diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index e3de8d73245..3bee580767a 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -7,7 +7,6 @@ use crate::beacon_block_streamer::{BeaconBlockStreamer, CheckCaches}; use crate::beacon_proposer_cache::{ BeaconProposerCache, EpochBlockProposers, ensure_state_can_determine_proposers_for_epoch, }; -use crate::blob_verification::{GossipBlobError, GossipVerifiedBlob}; use crate::block_times_cache::BlockTimesCache; use crate::block_verification::POS_PANDA_BANNER; use crate::block_verification::{ @@ -412,8 +411,6 @@ pub struct BeaconChain { pub(crate) observed_sync_aggregators: RwLock>, /// Maintains a record of which validators have proposed blocks for each slot. pub observed_block_producers: RwLock>, - /// Maintains a record of blob sidecars seen over the gossip network. - pub observed_blob_sidecars: RwLock>>, /// Maintains a record of column sidecars seen over the gossip network. pub observed_column_sidecars: RwLock>>, /// Maintains a record of slashable message seen over the gossip network or RPC. @@ -1131,13 +1128,18 @@ impl BeaconChain { .or_else(|| self.early_attester_cache.get_data_columns(block_root)); if let Some(mut all_cached_columns) = all_cached_columns_opt { - all_cached_columns.retain(|col| indices.contains(&col.index)); + all_cached_columns.retain(|col| indices.contains(col.index())); Ok(all_cached_columns) - } else { + } else if let Some(block) = self.get_blinded_block(&block_root)? { indices .iter() - .filter_map(|index| self.get_data_column(&block_root, index).transpose()) + .filter_map(|index| { + self.get_data_column(&block_root, index, block.fork_name_unchecked()) + .transpose() + }) .collect::>() + } else { + Ok(vec![]) } } @@ -1222,8 +1224,11 @@ impl BeaconChain { pub fn get_data_columns( &self, block_root: &Hash256, + fork_name: ForkName, ) -> Result>, Error> { - self.store.get_data_columns(block_root).map_err(Error::from) + self.store + .get_data_columns(block_root, fork_name) + .map_err(Error::from) } /// Returns the blobs at the given root, if any. @@ -1244,7 +1249,8 @@ impl BeaconChain { }; if self.spec.is_peer_das_enabled_for_epoch(block.epoch()) { - if let Some(columns) = self.store.get_data_columns(block_root)? { + let fork_name = self.spec.fork_name_at_epoch(block.epoch()); + if let Some(columns) = self.store.get_data_columns(block_root, fork_name)? { let num_required_columns = T::EthSpec::number_of_columns() / 2; let reconstruction_possible = columns.len() >= num_required_columns; if reconstruction_possible { @@ -1260,7 +1266,7 @@ impl BeaconChain { Ok(None) } } else { - self.get_blobs(block_root).map(|b| b.blobs()) + Ok(self.get_blobs(block_root)?.blobs()) } } @@ -1272,8 +1278,11 @@ impl BeaconChain { &self, block_root: &Hash256, column_index: &ColumnIndex, + fork_name: ForkName, ) -> Result>>, Error> { - Ok(self.store.get_data_column(block_root, column_index)?) + Ok(self + .store + .get_data_column(block_root, column_index, fork_name)?) } pub fn get_blinded_block( @@ -2194,19 +2203,6 @@ impl BeaconChain { }) } - #[instrument(skip_all, level = "trace")] - pub fn verify_blob_sidecar_for_gossip( - self: &Arc, - blob_sidecar: Arc>, - subnet_id: u64, - ) -> Result, GossipBlobError> { - metrics::inc_counter(&metrics::BLOBS_SIDECAR_PROCESSING_REQUESTS); - let _timer = metrics::start_timer(&metrics::BLOBS_SIDECAR_GOSSIP_VERIFICATION_TIMES); - GossipVerifiedBlob::new(blob_sidecar, subnet_id, self).inspect(|_| { - metrics::inc_counter(&metrics::BLOBS_SIDECAR_PROCESSING_SUCCESSES); - }) - } - /// Accepts some 'LightClientOptimisticUpdate' from the network and attempts to verify it pub fn verify_optimistic_update_for_gossip( self: &Arc, @@ -3002,35 +2998,6 @@ impl BeaconChain { .map_err(BeaconChainError::TokioJoin)? } - /// Cache the blob in the processing cache, process it, then evict it from the cache if it was - /// imported or errors. - #[instrument(skip_all, level = "debug")] - pub async fn process_gossip_blob( - self: &Arc, - blob: GossipVerifiedBlob, - ) -> Result { - let block_root = blob.block_root(); - - // If this block has already been imported to forkchoice it must have been available, so - // we don't need to process its blobs again. - if self - .canonical_head - .fork_choice_read_lock() - .contains_block(&block_root) - { - return Err(BlockError::DuplicateFullyImported(blob.block_root())); - } - - // No need to process and import blobs beyond the PeerDAS epoch. - if self.spec.is_peer_das_enabled_for_epoch(blob.epoch()) { - return Err(BlockError::BlobNotRequired(blob.slot())); - } - - self.emit_sse_blob_sidecar_events(&block_root, std::iter::once(blob.as_blob())); - - self.check_gossip_blob_availability_and_import(blob).await - } - /// Cache the data columns in the processing cache, process it, then evict it from the cache if it was /// imported or errors. #[instrument(skip_all, level = "debug")] @@ -3093,19 +3060,21 @@ impl BeaconChain { return Err(BlockError::DuplicateFullyImported(block_root)); } - // Reject RPC blobs referencing unknown parents. Otherwise we allow potentially invalid data - // into the da_checker, where invalid = descendant of invalid blocks. - // Note: blobs should have at least one item and all items have the same parent root. - if let Some(parent_root) = blobs - .iter() - .filter_map(|b| b.as_ref().map(|b| b.block_parent_root())) - .next() - && !self - .canonical_head - .fork_choice_read_lock() - .contains_block(&parent_root) - { - return Err(BlockError::ParentUnknown { parent_root }); + for blob in &blobs { + if let Some(blob) = blob.as_ref() { + // Reject RPC blobs referencing unknown parents. Otherwise we allow potentially invalid data + // into the da_checker, where invalid = descendant of invalid blocks. + // Note: blobs should have at least one item and all items have the same parent root. + if !self + .canonical_head + .fork_choice_read_lock() + .contains_block(&blob.block_parent_root()) + { + return Err(BlockError::ParentUnknown { + parent_root: blob.block_parent_root(), + }); + } + } } self.emit_sse_blob_sidecar_events(&block_root, blobs.iter().flatten().map(Arc::as_ref)); @@ -3132,9 +3101,6 @@ impl BeaconChain { } match &engine_get_blobs_output { - EngineGetBlobsOutput::Blobs(blobs) => { - self.emit_sse_blob_sidecar_events(&block_root, blobs.iter().map(|b| b.as_blob())); - } EngineGetBlobsOutput::CustodyColumns(columns) => { self.emit_sse_data_column_sidecar_events( &block_root, @@ -3183,7 +3149,7 @@ impl BeaconChain { .cached_data_column_indexes(block_root) .unwrap_or_default(); let new_data_columns = - data_columns_iter.filter(|b| !imported_data_columns.contains(&b.index)); + data_columns_iter.filter(|b| !imported_data_columns.contains(b.index())); for data_column in new_data_columns { event_handler.register(EventKind::DataColumnSidecar( @@ -3223,7 +3189,13 @@ impl BeaconChain { // Reject RPC columns referencing unknown parents. Otherwise we allow potentially invalid data // into the da_checker, where invalid = descendant of invalid blocks. // Note: custody_columns should have at least one item and all items have the same parent root. - if let Some(parent_root) = custody_columns.iter().map(|c| c.block_parent_root()).next() + if let Some(parent_root) = custody_columns + .iter() + .filter_map(|c| match c.as_ref() { + DataColumnSidecar::Fulu(column) => Some(column.block_parent_root()), + _ => None, + }) + .next() && !self .canonical_head .fork_choice_read_lock() @@ -3515,24 +3487,6 @@ impl BeaconChain { .await } - /// Checks if the provided blob can make any cached blocks available, and imports immediately - /// if so, otherwise caches the blob in the data availability checker. - async fn check_gossip_blob_availability_and_import( - self: &Arc, - blob: GossipVerifiedBlob, - ) -> Result { - let slot = blob.slot(); - if let Some(slasher) = self.slasher.as_ref() { - slasher.accept_block_header(blob.signed_block_header()); - } - let availability = self - .data_availability_checker - .put_gossip_verified_blobs(blob.block_root(), std::iter::once(blob))?; - - self.process_availability(slot, availability, || Ok(())) - .await - } - /// Checks if the provided data column can make any cached blocks available, and imports immediately /// if so, otherwise caches the data column in the data availability checker. async fn check_gossip_data_columns_availability_and_import( @@ -3543,8 +3497,10 @@ impl BeaconChain { publish_fn: impl FnOnce() -> Result<(), BlockError>, ) -> Result { if let Some(slasher) = self.slasher.as_ref() { - for data_colum in &data_columns { - slasher.accept_block_header(data_colum.signed_block_header()); + for data_column in &data_columns { + if let DataColumnSidecar::Fulu(c) = data_column.as_data_column() { + slasher.accept_block_header(c.signed_block_header.clone()); + } } } @@ -3596,7 +3552,7 @@ impl BeaconChain { ) -> Result { self.check_blob_header_signature_and_slashability( block_root, - blobs.iter().flatten().map(Arc::as_ref), + blobs.iter().flatten().map(|b| b.as_ref()), )?; let availability = self .data_availability_checker @@ -3613,18 +3569,15 @@ impl BeaconChain { engine_get_blobs_output: EngineGetBlobsOutput, ) -> Result { let availability = match engine_get_blobs_output { - EngineGetBlobsOutput::Blobs(blobs) => { - self.check_blob_header_signature_and_slashability( - block_root, - blobs.iter().map(|b| b.as_blob()), - )?; - self.data_availability_checker - .put_kzg_verified_blobs(block_root, blobs)? - } EngineGetBlobsOutput::CustodyColumns(data_columns) => { self.check_data_column_sidecar_header_signature_and_slashability( block_root, - data_columns.iter().map(|c| c.as_data_column()), + data_columns + .iter() + .filter_map(|c| match c.as_data_column() { + DataColumnSidecar::Fulu(column) => Some(column), + _ => None, + }), )?; self.data_availability_checker .put_kzg_verified_custody_data_columns(block_root, data_columns)? @@ -3645,7 +3598,10 @@ impl BeaconChain { ) -> Result { self.check_data_column_sidecar_header_signature_and_slashability( block_root, - custody_columns.iter().map(|c| c.as_ref()), + custody_columns.iter().filter_map(|c| match c.as_ref() { + DataColumnSidecar::Fulu(fulu) => Some(fulu), + _ => None, + }), )?; // This slot value is purely informative for the consumers of @@ -3663,7 +3619,7 @@ impl BeaconChain { fn check_data_column_sidecar_header_signature_and_slashability<'a>( self: &Arc, block_root: Hash256, - custody_columns: impl IntoIterator>, + custody_columns: impl IntoIterator>, ) -> Result<(), BlockError> { let mut slashable_cache = self.observed_slashable.write(); // Process all unique block headers - previous logic assumed all headers were identical and @@ -7366,7 +7322,7 @@ impl BeaconChain { // Supernodes need to persist all sampled custody columns if columns_to_custody.len() != self.spec.number_of_custody_groups as usize { data_columns - .retain(|data_column| columns_to_custody.contains(&data_column.index)); + .retain(|data_column| columns_to_custody.contains(data_column.index())); } debug!( %block_root, @@ -7379,7 +7335,11 @@ impl BeaconChain { } /// Retrieves block roots (in ascending slot order) within some slot range from fork choice. - pub fn block_roots_from_fork_choice(&self, start_slot: u64, count: u64) -> Vec { + pub fn block_roots_from_fork_choice( + &self, + start_slot: u64, + count: u64, + ) -> Vec<(Hash256, Slot)> { let head_block_root = self.canonical_head.cached_head().head_block_root(); let fork_choice_read_lock = self.canonical_head.fork_choice_read_lock(); let block_roots_iter = fork_choice_read_lock @@ -7390,7 +7350,7 @@ impl BeaconChain { for (root, slot) in block_roots_iter { if slot < end_slot && slot >= start_slot { - roots.push(root); + roots.push((root, slot)); } if slot < start_slot { break; diff --git a/beacon_node/beacon_chain/src/blob_verification.rs b/beacon_node/beacon_chain/src/blob_verification.rs index 45374f509b2..e6e5a34f638 100644 --- a/beacon_node/beacon_chain/src/blob_verification.rs +++ b/beacon_node/beacon_chain/src/blob_verification.rs @@ -1,24 +1,13 @@ -use educe::Educe; -use slot_clock::SlotClock; -use std::marker::PhantomData; -use std::sync::Arc; - -use crate::beacon_chain::{BeaconChain, BeaconChainTypes}; -use crate::block_verification::{ - BlockSlashInfo, get_validator_pubkey_cache, process_block_slash_info, -}; use crate::kzg_utils::{validate_blob, validate_blobs}; -use crate::observed_data_sidecars::{ObservationStrategy, Observe}; use crate::{BeaconChainError, metrics}; +use educe::Educe; use kzg::{Error as KzgError, Kzg, KzgCommitment}; use ssz_derive::{Decode, Encode}; +use std::sync::Arc; use std::time::Duration; -use tracing::{debug, instrument}; +use tracing::instrument; use tree_hash::TreeHash; -use types::data::BlobIdentifier; -use types::{ - BeaconStateError, BlobSidecar, Epoch, EthSpec, Hash256, SignedBeaconBlockHeader, Slot, -}; +use types::{BeaconStateError, BlobSidecar, EthSpec, Hash256, Slot}; /// An error occurred while validating a gossip blob. #[derive(Debug)] @@ -156,93 +145,6 @@ impl From for GossipBlobError { } } -/// A wrapper around a `BlobSidecar` that indicates it has been approved for re-gossiping on -/// the p2p network. -#[derive(Debug)] -pub struct GossipVerifiedBlob { - block_root: Hash256, - blob: KzgVerifiedBlob, - _phantom: PhantomData, -} - -impl Clone for GossipVerifiedBlob { - fn clone(&self) -> Self { - Self { - block_root: self.block_root, - blob: self.blob.clone(), - _phantom: PhantomData, - } - } -} - -impl GossipVerifiedBlob { - pub fn new( - blob: Arc>, - subnet_id: u64, - chain: &BeaconChain, - ) -> Result { - let header = blob.signed_block_header.clone(); - // We only process slashing info if the gossip verification failed - // since we do not process the blob any further in that case. - validate_blob_sidecar_for_gossip::(blob, subnet_id, chain).map_err(|e| { - process_block_slash_info::<_, GossipBlobError>( - chain, - BlockSlashInfo::from_early_error_blob(header, e), - ) - }) - } - /// Construct a `GossipVerifiedBlob` that is assumed to be valid. - /// - /// This should ONLY be used for testing. - pub fn __assumed_valid(blob: Arc>) -> Self { - Self { - block_root: blob.block_root(), - blob: KzgVerifiedBlob { - blob, - seen_timestamp: Duration::from_secs(0), - }, - _phantom: PhantomData, - } - } - pub fn id(&self) -> BlobIdentifier { - BlobIdentifier { - block_root: self.block_root, - index: self.blob.blob_index(), - } - } - pub fn block_root(&self) -> Hash256 { - self.block_root - } - pub fn slot(&self) -> Slot { - self.blob.blob.slot() - } - pub fn epoch(&self) -> Epoch { - self.blob.blob.epoch() - } - pub fn index(&self) -> u64 { - self.blob.blob.index - } - pub fn kzg_commitment(&self) -> KzgCommitment { - self.blob.blob.kzg_commitment - } - pub fn signed_block_header(&self) -> SignedBeaconBlockHeader { - self.blob.blob.signed_block_header.clone() - } - pub fn block_proposer_index(&self) -> u64 { - self.blob.blob.block_proposer_index() - } - pub fn into_inner(self) -> KzgVerifiedBlob { - self.blob - } - pub fn as_blob(&self) -> &BlobSidecar { - self.blob.as_blob() - } - /// This is cheap as we're calling clone on an Arc - pub fn clone_blob(&self) -> Arc> { - self.blob.clone_blob() - } -} - /// Wrapper over a `BlobSidecar` for which we have completed kzg verification. /// i.e. `verify_blob_kzg_proof(blob, commitment, proof) == true`. #[derive(Debug, Educe, Clone, Encode, Decode)] @@ -387,223 +289,6 @@ where validate_blobs::(kzg, commitments.as_slice(), blobs, proofs.as_slice()) } -pub fn validate_blob_sidecar_for_gossip( - blob_sidecar: Arc>, - subnet: u64, - chain: &BeaconChain, -) -> Result, GossipBlobError> { - let blob_slot = blob_sidecar.slot(); - let blob_index = blob_sidecar.index; - let block_parent_root = blob_sidecar.block_parent_root(); - let blob_proposer_index = blob_sidecar.block_proposer_index(); - let block_root = blob_sidecar.block_root(); - let blob_epoch = blob_slot.epoch(T::EthSpec::slots_per_epoch()); - let signed_block_header = &blob_sidecar.signed_block_header; - - let seen_timestamp = chain.slot_clock.now_duration().unwrap_or_default(); - - // This condition is not possible if we have received the blob from the network - // since we only subscribe to `MaxBlobsPerBlock` subnets over gossip network. - // We include this check only for completeness. - // Getting this error would imply something very wrong with our networking decoding logic. - if blob_index >= chain.spec.max_blobs_per_block(blob_epoch) { - return Err(GossipBlobError::InvalidSubnet { - expected: subnet, - received: blob_index, - }); - } - - // Verify that the blob_sidecar was received on the correct subnet. - if blob_index != subnet { - return Err(GossipBlobError::InvalidSubnet { - expected: subnet, - received: blob_index, - }); - } - - // Verify that the sidecar is not from a future slot. - let latest_permissible_slot = chain - .slot_clock - .now_with_future_tolerance(chain.spec.maximum_gossip_clock_disparity()) - .ok_or(BeaconChainError::UnableToReadSlot)?; - if blob_slot > latest_permissible_slot { - return Err(GossipBlobError::FutureSlot { - message_slot: blob_slot, - latest_permissible_slot, - }); - } - - // Verify that the sidecar slot is greater than the latest finalized slot - let latest_finalized_slot = chain - .head() - .finalized_checkpoint() - .epoch - .start_slot(T::EthSpec::slots_per_epoch()); - if blob_slot <= latest_finalized_slot { - return Err(GossipBlobError::PastFinalizedSlot { - blob_slot, - finalized_slot: latest_finalized_slot, - }); - } - - // Verify that this is the first blob sidecar received for the tuple: - // (block_header.slot, block_header.proposer_index, blob_sidecar.index) - if chain - .observed_blob_sidecars - .read() - .proposer_is_known(&blob_sidecar) - .map_err(|e| GossipBlobError::BeaconChainError(Box::new(e.into())))? - { - return Err(GossipBlobError::RepeatBlob { - proposer: blob_proposer_index, - slot: blob_slot, - index: blob_index, - }); - } - - // Verify the inclusion proof in the sidecar - let _timer = metrics::start_timer(&metrics::BLOB_SIDECAR_INCLUSION_PROOF_VERIFICATION); - if !blob_sidecar.verify_blob_sidecar_inclusion_proof() { - return Err(GossipBlobError::InvalidInclusionProof); - } - drop(_timer); - - let fork_choice = chain.canonical_head.fork_choice_read_lock(); - - // We have already verified that the blob is past finalization, so we can - // just check fork choice for the block's parent. - let Some(parent_block) = fork_choice.get_block(&block_parent_root) else { - return Err(GossipBlobError::ParentUnknown { - parent_root: block_parent_root, - }); - }; - - // Do not process a blob that does not descend from the finalized root. - // We just loaded the parent_block, so we can be sure that it exists in fork choice. - if !fork_choice.is_finalized_checkpoint_or_descendant(block_parent_root) { - return Err(GossipBlobError::NotFinalizedDescendant { block_parent_root }); - } - drop(fork_choice); - - if parent_block.slot >= blob_slot { - return Err(GossipBlobError::BlobIsNotLaterThanParent { - blob_slot, - parent_slot: parent_block.slot, - }); - } - - let proposer_shuffling_root = - parent_block.proposer_shuffling_root_for_child_block(blob_epoch, &chain.spec); - - let proposer = chain.with_proposer_cache( - proposer_shuffling_root, - blob_epoch, - |proposers| proposers.get_slot::(blob_slot), - || { - debug!( - %block_root, - index = %blob_index, - "Proposer shuffling cache miss for blob verification" - ); - chain - .store - .get_advanced_hot_state(block_parent_root, blob_slot, parent_block.state_root) - .map_err(|e| GossipBlobError::BeaconChainError(Box::new(e.into())))? - .ok_or_else(|| { - GossipBlobError::BeaconChainError(Box::new(BeaconChainError::DBInconsistent( - format!("Missing state for parent block {block_parent_root:?}",), - ))) - }) - }, - )?; - let proposer_index = proposer.index; - let fork = proposer.fork; - - // Signature verify the signed block header. - let signature_is_valid = { - let pubkey_cache = - get_validator_pubkey_cache(chain).map_err(|_| GossipBlobError::PubkeyCacheTimeout)?; - - let pubkey = pubkey_cache - .get(proposer_index) - .ok_or_else(|| GossipBlobError::UnknownValidator(proposer_index as u64))?; - signed_block_header.verify_signature::( - pubkey, - &fork, - chain.genesis_validators_root, - &chain.spec, - ) - }; - - if !signature_is_valid { - return Err(GossipBlobError::ProposalSignatureInvalid); - } - - if proposer_index != blob_proposer_index as usize { - return Err(GossipBlobError::ProposerIndexMismatch { - sidecar: blob_proposer_index as usize, - local: proposer_index, - }); - } - - // Kzg verification for gossip blob sidecar - let kzg = chain.kzg.as_ref(); - - let kzg_verified_blob = KzgVerifiedBlob::new(blob_sidecar.clone(), kzg, seen_timestamp) - .map_err(GossipBlobError::KzgError)?; - let blob_sidecar = &kzg_verified_blob.blob; - - chain - .observed_slashable - .write() - .observe_slashable( - blob_sidecar.slot(), - blob_sidecar.block_proposer_index(), - block_root, - ) - .map_err(|e| GossipBlobError::BeaconChainError(Box::new(e.into())))?; - - if O::observe() { - observe_gossip_blob(&kzg_verified_blob.blob, chain)?; - } - - Ok(GossipVerifiedBlob { - block_root, - blob: kzg_verified_blob, - _phantom: PhantomData, - }) -} - -pub fn observe_gossip_blob( - blob_sidecar: &BlobSidecar, - chain: &BeaconChain, -) -> Result<(), GossipBlobError> { - // Now the signature is valid, store the proposal so we don't accept another blob sidecar - // with the same `BlobIdentifier`. It's important to double-check that the proposer still - // hasn't been observed so we don't have a race-condition when verifying two blocks - // simultaneously. - // - // Note: If this BlobSidecar goes on to fail full verification, we do not evict it from the - // seen_cache as alternate blob_sidecars for the same identifier can still be retrieved over - // rpc. Evicting them from this cache would allow faster propagation over gossip. So we - // allow retrieval of potentially valid blocks over rpc, but try to punish the proposer for - // signing invalid messages. Issue for more background - // https://github.com/ethereum/consensus-specs/issues/3261 - if chain - .observed_blob_sidecars - .write() - .observe_sidecar(blob_sidecar) - .map_err(|e| GossipBlobError::BeaconChainError(Box::new(e.into())))? - { - return Err(GossipBlobError::RepeatBlob { - proposer: blob_sidecar.block_proposer_index(), - slot: blob_sidecar.slot(), - index: blob_sidecar.index, - }); - } - Ok(()) -} - /// Returns the canonical root of the given `blob`. /// /// Use this function to ensure that we report the blob hashing time Prometheus metric. diff --git a/beacon_node/beacon_chain/src/builder.rs b/beacon_node/beacon_chain/src/builder.rs index dc38fc1c292..5559b2dcbf1 100644 --- a/beacon_node/beacon_chain/src/builder.rs +++ b/beacon_node/beacon_chain/src/builder.rs @@ -9,7 +9,7 @@ use crate::data_availability_checker::DataAvailabilityChecker; use crate::fork_choice_signal::ForkChoiceSignalTx; use crate::fork_revert::{reset_fork_choice_to_finalization, revert_to_fork_boundary}; use crate::graffiti_calculator::{GraffitiCalculator, GraffitiOrigin}; -use crate::kzg_utils::build_data_column_sidecars; +use crate::kzg_utils::{build_data_column_sidecars_fulu, build_data_column_sidecars_gloas}; use crate::light_client_server_cache::LightClientServerCache; use crate::migrate::{BackgroundMigrator, MigratorConfig}; use crate::observed_data_sidecars::ObservedDataSidecars; @@ -1006,7 +1006,6 @@ where // TODO: allow for persisting and loading the pool from disk. observed_block_producers: <_>::default(), observed_column_sidecars: RwLock::new(ObservedDataSidecars::new(self.spec.clone())), - observed_blob_sidecars: RwLock::new(ObservedDataSidecars::new(self.spec.clone())), observed_slashable: <_>::default(), observed_voluntary_exits: <_>::default(), observed_proposer_slashings: <_>::default(), @@ -1213,17 +1212,29 @@ fn build_data_columns_from_blobs( .blob_kzg_commitments() .cloned() .map_err(|e| format!("Unexpected pre Deneb block: {e:?}"))?; - let kzg_commitments_inclusion_proof = beacon_block_body - .kzg_commitments_merkle_proof() - .map_err(|e| format!("Failed to compute kzg commitments merkle proof: {e:?}"))?; - build_data_column_sidecars( - kzg_commitments, - kzg_commitments_inclusion_proof, - block.signed_block_header(), - blob_cells_and_proofs_vec, - spec, - ) - .map_err(|e| format!("Failed to compute weak subjectivity data_columns: {e:?}"))? + + if block.fork_name_unchecked().gloas_enabled() { + build_data_column_sidecars_gloas( + kzg_commitments, + block.signed_block_header(), + blob_cells_and_proofs_vec, + spec, + ) + .map_err(|e| format!("Failed to compute weak subjectivity data_columns: {e:?}"))? + } else { + let kzg_commitments_inclusion_proof = beacon_block_body + .kzg_commitments_merkle_proof() + .map_err(|e| format!("Failed to compute kzg commitments merkle proof: {e:?}"))?; + + build_data_column_sidecars_fulu( + kzg_commitments, + kzg_commitments_inclusion_proof, + block.signed_block_header(), + blob_cells_and_proofs_vec, + spec, + ) + .map_err(|e| format!("Failed to compute weak subjectivity data_columns: {e:?}"))? + } }; Ok(data_columns) } diff --git a/beacon_node/beacon_chain/src/canonical_head.rs b/beacon_node/beacon_chain/src/canonical_head.rs index 76c08c5e39c..4c7a234c09a 100644 --- a/beacon_node/beacon_chain/src/canonical_head.rs +++ b/beacon_node/beacon_chain/src/canonical_head.rs @@ -944,13 +944,6 @@ impl BeaconChain { .start_slot(T::EthSpec::slots_per_epoch()), ); - self.observed_blob_sidecars.write().prune( - new_view - .finalized_checkpoint - .epoch - .start_slot(T::EthSpec::slots_per_epoch()), - ); - self.observed_slashable.write().prune( new_view .finalized_checkpoint diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index 7aec24b8e52..051a3f563a4 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -1,6 +1,4 @@ -use crate::blob_verification::{ - GossipVerifiedBlob, KzgVerifiedBlob, KzgVerifiedBlobList, verify_kzg_for_blob_list, -}; +use crate::blob_verification::{KzgVerifiedBlob, KzgVerifiedBlobList, verify_kzg_for_blob_list}; use crate::block_verification_types::{ AvailabilityPendingExecutedBlock, AvailableExecutedBlock, RpcBlock, }; @@ -19,10 +17,10 @@ use std::sync::Arc; use std::time::Duration; use task_executor::TaskExecutor; use tracing::{debug, error, instrument}; -use types::data::{BlobIdentifier, BlobSidecar, FixedBlobSidecarList}; +use types::data::{BlobIdentifier, FixedBlobSidecarList}; use types::{ - BlobSidecarList, BlockImportSource, ChainSpec, DataColumnSidecar, DataColumnSidecarList, Epoch, - EthSpec, Hash256, SignedBeaconBlock, Slot, + BlobSidecar, BlobSidecarList, BlockImportSource, ChainSpec, DataColumnSidecar, + DataColumnSidecarList, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot, }; mod error; @@ -187,7 +185,7 @@ impl DataAvailabilityChecker { self.availability_cache .peek_pending_components(block_root, |components| { components.is_some_and(|components| { - let cached_column_opt = components.get_cached_data_column(data_column.index); + let cached_column_opt = components.get_cached_data_column(*data_column.index()); cached_column_opt.is_some_and(|cached| *cached == *data_column) }) }) @@ -269,24 +267,6 @@ impl DataAvailabilityChecker { .put_kzg_verified_data_columns(block_root, verified_custody_columns) } - /// Check if we've cached other blobs for this block. If it completes a set and we also - /// have a block cached, return the `Availability` variant triggering block import. - /// Otherwise cache the blob sidecar. - /// - /// This should only accept gossip verified blobs, so we should not have to worry about dupes. - #[instrument(skip_all, level = "trace")] - pub fn put_gossip_verified_blobs< - I: IntoIterator>, - O: ObservationStrategy, - >( - &self, - block_root: Hash256, - blobs: I, - ) -> Result, AvailabilityCheckError> { - self.availability_cache - .put_kzg_verified_blobs(block_root, blobs.into_iter().map(|b| b.into_inner())) - } - #[instrument(skip_all, level = "trace")] pub fn put_kzg_verified_blobs>>( &self, @@ -877,7 +857,9 @@ mod test { use std::time::Duration; use store::HotColdDB; use types::data::DataColumn; - use types::{ChainSpec, ColumnIndex, EthSpec, ForkName, MainnetEthSpec, Slot}; + use types::{ + ChainSpec, ColumnIndex, DataColumnSidecarFulu, EthSpec, ForkName, MainnetEthSpec, Slot, + }; type E = MainnetEthSpec; type T = EphemeralHarnessType; @@ -932,7 +914,7 @@ mod test { cgc_change_slot, data_columns .into_iter() - .filter(|d| requested_columns.contains(&d.index)) + .filter(|d| requested_columns.contains(d.index())) .collect(), ) .expect("should put rpc custody columns"); @@ -1007,7 +989,7 @@ mod test { let requested_columns = &custody_columns[..10]; let gossip_columns = data_columns .into_iter() - .filter(|d| requested_columns.contains(&d.index)) + .filter(|d| requested_columns.contains(d.index())) .map(GossipVerifiedDataColumn::::__new_for_testing) .collect::>(); da_checker @@ -1039,7 +1021,7 @@ mod test { /// Regression test for KZG verification truncation bug (https://github.com/sigp/lighthouse/pull/7927) #[test] - fn verify_kzg_for_rpc_blocks_should_not_truncate_data_columns() { + fn verify_kzg_for_rpc_blocks_should_not_truncate_data_columns_fulu() { let spec = Arc::new(ForkName::Fulu.make_genesis_spec(E::default_spec())); let mut rng = StdRng::seed_from_u64(0xDEADBEEF0BAD5EEDu64); let da_checker = new_da_checker(spec.clone()); @@ -1065,10 +1047,17 @@ mod test { data_columns .into_iter() .map(|d| { - let invalid_sidecar = DataColumnSidecar { + let invalid_sidecar = DataColumnSidecar::Fulu(DataColumnSidecarFulu { column: DataColumn::::empty(), - ..d.as_ref().clone() - }; + index: *d.index(), + kzg_commitments: d.kzg_commitments().clone(), + kzg_proofs: d.kzg_proofs().clone(), + signed_block_header: d.signed_block_header().unwrap().clone(), + kzg_commitments_inclusion_proof: d + .kzg_commitments_inclusion_proof() + .unwrap() + .clone(), + }); CustodyDataColumn::from_asserted_custody(Arc::new(invalid_sidecar)) }) .collect::>() @@ -1126,7 +1115,7 @@ mod test { let custody_columns = custody_context.custody_columns_for_epoch(None, &spec); let custody_columns = custody_columns .iter() - .filter_map(|&col_idx| data_columns.iter().find(|d| d.index == col_idx).cloned()) + .filter_map(|&col_idx| data_columns.iter().find(|d| *d.index() == col_idx).cloned()) .take(64) .map(|d| { KzgVerifiedCustodyDataColumn::from_asserted_custody( diff --git a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs index c249af2d400..83d6ee87618 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs @@ -823,9 +823,11 @@ impl DataAvailabilityCheckerInner { mod test { use super::*; - use crate::test_utils::generate_data_column_indices_rand_order; + use crate::data_column_verification::{GossipVerifiedDataColumn, KzgVerifiedCustodyDataColumn}; + use crate::test_utils::{ + generate_data_column_indices_rand_order, generate_data_column_sidecars_from_block, + }; use crate::{ - blob_verification::GossipVerifiedBlob, block_verification::PayloadVerificationOutcome, block_verification_types::{AsBlock, BlockImportData}, custody_context::NodeCustodyType, @@ -839,6 +841,7 @@ mod test { use store::{HotColdDB, ItemStore, StoreConfig, database::interface::BeaconNodeBackend}; use tempfile::{TempDir, tempdir}; use tracing::{debug_span, info}; + use types::data::DataColumnSubnetId; use types::new_non_zero_usize; use types::{ExecPayload, MinimalEthSpec}; @@ -865,8 +868,8 @@ mod test { .expect("disk store should initialize") } - // get a beacon chain harness advanced to just before deneb fork - async fn get_deneb_chain( + // get a beacon chain harness advanced to just before fulu fork + async fn get_fulu_chain( db_path: &TempDir, ) -> BeaconChainHarness> { let altair_fork_epoch = Epoch::new(1); @@ -874,13 +877,17 @@ mod test { let bellatrix_fork_slot = bellatrix_fork_epoch.start_slot(E::slots_per_epoch()); let capella_fork_epoch = Epoch::new(3); let deneb_fork_epoch = Epoch::new(4); - let deneb_fork_slot = deneb_fork_epoch.start_slot(E::slots_per_epoch()); + let electra_fork_epoch = Epoch::new(5); + let fulu_fork_epoch = Epoch::new(6); + let fulu_fork_slot = fulu_fork_epoch.start_slot(E::slots_per_epoch()); let mut spec = E::default_spec(); spec.altair_fork_epoch = Some(altair_fork_epoch); spec.bellatrix_fork_epoch = Some(bellatrix_fork_epoch); spec.capella_fork_epoch = Some(capella_fork_epoch); spec.deneb_fork_epoch = Some(deneb_fork_epoch); + spec.electra_fork_epoch = Some(electra_fork_epoch); + spec.fulu_fork_epoch = Some(fulu_fork_epoch); let spec = Arc::new(spec); let chain_store = get_store_with_spec::(db_path, spec.clone()); @@ -912,8 +919,8 @@ mod test { .execution_block_generator() .move_to_terminal_block() .unwrap(); - // go right before deneb slot - harness.extend_to_slot(deneb_fork_slot - 1).await; + // go right before fulu slot + harness.extend_to_slot(fulu_fork_slot - 1).await; harness } @@ -922,7 +929,7 @@ mod test { harness: &BeaconChainHarness>, ) -> ( AvailabilityPendingExecutedBlock, - Vec>>, + Vec>>, ) where E: EthSpec, @@ -940,7 +947,7 @@ mod test { .expect("should get block") .expect("should have block"); - let (signed_beacon_block_hash, (block, maybe_blobs), state) = harness + let (signed_beacon_block_hash, (block, _maybe_blobs), state) = harness .add_block_at_slot(target_slot, parent_state) .await .expect("should add block"); @@ -958,27 +965,25 @@ mod test { .message() .body() .blob_kzg_commitments() - .expect("should be deneb fork") + .expect("should be fulu fork") .clone(), ) { info!(commitment = ?comm, "kzg commitment"); } info!("done printing kzg commitments"); - let gossip_verified_blobs = if let Some((kzg_proofs, blobs)) = maybe_blobs { - let sidecars = - BlobSidecar::build_sidecars(blobs, &block, kzg_proofs, &chain.spec).unwrap(); - Vec::from(sidecars) - .into_iter() - .map(|sidecar| { - let subnet = sidecar.index; - GossipVerifiedBlob::new(sidecar, subnet, &harness.chain) - .expect("should validate blob") - }) - .collect() - } else { - vec![] - }; + // Generate data columns from the block + let data_columns = generate_data_column_sidecars_from_block(&block, &harness.spec); + + let gossip_verified_columns: Vec<_> = data_columns + .into_iter() + .map(|sidecar| { + let subnet_id = + DataColumnSubnetId::from_column_index(*sidecar.index(), &harness.spec); + GossipVerifiedDataColumn::new(sidecar, subnet_id, &harness.chain) + .expect("should validate data column") + }) + .collect(); let slot = block.slot(); let consensus_context = ConsensusContext::::new(slot); @@ -1000,7 +1005,7 @@ mod test { payload_verification_outcome, }; - (availability_pending_block, gossip_verified_blobs) + (availability_pending_block, gossip_verified_columns) } async fn setup_harness_and_cache( @@ -1020,7 +1025,7 @@ mod test { { create_test_tracing_subscriber(); let chain_db_path = tempdir().expect("should get temp dir"); - let harness = get_deneb_chain(&chain_db_path).await; + let harness = get_fulu_chain(&chain_db_path).await; let spec = harness.spec.clone(); let test_store = harness.chain.store.clone(); let capacity_non_zero = new_non_zero_usize(capacity); @@ -1048,20 +1053,27 @@ mod test { let capacity = 4; let (harness, cache, _path) = setup_harness_and_cache::(capacity).await; - let (pending_block, blobs) = availability_pending_block(&harness).await; + let (pending_block, columns) = availability_pending_block(&harness).await; let root = pending_block.import_data.block_root; + let epoch = pending_block.block.epoch(); - let blobs_expected = pending_block.num_blobs_expected(); + let num_blobs_expected = pending_block.num_blobs_expected(); + let columns_expected = cache + .custody_context + .num_of_data_columns_to_sample(epoch, &harness.spec); + + // All columns are returned from availability_pending_block (E::number_of_columns()) + // but we only need custody columns assert_eq!( - blobs.len(), - blobs_expected, - "should have expected number of blobs" + columns.len(), + E::number_of_columns(), + "should have all data columns from block" ); assert!(cache.critical.read().is_empty(), "cache should be empty"); let availability = cache .put_executed_block(pending_block) .expect("should put block"); - if blobs_expected == 0 { + if num_blobs_expected == 0 { assert!( matches!(availability, Availability::Available(_)), "block doesn't have blobs, should be available" @@ -1074,7 +1086,7 @@ mod test { } else { assert!( matches!(availability, Availability::MissingComponents(_)), - "should be pending blobs" + "should be pending columns" ); assert_eq!( cache.critical.read().len(), @@ -1087,13 +1099,26 @@ mod test { ); } - let mut kzg_verified_blobs = Vec::new(); - for (blob_index, gossip_blob) in blobs.into_iter().enumerate() { - kzg_verified_blobs.push(gossip_blob.into_inner()); + // Get custody column indices for this epoch + let custody_column_indices = cache + .custody_context + .custody_columns_for_epoch(Some(epoch), &harness.spec); + + // Filter to only custody columns + let custody_columns: Vec<_> = columns + .into_iter() + .filter(|col| custody_column_indices.contains(&col.index())) + .collect(); + + let mut kzg_verified_columns = Vec::new(); + for (col_index, gossip_column) in custody_columns.into_iter().enumerate() { + kzg_verified_columns.push(KzgVerifiedCustodyDataColumn::from_asserted_custody( + gossip_column.into_inner(), + )); let availability = cache - .put_kzg_verified_blobs(root, kzg_verified_blobs.clone()) - .expect("should put blob"); - if blob_index == blobs_expected - 1 { + .put_kzg_verified_data_columns(root, kzg_verified_columns.clone()) + .expect("should put column"); + if col_index == columns_expected - 1 { assert!(matches!(availability, Availability::Available(_))); } else { assert!(matches!(availability, Availability::MissingComponents(_))); @@ -1101,20 +1126,36 @@ mod test { } } - let (pending_block, blobs) = availability_pending_block(&harness).await; - let blobs_expected = pending_block.num_blobs_expected(); + let (pending_block, columns) = availability_pending_block(&harness).await; + let _num_blobs_expected = pending_block.num_blobs_expected(); + let epoch = pending_block.block.epoch(); + // All columns returned assert_eq!( - blobs.len(), - blobs_expected, - "should have expected number of blobs" + columns.len(), + E::number_of_columns(), + "should have all data columns" ); let root = pending_block.import_data.block_root; - let mut kzg_verified_blobs = vec![]; - for gossip_blob in blobs { - kzg_verified_blobs.push(gossip_blob.into_inner()); + + // Get custody column indices for this epoch + let custody_column_indices = cache + .custody_context + .custody_columns_for_epoch(Some(epoch), &harness.spec); + + // Filter to only custody columns + let custody_columns: Vec<_> = columns + .into_iter() + .filter(|col| custody_column_indices.contains(&col.index())) + .collect(); + + let mut kzg_verified_columns = vec![]; + for gossip_column in custody_columns { + kzg_verified_columns.push(KzgVerifiedCustodyDataColumn::from_asserted_custody( + gossip_column.into_inner(), + )); let availability = cache - .put_kzg_verified_blobs(root, kzg_verified_blobs.clone()) - .expect("should put blob"); + .put_kzg_verified_data_columns(root, kzg_verified_columns.clone()) + .expect("should put column"); assert!( matches!(availability, Availability::MissingComponents(_)), "should be pending block" @@ -1151,11 +1192,11 @@ mod test { let mut pending_blocks = VecDeque::new(); let mut states = Vec::new(); let mut state_roots = Vec::new(); - // Get enough blocks to fill the cache to capacity, ensuring all blocks have blobs + // Get enough blocks to fill the cache to capacity, ensuring all blocks have data columns while pending_blocks.len() < capacity { let (mut pending_block, _) = availability_pending_block(&harness).await; if pending_block.num_blobs_expected() == 0 { - // we need blocks with blobs + // we need blocks with data (columns) continue; } let state_root = pending_block.import_data.state.canonical_root().unwrap(); @@ -1200,10 +1241,10 @@ mod test { .expect("should exist"); pushed_diet_blocks.push_back(diet_block); - // should be unavailable since we made sure all blocks had blobs + // should be unavailable since we made sure all blocks had data columns assert!( matches!(availability, Availability::MissingComponents(_)), - "should be pending blobs" + "should be pending columns" ); if i >= STATE_LRU_CAPACITY { diff --git a/beacon_node/beacon_chain/src/data_column_verification.rs b/beacon_node/beacon_chain/src/data_column_verification.rs index 7bb139756d9..299243e35fa 100644 --- a/beacon_node/beacon_chain/src/data_column_verification.rs +++ b/beacon_node/beacon_chain/src/data_column_verification.rs @@ -9,7 +9,7 @@ use fork_choice::ProtoBlock; use kzg::{Error as KzgError, Kzg}; use proto_array::Block; use slot_clock::SlotClock; -use ssz_derive::{Decode, Encode}; +use ssz_derive::Encode; use ssz_types::VariableList; use std::iter; use std::marker::PhantomData; @@ -17,8 +17,8 @@ use std::sync::Arc; use tracing::{debug, instrument}; use types::data::ColumnIndex; use types::{ - BeaconStateError, ChainSpec, DataColumnSidecar, DataColumnSubnetId, EthSpec, Hash256, - SignedBeaconBlockHeader, Slot, + BeaconStateError, ChainSpec, DataColumnSidecar, DataColumnSidecarFulu, DataColumnSubnetId, + EthSpec, Hash256, Slot, }; /// An error occurred while validating a gossip data column. @@ -123,11 +123,7 @@ pub enum GossipDataColumnError { /// ## Peer scoring /// /// The peer isn't faulty, but we do not forward it over gossip. - PriorKnown { - proposer: u64, - slot: Slot, - index: ColumnIndex, - }, + PriorKnown { slot: Slot, index: ColumnIndex }, /// A column has already been processed from non-gossip source and have not yet been seen on /// the gossip network. /// This column should be accepted and forwarded over gossip. @@ -209,17 +205,38 @@ impl GossipVerifiedDataColumn subnet_id: DataColumnSubnetId, chain: &BeaconChain, ) -> Result { - let header = column_sidecar.signed_block_header.clone(); - // We only process slashing info if the gossip verification failed - // since we do not process the data column any further in that case. - validate_data_column_sidecar_for_gossip::(column_sidecar, subnet_id, chain).map_err( - |e| { - process_block_slash_info::<_, GossipDataColumnError>( + match column_sidecar.as_ref() { + DataColumnSidecar::Fulu(c) => { + let header = c.signed_block_header.clone(); + // We only process slashing info if the gossip verification failed + // since we do not process the data column any further in that case. + validate_data_column_sidecar_for_gossip_fulu::( + column_sidecar, + subnet_id, chain, - BlockSlashInfo::from_early_error_data_column(header, e), ) - }, - ) + .map_err(|e| { + process_block_slash_info::<_, GossipDataColumnError>( + chain, + BlockSlashInfo::from_early_error_data_column(header, e), + ) + }) + } + DataColumnSidecar::Gloas(_) => { + todo!() + } + } + } + + /// Construct a `GossipVerifiedBlob` that is assumed to be valid. + /// + /// This should ONLY be used for testing. + pub fn __assumed_valid(column: Arc>) -> Self { + Self { + block_root: column.block_root(), + data_column: KzgVerifiedDataColumn { data: column }, + _phantom: PhantomData, + } } /// Create a `GossipVerifiedDataColumn` from `DataColumnSidecar` for block production ONLY. @@ -283,11 +300,7 @@ impl GossipVerifiedDataColumn } pub fn index(&self) -> ColumnIndex { - self.data_column.data.index - } - - pub fn signed_block_header(&self) -> SignedBeaconBlockHeader { - self.data_column.data.signed_block_header.clone() + *self.data_column.data.index() } pub fn into_inner(self) -> KzgVerifiedDataColumn { @@ -296,7 +309,7 @@ impl GossipVerifiedDataColumn } /// Wrapper over a `DataColumnSidecar` for which we have completed kzg verification. -#[derive(Debug, Educe, Clone, Encode, Decode)] +#[derive(Debug, Educe, Clone, Encode)] #[educe(PartialEq, Eq)] #[ssz(struct_behaviour = "transparent")] pub struct KzgVerifiedDataColumn { @@ -345,7 +358,7 @@ impl KzgVerifiedDataColumn { } pub fn index(&self) -> ColumnIndex { - self.data.index + *self.data.index() } } @@ -353,7 +366,7 @@ pub type CustodyDataColumnList = VariableList, ::NumberOfColumns>; /// Data column that we must custody -#[derive(Debug, Educe, Clone, Encode, Decode)] +#[derive(Debug, Educe, Clone, Encode)] #[educe(PartialEq, Eq, Hash(bound(E: EthSpec)))] #[ssz(struct_behaviour = "transparent")] pub struct CustodyDataColumn { @@ -378,12 +391,12 @@ impl CustodyDataColumn { self.data.clone() } pub fn index(&self) -> u64 { - self.data.index + *self.data.index() } } /// Data column that we must custody and has completed kzg verification -#[derive(Debug, Educe, Clone, Encode, Decode)] +#[derive(Debug, Educe, Clone, Encode)] #[educe(PartialEq, Eq)] #[ssz(struct_behaviour = "transparent")] pub struct KzgVerifiedCustodyDataColumn { @@ -443,7 +456,7 @@ impl KzgVerifiedCustodyDataColumn { self.data.clone() } pub fn index(&self) -> ColumnIndex { - self.data.index + *self.data.index() } } @@ -478,11 +491,15 @@ where } #[instrument(skip_all, level = "debug")] -pub fn validate_data_column_sidecar_for_gossip( +pub fn validate_data_column_sidecar_for_gossip_fulu( data_column: Arc>, subnet: DataColumnSubnetId, chain: &BeaconChain, ) -> Result, GossipDataColumnError> { + let DataColumnSidecar::Fulu(data_column_fulu) = data_column.as_ref() else { + todo!() + }; + let column_slot = data_column.slot(); verify_data_column_sidecar(&data_column, &chain.spec)?; verify_index_matches_subnet(&data_column, subnet, &chain.spec)?; @@ -506,10 +523,10 @@ pub fn validate_data_column_sidecar_for_gossip( data_column: &DataColumnSidecar, spec: &ChainSpec, ) -> Result<(), GossipDataColumnError> { - if data_column.index >= E::number_of_columns() as u64 { - return Err(GossipDataColumnError::InvalidColumnIndex(data_column.index)); + if *data_column.index() >= E::number_of_columns() as u64 { + return Err(GossipDataColumnError::InvalidColumnIndex( + *data_column.index(), + )); } - if data_column.kzg_commitments.is_empty() { + if data_column.kzg_commitments().is_empty() { return Err(GossipDataColumnError::UnexpectedDataColumn); } - let cells_len = data_column.column.len(); - let commitments_len = data_column.kzg_commitments.len(); - let proofs_len = data_column.kzg_proofs.len(); + let cells_len = data_column.column().len(); + let commitments_len = data_column.kzg_commitments().len(); + let proofs_len = data_column.kzg_proofs().len(); let max_blobs_per_block = spec.max_blobs_per_block(data_column.epoch()) as usize; if commitments_len > max_blobs_per_block { @@ -589,16 +608,15 @@ fn verify_is_unknown_sidecar( .map_err(|e| GossipDataColumnError::BeaconChainError(Box::new(e.into())))? { return Err(GossipDataColumnError::PriorKnown { - proposer: column_sidecar.block_proposer_index(), slot: column_sidecar.slot(), - index: column_sidecar.index, + index: *column_sidecar.index(), }); } Ok(()) } fn verify_column_inclusion_proof( - data_column: &DataColumnSidecar, + data_column: &DataColumnSidecarFulu, ) -> Result<(), GossipDataColumnError> { let _timer = metrics::start_timer(&metrics::DATA_COLUMN_SIDECAR_INCLUSION_PROOF_VERIFICATION); if !data_column.verify_inclusion_proof() { @@ -622,7 +640,7 @@ fn verify_slot_higher_than_parent( } fn verify_parent_block_and_finalized_descendant( - data_column: Arc>, + data_column: &DataColumnSidecarFulu, chain: &BeaconChain, ) -> Result { let fork_choice = chain.canonical_head.fork_choice_read_lock(); @@ -646,7 +664,7 @@ fn verify_parent_block_and_finalized_descendant( } fn verify_proposer_and_signature( - data_column: &DataColumnSidecar, + data_column: &DataColumnSidecarFulu, parent_block: &ProtoBlock, chain: &BeaconChain, ) -> Result<(), GossipDataColumnError> { @@ -723,7 +741,7 @@ fn verify_index_matches_subnet( subnet: DataColumnSubnetId, spec: &ChainSpec, ) -> Result<(), GossipDataColumnError> { - let expected_subnet = DataColumnSubnetId::from_column_index(data_column.index, spec); + let expected_subnet = DataColumnSubnetId::from_column_index(*data_column.index(), spec); if expected_subnet != subnet { return Err(GossipDataColumnError::InvalidSubnetId { received: subnet.into(), @@ -790,9 +808,8 @@ pub fn observe_gossip_data_column( .map_err(|e| GossipDataColumnError::BeaconChainError(Box::new(e.into())))? { return Err(GossipDataColumnError::PriorKnown { - proposer: data_column_sidecar.block_proposer_index(), slot: data_column_sidecar.slot(), - index: data_column_sidecar.index, + index: *data_column_sidecar.index(), }); } Ok(()) @@ -801,7 +818,8 @@ pub fn observe_gossip_data_column( #[cfg(test)] mod test { use crate::data_column_verification::{ - GossipDataColumnError, GossipVerifiedDataColumn, validate_data_column_sidecar_for_gossip, + GossipDataColumnError, GossipVerifiedDataColumn, + validate_data_column_sidecar_for_gossip_fulu, }; use crate::observed_data_sidecars::Observe; use crate::test_utils::{ @@ -810,12 +828,15 @@ mod test { use eth2::types::BlobsBundle; use execution_layer::test_utils::generate_blobs; use std::sync::Arc; - use types::{DataColumnSidecar, DataColumnSubnetId, EthSpec, ForkName, MainnetEthSpec}; + use types::{ + DataColumnSidecar, DataColumnSidecarFulu, DataColumnSubnetId, EthSpec, ForkName, + MainnetEthSpec, + }; type E = MainnetEthSpec; #[tokio::test] - async fn test_validate_data_column_sidecar_for_gossip() { + async fn test_validate_data_column_sidecar_for_gossip_fulu() { // Setting up harness is slow, we initialise once and use it for all gossip validation tests. let spec = ForkName::Fulu.make_genesis_spec(E::default_spec()); let harness = BeaconChainHarness::builder(E::default()) @@ -827,19 +848,19 @@ mod test { harness.advance_slot(); let verify_fn = |column_sidecar: DataColumnSidecar| { - let col_index = column_sidecar.index; - validate_data_column_sidecar_for_gossip::<_, Observe>( + let col_index = *column_sidecar.index(); + validate_data_column_sidecar_for_gossip_fulu::<_, Observe>( column_sidecar.into(), DataColumnSubnetId::from_column_index(col_index, &harness.spec), &harness.chain, ) }; - empty_data_column_sidecars_fails_validation(&harness, &verify_fn).await; + empty_data_column_sidecars_fails_validation_fulu(&harness, &verify_fn).await; data_column_sidecar_commitments_exceed_max_blobs_per_block(&harness, &verify_fn).await; } #[tokio::test] - async fn test_new_for_block_publishing() { + async fn test_new_for_block_publishing_fulu() { // Setting up harness is slow, we initialise once and use it for all gossip validation tests. let spec = ForkName::Fulu.make_genesis_spec(E::default_spec()); let harness = BeaconChainHarness::builder(E::default()) @@ -856,11 +877,11 @@ mod test { &harness.chain, ) }; - empty_data_column_sidecars_fails_validation(&harness, &verify_fn).await; + empty_data_column_sidecars_fails_validation_fulu(&harness, &verify_fn).await; data_column_sidecar_commitments_exceed_max_blobs_per_block(&harness, &verify_fn).await; } - async fn empty_data_column_sidecars_fails_validation( + async fn empty_data_column_sidecars_fails_validation_fulu( harness: &BeaconChainHarness>, verify_fn: &impl Fn(DataColumnSidecar) -> Result, ) { @@ -873,7 +894,7 @@ mod test { .await; let index = 0; - let column_sidecar = DataColumnSidecar:: { + let column_sidecar: DataColumnSidecar = DataColumnSidecar::Fulu(DataColumnSidecarFulu { index, column: vec![].try_into().unwrap(), kzg_commitments: vec![].try_into().unwrap(), @@ -884,7 +905,7 @@ mod test { .body() .kzg_commitments_merkle_proof() .unwrap(), - }; + }); let result = verify_fn(column_sidecar); assert!(matches!( diff --git a/beacon_node/beacon_chain/src/fetch_blobs/fetch_blobs_beacon_adapter.rs b/beacon_node/beacon_chain/src/fetch_blobs/fetch_blobs_beacon_adapter.rs index 9526921da73..2ea44cac94c 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs/fetch_blobs_beacon_adapter.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs/fetch_blobs_beacon_adapter.rs @@ -1,7 +1,6 @@ use crate::fetch_blobs::{EngineGetBlobsOutput, FetchEngineBlobError}; -use crate::observed_block_producers::ProposalKey; use crate::{AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes}; -use execution_layer::json_structures::{BlobAndProofV1, BlobAndProofV2}; +use execution_layer::json_structures::BlobAndProofV2; use kzg::Kzg; #[cfg(test)] use mockall::automock; @@ -35,22 +34,6 @@ impl FetchBlobsBeaconAdapter { &self.chain.task_executor } - pub(crate) async fn get_blobs_v1( - &self, - versioned_hashes: Vec, - ) -> Result>>, FetchEngineBlobError> { - let execution_layer = self - .chain - .execution_layer - .as_ref() - .ok_or(FetchEngineBlobError::ExecutionLayerMissing)?; - - execution_layer - .get_blobs_v1(versioned_hashes) - .await - .map_err(FetchEngineBlobError::RequestFailed) - } - pub(crate) async fn get_blobs_v2( &self, versioned_hashes: Vec, @@ -67,36 +50,14 @@ impl FetchBlobsBeaconAdapter { .map_err(FetchEngineBlobError::RequestFailed) } - pub(crate) fn blobs_known_for_proposal( - &self, - proposer: u64, - slot: Slot, - ) -> Option> { - let proposer_key = ProposalKey::new(proposer, slot); - self.chain - .observed_blob_sidecars - .read() - .known_for_proposal(&proposer_key) - .cloned() - } - - pub(crate) fn data_column_known_for_proposal( - &self, - proposal_key: ProposalKey, - ) -> Option> { + pub(crate) fn data_column_known_for_slot(&self, slot: Slot) -> Option> { self.chain .observed_column_sidecars .read() - .known_for_proposal(&proposal_key) + .known_for_slot(&slot) .cloned() } - pub(crate) fn cached_blob_indexes(&self, block_root: &Hash256) -> Option> { - self.chain - .data_availability_checker - .cached_blob_indexes(block_root) - } - pub(crate) fn cached_data_column_indexes(&self, block_root: &Hash256) -> Option> { self.chain .data_availability_checker diff --git a/beacon_node/beacon_chain/src/fetch_blobs/mod.rs b/beacon_node/beacon_chain/src/fetch_blobs/mod.rs index 6559f24d23d..778687b6da9 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs/mod.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs/mod.rs @@ -12,39 +12,36 @@ mod fetch_blobs_beacon_adapter; #[cfg(test)] mod tests; -use crate::blob_verification::{GossipBlobError, KzgVerifiedBlob}; +use crate::blob_verification::GossipBlobError; use crate::block_verification_types::AsBlock; use crate::data_column_verification::{KzgVerifiedCustodyDataColumn, KzgVerifiedDataColumn}; #[cfg_attr(test, double)] use crate::fetch_blobs::fetch_blobs_beacon_adapter::FetchBlobsBeaconAdapter; use crate::kzg_utils::blobs_to_data_column_sidecars; -use crate::observed_block_producers::ProposalKey; -use crate::validator_monitor::timestamp_now; use crate::{ AvailabilityProcessingStatus, BeaconChain, BeaconChainError, BeaconChainTypes, BlockError, metrics, }; use execution_layer::Error as ExecutionLayerError; -use execution_layer::json_structures::{BlobAndProofV1, BlobAndProofV2}; +use execution_layer::json_structures::BlobAndProofV2; use metrics::{TryExt, inc_counter}; #[cfg(test)] use mockall_double::double; -use ssz_types::FixedVector; use state_processing::per_block_processing::deneb::kzg_commitment_to_versioned_hash; use std::sync::Arc; use tracing::{Span, debug, instrument, warn}; use types::data::{BlobSidecarError, DataColumnSidecarError}; use types::{ - BeaconStateError, Blob, BlobSidecar, ColumnIndex, EthSpec, FullPayload, Hash256, KzgProofs, - SignedBeaconBlock, SignedBeaconBlockHeader, VersionedHash, + BeaconStateError, Blob, ColumnIndex, FullPayload, Hash256, KzgProofs, SignedBeaconBlock, + VersionedHash, }; /// Result from engine get blobs to be passed onto `DataAvailabilityChecker` and published to the -/// gossip network. The blobs / data columns have not been marked as observed yet, as they may not +/// gossip network. The data columns have not been marked as observed yet, as they may not /// be published immediately. #[derive(Debug)] +// TODO(gloas) this doesn't need to be an enum pub enum EngineGetBlobsOutput { - Blobs(Vec>), /// A filtered list of custody data columns to be imported into the `DataAvailabilityChecker`. CustodyColumns(Vec>), } @@ -129,108 +126,10 @@ async fn fetch_and_process_engine_blobs_inner( ) .await } else { - fetch_and_process_blobs_v1( - chain_adapter, - block_root, - block, - versioned_hashes, - publish_fn, - ) - .await - } -} - -#[instrument(skip_all, level = "debug")] -async fn fetch_and_process_blobs_v1( - chain_adapter: FetchBlobsBeaconAdapter, - block_root: Hash256, - block: Arc>, - versioned_hashes: Vec, - publish_fn: impl Fn(EngineGetBlobsOutput) + Send + Sized, -) -> Result, FetchEngineBlobError> { - let num_expected_blobs = versioned_hashes.len(); - metrics::observe(&metrics::BLOBS_FROM_EL_EXPECTED, num_expected_blobs as f64); - debug!(num_expected_blobs, "Fetching blobs from the EL"); - let response = chain_adapter - .get_blobs_v1(versioned_hashes) - .await - .inspect_err(|_| { - inc_counter(&metrics::BLOBS_FROM_EL_ERROR_TOTAL); - })?; - - let num_fetched_blobs = response.iter().filter(|opt| opt.is_some()).count(); - metrics::observe(&metrics::BLOBS_FROM_EL_RECEIVED, num_fetched_blobs as f64); - - if num_fetched_blobs == 0 { - debug!(num_expected_blobs, "No blobs fetched from the EL"); - inc_counter(&metrics::BLOBS_FROM_EL_MISS_TOTAL); - return Ok(None); - } else { - debug!( - num_expected_blobs, - num_fetched_blobs, "Received blobs from the EL" - ); - inc_counter(&metrics::BLOBS_FROM_EL_HIT_TOTAL); - } - - if chain_adapter.fork_choice_contains_block(&block_root) { - // Avoid computing sidecars if the block has already been imported. - debug!( - info = "block has already been imported", - "Ignoring EL blobs response" - ); - return Ok(None); + Err(FetchEngineBlobError::InternalError( + "fetch blobs v1 no longer supported".to_owned(), + )) } - - let (signed_block_header, kzg_commitments_proof) = block - .signed_block_header_and_kzg_commitments_proof() - .map_err(FetchEngineBlobError::BeaconStateError)?; - - let mut blob_sidecar_list = build_blob_sidecars( - &block, - response, - signed_block_header, - &kzg_commitments_proof, - )?; - - if let Some(observed_blobs) = - chain_adapter.blobs_known_for_proposal(block.message().proposer_index(), block.slot()) - { - blob_sidecar_list.retain(|blob| !observed_blobs.contains(&blob.blob_index())); - if blob_sidecar_list.is_empty() { - debug!( - info = "blobs have already been seen on gossip", - "Ignoring EL blobs response" - ); - return Ok(None); - } - } - - if let Some(known_blobs) = chain_adapter.cached_blob_indexes(&block_root) { - blob_sidecar_list.retain(|blob| !known_blobs.contains(&blob.blob_index())); - if blob_sidecar_list.is_empty() { - debug!( - info = "blobs have already been imported into data availability checker", - "Ignoring EL blobs response" - ); - return Ok(None); - } - } - - // Up until this point we have not observed the blobs in the gossip cache, which allows them to - // arrive independently while this function is running. In `publish_fn` we will observe them - // and then publish any blobs that had not already been observed. - publish_fn(EngineGetBlobsOutput::Blobs(blob_sidecar_list.clone())); - - let availability_processing_status = chain_adapter - .process_engine_blobs( - block.slot(), - block_root, - EngineGetBlobsOutput::Blobs(blob_sidecar_list), - ) - .await?; - - Ok(Some(availability_processing_status)) } #[instrument(skip_all, level = "debug")] @@ -380,7 +279,7 @@ async fn compute_custody_columns_to_import( .map(|data_columns| { data_columns .into_iter() - .filter(|col| custody_columns_indices.contains(&col.index)) + .filter(|col| custody_columns_indices.contains(col.index())) .map(|col| { KzgVerifiedCustodyDataColumn::from_asserted_custody( KzgVerifiedDataColumn::from_execution_verified(col), @@ -391,9 +290,9 @@ async fn compute_custody_columns_to_import( .map_err(FetchEngineBlobError::DataColumnSidecarError)?; // Only consider columns that are not already observed on gossip. - if let Some(observed_columns) = chain_adapter_cloned.data_column_known_for_proposal( - ProposalKey::new(block.message().proposer_index(), block.slot()), - ) { + if let Some(observed_columns) = + chain_adapter_cloned.data_column_known_for_slot(block.slot()) + { custody_columns.retain(|col| !observed_columns.contains(&col.index())); if custody_columns.is_empty() { return Ok(vec![]); @@ -418,34 +317,3 @@ async fn compute_custody_columns_to_import( .await .map_err(FetchEngineBlobError::TokioJoin)? } - -fn build_blob_sidecars( - block: &Arc>>, - response: Vec>>, - signed_block_header: SignedBeaconBlockHeader, - kzg_commitments_inclusion_proof: &FixedVector, -) -> Result>, FetchEngineBlobError> { - let mut sidecars = vec![]; - for (index, blob_and_proof) in response - .into_iter() - .enumerate() - .filter_map(|(index, opt_blob)| Some((index, opt_blob?))) - { - let blob_sidecar = BlobSidecar::new_with_existing_proof( - index, - blob_and_proof.blob, - block, - signed_block_header.clone(), - kzg_commitments_inclusion_proof, - blob_and_proof.proof, - ) - .map_err(FetchEngineBlobError::BlobSidecarError)?; - - sidecars.push(KzgVerifiedBlob::from_execution_verified( - Arc::new(blob_sidecar), - timestamp_now(), - )); - } - - Ok(sidecars) -} diff --git a/beacon_node/beacon_chain/src/fetch_blobs/tests.rs b/beacon_node/beacon_chain/src/fetch_blobs/tests.rs index cbe2f78fbda..1c16779a64b 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs/tests.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs/tests.rs @@ -156,7 +156,7 @@ mod get_blobs_v2 { mock_fork_choice_contains_block(&mut mock_adapter, vec![]); // All data columns already seen on gossip mock_adapter - .expect_data_column_known_for_proposal() + .expect_data_column_known_for_slot() .returning(|_| Some(hashset![0, 1, 2])); // No blobs should be processed mock_adapter.expect_process_engine_blobs().times(0); @@ -193,7 +193,7 @@ mod get_blobs_v2 { mock_get_blobs_v2_response(&mut mock_adapter, Some(blobs_and_proofs)); mock_fork_choice_contains_block(&mut mock_adapter, vec![]); mock_adapter - .expect_data_column_known_for_proposal() + .expect_data_column_known_for_slot() .returning(|_| None); mock_adapter .expect_cached_data_column_indexes() @@ -249,278 +249,6 @@ mod get_blobs_v2 { } } -mod get_blobs_v1 { - use super::*; - use crate::block_verification_types::AsBlock; - use std::collections::HashSet; - use types::ColumnIndex; - - const ELECTRA_FORK: ForkName = ForkName::Electra; - - #[tokio::test(flavor = "multi_thread", worker_threads = 2)] - async fn test_fetch_blobs_v1_no_blobs_in_block() { - let mut mock_adapter = mock_beacon_adapter(ELECTRA_FORK); - let spec = mock_adapter.spec(); - let (publish_fn, _s) = mock_publish_fn(); - let block_no_blobs = - SignedBeaconBlock::from_block(BeaconBlock::empty(spec), Signature::empty()); - let block_root = block_no_blobs.canonical_root(); - - // Expectations: engine fetch blobs should not be triggered - mock_adapter.expect_get_blobs_v1().times(0); - - // WHEN: Trigger fetch blobs on the block - let custody_columns: [ColumnIndex; 3] = [0, 1, 2]; - let processing_status = fetch_and_process_engine_blobs_inner( - mock_adapter, - block_root, - Arc::new(block_no_blobs), - &custody_columns, - publish_fn, - ) - .await - .expect("fetch blobs should succeed"); - - // THEN: No blob is processed - assert_eq!(processing_status, None); - } - - #[tokio::test(flavor = "multi_thread", worker_threads = 2)] - async fn test_fetch_blobs_v1_no_blobs_returned() { - let mut mock_adapter = mock_beacon_adapter(ELECTRA_FORK); - let (publish_fn, _) = mock_publish_fn(); - let (block, _blobs_and_proofs) = create_test_block_and_blobs(&mock_adapter, 2); - let block_root = block.canonical_root(); - - // GIVEN: No blobs in EL response - let expected_blob_count = block.message().body().blob_kzg_commitments().unwrap().len(); - mock_get_blobs_v1_response(&mut mock_adapter, vec![None; expected_blob_count]); - - // WHEN: Trigger fetch blobs on the block - let custody_columns: [ColumnIndex; 3] = [0, 1, 2]; - let processing_status = fetch_and_process_engine_blobs_inner( - mock_adapter, - block_root, - block, - &custody_columns, - publish_fn, - ) - .await - .expect("fetch blobs should succeed"); - - // THEN: No blob is processed - assert_eq!(processing_status, None); - } - - #[tokio::test(flavor = "multi_thread", worker_threads = 2)] - async fn test_fetch_blobs_v1_partial_blobs_returned() { - let mut mock_adapter = mock_beacon_adapter(ELECTRA_FORK); - let (publish_fn, publish_fn_args) = mock_publish_fn(); - let blob_count = 2; - let (block, blobs_and_proofs) = create_test_block_and_blobs(&mock_adapter, blob_count); - let block_slot = block.slot(); - let block_root = block.canonical_root(); - - // GIVEN: Missing a blob in EL response (remove 1 blob from response) - let mut blob_and_proof_opts = blobs_and_proofs.into_iter().map(Some).collect::>(); - blob_and_proof_opts.first_mut().unwrap().take(); - mock_get_blobs_v1_response(&mut mock_adapter, blob_and_proof_opts); - // AND block is not imported into fork choice - mock_fork_choice_contains_block(&mut mock_adapter, vec![]); - // AND all blobs have not yet been seen - mock_adapter - .expect_cached_blob_indexes() - .returning(|_| None); - mock_adapter - .expect_blobs_known_for_proposal() - .returning(|_, _| None); - // Returned blobs should be processed - mock_process_engine_blobs_result( - &mut mock_adapter, - Ok(AvailabilityProcessingStatus::MissingComponents( - block_slot, block_root, - )), - ); - - // WHEN: Trigger fetch blobs on the block - let custody_columns: [ColumnIndex; 3] = [0, 1, 2]; - let processing_status = fetch_and_process_engine_blobs_inner( - mock_adapter, - block_root, - block, - &custody_columns, - publish_fn, - ) - .await - .expect("fetch blobs should succeed"); - - // THEN: Returned blobs are processed and published - assert_eq!( - processing_status, - Some(AvailabilityProcessingStatus::MissingComponents( - block_slot, block_root, - )) - ); - assert!( - matches!( - extract_published_blobs(publish_fn_args), - EngineGetBlobsOutput::Blobs(blobs) if blobs.len() == blob_count - 1 - ), - "partial blob results should still be published" - ); - } - - #[tokio::test(flavor = "multi_thread", worker_threads = 2)] - async fn test_fetch_blobs_v1_block_imported_after_el_response() { - let mut mock_adapter = mock_beacon_adapter(ELECTRA_FORK); - let (publish_fn, publish_fn_args) = mock_publish_fn(); - let (block, blobs_and_proofs) = create_test_block_and_blobs(&mock_adapter, 2); - let block_root = block.canonical_root(); - - // GIVEN: All blobs returned, but fork choice already imported the block - let blob_and_proof_opts = blobs_and_proofs.into_iter().map(Some).collect::>(); - mock_get_blobs_v1_response(&mut mock_adapter, blob_and_proof_opts); - mock_fork_choice_contains_block(&mut mock_adapter, vec![block.canonical_root()]); - - // WHEN: Trigger fetch blobs on the block - let custody_columns: [ColumnIndex; 3] = [0, 1, 2]; - let processing_status = fetch_and_process_engine_blobs_inner( - mock_adapter, - block_root, - block, - &custody_columns, - publish_fn, - ) - .await - .expect("fetch blobs should succeed"); - - // THEN: Returned blobs should NOT be processed or published. - assert_eq!(processing_status, None); - assert_eq!( - publish_fn_args.lock().unwrap().len(), - 0, - "no blobs should be published" - ); - } - - #[tokio::test(flavor = "multi_thread", worker_threads = 2)] - async fn test_fetch_blobs_v1_no_new_blobs_to_import() { - let mut mock_adapter = mock_beacon_adapter(ELECTRA_FORK); - let (publish_fn, publish_fn_args) = mock_publish_fn(); - let (block, blobs_and_proofs) = create_test_block_and_blobs(&mock_adapter, 2); - let block_root = block.canonical_root(); - - // **GIVEN**: - // All blobs returned - let blob_and_proof_opts = blobs_and_proofs.into_iter().map(Some).collect::>(); - let all_blob_indices = blob_and_proof_opts - .iter() - .enumerate() - .map(|(i, _)| i as u64) - .collect::>(); - - mock_get_blobs_v1_response(&mut mock_adapter, blob_and_proof_opts); - // block not yet imported into fork choice - mock_fork_choice_contains_block(&mut mock_adapter, vec![]); - // All blobs already seen on gossip - mock_adapter - .expect_cached_blob_indexes() - .returning(|_| None); - mock_adapter - .expect_blobs_known_for_proposal() - .returning(move |_, _| Some(all_blob_indices.clone())); - - // **WHEN**: Trigger `fetch_blobs` on the block - let custody_columns: [ColumnIndex; 3] = [0, 1, 2]; - let processing_status = fetch_and_process_engine_blobs_inner( - mock_adapter, - block_root, - block, - &custody_columns, - publish_fn, - ) - .await - .expect("fetch blobs should succeed"); - - // **THEN**: Should NOT be processed and no blobs should be published. - assert_eq!(processing_status, None); - assert_eq!( - publish_fn_args.lock().unwrap().len(), - 0, - "no blobs should be published" - ); - } - - #[tokio::test(flavor = "multi_thread", worker_threads = 2)] - async fn test_fetch_blobs_v1_success() { - let mut mock_adapter = mock_beacon_adapter(ELECTRA_FORK); - let (publish_fn, publish_fn_args) = mock_publish_fn(); - let blob_count = 2; - let (block, blobs_and_proofs) = create_test_block_and_blobs(&mock_adapter, blob_count); - let block_root = block.canonical_root(); - - // All blobs returned, fork choice doesn't contain block - let blob_and_proof_opts = blobs_and_proofs.into_iter().map(Some).collect::>(); - mock_get_blobs_v1_response(&mut mock_adapter, blob_and_proof_opts); - mock_fork_choice_contains_block(&mut mock_adapter, vec![]); - mock_adapter - .expect_cached_blob_indexes() - .returning(|_| None); - mock_adapter - .expect_blobs_known_for_proposal() - .returning(|_, _| None); - mock_process_engine_blobs_result( - &mut mock_adapter, - Ok(AvailabilityProcessingStatus::Imported(block_root)), - ); - - // Trigger fetch blobs on the block - let custody_columns: [ColumnIndex; 3] = [0, 1, 2]; - let processing_status = fetch_and_process_engine_blobs_inner( - mock_adapter, - block_root, - block, - &custody_columns, - publish_fn, - ) - .await - .expect("fetch blobs should succeed"); - - // THEN all fetched blobs are processed and published - assert_eq!( - processing_status, - Some(AvailabilityProcessingStatus::Imported(block_root)) - ); - - let published_blobs = extract_published_blobs(publish_fn_args); - assert!( - matches!( - published_blobs, - EngineGetBlobsOutput::Blobs(blobs) if blobs.len() == blob_count - ), - "should publish fetched blobs" - ); - } - - fn mock_get_blobs_v1_response( - mock_adapter: &mut MockFetchBlobsBeaconAdapter, - blobs_and_proofs_opt: Vec>>, - ) { - let blobs_and_proofs_v1 = blobs_and_proofs_opt - .into_iter() - .map(|blob_and_proof_opt| { - blob_and_proof_opt.map(|blob_and_proof| match blob_and_proof { - BlobAndProof::V1(inner) => inner, - _ => panic!("BlobAndProofV1 not expected"), - }) - }) - .collect(); - mock_adapter - .expect_get_blobs_v1() - .return_once(move |_| Ok(blobs_and_proofs_v1)); - } -} - /// Extract the `EngineGetBlobsOutput` passed to the `publish_fn`. fn extract_published_blobs( publish_fn_args: Arc>>>, diff --git a/beacon_node/beacon_chain/src/historical_data_columns.rs b/beacon_node/beacon_chain/src/historical_data_columns.rs index 6cf947adcb1..d6977d99852 100644 --- a/beacon_node/beacon_chain/src/historical_data_columns.rs +++ b/beacon_node/beacon_chain/src/historical_data_columns.rs @@ -61,12 +61,12 @@ impl BeaconChain { let unique_column_indices = historical_data_column_sidecar_list .iter() - .map(|item| item.index) + .map(|item| *item.index()) .collect::>(); let mut slot_and_column_index_to_data_columns = historical_data_column_sidecar_list .iter() - .map(|data_column| ((data_column.slot(), data_column.index), data_column)) + .map(|data_column| ((data_column.slot(), *data_column.index()), data_column)) .collect::>(); let forward_blocks_iter = self @@ -80,13 +80,14 @@ impl BeaconChain { let (block_root, slot) = block_iter_result .map_err(|e| HistoricalDataColumnError::BeaconChainError(Box::new(e)))?; + let fork_name = self.spec.fork_name_at_slot::(slot); for column_index in unique_column_indices.clone() { if let Some(data_column) = slot_and_column_index_to_data_columns.remove(&(slot, column_index)) { if self .store - .get_data_column(&block_root, &data_column.index)? + .get_data_column(&block_root, data_column.index(), fork_name)? .is_some() { continue; diff --git a/beacon_node/beacon_chain/src/kzg_utils.rs b/beacon_node/beacon_chain/src/kzg_utils.rs index a1c255e3b3c..21242431720 100644 --- a/beacon_node/beacon_chain/src/kzg_utils.rs +++ b/beacon_node/beacon_chain/src/kzg_utils.rs @@ -6,12 +6,13 @@ use rayon::prelude::*; use ssz_types::{FixedVector, VariableList}; use std::sync::Arc; use tracing::instrument; +use tree_hash::TreeHash; use types::data::{Cell, DataColumn, DataColumnSidecarError}; use types::kzg_ext::KzgCommitments; use types::{ - Blob, BlobSidecar, BlobSidecarList, ChainSpec, DataColumnSidecar, DataColumnSidecarList, - EthSpec, Hash256, KzgCommitment, KzgProof, SignedBeaconBlock, SignedBeaconBlockHeader, - SignedBlindedBeaconBlock, + Blob, BlobSidecar, BlobSidecarList, ChainSpec, DataColumnSidecar, DataColumnSidecarFulu, + DataColumnSidecarGloas, DataColumnSidecarList, EthSpec, Hash256, KzgCommitment, KzgProof, + SignedBeaconBlock, SignedBeaconBlockHeader, SignedBlindedBeaconBlock, }; /// Converts a blob ssz List object to an array to be used with the kzg @@ -59,22 +60,22 @@ where let mut commitments = Vec::new(); for data_column in data_column_iter { - let col_index = data_column.index; + let col_index = *data_column.index(); - if data_column.column.is_empty() { + if data_column.column().is_empty() { return Err((Some(col_index), KzgError::KzgVerificationFailed)); } - for cell in &data_column.column { + for cell in data_column.column() { cells.push(ssz_cell_to_crypto_cell::(cell).map_err(|e| (Some(col_index), e))?); column_indices.push(col_index); } - for &proof in &data_column.kzg_proofs { + for &proof in data_column.kzg_proofs() { proofs.push(Bytes48::from(proof)); } - for &commitment in &data_column.kzg_commitments { + for &commitment in data_column.kzg_commitments() { commitments.push(Bytes48::from(commitment)); } @@ -171,7 +172,6 @@ pub fn blobs_to_data_column_sidecars( .body() .blob_kzg_commitments() .map_err(|_err| DataColumnSidecarError::PreDeneb)?; - let kzg_commitments_inclusion_proof = block.message().body().kzg_commitments_merkle_proof()?; let signed_block_header = block.signed_block_header(); if cell_proofs.len() != blobs.len() * E::number_of_columns() { @@ -207,14 +207,26 @@ pub fn blobs_to_data_column_sidecars( }) .collect::, KzgError>>()?; - build_data_column_sidecars( - kzg_commitments.clone(), - kzg_commitments_inclusion_proof, - signed_block_header, - blob_cells_and_proofs_vec, - spec, - ) - .map_err(DataColumnSidecarError::BuildSidecarFailed) + if block.fork_name_unchecked().gloas_enabled() { + build_data_column_sidecars_gloas( + kzg_commitments.clone(), + signed_block_header, + blob_cells_and_proofs_vec, + spec, + ) + .map_err(DataColumnSidecarError::BuildSidecarFailed) + } else { + let kzg_commitments_inclusion_proof = + block.message().body().kzg_commitments_merkle_proof()?; + build_data_column_sidecars_fulu( + kzg_commitments.clone(), + kzg_commitments_inclusion_proof, + signed_block_header, + blob_cells_and_proofs_vec, + spec, + ) + .map_err(DataColumnSidecarError::BuildSidecarFailed) + } } pub fn compute_cells(blobs: &[&Blob], kzg: &Kzg) -> Result, KzgError> { @@ -235,13 +247,20 @@ pub fn compute_cells(blobs: &[&Blob], kzg: &Kzg) -> Result( +pub(crate) fn build_data_column_sidecars_fulu( kzg_commitments: KzgCommitments, kzg_commitments_inclusion_proof: FixedVector, signed_block_header: SignedBeaconBlockHeader, blob_cells_and_proofs_vec: Vec, spec: &ChainSpec, ) -> Result, String> { + if spec + .fork_name_at_slot::(signed_block_header.message.slot) + .gloas_enabled() + { + return Err("Attempting to construct Fulu data columns post-Gloas".to_owned()); + } + let number_of_columns = E::number_of_columns(); let max_blobs_per_block = spec .max_blobs_per_block(signed_block_header.message.slot.epoch(E::slots_per_epoch())) @@ -283,7 +302,7 @@ pub(crate) fn build_data_column_sidecars( .enumerate() .map( |(index, (col, proofs))| -> Result>, String> { - Ok(Arc::new(DataColumnSidecar { + Ok(Arc::new(DataColumnSidecar::Fulu(DataColumnSidecarFulu { index: index as u64, column: DataColumn::::try_from(col) .map_err(|e| format!("MaxBlobCommitmentsPerBlock exceeded: {e:?}"))?, @@ -292,7 +311,81 @@ pub(crate) fn build_data_column_sidecars( .map_err(|e| format!("MaxBlobCommitmentsPerBlock exceeded: {e:?}"))?, signed_block_header: signed_block_header.clone(), kzg_commitments_inclusion_proof: kzg_commitments_inclusion_proof.clone(), - })) + }))) + }, + ) + .collect(); + + sidecars +} + +pub(crate) fn build_data_column_sidecars_gloas( + kzg_commitments: KzgCommitments, + signed_block_header: SignedBeaconBlockHeader, + blob_cells_and_proofs_vec: Vec, + spec: &ChainSpec, +) -> Result, String> { + if spec + .fork_name_at_slot::(signed_block_header.message.slot) + .gloas_enabled() + { + return Err("Attempting to construct Gloas data columns pre-Gloas".to_owned()); + } + + let number_of_columns = E::number_of_columns(); + let max_blobs_per_block = spec + .max_blobs_per_block(signed_block_header.message.slot.epoch(E::slots_per_epoch())) + as usize; + let mut columns = vec![Vec::with_capacity(max_blobs_per_block); number_of_columns]; + let mut column_kzg_proofs = vec![Vec::with_capacity(max_blobs_per_block); number_of_columns]; + + for (blob_cells, blob_cell_proofs) in blob_cells_and_proofs_vec { + // we iterate over each column, and we construct the column from "top to bottom", + // pushing on the cell and the corresponding proof at each column index. we do this for + // each blob (i.e. the outer loop). + for col in 0..number_of_columns { + let cell = blob_cells + .get(col) + .ok_or(format!("Missing blob cell at index {col}"))?; + let cell: Vec = cell.to_vec(); + let cell = + Cell::::try_from(cell).map_err(|e| format!("BytesPerCell exceeded: {e:?}"))?; + + let proof = blob_cell_proofs + .get(col) + .ok_or(format!("Missing blob cell KZG proof at index {col}"))?; + + let column = columns + .get_mut(col) + .ok_or(format!("Missing data column at index {col}"))?; + let column_proofs = column_kzg_proofs + .get_mut(col) + .ok_or(format!("Missing data column proofs at index {col}"))?; + + column.push(cell); + column_proofs.push(*proof); + } + } + + let beacon_block_root = signed_block_header.message.tree_hash_root(); + let slot = signed_block_header.message.slot; + + let sidecars: Result>>, String> = columns + .into_iter() + .zip(column_kzg_proofs) + .enumerate() + .map( + |(index, (col, proofs))| -> Result>, String> { + Ok(Arc::new(DataColumnSidecar::Gloas(DataColumnSidecarGloas { + index: index as u64, + column: DataColumn::::try_from(col) + .map_err(|e| format!("MaxBlobCommitmentsPerBlock exceeded: {e:?}"))?, + kzg_commitments: kzg_commitments.clone(), + kzg_proofs: VariableList::try_from(proofs) + .map_err(|e| format!("MaxBlobCommitmentsPerBlock exceeded: {e:?}"))?, + beacon_block_root, + slot, + }))) }, ) .collect(); @@ -314,7 +407,7 @@ pub fn reconstruct_blobs( spec: &ChainSpec, ) -> Result, String> { // Sort data columns by index to ensure ascending order for KZG operations - data_columns.sort_unstable_by_key(|dc| dc.index); + data_columns.sort_unstable_by_key(|dc| *dc.index()); let first_data_column = data_columns .first() @@ -323,7 +416,7 @@ pub fn reconstruct_blobs( let blob_indices: Vec = match blob_indices_opt { Some(indices) => indices.into_iter().map(|i| i as usize).collect(), None => { - let num_of_blobs = first_data_column.kzg_commitments.len(); + let num_of_blobs = first_data_column.kzg_commitments().len(); (0..num_of_blobs).collect() } }; @@ -335,7 +428,7 @@ pub fn reconstruct_blobs( let mut cell_ids: Vec = vec![]; for data_column in &data_columns { let cell = data_column - .column + .column() .get(row_index) .ok_or(format!("Missing data column at row index {row_index}")) .and_then(|cell| { @@ -343,7 +436,7 @@ pub fn reconstruct_blobs( })?; cells.push(cell); - cell_ids.push(data_column.index); + cell_ids.push(*data_column.index()); } let num_cells_original_blob = E::number_of_columns() / 2; @@ -374,8 +467,13 @@ pub fn reconstruct_blobs( row_index, blob, signed_block, - first_data_column.signed_block_header.clone(), - &first_data_column.kzg_commitments_inclusion_proof, + first_data_column + .signed_block_header() + .map_err(|e| format!("{e:?}"))? + .clone(), + first_data_column + .kzg_commitments_inclusion_proof() + .map_err(|e| format!("{e:?}"))?, kzg_proof, ) .map(Arc::new) @@ -395,7 +493,7 @@ pub fn reconstruct_data_columns( spec: &ChainSpec, ) -> Result, KzgError> { // Sort data columns by index to ensure ascending order for KZG operations - data_columns.sort_unstable_by_key(|dc| dc.index); + data_columns.sort_unstable_by_key(|dc| *dc.index()); let first_data_column = data_columns .first() @@ -403,37 +501,41 @@ pub fn reconstruct_data_columns( "data_columns should have at least one element".to_string(), ))?; - let num_of_blobs = first_data_column.kzg_commitments.len(); - - let blob_cells_and_proofs_vec = - (0..num_of_blobs) - .into_par_iter() - .map(|row_index| { - let mut cells: Vec = vec![]; - let mut cell_ids: Vec = vec![]; - for data_column in &data_columns { - let cell = data_column.column.get(row_index).ok_or( - KzgError::InconsistentArrayLength(format!( - "Missing data column at row index {row_index}" - )), - )?; - - cells.push(ssz_cell_to_crypto_cell::(cell)?); - cell_ids.push(data_column.index); - } - kzg.recover_cells_and_compute_kzg_proofs(&cell_ids, &cells) - }) - .collect::, KzgError>>()?; - - // Clone sidecar elements from existing data column, no need to re-compute - build_data_column_sidecars( - first_data_column.kzg_commitments.clone(), - first_data_column.kzg_commitments_inclusion_proof.clone(), - first_data_column.signed_block_header.clone(), - blob_cells_and_proofs_vec, - spec, - ) - .map_err(KzgError::ReconstructFailed) + let num_of_blobs = first_data_column.kzg_commitments().len(); + + let blob_cells_and_proofs_vec = (0..num_of_blobs) + .into_par_iter() + .map(|row_index| { + let mut cells: Vec = vec![]; + let mut cell_ids: Vec = vec![]; + for data_column in &data_columns { + let cell = data_column.column().get(row_index).ok_or( + KzgError::InconsistentArrayLength(format!( + "Missing data column at row index {row_index}" + )), + )?; + + cells.push(ssz_cell_to_crypto_cell::(cell)?); + cell_ids.push(*data_column.index()); + } + kzg.recover_cells_and_compute_kzg_proofs(&cell_ids, &cells) + }) + .collect::, KzgError>>()?; + match first_data_column.as_ref() { + DataColumnSidecar::Fulu(first_column) => { + // Clone sidecar elements from existing data column, no need to re-compute + build_data_column_sidecars_fulu( + first_column.kzg_commitments.clone(), + first_column.kzg_commitments_inclusion_proof.clone(), + first_column.signed_block_header.clone(), + blob_cells_and_proofs_vec, + spec, + ) + .map_err(KzgError::ReconstructFailed) + } + // TODO(gloas) resolve TODO + DataColumnSidecar::Gloas(_) => todo!(), + } } #[cfg(test)] @@ -460,7 +562,7 @@ mod test { let spec = ForkName::Fulu.make_genesis_spec(E::default_spec()); let kzg = get_kzg(); test_build_data_columns_empty(&kzg, &spec); - test_build_data_columns(&kzg, &spec); + test_build_data_columns_fulu(&kzg, &spec); test_reconstruct_data_columns(&kzg, &spec); test_reconstruct_data_columns_unordered(&kzg, &spec); test_reconstruct_blobs_from_data_columns(&kzg, &spec); @@ -495,7 +597,7 @@ mod test { } #[track_caller] - fn test_build_data_columns(kzg: &Kzg, spec: &ChainSpec) { + fn test_build_data_columns_fulu(kzg: &Kzg, spec: &ChainSpec) { // Using at least 2 blobs to make sure we're arranging the data columns correctly. let num_of_blobs = 2; let (signed_block, blobs, proofs) = @@ -520,18 +622,21 @@ mod test { assert_eq!(column_sidecars.len(), E::number_of_columns()); for (idx, col_sidecar) in column_sidecars.iter().enumerate() { - assert_eq!(col_sidecar.index, idx as u64); + assert_eq!(*col_sidecar.index(), idx as u64); - assert_eq!(col_sidecar.kzg_commitments.len(), num_of_blobs); - assert_eq!(col_sidecar.column.len(), num_of_blobs); - assert_eq!(col_sidecar.kzg_proofs.len(), num_of_blobs); + assert_eq!(col_sidecar.kzg_commitments().len(), num_of_blobs); + assert_eq!(col_sidecar.column().len(), num_of_blobs); + assert_eq!(col_sidecar.kzg_proofs().len(), num_of_blobs); - assert_eq!(col_sidecar.kzg_commitments, block_kzg_commitments); + assert_eq!(col_sidecar.kzg_commitments().clone(), block_kzg_commitments); assert_eq!( - col_sidecar.kzg_commitments_inclusion_proof, + col_sidecar + .kzg_commitments_inclusion_proof() + .unwrap() + .clone(), block_kzg_commitments_inclusion_proof ); - assert!(col_sidecar.verify_inclusion_proof()); + assert!(col_sidecar.as_fulu().unwrap().verify_inclusion_proof()); } } diff --git a/beacon_node/beacon_chain/src/observed_data_sidecars.rs b/beacon_node/beacon_chain/src/observed_data_sidecars.rs index 46a3678f167..9c38196afb5 100644 --- a/beacon_node/beacon_chain/src/observed_data_sidecars.rs +++ b/beacon_node/beacon_chain/src/observed_data_sidecars.rs @@ -3,11 +3,10 @@ //! Only `BlobSidecar`s that have completed proposer signature verification can be added //! to this cache to reduce DoS risks. -use crate::observed_block_producers::ProposalKey; use std::collections::{HashMap, HashSet}; use std::marker::PhantomData; use std::sync::Arc; -use types::{BlobSidecar, ChainSpec, DataColumnSidecar, EthSpec, Slot}; +use types::{ChainSpec, DataColumnSidecar, EthSpec, Slot}; #[derive(Debug, PartialEq)] pub enum Error { @@ -22,40 +21,17 @@ pub enum Error { pub trait ObservableDataSidecar { fn slot(&self) -> Slot; - fn block_proposer_index(&self) -> u64; fn index(&self) -> u64; fn max_num_of_items(spec: &ChainSpec, slot: Slot) -> usize; } -impl ObservableDataSidecar for BlobSidecar { - fn slot(&self) -> Slot { - self.slot() - } - - fn block_proposer_index(&self) -> u64 { - self.block_proposer_index() - } - - fn index(&self) -> u64 { - self.index - } - - fn max_num_of_items(spec: &ChainSpec, slot: Slot) -> usize { - spec.max_blobs_per_block(slot.epoch(E::slots_per_epoch())) as usize - } -} - impl ObservableDataSidecar for DataColumnSidecar { fn slot(&self) -> Slot { self.slot() } - fn block_proposer_index(&self) -> u64 { - self.block_proposer_index() - } - fn index(&self) -> u64 { - self.index + *self.index() } fn max_num_of_items(_spec: &ChainSpec, _slot: Slot) -> usize { @@ -73,8 +49,8 @@ impl ObservableDataSidecar for DataColumnSidecar { /// like checking the proposer signature. pub struct ObservedDataSidecars { finalized_slot: Slot, - /// Stores all received data indices for a given `(ValidatorIndex, Slot)` tuple. - items: HashMap>, + /// Stores all received data indices for a given `slot`. + items: HashMap>, spec: Arc, _phantom: PhantomData, } @@ -97,15 +73,9 @@ impl ObservedDataSidecars { pub fn observe_sidecar(&mut self, data_sidecar: &T) -> Result { self.sanitize_data_sidecar(data_sidecar)?; - let data_indices = self - .items - .entry(ProposalKey { - slot: data_sidecar.slot(), - proposer: data_sidecar.block_proposer_index(), - }) - .or_insert_with(|| { - HashSet::with_capacity(T::max_num_of_items(&self.spec, data_sidecar.slot())) - }); + let data_indices = self.items.entry(data_sidecar.slot()).or_insert_with(|| { + HashSet::with_capacity(T::max_num_of_items(&self.spec, data_sidecar.slot())) + }); let did_not_exist = data_indices.insert(data_sidecar.index()); Ok(!did_not_exist) @@ -116,16 +86,13 @@ impl ObservedDataSidecars { self.sanitize_data_sidecar(data_sidecar)?; let is_known = self .items - .get(&ProposalKey { - slot: data_sidecar.slot(), - proposer: data_sidecar.block_proposer_index(), - }) + .get(&data_sidecar.slot()) .is_some_and(|indices| indices.contains(&data_sidecar.index())); Ok(is_known) } - pub fn known_for_proposal(&self, proposal_key: &ProposalKey) -> Option<&HashSet> { - self.items.get(proposal_key) + pub fn known_for_slot(&self, slot: &Slot) -> Option<&HashSet> { + self.items.get(slot) } fn sanitize_data_sidecar(&self, data_sidecar: &T) -> Result<(), Error> { @@ -150,7 +117,7 @@ impl ObservedDataSidecars { } self.finalized_slot = finalized_slot; - self.items.retain(|k, _| k.slot > finalized_slot); + self.items.retain(|k, _| *k > finalized_slot); } } @@ -184,167 +151,43 @@ impl ObservationStrategy for DoNotObserve { mod tests { use super::*; use crate::test_utils::test_spec; - use bls::Hash256; + use bls::{Hash256, Signature}; use std::sync::Arc; - use types::{Epoch, MainnetEthSpec}; + use types::{ + BeaconBlockHeader, DataColumnSidecarFulu, Epoch, MainnetEthSpec, SignedBeaconBlockHeader, + }; type E = MainnetEthSpec; - fn get_blob_sidecar(slot: u64, proposer_index: u64, index: u64) -> Arc> { - let mut blob_sidecar = BlobSidecar::empty(); - blob_sidecar.signed_block_header.message.slot = slot.into(); - blob_sidecar.signed_block_header.message.proposer_index = proposer_index; - blob_sidecar.index = index; - Arc::new(blob_sidecar) - } - - #[test] - fn pruning() { - let spec = Arc::new(test_spec::()); - let mut cache = ObservedDataSidecars::>::new(spec); - - assert_eq!(cache.finalized_slot, 0, "finalized slot is zero"); - assert_eq!(cache.items.len(), 0, "no slots should be present"); - - // Slot 0, index 0 - let proposer_index_a = 420; - let sidecar_a = get_blob_sidecar(0, proposer_index_a, 0); - - assert_eq!( - cache.observe_sidecar(&sidecar_a), - Ok(false), - "can observe proposer, indicates proposer unobserved" - ); - - /* - * Preconditions. - */ - - assert_eq!(cache.finalized_slot, 0, "finalized slot is zero"); - assert_eq!( - cache.items.len(), - 1, - "only one (validator_index, slot) tuple should be present" - ); - - let cached_blob_indices = cache - .items - .get(&ProposalKey::new(proposer_index_a, Slot::new(0))) - .expect("slot zero should be present"); - assert_eq!( - cached_blob_indices.len(), - 1, - "only one proposer should be present" - ); - - /* - * Check that a prune at the genesis slot does nothing. - */ - - cache.prune(Slot::new(0)); - - assert_eq!(cache.finalized_slot, 0, "finalized slot is zero"); - assert_eq!(cache.items.len(), 1, "only one slot should be present"); - let cached_blob_indices = cache - .items - .get(&ProposalKey::new(proposer_index_a, Slot::new(0))) - .expect("slot zero should be present"); - assert_eq!( - cached_blob_indices.len(), - 1, - "only one proposer should be present" - ); - - /* - * Check that a prune empties the cache - */ - - cache.prune(E::slots_per_epoch().into()); - assert_eq!( - cache.finalized_slot, - Slot::from(E::slots_per_epoch()), - "finalized slot is updated" - ); - assert_eq!(cache.items.len(), 0, "no items left"); - - /* - * Check that we can't insert a finalized sidecar - */ - - // First slot of finalized epoch - let block_b = get_blob_sidecar(E::slots_per_epoch(), 419, 0); - - assert_eq!( - cache.observe_sidecar(&block_b), - Err(Error::FinalizedDataSidecar { - slot: E::slots_per_epoch().into(), - finalized_slot: E::slots_per_epoch().into(), - }), - "cant insert finalized sidecar" - ); - - assert_eq!(cache.items.len(), 0, "sidecar was not added"); - - /* - * Check that we _can_ insert a non-finalized block - */ - - let three_epochs = E::slots_per_epoch() * 3; - - // First slot of finalized epoch - let proposer_index_b = 421; - let block_b = get_blob_sidecar(three_epochs, proposer_index_b, 0); - - assert_eq!( - cache.observe_sidecar(&block_b), - Ok(false), - "can insert non-finalized block" - ); - - assert_eq!(cache.items.len(), 1, "only one slot should be present"); - let cached_blob_indices = cache - .items - .get(&ProposalKey::new(proposer_index_b, Slot::new(three_epochs))) - .expect("the three epochs slot should be present"); - assert_eq!( - cached_blob_indices.len(), - 1, - "only one proposer should be present" - ); - - /* - * Check that a prune doesnt wipe later blocks - */ + fn get_column_sidecar(slot: u64, proposer_index: u64, index: u64) -> Arc> { + let mut signed_block_header = SignedBeaconBlockHeader { + message: BeaconBlockHeader::empty(), + signature: Signature::empty(), + }; - let two_epochs = E::slots_per_epoch() * 2; - cache.prune(two_epochs.into()); + signed_block_header.message.slot = slot.into(); + signed_block_header.message.proposer_index = proposer_index; - assert_eq!( - cache.finalized_slot, - Slot::from(two_epochs), - "finalized slot is updated" - ); + let column_sidecar = DataColumnSidecar::Fulu(DataColumnSidecarFulu { + index, + column: <_>::default(), + kzg_commitments: <_>::default(), + kzg_proofs: <_>::default(), + signed_block_header, + kzg_commitments_inclusion_proof: <_>::default(), + }); - assert_eq!(cache.items.len(), 1, "only one slot should be present"); - let cached_blob_indices = cache - .items - .get(&ProposalKey::new(proposer_index_b, Slot::new(three_epochs))) - .expect("the three epochs slot should be present"); - assert_eq!( - cached_blob_indices.len(), - 1, - "only one proposer should be present" - ); + Arc::new(column_sidecar) } #[test] fn simple_observations() { let spec = Arc::new(test_spec::()); - let mut cache = ObservedDataSidecars::>::new(spec.clone()); + let mut cache = ObservedDataSidecars::>::new(spec.clone()); // Slot 0, index 0 let proposer_index_a = 420; - let sidecar_a = get_blob_sidecar(0, proposer_index_a, 0); + let sidecar_a = get_column_sidecar(0, proposer_index_a, 0); assert_eq!( cache.proposer_is_known(&sidecar_a), @@ -374,7 +217,7 @@ mod tests { assert_eq!(cache.items.len(), 1, "only one slot should be present"); let cached_blob_indices = cache .items - .get(&ProposalKey::new(proposer_index_a, Slot::new(0))) + .get(&Slot::new(0)) .expect("slot zero should be present"); assert_eq!( cached_blob_indices.len(), @@ -385,7 +228,7 @@ mod tests { // Slot 1, proposer 0 let proposer_index_b = 421; - let sidecar_b = get_blob_sidecar(1, proposer_index_b, 0); + let sidecar_b = get_column_sidecar(1, proposer_index_b, 0); assert_eq!( cache.proposer_is_known(&sidecar_b), @@ -412,7 +255,7 @@ mod tests { assert_eq!(cache.items.len(), 2, "two slots should be present"); let cached_blob_indices = cache .items - .get(&ProposalKey::new(proposer_index_a, Slot::new(0))) + .get(&Slot::new(0)) .expect("slot zero should be present"); assert_eq!( cached_blob_indices.len(), @@ -421,7 +264,7 @@ mod tests { ); let cached_blob_indices = cache .items - .get(&ProposalKey::new(proposer_index_b, Slot::new(1))) + .get(&Slot::new(1)) .expect("slot zero should be present"); assert_eq!( cached_blob_indices.len(), @@ -430,7 +273,7 @@ mod tests { ); // Slot 0, index 1 - let sidecar_c = get_blob_sidecar(0, proposer_index_a, 1); + let sidecar_c = get_column_sidecar(0, proposer_index_a, 1); assert_eq!( cache.proposer_is_known(&sidecar_c), @@ -457,7 +300,7 @@ mod tests { assert_eq!(cache.items.len(), 2, "two slots should be present"); let cached_blob_indices = cache .items - .get(&ProposalKey::new(proposer_index_a, Slot::new(0))) + .get(&Slot::new(0)) .expect("slot zero should be present"); assert_eq!( cached_blob_indices.len(), @@ -466,15 +309,22 @@ mod tests { ); // Create a sidecar sharing slot and proposer but with a different block root. - let mut sidecar_d: BlobSidecar = BlobSidecar { - index: sidecar_c.index, - blob: sidecar_c.blob.clone(), - kzg_commitment: sidecar_c.kzg_commitment, - kzg_proof: sidecar_c.kzg_proof, - signed_block_header: sidecar_c.signed_block_header.clone(), - kzg_commitment_inclusion_proof: sidecar_c.kzg_commitment_inclusion_proof.clone(), - }; - sidecar_d.signed_block_header.message.body_root = Hash256::repeat_byte(7); + let mut sidecar_d: DataColumnSidecar = DataColumnSidecar::Fulu(DataColumnSidecarFulu { + index: *sidecar_c.index(), + column: sidecar_c.column().clone(), + kzg_commitments: sidecar_c.kzg_commitments().clone(), + kzg_proofs: sidecar_c.kzg_proofs().clone(), + signed_block_header: sidecar_c.signed_block_header().unwrap().clone(), + kzg_commitments_inclusion_proof: sidecar_c + .kzg_commitments_inclusion_proof() + .unwrap() + .clone(), + }); + sidecar_d + .signed_block_header_mut() + .unwrap() + .message + .body_root = Hash256::repeat_byte(7); assert_eq!( cache.proposer_is_known(&sidecar_d), Ok(true), @@ -487,7 +337,7 @@ mod tests { ); let cached_blob_indices = cache .items - .get(&ProposalKey::new(proposer_index_a, Slot::new(0))) + .get(&Slot::new(0)) .expect("slot zero should be present"); assert_eq!( cached_blob_indices.len(), @@ -497,7 +347,7 @@ mod tests { // Try adding an out of bounds index let invalid_index = spec.max_blobs_per_block(Epoch::new(0)); - let sidecar_d = get_blob_sidecar(0, proposer_index_a, invalid_index); + let sidecar_d = get_column_sidecar(0, proposer_index_a, invalid_index); assert_eq!( cache.observe_sidecar(&sidecar_d), Err(Error::InvalidDataIndex(invalid_index)), diff --git a/beacon_node/beacon_chain/src/test_utils.rs b/beacon_node/beacon_chain/src/test_utils.rs index b6c235a4cb0..fd52842bcd2 100644 --- a/beacon_node/beacon_chain/src/test_utils.rs +++ b/beacon_node/beacon_chain/src/test_utils.rs @@ -1,9 +1,8 @@ -use crate::blob_verification::GossipVerifiedBlob; use crate::block_verification_types::{AsBlock, RpcBlock}; use crate::custody_context::NodeCustodyType; use crate::data_column_verification::CustodyDataColumn; use crate::graffiti_calculator::GraffitiSettings; -use crate::kzg_utils::build_data_column_sidecars; +use crate::kzg_utils::{build_data_column_sidecars_fulu, build_data_column_sidecars_gloas}; use crate::observed_operations::ObservationOutcome; pub use crate::persisted_beacon_chain::PersistedBeaconChain; use crate::{BeaconBlockResponseWrapper, get_block_root}; @@ -2441,7 +2440,12 @@ where // Blobs are stored as data columns from Fulu (PeerDAS) if self.spec.is_peer_das_enabled_for_epoch(block.epoch()) { - let columns = self.chain.get_data_columns(&block_root).unwrap().unwrap(); + let fork_name = self.spec.fork_name_at_epoch(block.epoch()); + let columns = self + .chain + .get_data_columns(&block_root, fork_name) + .unwrap() + .unwrap(); let custody_columns = columns .into_iter() .map(CustodyDataColumn::from_asserted_custody) @@ -2470,7 +2474,7 @@ where // currently have any knowledge of the columns being custodied. let columns = generate_data_column_sidecars_from_block(&block, &self.spec) .into_iter() - .filter(|d| sampling_columns.contains(&d.index)) + .filter(|d| sampling_columns.contains(d.index())) .map(CustodyDataColumn::from_asserted_custody) .collect::>(); RpcBlock::new_with_custody_columns(Some(block_root), block, columns)? @@ -3189,11 +3193,9 @@ where /// Simulate some of the blobs / data columns being seen on gossip. /// Converts the blobs to data columns if the slot is Fulu or later. - pub async fn process_gossip_blobs_or_columns<'a>( + pub async fn process_gossip_columns( &self, block: &SignedBeaconBlock, - blobs: impl Iterator>, - proofs: impl Iterator, custody_columns_opt: Option>, ) { let is_peerdas_enabled = self.chain.spec.is_peer_das_enabled_for_epoch(block.epoch()); @@ -3209,10 +3211,10 @@ where let verified_columns = generate_data_column_sidecars_from_block(block, &self.spec) .into_iter() - .filter(|c| custody_columns.contains(&c.index)) + .filter(|c| custody_columns.contains(c.index())) .map(|sidecar| { let subnet_id = - DataColumnSubnetId::from_column_index(sidecar.index, &self.spec); + DataColumnSubnetId::from_column_index(*sidecar.index(), &self.spec); self.chain .verify_data_column_sidecar_for_gossip(sidecar, subnet_id) }) @@ -3226,16 +3228,7 @@ where .unwrap(); } } else { - for (i, (kzg_proof, blob)) in proofs.into_iter().zip(blobs).enumerate() { - let sidecar = - Arc::new(BlobSidecar::new(i, blob.clone(), block, *kzg_proof).unwrap()); - let gossip_blob = GossipVerifiedBlob::new(sidecar, i as u64, &self.chain) - .expect("should obtain gossip verified blob"); - self.chain - .process_gossip_blob(gossip_blob) - .await - .expect("should import valid gossip verified blob"); - } + panic!("We no longer support blobs over gossip"); } } } @@ -3363,39 +3356,75 @@ pub fn generate_data_column_sidecars_from_block( .unwrap(); let signed_block_header = block.signed_block_header(); - // load the precomputed column sidecar to avoid computing them for every block in the tests. - let template_data_columns = RuntimeVariableList::>::from_ssz_bytes( - TEST_DATA_COLUMN_SIDECARS_SSZ, - E::number_of_columns(), - ) - .unwrap(); + // Load the precomputed column sidecar to avoid computing them for every block in the tests. + // Then repeat the cells and proofs for every blob + let blob_cells_and_proofs_vec = if block.fork_name_unchecked().gloas_enabled() { + let template_data_columns = + RuntimeVariableList::>::from_ssz_bytes( + TEST_DATA_COLUMN_SIDECARS_SSZ, + E::number_of_columns(), + ) + .unwrap(); - let (cells, proofs) = template_data_columns - .into_iter() - .map(|sidecar| { - let DataColumnSidecar { - column, kzg_proofs, .. - } = sidecar; - // There's only one cell per column for a single blob - let cell_bytes: Vec = column.into_iter().next().unwrap().into(); - let kzg_cell = cell_bytes.try_into().unwrap(); - let kzg_proof = kzg_proofs.into_iter().next().unwrap(); - (kzg_cell, kzg_proof) - }) - .collect::<(Vec<_>, Vec<_>)>(); + let (cells, proofs) = template_data_columns + .into_iter() + .map(|sidecar| { + let DataColumnSidecarGloas { + column, kzg_proofs, .. + } = sidecar; + // There's only one cell per column for a single blob + let cell_bytes: Vec = column.into_iter().next().unwrap().into(); + let kzg_cell = cell_bytes.try_into().unwrap(); + let kzg_proof = kzg_proofs.into_iter().next().unwrap(); + (kzg_cell, kzg_proof) + }) + .collect::<(Vec<_>, Vec<_>)>(); + + vec![(cells.try_into().unwrap(), proofs.try_into().unwrap()); kzg_commitments.len()] + } else { + // load the precomputed column sidecar to avoid computing them for every block in the tests. + let template_data_columns = + RuntimeVariableList::>::from_ssz_bytes( + TEST_DATA_COLUMN_SIDECARS_SSZ, + E::number_of_columns(), + ) + .unwrap(); - // Repeat the cells and proofs for every blob - let blob_cells_and_proofs_vec = - vec![(cells.try_into().unwrap(), proofs.try_into().unwrap()); kzg_commitments.len()]; + let (cells, proofs) = template_data_columns + .into_iter() + .map(|sidecar| { + let DataColumnSidecarFulu { + column, kzg_proofs, .. + } = sidecar; + // There's only one cell per column for a single blob + let cell_bytes: Vec = column.into_iter().next().unwrap().into(); + let kzg_cell = cell_bytes.try_into().unwrap(); + let kzg_proof = kzg_proofs.into_iter().next().unwrap(); + (kzg_cell, kzg_proof) + }) + .collect::<(Vec<_>, Vec<_>)>(); - build_data_column_sidecars( - kzg_commitments.clone(), - kzg_commitments_inclusion_proof, - signed_block_header, - blob_cells_and_proofs_vec, - spec, - ) - .unwrap() + vec![(cells.try_into().unwrap(), proofs.try_into().unwrap()); kzg_commitments.len()] + }; + + if block.fork_name_unchecked().gloas_enabled() { + build_data_column_sidecars_gloas( + kzg_commitments.clone(), + signed_block_header, + blob_cells_and_proofs_vec, + spec, + ) + .unwrap() + } else { + build_data_column_sidecars_fulu( + kzg_commitments.clone(), + kzg_commitments_inclusion_proof, + signed_block_header, + blob_cells_and_proofs_vec, + spec, + ) + .unwrap() + } } pub fn generate_data_column_indices_rand_order() -> Vec { diff --git a/beacon_node/beacon_chain/tests/block_verification.rs b/beacon_node/beacon_chain/tests/block_verification.rs index 2644b74b28e..155c6991b70 100644 --- a/beacon_node/beacon_chain/tests/block_verification.rs +++ b/beacon_node/beacon_chain/tests/block_verification.rs @@ -81,10 +81,12 @@ async fn get_chain_segment() -> (Vec>, Vec( ) { for old_custody_column_sidecar in data_columns.as_mut_slice() { let old_column_sidecar = old_custody_column_sidecar.as_data_column(); - let new_column_sidecar = Arc::new(DataColumnSidecar:: { - index: old_column_sidecar.index, - column: old_column_sidecar.column.clone(), - kzg_commitments: old_column_sidecar.kzg_commitments.clone(), - kzg_proofs: old_column_sidecar.kzg_proofs.clone(), + let new_column_sidecar = Arc::new(DataColumnSidecar::Fulu(DataColumnSidecarFulu { + index: *old_column_sidecar.index(), + column: old_column_sidecar.column().clone(), + kzg_commitments: old_column_sidecar.kzg_commitments().clone(), + kzg_proofs: old_column_sidecar.kzg_proofs().clone(), signed_block_header: signed_block.signed_block_header(), kzg_commitments_inclusion_proof: signed_block .message() .body() .kzg_commitments_merkle_proof() .unwrap(), - }); + })); *old_custody_column_sidecar = CustodyDataColumn::from_asserted_custody(new_column_sidecar); } } @@ -1297,20 +1299,8 @@ async fn verify_and_process_gossip_data_sidecars( data_sidecars: DataSidecars, ) { match data_sidecars { - DataSidecars::Blobs(blob_sidecars) => { - for blob_sidecar in blob_sidecars { - let blob_index = blob_sidecar.index; - let gossip_verified = harness - .chain - .verify_blob_sidecar_for_gossip(blob_sidecar.clone(), blob_index) - .expect("should obtain gossip verified blob"); - - harness - .chain - .process_gossip_blob(gossip_verified) - .await - .expect("should import valid gossip verified blob"); - } + DataSidecars::Blobs(_blob_sidecars) => { + panic!("We dont support blobs anymore"); } DataSidecars::DataColumns(column_sidecars) => { let gossip_verified = column_sidecars @@ -1362,14 +1352,9 @@ async fn verify_block_for_gossip_slashing_detection() { let verified_block = harness.chain.verify_block_for_gossip(block1).await.unwrap(); - if let Some((kzg_proofs, blobs)) = blobs1 { + if let Some((_, _)) = blobs1 { harness - .process_gossip_blobs_or_columns( - verified_block.block(), - blobs.iter(), - kzg_proofs.iter(), - None, - ) + .process_gossip_columns(verified_block.block(), None) .await; } harness diff --git a/beacon_node/beacon_chain/tests/events.rs b/beacon_node/beacon_chain/tests/events.rs index 35884541e19..c64d6ce7744 100644 --- a/beacon_node/beacon_chain/tests/events.rs +++ b/beacon_node/beacon_chain/tests/events.rs @@ -1,4 +1,3 @@ -use beacon_chain::blob_verification::GossipVerifiedBlob; use beacon_chain::data_column_verification::GossipVerifiedDataColumn; use beacon_chain::test_utils::{ BeaconChainHarness, fork_name_from_env, generate_data_column_sidecars_from_block, test_spec, @@ -9,48 +8,13 @@ use rand::rngs::StdRng; use std::sync::Arc; use types::data::FixedBlobSidecarList; use types::test_utils::TestRandom; -use types::{BlobSidecar, DataColumnSidecar, EthSpec, MinimalEthSpec, Slot}; +use types::{ + BlobSidecar, DataColumnSidecar, DataColumnSidecarFulu, DataColumnSidecarGloas, EthSpec, + MinimalEthSpec, Slot, +}; type E = MinimalEthSpec; -/// Verifies that a blob event is emitted when a gossip verified blob is received via gossip or the publish block API. -#[tokio::test] -async fn blob_sidecar_event_on_process_gossip_blob() { - if fork_name_from_env().is_some_and(|f| !f.deneb_enabled() || f.fulu_enabled()) { - return; - }; - - let spec = Arc::new(test_spec::()); - let harness = BeaconChainHarness::builder(E::default()) - .spec(spec) - .deterministic_keypairs(8) - .fresh_ephemeral_store() - .mock_execution_layer() - .build(); - - // subscribe to blob sidecar events - let event_handler = harness.chain.event_handler.as_ref().unwrap(); - let mut blob_event_receiver = event_handler.subscribe_blob_sidecar(); - - // build and process a gossip verified blob - let kzg = harness.chain.kzg.as_ref(); - let mut rng = StdRng::seed_from_u64(0xDEADBEEF0BAD5EEDu64); - let sidecar = BlobSidecar::random_valid(&mut rng, kzg) - .map(Arc::new) - .unwrap(); - let gossip_verified_blob = GossipVerifiedBlob::__assumed_valid(sidecar); - let expected_sse_blobs = SseBlobSidecar::from_blob_sidecar(gossip_verified_blob.as_blob()); - - let _ = harness - .chain - .process_gossip_blob(gossip_verified_blob) - .await - .unwrap(); - - let sidecar_event = blob_event_receiver.try_recv().unwrap(); - assert_eq!(sidecar_event, EventKind::BlobSidecar(expected_sse_blobs)); -} - /// Verifies that a data column event is emitted when a gossip verified data column is received via gossip or the publish block API. #[tokio::test] async fn data_column_sidecar_event_on_process_gossip_data_column() { @@ -73,13 +37,22 @@ async fn data_column_sidecar_event_on_process_gossip_data_column() { // build and process a gossip verified data column let mut rng = StdRng::seed_from_u64(0xDEADBEEF0BAD5EEDu64); let sidecar = { - // DA checker only accepts sampling columns, so we need to create one with a sampling index. - let mut random_sidecar = DataColumnSidecar::random_for_test(&mut rng); let slot = Slot::new(10); - let epoch = slot.epoch(E::slots_per_epoch()); - random_sidecar.signed_block_header.message.slot = slot; - random_sidecar.index = harness.chain.sampling_columns_for_epoch(epoch)[0]; - random_sidecar + let fork_name = harness.spec.fork_name_at_slot::(slot); + // DA checker only accepts sampling columns, so we need to create one with a sampling index. + if fork_name.gloas_enabled() { + let mut random_sidecar = DataColumnSidecarGloas::random_for_test(&mut rng); + let epoch = slot.epoch(E::slots_per_epoch()); + random_sidecar.slot = slot; + random_sidecar.index = harness.chain.sampling_columns_for_epoch(epoch)[0]; + DataColumnSidecar::Gloas(random_sidecar) + } else { + let mut random_sidecar = DataColumnSidecarFulu::random_for_test(&mut rng); + let epoch = slot.epoch(E::slots_per_epoch()); + random_sidecar.signed_block_header.message.slot = slot; + random_sidecar.index = harness.chain.sampling_columns_for_epoch(epoch)[0]; + DataColumnSidecar::Fulu(random_sidecar) + } }; let gossip_verified_data_column = GossipVerifiedDataColumn::__new_for_testing(Arc::new(sidecar)); diff --git a/beacon_node/beacon_chain/tests/store_tests.rs b/beacon_node/beacon_chain/tests/store_tests.rs index ba0621ae720..c3921b7cb62 100644 --- a/beacon_node/beacon_chain/tests/store_tests.rs +++ b/beacon_node/beacon_chain/tests/store_tests.rs @@ -3380,7 +3380,7 @@ async fn test_import_historical_data_columns_batch() { .await; harness.advance_slot(); - let block_root_iter = harness + let block_root_and_slot = harness .chain .forwards_iter_block_roots_until(start_slot, end_slot) .unwrap(); @@ -3388,9 +3388,14 @@ async fn test_import_historical_data_columns_batch() { let mut data_columns_list = vec![]; // Get all data columns for epoch 0 - for block in block_root_iter { - let (block_root, _) = block.unwrap(); - let data_columns = harness.chain.store.get_data_columns(&block_root).unwrap(); + for block_root_and_slot in block_root_and_slot { + let (block_root, slot) = block_root_and_slot.unwrap(); + let fork_name = harness.spec.fork_name_at_slot::(slot); + let data_columns = harness + .chain + .store + .get_data_columns(&block_root, fork_name) + .unwrap(); for data_column in data_columns.unwrap_or_default() { data_columns_list.push(data_column); } @@ -3415,15 +3420,20 @@ async fn test_import_historical_data_columns_batch() { .try_prune_blobs(true, Epoch::new(2)) .unwrap(); - let block_root_iter = harness + let block_root_and_slot_iter = harness .chain .forwards_iter_block_roots_until(start_slot, end_slot) .unwrap(); // Assert that data columns no longer exist for epoch 0 - for block in block_root_iter { - let (block_root, _) = block.unwrap(); - let data_columns = harness.chain.store.get_data_columns(&block_root).unwrap(); + for block_root_and_slot in block_root_and_slot_iter { + let (block_root, slot) = block_root_and_slot.unwrap(); + let fork_name = harness.spec.fork_name_at_slot::(slot); + let data_columns = harness + .chain + .store + .get_data_columns(&block_root, fork_name) + .unwrap(); assert!(data_columns.is_none()) } @@ -3433,14 +3443,14 @@ async fn test_import_historical_data_columns_batch() { .import_historical_data_column_batch(Epoch::new(0), data_columns_list, cgc) .unwrap(); - let block_root_iter = harness + let block_root_and_slot_iter = harness .chain .forwards_iter_block_roots_until(start_slot, end_slot) .unwrap(); // Assert that data columns now exist for epoch 0 - for block in block_root_iter { - let (block_root, _) = block.unwrap(); + for block_root_and_slot in block_root_and_slot_iter { + let (block_root, slot) = block_root_and_slot.unwrap(); if !harness .get_block(block_root.into()) .unwrap() @@ -3450,7 +3460,12 @@ async fn test_import_historical_data_columns_batch() { .unwrap() .is_empty() { - let data_columns = harness.chain.store.get_data_columns(&block_root).unwrap(); + let fork_name = harness.spec.fork_name_at_slot::(slot); + let data_columns = harness + .chain + .store + .get_data_columns(&block_root, fork_name) + .unwrap(); assert!(data_columns.is_some()) }; } @@ -3478,7 +3493,7 @@ async fn test_import_historical_data_columns_batch_mismatched_block_root() { .await; harness.advance_slot(); - let block_root_iter = harness + let block_root_and_slot_iter = harness .chain .forwards_iter_block_roots_until(start_slot, end_slot) .unwrap(); @@ -3487,14 +3502,23 @@ async fn test_import_historical_data_columns_batch_mismatched_block_root() { // Get all data columns from start_slot to end_slot // and mutate the data columns with an invalid block root - for block in block_root_iter { - let (block_root, _) = block.unwrap(); - let data_columns = harness.chain.store.get_data_columns(&block_root).unwrap(); + for block_root_and_slot in block_root_and_slot_iter { + let (block_root, slot) = block_root_and_slot.unwrap(); + let fork_name = harness.spec.fork_name_at_slot::(slot); + let data_columns = harness + .chain + .store + .get_data_columns(&block_root, fork_name) + .unwrap(); for data_column in data_columns.unwrap_or_default() { let mut data_column = (*data_column).clone(); - if data_column.index % 2 == 0 { - data_column.signed_block_header.message.body_root = Hash256::ZERO; + if data_column.index() % 2 == 0 { + data_column + .signed_block_header_mut() + .unwrap() + .message + .body_root = Hash256::ZERO; } data_columns_list.push(Arc::new(data_column)); @@ -3519,15 +3543,20 @@ async fn test_import_historical_data_columns_batch_mismatched_block_root() { .try_prune_blobs(true, Epoch::new(2)) .unwrap(); - let block_root_iter = harness + let block_root_and_slot_iter = harness .chain .forwards_iter_block_roots_until(start_slot, end_slot) .unwrap(); // Assert there are no columns between start_slot and end_slot - for block in block_root_iter { - let (block_root, _) = block.unwrap(); - let data_columns = harness.chain.store.get_data_columns(&block_root).unwrap(); + for block_root_and_slot in block_root_and_slot_iter { + let (block_root, slot) = block_root_and_slot.unwrap(); + let fork_name = harness.spec.fork_name_at_slot::(slot); + let data_columns = harness + .chain + .store + .get_data_columns(&block_root, fork_name) + .unwrap(); assert!(data_columns.is_none()) } @@ -3573,20 +3602,29 @@ async fn test_import_historical_data_columns_batch_no_block_found() { .await; harness.advance_slot(); - let block_root_iter = harness + let block_root_and_slot_iter = harness .chain .forwards_iter_block_roots_until(start_slot, end_slot) .unwrap(); let mut data_columns_list = vec![]; - for block in block_root_iter { - let (block_root, _) = block.unwrap(); - let data_columns = harness.chain.store.get_data_columns(&block_root).unwrap(); + for block_root_and_slot in block_root_and_slot_iter { + let (block_root, slot) = block_root_and_slot.unwrap(); + let fork_name = harness.spec.fork_name_at_slot::(slot); + let data_columns = harness + .chain + .store + .get_data_columns(&block_root, fork_name) + .unwrap(); for data_column in data_columns.unwrap_or_default() { let mut data_column = (*data_column).clone(); - data_column.signed_block_header.message.body_root = Hash256::ZERO; + data_column + .signed_block_header_mut() + .unwrap() + .message + .body_root = Hash256::ZERO; data_columns_list.push(Arc::new(data_column)); } } @@ -3609,14 +3647,19 @@ async fn test_import_historical_data_columns_batch_no_block_found() { .try_prune_blobs(true, Epoch::new(2)) .unwrap(); - let block_root_iter = harness + let block_root_and_slot_iter = harness .chain .forwards_iter_block_roots_until(start_slot, end_slot) .unwrap(); - for block in block_root_iter { - let (block_root, _) = block.unwrap(); - let data_columns = harness.chain.store.get_data_columns(&block_root).unwrap(); + for block_root_and_slot in block_root_and_slot_iter { + let (block_root, slot) = block_root_and_slot.unwrap(); + let fork_name = harness.spec.fork_name_at_slot::(slot); + let data_columns = harness + .chain + .store + .get_data_columns(&block_root, fork_name) + .unwrap(); assert!(data_columns.is_none()) } @@ -4995,7 +5038,13 @@ fn check_data_column_existence( .unwrap() .map(Result::unwrap) { - if let Some(columns) = harness.chain.store.get_data_columns(&block_root).unwrap() { + let fork_name = harness.spec.fork_name_at_slot::(slot); + if let Some(columns) = harness + .chain + .store + .get_data_columns(&block_root, fork_name) + .unwrap() + { assert!(should_exist, "columns at slot {slot} exist but should not"); columns_seen += columns.len(); } else { diff --git a/beacon_node/http_api/src/block_id.rs b/beacon_node/http_api/src/block_id.rs index 6a0cdc33a08..210801f1d01 100644 --- a/beacon_node/http_api/src/block_id.rs +++ b/beacon_node/http_api/src/block_id.rs @@ -281,7 +281,9 @@ impl BlockId { warp_utils::reject::custom_not_found(format!("beacon block with root {}", root)) })?; - if !chain.spec.is_peer_das_enabled_for_epoch(block.epoch()) { + let fork_name = chain.spec.fork_name_at_epoch(block.epoch()); + + if !fork_name.fulu_enabled() { return Err(warp_utils::reject::custom_bad_request( "block is pre-Fulu and has no data columns".to_string(), )); @@ -290,12 +292,12 @@ impl BlockId { let data_column_sidecars = if let Some(indices) = query.indices { indices .iter() - .filter_map(|index| chain.get_data_column(&root, index).transpose()) + .filter_map(|index| chain.get_data_column(&root, index, fork_name).transpose()) .collect::, _>>() .map_err(warp_utils::reject::unhandled_error)? } else { chain - .get_data_columns(&root) + .get_data_columns(&root, fork_name) .map_err(warp_utils::reject::unhandled_error)? .unwrap_or_default() }; @@ -431,20 +433,18 @@ impl BlockId { warp_utils::reject::custom_not_found(format!("no blobs stored for block {root}")) })?; - let blob_sidecar_list_filtered = match indices { - Some(vec) => { - let list: Vec<_> = vec - .into_iter() - .flat_map(|index| blob_sidecar_list.get(index as usize).cloned()) - .collect(); + let blob_sidecar_list: Vec<_> = blob_sidecar_list.into_iter().collect(); - BlobSidecarList::new(list, max_blobs_per_block) - .map_err(|e| warp_utils::reject::custom_server_error(format!("{:?}", e)))? - } + let blob_sidecar_list = match indices { + Some(indices) => indices + .into_iter() + .filter_map(|i| blob_sidecar_list.get(i as usize).cloned()) + .collect(), None => blob_sidecar_list, }; - Ok(blob_sidecar_list_filtered) + BlobSidecarList::new(blob_sidecar_list, max_blobs_per_block) + .map_err(|e| warp_utils::reject::custom_server_error(format!("{:?}", e))) } fn get_blobs_from_data_columns( @@ -462,17 +462,18 @@ impl BlockId { let num_found_column_keys = column_indices.len(); let num_required_columns = T::EthSpec::number_of_columns() / 2; let is_blob_available = num_found_column_keys >= num_required_columns; + let fork_name = chain.spec.fork_name_at_epoch(block.epoch()); if is_blob_available { let data_columns = column_indices .into_iter() - .filter_map( - |column_index| match chain.get_data_column(&root, &column_index) { + .filter_map(|column_index| { + match chain.get_data_column(&root, &column_index, fork_name) { Ok(Some(data_column)) => Some(Ok(data_column)), Ok(None) => None, Err(e) => Some(Err(warp_utils::reject::unhandled_error(e))), - }, - ) + } + }) .collect::, _>>()?; reconstruct_blobs(&chain.kzg, data_columns, blob_indices, block, &chain.spec).map_err( diff --git a/beacon_node/http_api/src/publish_blocks.rs b/beacon_node/http_api/src/publish_blocks.rs index b54c071eb80..ff54e603a71 100644 --- a/beacon_node/http_api/src/publish_blocks.rs +++ b/beacon_node/http_api/src/publish_blocks.rs @@ -1,7 +1,6 @@ use crate::metrics; use std::future::Future; -use beacon_chain::blob_verification::{GossipBlobError, GossipVerifiedBlob}; use beacon_chain::block_verification_types::{AsBlock, RpcBlock}; use beacon_chain::data_column_verification::GossipVerifiedDataColumn; use beacon_chain::validator_monitor::{get_block_delay_ms, timestamp_now}; @@ -31,9 +30,9 @@ use tokio::sync::mpsc::UnboundedSender; use tracing::{Span, debug, debug_span, error, field, info, instrument, warn}; use tree_hash::TreeHash; use types::{ - AbstractExecPayload, BeaconBlockRef, BlobSidecar, BlobsList, BlockImportSource, - DataColumnSubnetId, EthSpec, ExecPayload, ExecutionBlockHash, ForkName, FullPayload, - FullPayloadBellatrix, Hash256, KzgProofs, SignedBeaconBlock, SignedBlindedBeaconBlock, + AbstractExecPayload, BeaconBlockRef, BlobsList, BlockImportSource, DataColumnSubnetId, EthSpec, + ExecPayload, ExecutionBlockHash, ForkName, FullPayload, FullPayloadBellatrix, Hash256, + KzgProofs, SignedBeaconBlock, SignedBlindedBeaconBlock, }; use warp::{Rejection, Reply, reply::Response}; @@ -199,23 +198,8 @@ pub async fn publish_block>( Ok(()) }; - // Wait for blobs/columns to get gossip verified before proceeding further as we need them for import. - let (gossip_verified_blobs, gossip_verified_columns) = build_sidecar_task_handle.await?; - - for blob in gossip_verified_blobs.into_iter().flatten() { - publish_blob_sidecars(network_tx, &blob).map_err(|_| { - warp_utils::reject::custom_server_error("unable to publish blob sidecars".into()) - })?; - if let Err(e) = Box::pin(chain.process_gossip_blob(blob)).await { - let msg = format!("Invalid blob: {e}"); - return if let BroadcastValidation::Gossip = validation_level { - Err(warp_utils::reject::broadcast_without_import(msg)) - } else { - error!(reason = &msg, "Invalid blob provided to HTTP API"); - Err(warp_utils::reject::custom_bad_request(msg)) - }; - } - } + // Wait for columns to get gossip verified before proceeding further as we need them for import. + let gossip_verified_columns = build_sidecar_task_handle.await?; if !gossip_verified_columns.is_empty() { if let Some(data_column_publishing_delay) = data_column_publishing_delay_for_testing { @@ -344,13 +328,7 @@ pub async fn publish_block>( } } -type BuildDataSidecarTaskResult = Result< - ( - Vec>>, - Vec>, - ), - Rejection, ->; +type BuildDataSidecarTaskResult = Result>, Rejection>; /// Convert blobs to either: /// @@ -368,22 +346,10 @@ fn spawn_build_data_sidecar_task( .spawn_blocking_handle( move || { let Some((kzg_proofs, blobs)) = proofs_and_blobs else { - return Ok((vec![], vec![])); + return Ok(vec![]); }; let _guard = debug_span!(parent: current_span, "build_data_sidecars").entered(); - - let peer_das_enabled = chain.spec.is_peer_das_enabled_for_epoch(block.epoch()); - if !peer_das_enabled { - // Pre-PeerDAS: construct blob sidecars for the network. - let gossip_verified_blobs = - build_gossip_verified_blobs(&chain, &block, blobs, kzg_proofs)?; - Ok((gossip_verified_blobs, vec![])) - } else { - // Post PeerDAS: construct data columns. - let gossip_verified_data_columns = - build_data_columns(&chain, &block, blobs, kzg_proofs)?; - Ok((vec![], gossip_verified_data_columns)) - } + build_data_columns(&chain, &block, blobs, kzg_proofs) }, "build_data_sidecars", ) @@ -427,76 +393,6 @@ fn build_data_columns( Ok(gossip_verified_data_columns) } -fn build_gossip_verified_blobs( - chain: &BeaconChain, - block: &SignedBeaconBlock>, - blobs: BlobsList, - kzg_proofs: KzgProofs, -) -> Result>>, Rejection> { - let slot = block.slot(); - let gossip_verified_blobs = kzg_proofs - .into_iter() - .zip(blobs) - .enumerate() - .map(|(i, (proof, unverified_blob))| { - let timer = metrics::start_timer( - &beacon_chain::metrics::BLOB_SIDECAR_INCLUSION_PROOF_COMPUTATION, - ); - let blob_sidecar = BlobSidecar::new(i, unverified_blob, block, proof) - .map(Arc::new) - .map_err(|e| { - error!( - error = ?e, - blob_index = i, - %slot, - "Invalid blob - not publishing block" - ); - warp_utils::reject::custom_bad_request(format!("{e:?}")) - })?; - drop(timer); - - let gossip_verified_blob = - GossipVerifiedBlob::new(blob_sidecar.clone(), blob_sidecar.index, chain); - - match gossip_verified_blob { - Ok(blob) => Ok(Some(blob)), - Err(GossipBlobError::RepeatBlob { proposer, .. }) => { - // Log the error but do not abort publication, we may need to publish the block - // or some of the other blobs if the block & blobs are only partially published - // by the other publisher. - debug!( - blob_index = blob_sidecar.index, - %slot, - proposer, - "Blob for publication already known" - ); - Ok(None) - } - Err(e) => { - error!( - blob_index = blob_sidecar.index, - %slot, - error = ?e, - "Blob for publication is gossip-invalid" - ); - Err(warp_utils::reject::custom_bad_request(e.to_string())) - } - } - }) - .collect::, Rejection>>()?; - - Ok(gossip_verified_blobs) -} - -fn publish_blob_sidecars( - sender_clone: &UnboundedSender>, - blob: &GossipVerifiedBlob, -) -> Result<(), BlockError> { - let pubsub_message = PubsubMessage::BlobSidecar(Box::new((blob.index(), blob.clone_blob()))); - crate::utils::publish_pubsub_message(sender_clone, pubsub_message) - .map_err(|_| BlockError::BeaconChainError(Box::new(BeaconChainError::UnableToPublish))) -} - fn publish_column_sidecars( sender_clone: &UnboundedSender>, data_column_sidecars: &[GossipVerifiedDataColumn], @@ -515,14 +411,14 @@ fn publish_column_sidecars( data_column_sidecars.shuffle(&mut **chain.rng.lock()); let dropped_indices = data_column_sidecars .drain(columns_to_keep..) - .map(|d| d.index) + .map(|d| *d.index()) .collect::>(); debug!(indices = ?dropped_indices, "Dropping data columns from publishing"); } let pubsub_messages = data_column_sidecars .into_iter() .map(|data_col| { - let subnet = DataColumnSubnetId::from_column_index(data_col.index, &chain.spec); + let subnet = DataColumnSubnetId::from_column_index(*data_col.index(), &chain.spec); PubsubMessage::DataColumnSidecar(Box::new((subnet, data_col))) }) .collect::>(); diff --git a/beacon_node/http_api/tests/broadcast_validation_tests.rs b/beacon_node/http_api/tests/broadcast_validation_tests.rs index 357b78cf41c..b6f7b70bc15 100644 --- a/beacon_node/http_api/tests/broadcast_validation_tests.rs +++ b/beacon_node/http_api/tests/broadcast_validation_tests.rs @@ -1629,7 +1629,7 @@ pub async fn block_seen_on_gossip_without_blobs_or_columns() { /// This test checks that an HTTP POST request with the block & blobs/columns succeeds with a 200 response /// even if the block has already been seen on gossip without all blobs/columns. #[tokio::test(flavor = "multi_thread", worker_threads = 2)] -pub async fn block_seen_on_gossip_with_some_blobs_or_columns() { +pub async fn block_seen_on_gossip_with_columns() { let validation_level: Option = Some(BroadcastValidation::Gossip); // Validator count needs to be at least 32 or proposer boost gets set to 0 when computing @@ -1639,7 +1639,7 @@ pub async fn block_seen_on_gossip_with_some_blobs_or_columns() { let tester = InteractiveTester::::new(None, validator_count).await; let state = tester.harness.get_current_state(); let fork_name = state.fork_name(&tester.harness.spec).unwrap(); - if !fork_name.deneb_enabled() { + if !fork_name.fulu_enabled() { return; } @@ -1671,9 +1671,6 @@ pub async fn block_seen_on_gossip_with_some_blobs_or_columns() { blobs.0.len() ); - let partial_kzg_proofs = [*blobs.0.first().unwrap()]; - let partial_blobs = [blobs.1.first().unwrap().clone()]; - // Simulate the block being seen on gossip. block .clone() @@ -1683,12 +1680,7 @@ pub async fn block_seen_on_gossip_with_some_blobs_or_columns() { // Simulate some of the blobs being seen on gossip. tester .harness - .process_gossip_blobs_or_columns( - &block, - partial_blobs.iter(), - partial_kzg_proofs.iter(), - Some(get_custody_columns(&tester, block.slot())), - ) + .process_gossip_columns(&block, Some(get_custody_columns(&tester, block.slot()))) .await; // It should not yet be added to fork choice because all blobs have not been seen. @@ -1721,7 +1713,7 @@ pub async fn block_seen_on_gossip_with_some_blobs_or_columns() { /// This test checks that an HTTP POST request with the block & blobs/columns succeeds with a 200 response /// even if the blobs/columns have already been seen on gossip. #[tokio::test(flavor = "multi_thread", worker_threads = 2)] -pub async fn blobs_or_columns_seen_on_gossip_without_block() { +pub async fn columns_seen_on_gossip_without_block() { let spec = test_spec::(); let validation_level: Option = Some(BroadcastValidation::Gossip); @@ -1732,7 +1724,7 @@ pub async fn blobs_or_columns_seen_on_gossip_without_block() { let tester = InteractiveTester::::new(Some(spec.clone()), validator_count).await; let state = tester.harness.get_current_state(); let fork_name = state.fork_name(&tester.harness.spec).unwrap(); - if !fork_name.deneb_enabled() { + if !fork_name.fulu_enabled() { return; } @@ -1758,12 +1750,7 @@ pub async fn blobs_or_columns_seen_on_gossip_without_block() { // Simulate the blobs being seen on gossip. tester .harness - .process_gossip_blobs_or_columns( - &block, - blobs.iter(), - kzg_proofs.iter(), - Some(get_custody_columns(&tester, block.slot())), - ) + .process_gossip_columns(&block, Some(get_custody_columns(&tester, block.slot()))) .await; // It should not yet be added to fork choice because the block has not been seen. @@ -1796,7 +1783,7 @@ pub async fn blobs_or_columns_seen_on_gossip_without_block() { /// This test checks that an HTTP POST request with the block succeeds with a 200 response /// if just the blobs have already been seen on gossip. #[tokio::test(flavor = "multi_thread", worker_threads = 2)] -async fn blobs_or_columns_seen_on_gossip_without_block_and_no_http_blobs_or_columns() { +async fn columns_seen_on_gossip_without_block_and_no_http_columns() { let validation_level: Option = Some(BroadcastValidation::Gossip); // Validator count needs to be at least 32 or proposer boost gets set to 0 when computing @@ -1806,7 +1793,7 @@ async fn blobs_or_columns_seen_on_gossip_without_block_and_no_http_blobs_or_colu let tester = InteractiveTester::::new(None, validator_count).await; let state = tester.harness.get_current_state(); let fork_name = state.fork_name(&tester.harness.spec).unwrap(); - if !fork_name.deneb_enabled() { + if !fork_name.fulu_enabled() { return; } @@ -1827,18 +1814,13 @@ async fn blobs_or_columns_seen_on_gossip_without_block_and_no_http_blobs_or_colu let state_a = tester.harness.get_current_state(); let ((block, blobs), _) = tester.harness.make_block(state_a, slot_b).await; - let (kzg_proofs, blobs) = blobs.expect("should have some blobs"); + let (_, blobs) = blobs.expect("should have some blobs"); assert!(!blobs.is_empty()); // Simulate the blobs being seen on gossip. tester .harness - .process_gossip_blobs_or_columns( - &block, - blobs.iter(), - kzg_proofs.iter(), - Some(get_custody_columns(&tester, block.slot())), - ) + .process_gossip_columns(&block, Some(get_custody_columns(&tester, block.slot()))) .await; // It should not yet be added to fork choice because the block has not been seen. @@ -1872,7 +1854,7 @@ async fn blobs_or_columns_seen_on_gossip_without_block_and_no_http_blobs_or_colu } #[tokio::test(flavor = "multi_thread", worker_threads = 2)] -async fn slashable_blobs_or_columns_seen_on_gossip_cause_failure() { +async fn slashable_columns_seen_on_gossip_cause_failure() { let validation_level: Option = Some(BroadcastValidation::ConsensusAndEquivocation); @@ -1883,7 +1865,7 @@ async fn slashable_blobs_or_columns_seen_on_gossip_cause_failure() { let tester = InteractiveTester::::new(None, validator_count).await; let state = tester.harness.get_current_state(); let fork_name = state.fork_name(&tester.harness.spec).unwrap(); - if !fork_name.deneb_enabled() { + if !fork_name.fulu_enabled() { return; } @@ -1904,19 +1886,13 @@ async fn slashable_blobs_or_columns_seen_on_gossip_cause_failure() { let state_a = tester.harness.get_current_state(); let ((block_a, blobs_a), _) = tester.harness.make_block(state_a.clone(), slot_b).await; - let ((block_b, blobs_b), _) = tester.harness.make_block(state_a, slot_b).await; + let ((block_b, _), _) = tester.harness.make_block(state_a, slot_b).await; let (kzg_proofs_a, blobs_a) = blobs_a.expect("should have some blobs"); - let (kzg_proofs_b, blobs_b) = blobs_b.expect("should have some blobs"); // Simulate the blobs of block B being seen on gossip. tester .harness - .process_gossip_blobs_or_columns( - &block_b, - blobs_b.iter(), - kzg_proofs_b.iter(), - Some(get_custody_columns(&tester, block_b.slot())), - ) + .process_gossip_columns(&block_b, Some(get_custody_columns(&tester, block_b.slot()))) .await; // It should not yet be added to fork choice because block B has not been seen. diff --git a/beacon_node/lighthouse_network/src/rpc/codec.rs b/beacon_node/lighthouse_network/src/rpc/codec.rs index 3611f023917..36d9726dd9c 100644 --- a/beacon_node/lighthouse_network/src/rpc/codec.rs +++ b/beacon_node/lighthouse_network/src/rpc/codec.rs @@ -693,7 +693,7 @@ fn handle_rpc_response( Some(fork_name) => { if fork_name.fulu_enabled() { Ok(Some(RpcSuccessResponse::DataColumnsByRoot(Arc::new( - DataColumnSidecar::from_ssz_bytes(decoded_buffer)?, + DataColumnSidecar::from_ssz_bytes_for_fork(decoded_buffer, fork_name)?, )))) } else { Err(RPCError::ErrorResponse( @@ -714,7 +714,7 @@ fn handle_rpc_response( Some(fork_name) => { if fork_name.fulu_enabled() { Ok(Some(RpcSuccessResponse::DataColumnsByRange(Arc::new( - DataColumnSidecar::from_ssz_bytes(decoded_buffer)?, + DataColumnSidecar::from_ssz_bytes_for_fork(decoded_buffer, fork_name)?, )))) } else { Err(RPCError::ErrorResponse( @@ -916,6 +916,7 @@ mod tests { SignedBeaconBlockHeader, Slot, data::{BlobIdentifier, Cell}, }; + use types::{BlobSidecar, DataColumnSidecarFulu}; type Spec = types::MainnetEthSpec; @@ -977,7 +978,7 @@ mod tests { fn empty_data_column_sidecar(spec: &ChainSpec) -> Arc> { // The context bytes are now derived from the block epoch, so we need to have the slot set // here. - let data_column_sidecar = DataColumnSidecar { + let data_column_sidecar = DataColumnSidecar::Fulu(DataColumnSidecarFulu { index: 0, column: VariableList::new(vec![Cell::::default()]).unwrap(), kzg_commitments: VariableList::new(vec![KzgCommitment::empty_for_testing()]).unwrap(), @@ -993,7 +994,7 @@ mod tests { signature: Signature::empty(), }, kzg_commitments_inclusion_proof: Default::default(), - }; + }); Arc::new(data_column_sidecar) } diff --git a/beacon_node/lighthouse_network/src/rpc/methods.rs b/beacon_node/lighthouse_network/src/rpc/methods.rs index 0539877c722..5a9a683b758 100644 --- a/beacon_node/lighthouse_network/src/rpc/methods.rs +++ b/beacon_node/lighthouse_network/src/rpc/methods.rs @@ -15,9 +15,9 @@ use superstruct::superstruct; use types::data::BlobIdentifier; use types::light_client::consts::MAX_REQUEST_LIGHT_CLIENT_UPDATES; use types::{ - ChainSpec, ColumnIndex, DataColumnSidecar, DataColumnsByRootIdentifier, Epoch, EthSpec, - ForkContext, Hash256, LightClientBootstrap, LightClientFinalityUpdate, - LightClientOptimisticUpdate, LightClientUpdate, SignedBeaconBlock, Slot, data::BlobSidecar, + BlobSidecar, ChainSpec, ColumnIndex, DataColumnSidecar, DataColumnsByRootIdentifier, Epoch, + EthSpec, ForkContext, Hash256, LightClientBootstrap, LightClientFinalityUpdate, + LightClientOptimisticUpdate, LightClientUpdate, SignedBeaconBlock, Slot, }; /// Maximum length of error message. @@ -762,12 +762,8 @@ impl RpcSuccessResponse { pub fn slot(&self) -> Option { match self { Self::BlocksByRange(r) | Self::BlocksByRoot(r) => Some(r.slot()), - Self::BlobsByRange(r) | Self::BlobsByRoot(r) => { - Some(r.signed_block_header.message.slot) - } - Self::DataColumnsByRange(r) | Self::DataColumnsByRoot(r) => { - Some(r.signed_block_header.message.slot) - } + Self::BlobsByRange(r) | Self::BlobsByRoot(r) => Some(r.slot()), + Self::DataColumnsByRange(r) | Self::DataColumnsByRoot(r) => Some(r.slot()), Self::LightClientBootstrap(r) => Some(r.get_slot()), Self::LightClientFinalityUpdate(r) => Some(r.get_attested_header_slot()), Self::LightClientOptimisticUpdate(r) => Some(r.get_slot()), diff --git a/beacon_node/lighthouse_network/src/rpc/protocol.rs b/beacon_node/lighthouse_network/src/rpc/protocol.rs index 366515d42f6..f0ac9d00f9e 100644 --- a/beacon_node/lighthouse_network/src/rpc/protocol.rs +++ b/beacon_node/lighthouse_network/src/rpc/protocol.rs @@ -17,11 +17,12 @@ use tokio_util::{ compat::{Compat, FuturesAsyncReadCompatExt}, }; use types::{ - BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BlobSidecar, ChainSpec, DataColumnSidecar, - EmptyBlock, Epoch, EthSpec, EthSpecId, ForkContext, ForkName, LightClientBootstrap, - LightClientBootstrapAltair, LightClientFinalityUpdate, LightClientFinalityUpdateAltair, - LightClientOptimisticUpdate, LightClientOptimisticUpdateAltair, LightClientUpdate, - MainnetEthSpec, MinimalEthSpec, SignedBeaconBlock, + BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BlobSidecar, ChainSpec, DataColumnSidecarFulu, + DataColumnSidecarGloas, EmptyBlock, Epoch, EthSpec, EthSpecId, ForkContext, ForkName, + LightClientBootstrap, LightClientBootstrapAltair, LightClientFinalityUpdate, + LightClientFinalityUpdateAltair, LightClientOptimisticUpdate, + LightClientOptimisticUpdateAltair, LightClientUpdate, MainnetEthSpec, MinimalEthSpec, + SignedBeaconBlock, }; // Note: Hardcoding the `EthSpec` type for `SignedBeaconBlock` as min/max values is @@ -640,10 +641,23 @@ pub fn rpc_data_column_limits( current_digest_epoch: Epoch, spec: &ChainSpec, ) -> RpcLimits { - RpcLimits::new( - DataColumnSidecar::::min_size(), - DataColumnSidecar::::max_size(spec.max_blobs_per_block(current_digest_epoch) as usize), - ) + let fork_name = spec.fork_name_at_epoch(current_digest_epoch); + + if fork_name.gloas_enabled() { + RpcLimits::new( + DataColumnSidecarGloas::::min_size(), + DataColumnSidecarGloas::::max_size( + spec.max_blobs_per_block(current_digest_epoch) as usize + ), + ) + } else { + RpcLimits::new( + DataColumnSidecarFulu::::min_size(), + DataColumnSidecarFulu::::max_size( + spec.max_blobs_per_block(current_digest_epoch) as usize + ), + ) + } } /* Inbound upgrade */ diff --git a/beacon_node/lighthouse_network/src/service/gossip_cache.rs b/beacon_node/lighthouse_network/src/service/gossip_cache.rs index 120b9e6c245..d09a4a969c0 100644 --- a/beacon_node/lighthouse_network/src/service/gossip_cache.rs +++ b/beacon_node/lighthouse_network/src/service/gossip_cache.rs @@ -20,8 +20,6 @@ pub struct GossipCache { topic_msgs: HashMap, Key>>, /// Timeout for blocks. beacon_block: Option, - /// Timeout for blobs. - blob_sidecar: Option, /// Timeout for data columns. data_column_sidecar: Option, /// Timeout for aggregate attestations. @@ -51,8 +49,6 @@ pub struct GossipCacheBuilder { default_timeout: Option, /// Timeout for blocks. beacon_block: Option, - /// Timeout for blob sidecars. - blob_sidecar: Option, /// Timeout for data column sidecars. data_column_sidecar: Option, /// Timeout for aggregate attestations. @@ -155,7 +151,6 @@ impl GossipCacheBuilder { let GossipCacheBuilder { default_timeout, beacon_block, - blob_sidecar, data_column_sidecar, aggregates, attestation, @@ -172,7 +167,6 @@ impl GossipCacheBuilder { expirations: DelayQueue::default(), topic_msgs: HashMap::default(), beacon_block: beacon_block.or(default_timeout), - blob_sidecar: blob_sidecar.or(default_timeout), data_column_sidecar: data_column_sidecar.or(default_timeout), aggregates: aggregates.or(default_timeout), attestation: attestation.or(default_timeout), @@ -199,7 +193,6 @@ impl GossipCache { pub fn insert(&mut self, topic: GossipTopic, data: Vec) { let expire_timeout = match topic.kind() { GossipKind::BeaconBlock => self.beacon_block, - GossipKind::BlobSidecar(_) => self.blob_sidecar, GossipKind::DataColumnSidecar(_) => self.data_column_sidecar, GossipKind::BeaconAggregateAndProof => self.aggregates, GossipKind::Attestation(_) => self.attestation, diff --git a/beacon_node/lighthouse_network/src/service/utils.rs b/beacon_node/lighthouse_network/src/service/utils.rs index 63f22be5e2c..c1d732bd24c 100644 --- a/beacon_node/lighthouse_network/src/service/utils.rs +++ b/beacon_node/lighthouse_network/src/service/utils.rs @@ -281,10 +281,6 @@ pub(crate) fn create_whitelist_filter( for id in 0..sync_committee_subnet_count { add(SyncCommitteeMessage(SyncSubnetId::new(id))); } - let blob_subnet_count = spec.blob_sidecar_subnet_count_max(); - for id in 0..blob_subnet_count { - add(BlobSidecar(id)); - } for id in 0..spec.data_column_sidecar_subnet_count { add(DataColumnSidecar(DataColumnSubnetId::new(id))); } diff --git a/beacon_node/lighthouse_network/src/types/pubsub.rs b/beacon_node/lighthouse_network/src/types/pubsub.rs index 72f2873def9..6d4f301de96 100644 --- a/beacon_node/lighthouse_network/src/types/pubsub.rs +++ b/beacon_node/lighthouse_network/src/types/pubsub.rs @@ -7,11 +7,11 @@ use ssz::{Decode, Encode}; use std::io::{Error, ErrorKind}; use std::sync::Arc; use types::{ - AttesterSlashing, AttesterSlashingBase, AttesterSlashingElectra, BlobSidecar, - DataColumnSidecar, DataColumnSubnetId, EthSpec, ForkContext, ForkName, - LightClientFinalityUpdate, LightClientOptimisticUpdate, ProposerSlashing, - SignedAggregateAndProof, SignedAggregateAndProofBase, SignedAggregateAndProofElectra, - SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockBellatrix, + AttesterSlashing, AttesterSlashingBase, AttesterSlashingElectra, DataColumnSidecar, + DataColumnSubnetId, EthSpec, ForkContext, ForkName, LightClientFinalityUpdate, + LightClientOptimisticUpdate, ProposerSlashing, SignedAggregateAndProof, + SignedAggregateAndProofBase, SignedAggregateAndProofElectra, SignedBeaconBlock, + SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockBellatrix, SignedBeaconBlockCapella, SignedBeaconBlockDeneb, SignedBeaconBlockElectra, SignedBeaconBlockFulu, SignedBeaconBlockGloas, SignedBlsToExecutionChange, SignedContributionAndProof, SignedVoluntaryExit, SingleAttestation, SubnetId, @@ -22,8 +22,6 @@ use types::{ pub enum PubsubMessage { /// Gossipsub message providing notification of a new block. BeaconBlock(Arc>), - /// Gossipsub message providing notification of a [`BlobSidecar`] along with the subnet id where it was received. - BlobSidecar(Box<(u64, Arc>)>), /// Gossipsub message providing notification of a [`DataColumnSidecar`] along with the subnet id where it was received. DataColumnSidecar(Box<(DataColumnSubnetId, Arc>)>), /// Gossipsub message providing notification of a Aggregate attestation and associated proof. @@ -129,9 +127,6 @@ impl PubsubMessage { pub fn kind(&self) -> GossipKind { match self { PubsubMessage::BeaconBlock(_) => GossipKind::BeaconBlock, - PubsubMessage::BlobSidecar(blob_sidecar_data) => { - GossipKind::BlobSidecar(blob_sidecar_data.0) - } PubsubMessage::DataColumnSidecar(column_sidecar_data) => { GossipKind::DataColumnSidecar(column_sidecar_data.0) } @@ -252,31 +247,11 @@ impl PubsubMessage { }; Ok(PubsubMessage::BeaconBlock(Arc::new(beacon_block))) } - GossipKind::BlobSidecar(blob_index) => { - if let Some(fork_name) = - fork_context.get_fork_from_context_bytes(gossip_topic.fork_digest) - && fork_name.deneb_enabled() - { - let blob_sidecar = Arc::new( - BlobSidecar::from_ssz_bytes(data) - .map_err(|e| format!("{:?}", e))?, - ); - return Ok(PubsubMessage::BlobSidecar(Box::new(( - *blob_index, - blob_sidecar, - )))); - } - - Err(format!( - "beacon_blobs_and_sidecar topic invalid for given fork digest {:?}", - gossip_topic.fork_digest - )) - } GossipKind::DataColumnSidecar(subnet_id) => { match fork_context.get_fork_from_context_bytes(gossip_topic.fork_digest) { Some(fork) if fork.fulu_enabled() => { let col_sidecar = Arc::new( - DataColumnSidecar::from_ssz_bytes(data) + DataColumnSidecar::from_ssz_bytes_for_fork(data, *fork) .map_err(|e| format!("{:?}", e))?, ); Ok(PubsubMessage::DataColumnSidecar(Box::new(( @@ -401,7 +376,6 @@ impl PubsubMessage { // messages for us. match &self { PubsubMessage::BeaconBlock(data) => data.as_ssz_bytes(), - PubsubMessage::BlobSidecar(data) => data.1.as_ssz_bytes(), PubsubMessage::DataColumnSidecar(data) => data.1.as_ssz_bytes(), PubsubMessage::AggregateAndProofAttestation(data) => data.as_ssz_bytes(), PubsubMessage::VoluntaryExit(data) => data.as_ssz_bytes(), @@ -426,17 +400,11 @@ impl std::fmt::Display for PubsubMessage { block.slot(), block.message().proposer_index() ), - PubsubMessage::BlobSidecar(data) => write!( - f, - "BlobSidecar: slot: {}, blob index: {}", - data.1.slot(), - data.1.index, - ), PubsubMessage::DataColumnSidecar(data) => write!( f, "DataColumnSidecar: slot: {}, column index: {}", data.1.slot(), - data.1.index, + data.1.index(), ), PubsubMessage::AggregateAndProofAttestation(att) => write!( f, diff --git a/beacon_node/lighthouse_network/src/types/topics.rs b/beacon_node/lighthouse_network/src/types/topics.rs index 0c988f35c39..861d33ff688 100644 --- a/beacon_node/lighthouse_network/src/types/topics.rs +++ b/beacon_node/lighthouse_network/src/types/topics.rs @@ -15,7 +15,6 @@ pub const SSZ_SNAPPY_ENCODING_POSTFIX: &str = "ssz_snappy"; pub const BEACON_BLOCK_TOPIC: &str = "beacon_block"; pub const BEACON_AGGREGATE_AND_PROOF_TOPIC: &str = "beacon_aggregate_and_proof"; pub const BEACON_ATTESTATION_PREFIX: &str = "beacon_attestation_"; -pub const BLOB_SIDECAR_PREFIX: &str = "blob_sidecar_"; pub const DATA_COLUMN_SIDECAR_PREFIX: &str = "data_column_sidecar_"; pub const VOLUNTARY_EXIT_TOPIC: &str = "voluntary_exit"; pub const PROPOSER_SLASHING_TOPIC: &str = "proposer_slashing"; @@ -72,13 +71,6 @@ pub fn core_topics_to_subscribe( topics.push(GossipKind::BlsToExecutionChange); } - if fork_name.deneb_enabled() && !fork_name.fulu_enabled() { - // All of deneb blob topics are core topics - for i in 0..spec.blob_sidecar_subnet_count(fork_name) { - topics.push(GossipKind::BlobSidecar(i)); - } - } - if fork_name.fulu_enabled() { for subnet in &opts.sampling_subnets { topics.push(GossipKind::DataColumnSidecar(*subnet)); @@ -101,7 +93,6 @@ pub fn is_fork_non_core_topic(topic: &GossipTopic, _fork_name: ForkName) -> bool // All these topics are core-only GossipKind::BeaconBlock | GossipKind::BeaconAggregateAndProof - | GossipKind::BlobSidecar(_) | GossipKind::DataColumnSidecar(_) | GossipKind::VoluntaryExit | GossipKind::ProposerSlashing @@ -145,8 +136,6 @@ pub enum GossipKind { BeaconBlock, /// Topic for publishing aggregate attestations and proofs. BeaconAggregateAndProof, - /// Topic for publishing BlobSidecars. - BlobSidecar(u64), /// Topic for publishing DataColumnSidecars. DataColumnSidecar(DataColumnSubnetId), /// Topic for publishing raw attestations on a particular subnet. @@ -178,9 +167,6 @@ impl std::fmt::Display for GossipKind { GossipKind::SyncCommitteeMessage(subnet_id) => { write!(f, "sync_committee_{}", **subnet_id) } - GossipKind::BlobSidecar(blob_index) => { - write!(f, "{}{}", BLOB_SIDECAR_PREFIX, blob_index) - } GossipKind::DataColumnSidecar(column_subnet_id) => { write!(f, "{}{}", DATA_COLUMN_SIDECAR_PREFIX, **column_subnet_id) } @@ -307,9 +293,6 @@ impl std::fmt::Display for GossipTopic { GossipKind::SyncCommitteeMessage(index) => { format!("{}{}", SYNC_COMMITTEE_PREFIX_TOPIC, *index) } - GossipKind::BlobSidecar(blob_index) => { - format!("{}{}", BLOB_SIDECAR_PREFIX, blob_index) - } GossipKind::DataColumnSidecar(column_subnet_id) => { format!("{}{}", DATA_COLUMN_SIDECAR_PREFIX, *column_subnet_id) } @@ -355,8 +338,6 @@ fn subnet_topic_index(topic: &str) -> Option { return Some(GossipKind::SyncCommitteeMessage(SyncSubnetId::new( index.parse::().ok()?, ))); - } else if let Some(index) = topic.strip_prefix(BLOB_SIDECAR_PREFIX) { - return Some(GossipKind::BlobSidecar(index.parse::().ok()?)); } else if let Some(index) = topic.strip_prefix(DATA_COLUMN_SIDECAR_PREFIX) { return Some(GossipKind::DataColumnSidecar(DataColumnSubnetId::new( index.parse::().ok()?, @@ -530,17 +511,6 @@ mod tests { } } - #[test] - fn blobs_are_not_subscribed_in_peerdas() { - let spec = get_spec(); - let s = get_sampling_subnets(); - let topic_config = get_topic_config(&s); - assert!( - !core_topics_to_subscribe::(ForkName::Fulu, &topic_config, &spec,) - .contains(&GossipKind::BlobSidecar(0)) - ); - } - #[test] fn columns_are_subscribed_in_peerdas() { let spec = get_spec(); diff --git a/beacon_node/lighthouse_network/tests/rpc_tests.rs b/beacon_node/lighthouse_network/tests/rpc_tests.rs index 2a17a04b905..d68b69c6581 100644 --- a/beacon_node/lighthouse_network/tests/rpc_tests.rs +++ b/beacon_node/lighthouse_network/tests/rpc_tests.rs @@ -17,9 +17,9 @@ use tokio::time::sleep; use tracing::{Instrument, debug, error, info, info_span, warn}; use types::{ BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockBellatrix, BeaconBlockHeader, - BlobSidecar, ChainSpec, DataColumnSidecar, DataColumnsByRootIdentifier, EmptyBlock, Epoch, - EthSpec, ForkName, Hash256, KzgCommitment, KzgProof, MinimalEthSpec, SignedBeaconBlock, - SignedBeaconBlockHeader, Slot, + BlobSidecar, ChainSpec, DataColumnSidecar, DataColumnSidecarFulu, DataColumnsByRootIdentifier, + EmptyBlock, Epoch, EthSpec, ForkName, Hash256, KzgCommitment, KzgProof, MinimalEthSpec, + SignedBeaconBlock, SignedBeaconBlockHeader, Slot, }; type E = MinimalEthSpec; @@ -1007,7 +1007,7 @@ fn test_tcp_columns_by_root_chunked_rpc() { let rpc_request = RequestType::DataColumnsByRoot(req); // DataColumnsByRoot Response - let data_column = Arc::new(DataColumnSidecar { + let data_column = Arc::new(DataColumnSidecar::Fulu(DataColumnSidecarFulu { index: 1, signed_block_header: SignedBeaconBlockHeader { message: BeaconBlockHeader { @@ -1030,7 +1030,7 @@ fn test_tcp_columns_by_root_chunked_rpc() { ] .try_into() .unwrap(), - }); + })); let rpc_response = Response::DataColumnsByRoot(Some(data_column.clone())); @@ -1152,7 +1152,7 @@ fn test_tcp_columns_by_range_chunked_rpc() { }); // DataColumnsByRange Response - let data_column = Arc::new(DataColumnSidecar { + let data_column = Arc::new(DataColumnSidecar::Fulu(DataColumnSidecarFulu { index: 1, signed_block_header: SignedBeaconBlockHeader { message: BeaconBlockHeader { @@ -1175,7 +1175,7 @@ fn test_tcp_columns_by_range_chunked_rpc() { ] .try_into() .unwrap(), - }); + })); let rpc_response = Response::DataColumnsByRange(Some(data_column.clone())); diff --git a/beacon_node/network/src/metrics.rs b/beacon_node/network/src/metrics.rs index cea06a28c86..89223617bfa 100644 --- a/beacon_node/network/src/metrics.rs +++ b/beacon_node/network/src/metrics.rs @@ -128,13 +128,6 @@ pub static BEACON_PROCESSOR_GOSSIP_BLOCK_EARLY_SECONDS: LazyLock> = - LazyLock::new(|| { - try_create_int_counter( - "beacon_processor_gossip_blob_verified_total", - "Total number of gossip blob verified for propagation.", - ) - }); pub static BEACON_PROCESSOR_GOSSIP_DATA_COLUMN_SIDECAR_VERIFIED_TOTAL: LazyLock< Result, > = LazyLock::new(|| { @@ -543,12 +536,6 @@ pub static BEACON_BLOCK_DELAY_GOSSIP_ARRIVED_LATE_TOTAL: LazyLock> = LazyLock::new(|| { - try_create_int_gauge( - "beacon_blob_delay_gossip_last_delay", - "The first time we see this blob as a delay from the start of the slot", - ) -}); pub static BEACON_DATA_COLUMN_GOSSIP_PROPAGATION_VERIFICATION_DELAY_TIME: LazyLock< Result, @@ -575,14 +562,6 @@ pub static BEACON_DATA_COLUMN_GOSSIP_SLOT_START_DELAY_TIME: LazyLock> = LazyLock::new( - || { - try_create_int_gauge( - "beacon_blob_delay_gossip_verification", - "Keeps track of the time delay from the start of the slot to the point we propagate the blob", - ) - }, -); pub static BEACON_BLOB_DELAY_FULL_VERIFICATION: LazyLock> = LazyLock::new(|| { try_create_int_gauge( "beacon_blob_last_full_verification_delay", @@ -606,15 +585,6 @@ pub static BEACON_BLOB_RPC_SLOT_START_DELAY_TIME: LazyLock> = }, ); -pub static BEACON_BLOB_GOSSIP_ARRIVED_LATE_TOTAL: LazyLock> = LazyLock::new( - || { - try_create_int_counter( - "beacon_blob_gossip_arrived_late_total", - "Count of times when a gossip blob arrived from the network later than the attestation deadline.", - ) - }, -); - /* * Light client update reprocessing queue metrics. */ diff --git a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs index ca259129348..3c44380b23b 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -4,7 +4,6 @@ use crate::{ service::NetworkMessage, sync::SyncMessage, }; -use beacon_chain::blob_verification::{GossipBlobError, GossipVerifiedBlob}; use beacon_chain::block_verification_types::AsBlock; use beacon_chain::data_column_verification::{GossipDataColumnError, GossipVerifiedDataColumn}; use beacon_chain::store::Error; @@ -21,9 +20,7 @@ use beacon_chain::{ }; use beacon_processor::{Work, WorkEvent}; use lighthouse_network::{Client, MessageAcceptance, MessageId, PeerAction, PeerId, ReportSource}; -use lighthouse_tracing::{ - SPAN_PROCESS_GOSSIP_BLOB, SPAN_PROCESS_GOSSIP_BLOCK, SPAN_PROCESS_GOSSIP_DATA_COLUMN, -}; +use lighthouse_tracing::{SPAN_PROCESS_GOSSIP_BLOCK, SPAN_PROCESS_GOSSIP_DATA_COLUMN}; use logging::crit; use operation_pool::ReceivedPreCapella; use slot_clock::SlotClock; @@ -36,7 +33,7 @@ use std::time::{Duration, Instant, SystemTime, UNIX_EPOCH}; use store::hot_cold_store::HotColdDBError; use tracing::{Instrument, Span, debug, error, info, instrument, trace, warn}; use types::{ - Attestation, AttestationData, AttestationRef, AttesterSlashing, BlobSidecar, DataColumnSidecar, + Attestation, AttestationData, AttestationRef, AttesterSlashing, DataColumnSidecar, DataColumnSubnetId, EthSpec, Hash256, IndexedAttestation, LightClientFinalityUpdate, LightClientOptimisticUpdate, ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedBlsToExecutionChange, SignedContributionAndProof, SignedVoluntaryExit, SingleAttestation, @@ -609,7 +606,7 @@ impl NetworkBeaconProcessor { parent = None, level = "debug", skip_all, - fields(slot = %column_sidecar.slot(), block_root = ?column_sidecar.block_root(), index = column_sidecar.index), + fields(slot = %column_sidecar.slot(), block_root = ?column_sidecar.block_root(), index = column_sidecar.index()), )] pub async fn process_gossip_data_column_sidecar( self: &Arc, @@ -621,7 +618,7 @@ impl NetworkBeaconProcessor { ) { let slot = column_sidecar.slot(); let block_root = column_sidecar.block_root(); - let index = column_sidecar.index; + let index = *column_sidecar.index(); let delay = get_slot_delay_ms(seen_duration, slot, &self.chain.slot_clock); // Log metrics to track delay from other nodes on the network. metrics::observe_duration( @@ -767,252 +764,6 @@ impl NetworkBeaconProcessor { } } - #[allow(clippy::too_many_arguments)] - #[instrument( - name = SPAN_PROCESS_GOSSIP_BLOB, - parent = None, - level = "debug", - skip_all, - fields( - slot = ?blob_sidecar.slot(), - block_root = ?blob_sidecar.block_root(), - index = blob_sidecar.index), - )] - pub async fn process_gossip_blob( - self: &Arc, - message_id: MessageId, - peer_id: PeerId, - _peer_client: Client, - blob_index: u64, - blob_sidecar: Arc>, - seen_duration: Duration, - ) { - let slot = blob_sidecar.slot(); - let root = blob_sidecar.block_root(); - let index = blob_sidecar.index; - let commitment = blob_sidecar.kzg_commitment; - let delay = get_slot_delay_ms(seen_duration, slot, &self.chain.slot_clock); - // Log metrics to track delay from other nodes on the network. - metrics::set_gauge(&metrics::BEACON_BLOB_DELAY_GOSSIP, delay.as_millis() as i64); - match self - .chain - .verify_blob_sidecar_for_gossip(blob_sidecar.clone(), blob_index) - { - Ok(gossip_verified_blob) => { - metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOB_VERIFIED_TOTAL); - - if delay >= self.chain.slot_clock.unagg_attestation_production_delay() { - metrics::inc_counter(&metrics::BEACON_BLOB_GOSSIP_ARRIVED_LATE_TOTAL); - debug!( - block_root = ?gossip_verified_blob.block_root(), - proposer_index = gossip_verified_blob.block_proposer_index(), - slot = %gossip_verified_blob.slot(), - delay = ?delay, - commitment = %gossip_verified_blob.kzg_commitment(), - "Gossip blob arrived late" - ); - } - - debug!( - %slot, - %root, - %index, - commitment = %gossip_verified_blob.kzg_commitment(), - "Successfully verified gossip blob" - ); - - self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Accept); - - // Log metrics to keep track of propagation delay times. - if let Some(duration) = SystemTime::now() - .duration_since(UNIX_EPOCH) - .ok() - .and_then(|now| now.checked_sub(seen_duration)) - { - metrics::set_gauge( - &metrics::BEACON_BLOB_DELAY_GOSSIP_VERIFICATION, - duration.as_millis() as i64, - ); - } - self.process_gossip_verified_blob(peer_id, gossip_verified_blob, seen_duration) - .await - } - Err(err) => { - match err { - GossipBlobError::ParentUnknown { parent_root } => { - debug!( - action = "requesting parent", - block_root = %root, - parent_root = %parent_root, - %commitment, - "Unknown parent hash for blob" - ); - self.send_sync_message(SyncMessage::UnknownParentBlob( - peer_id, - blob_sidecar, - )); - } - GossipBlobError::PubkeyCacheTimeout | GossipBlobError::BeaconChainError(_) => { - crit!( - error = ?err, - "Internal error when verifying blob sidecar" - ) - } - GossipBlobError::ProposalSignatureInvalid - | GossipBlobError::UnknownValidator(_) - | GossipBlobError::ProposerIndexMismatch { .. } - | GossipBlobError::BlobIsNotLaterThanParent { .. } - | GossipBlobError::InvalidSubnet { .. } - | GossipBlobError::InvalidInclusionProof - | GossipBlobError::KzgError(_) - | GossipBlobError::NotFinalizedDescendant { .. } => { - warn!( - error = ?err, - %slot, - %root, - %index, - %commitment, - "Could not verify blob sidecar for gossip. Rejecting the blob sidecar" - ); - // Prevent recurring behaviour by penalizing the peer. - self.gossip_penalize_peer( - peer_id, - PeerAction::LowToleranceError, - "gossip_blob_low", - ); - self.propagate_validation_result( - message_id, - peer_id, - MessageAcceptance::Reject, - ); - } - GossipBlobError::RepeatBlob { .. } => { - // We may have received the blob from the EL. Do not penalise the peer. - // Gossip filter should filter any duplicates received after this. - debug!( - %slot, - %root, - %index, - "Received already available blob sidecar. Ignoring the blob sidecar" - ) - } - GossipBlobError::FutureSlot { .. } => { - debug!( - error = ?err, - %slot, - %root, - %index, - %commitment, - "Could not verify blob sidecar for gossip. Ignoring the blob sidecar" - ); - // Prevent recurring behaviour by penalizing the peer slightly. - self.gossip_penalize_peer( - peer_id, - PeerAction::HighToleranceError, - "gossip_blob_high", - ); - self.propagate_validation_result( - message_id, - peer_id, - MessageAcceptance::Ignore, - ); - } - GossipBlobError::PastFinalizedSlot { .. } => { - debug!( - error = ?err, - %slot, - %root, - %index, - %commitment, - "Could not verify blob sidecar for gossip. Ignoring the blob sidecar" - ); - // Prevent recurring behaviour by penalizing the peer. A low-tolerance - // error is fine because there's no reason for peers to be propagating old - // blobs on gossip, even if their view of finality is lagging. - self.gossip_penalize_peer( - peer_id, - PeerAction::LowToleranceError, - "gossip_blob_low", - ); - self.propagate_validation_result( - message_id, - peer_id, - MessageAcceptance::Ignore, - ); - } - } - } - } - } - - async fn process_gossip_verified_blob( - self: &Arc, - peer_id: PeerId, - verified_blob: GossipVerifiedBlob, - _seen_duration: Duration, - ) { - let processing_start_time = Instant::now(); - let block_root = verified_blob.block_root(); - let blob_slot = verified_blob.slot(); - let blob_index = verified_blob.id().index; - - let result = self.chain.process_gossip_blob(verified_blob).await; - register_process_result_metrics(&result, metrics::BlockSource::Gossip, "blob"); - - match &result { - Ok(AvailabilityProcessingStatus::Imported(block_root)) => { - debug!( - %block_root, - "Gossipsub blob processed - imported fully available block" - ); - self.chain.recompute_head_at_current_slot().await; - - metrics::set_gauge( - &metrics::BEACON_BLOB_DELAY_FULL_VERIFICATION, - processing_start_time.elapsed().as_millis() as i64, - ); - } - Ok(AvailabilityProcessingStatus::MissingComponents(slot, block_root)) => { - debug!( - %slot, - %blob_index, - %block_root, - "Processed gossip blob - waiting for other components" - ); - } - Err(BlockError::DuplicateFullyImported(_)) => { - debug!( - ?block_root, - blob_index, "Ignoring gossip blob already imported" - ); - } - Err(err) => { - debug!( - outcome = ?err, - ?block_root, - %blob_slot, - blob_index, - "Invalid gossip blob" - ); - self.gossip_penalize_peer( - peer_id, - PeerAction::MidToleranceError, - "bad_gossip_blob_ssz", - ); - } - } - - // If a block is in the da_checker, sync maybe awaiting for an event when block is finally - // imported. A block can become imported both after processing a block or blob. If a - // importing a block results in `Imported`, notify. Do not notify of blob errors. - if matches!(result, Ok(AvailabilityProcessingStatus::Imported(_))) { - self.send_sync_message(SyncMessage::GossipBlockProcessResult { - block_root, - imported: true, - }); - } - } - async fn process_gossip_verified_data_column( self: &Arc, peer_id: PeerId, diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index fd9c2c1e55c..b30d40f4a1f 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -1,6 +1,5 @@ use crate::sync::manager::BlockProcessType; use crate::{service::NetworkMessage, sync::manager::SyncMessage}; -use beacon_chain::blob_verification::{GossipBlobError, observe_gossip_blob}; use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::data_column_verification::{GossipDataColumnError, observe_gossip_data_column}; use beacon_chain::fetch_blobs::{ @@ -63,9 +62,6 @@ pub struct NetworkBeaconProcessor { pub executor: TaskExecutor, } -// Publish blobs in batches of exponentially increasing size. -const BLOB_PUBLICATION_EXP_FACTOR: usize = 2; - impl NetworkBeaconProcessor { fn try_send(&self, event: BeaconWorkEvent) -> Result<(), Error> { self.beacon_processor_send.try_send(event) @@ -191,36 +187,6 @@ impl NetworkBeaconProcessor { }) } - /// Create a new `Work` event for some blob sidecar. - pub fn send_gossip_blob_sidecar( - self: &Arc, - message_id: MessageId, - peer_id: PeerId, - peer_client: Client, - blob_index: u64, - blob_sidecar: Arc>, - seen_timestamp: Duration, - ) -> Result<(), Error> { - let processor = self.clone(); - let process_fn = async move { - processor - .process_gossip_blob( - message_id, - peer_id, - peer_client, - blob_index, - blob_sidecar, - seen_timestamp, - ) - .await - }; - - self.try_send(BeaconWorkEvent { - drop_during_sync: false, - work: Work::GossipBlobSidecar(Box::pin(process_fn)), - }) - } - /// Create a new `Work` event for some data column sidecar. pub fn send_gossip_data_column_sidecar( self: &Arc, @@ -767,12 +733,6 @@ impl NetworkBeaconProcessor { let publish_fn = move |blobs_or_data_column| { if publish_blobs { match blobs_or_data_column { - EngineGetBlobsOutput::Blobs(blobs) => { - self_cloned.publish_blobs_gradually( - blobs.into_iter().map(|b| b.to_blob()).collect(), - block_root, - ); - } EngineGetBlobsOutput::CustodyColumns(columns) => { self_cloned.publish_data_columns_gradually( columns.into_iter().map(|c| c.clone_arc()).collect(), @@ -879,84 +839,6 @@ impl NetworkBeaconProcessor { } } - /// This function gradually publishes blobs to the network in randomised batches. - /// - /// This is an optimisation to reduce outbound bandwidth and ensures each blob is published - /// by some nodes on the network as soon as possible. Our hope is that some blobs arrive from - /// other nodes in the meantime, obviating the need for us to publish them. If no other - /// publisher exists for a blob, it will eventually get published here. - fn publish_blobs_gradually( - self: &Arc, - mut blobs: Vec>>, - block_root: Hash256, - ) { - let self_clone = self.clone(); - - self.executor.spawn( - async move { - let chain = self_clone.chain.clone(); - let publish_fn = |blobs: Vec>>| { - self_clone.send_network_message(NetworkMessage::Publish { - messages: blobs - .into_iter() - .map(|blob| PubsubMessage::BlobSidecar(Box::new((blob.index, blob)))) - .collect(), - }); - }; - - // Permute the blobs and split them into batches. - // The hope is that we won't need to publish some blobs because we will receive them - // on gossip from other nodes. - blobs.shuffle(&mut rand::rng()); - - let blob_publication_batch_interval = chain.config.blob_publication_batch_interval; - let mut publish_count = 0usize; - let blob_count = blobs.len(); - let mut blobs_iter = blobs.into_iter().peekable(); - let mut batch_size = 1usize; - - while blobs_iter.peek().is_some() { - let batch = blobs_iter.by_ref().take(batch_size); - let publishable = batch - .filter_map(|blob| match observe_gossip_blob(&blob, &chain) { - Ok(()) => Some(blob), - Err(GossipBlobError::RepeatBlob { .. }) => None, - Err(e) => { - warn!( - error = ?e, - "Previously verified blob is invalid" - ); - None - } - }) - .collect::>(); - - if !publishable.is_empty() { - debug!( - publish_count = publishable.len(), - ?block_root, - "Publishing blob batch" - ); - publish_count += publishable.len(); - publish_fn(publishable); - } - - tokio::time::sleep(blob_publication_batch_interval).await; - batch_size *= BLOB_PUBLICATION_EXP_FACTOR; - } - - debug!( - batch_interval = blob_publication_batch_interval.as_millis(), - blob_count, - publish_count, - ?block_root, - "Batch blob publication complete" - ) - }, - "gradual_blob_publication", - ); - } - /// This function gradually publishes data columns to the network in randomised batches. /// /// This is an optimisation to reduce outbound bandwidth and ensures each column is published @@ -980,7 +862,7 @@ impl NetworkBeaconProcessor { .into_iter() .map(|d| { let subnet = - DataColumnSubnetId::from_column_index(d.index, &chain.spec); + DataColumnSubnetId::from_column_index(*d.index(), &chain.spec); PubsubMessage::DataColumnSidecar(Box::new((subnet, d))) }) .collect(), diff --git a/beacon_node/network/src/network_beacon_processor/rpc_methods.rs b/beacon_node/network/src/network_beacon_processor/rpc_methods.rs index e443eb78d89..8d58e40ae6c 100644 --- a/beacon_node/network/src/network_beacon_processor/rpc_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/rpc_methods.rs @@ -753,7 +753,10 @@ impl NetworkBeaconProcessor { ) .ok_or((RpcErrorResponse::ServerError, "shutting down"))? .await - .map_err(|_| (RpcErrorResponse::ServerError, "tokio join"))??; + .map_err(|_| (RpcErrorResponse::ServerError, "tokio join"))?? + .iter() + .map(|(root, _)| *root) + .collect::>(); let current_slot = self .chain @@ -866,7 +869,7 @@ impl NetworkBeaconProcessor { req_start_slot: u64, req_count: u64, req_type: &str, - ) -> Result, (RpcErrorResponse, &'static str)> { + ) -> Result, (RpcErrorResponse, &'static str)> { let start_time = std::time::Instant::now(); let finalized_slot = self .chain @@ -876,7 +879,7 @@ impl NetworkBeaconProcessor { .epoch .start_slot(T::EthSpec::slots_per_epoch()); - let (block_roots, source) = if req_start_slot >= finalized_slot.as_u64() { + let (block_roots_and_slots, source) = if req_start_slot >= finalized_slot.as_u64() { // If the entire requested range is after finalization, use fork_choice ( self.chain @@ -920,14 +923,14 @@ impl NetworkBeaconProcessor { req_type, start_slot = %req_start_slot, req_count, - roots_count = block_roots.len(), + roots_count = block_roots_and_slots.len(), source, elapsed = ?elapsed, %finalized_slot, "Range request block roots retrieved" ); - Ok(block_roots) + Ok(block_roots_and_slots) } /// Get block roots for a `BlocksByRangeRequest` from the store using roots iterator. @@ -935,7 +938,7 @@ impl NetworkBeaconProcessor { &self, start_slot: u64, count: u64, - ) -> Result, (RpcErrorResponse, &'static str)> { + ) -> Result, (RpcErrorResponse, &'static str)> { let forwards_block_root_iter = match self.chain.forwards_iter_block_roots(Slot::from(start_slot)) { Ok(iter) => iter, @@ -981,11 +984,7 @@ impl NetworkBeaconProcessor { }; // remove all skip slots i.e. duplicated roots - Ok(block_roots - .into_iter() - .map(|(root, _)| root) - .unique() - .collect::>()) + Ok(block_roots.into_iter().unique().collect::>()) } /// Handle a `BlobsByRange` request from the peer. @@ -1092,7 +1091,7 @@ impl NetworkBeaconProcessor { }; } - let block_roots = + let block_roots_and_slots = self.get_block_roots_for_slot_range(req.start_slot, effective_count, "BlobsByRange")?; let current_slot = self @@ -1113,7 +1112,7 @@ impl NetworkBeaconProcessor { let mut blobs_sent = 0; - for root in block_roots { + for (root, _) in block_roots_and_slots { match self.chain.get_blobs(&root) { Ok(blob_sidecar_list) => { for blob_sidecar in blob_sidecar_list.iter() { @@ -1252,7 +1251,7 @@ impl NetworkBeaconProcessor { }; } - let block_roots = + let block_roots_and_slots = self.get_block_roots_for_slot_range(req.start_slot, req.count, "DataColumnsByRange")?; let mut data_columns_sent = 0; @@ -1269,9 +1268,10 @@ impl NetworkBeaconProcessor { .filter(|c| available_columns.contains(c)) .collect::>(); - for root in block_roots { + for (root, slot) in block_roots_and_slots { + let fork_name = self.chain.spec.fork_name_at_slot::(slot); for index in &indices_to_retrieve { - match self.chain.get_data_column(&root, index) { + match self.chain.get_data_column(&root, index, fork_name) { Ok(Some(data_column_sidecar)) => { // Due to skip slots, data columns could be out of the range, we ensure they // are in the range before sending diff --git a/beacon_node/network/src/network_beacon_processor/sync_methods.rs b/beacon_node/network/src/network_beacon_processor/sync_methods.rs index 38de6dffada..146bb90965c 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -374,7 +374,10 @@ impl NetworkBeaconProcessor { metrics::observe_duration(&metrics::BEACON_BLOB_RPC_SLOT_START_DELAY_TIME, delay); } - let mut indices = custody_columns.iter().map(|d| d.index).collect::>(); + let mut indices = custody_columns + .iter() + .map(|d| *d.index()) + .collect::>(); indices.sort_unstable(); debug!( ?indices, diff --git a/beacon_node/network/src/network_beacon_processor/tests.rs b/beacon_node/network/src/network_beacon_processor/tests.rs index 49da522c9a1..8bd8636310e 100644 --- a/beacon_node/network/src/network_beacon_processor/tests.rs +++ b/beacon_node/network/src/network_beacon_processor/tests.rs @@ -10,7 +10,7 @@ use crate::{ }; use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::custody_context::NodeCustodyType; -use beacon_chain::data_column_verification::validate_data_column_sidecar_for_gossip; +use beacon_chain::data_column_verification::validate_data_column_sidecar_for_gossip_fulu; use beacon_chain::kzg_utils::blobs_to_data_column_sidecars; use beacon_chain::observed_data_sidecars::DoNotObserve; use beacon_chain::test_utils::{ @@ -39,12 +39,14 @@ use std::iter::Iterator; use std::sync::Arc; use std::time::Duration; use tokio::sync::mpsc; -use types::data::{BlobIdentifier, FixedBlobSidecarList}; use types::{ - AttesterSlashing, BlobSidecar, BlobSidecarList, ChainSpec, DataColumnSidecarList, - DataColumnSubnetId, Epoch, EthSpec, Hash256, MainnetEthSpec, ProposerSlashing, - SignedAggregateAndProof, SignedBeaconBlock, SignedVoluntaryExit, SingleAttestation, Slot, - SubnetId, + AttesterSlashing, BlobSidecar, ChainSpec, DataColumnSidecarList, DataColumnSubnetId, Epoch, + EthSpec, Hash256, MainnetEthSpec, ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, + SignedVoluntaryExit, SingleAttestation, Slot, SubnetId, +}; +use types::{ + BlobSidecarList, + data::{BlobIdentifier, FixedBlobSidecarList}, }; type E = MainnetEthSpec; @@ -309,7 +311,7 @@ impl TestRig { ) .unwrap() .into_iter() - .filter(|c| sampling_indices.contains(&c.index)) + .filter(|c| sampling_indices.contains(c.index())) .collect::>(); (None, Some(custody_columns)) @@ -363,22 +365,6 @@ impl TestRig { .unwrap(); } - pub fn enqueue_gossip_blob(&self, blob_index: usize) { - if let Some(blobs) = self.next_blobs.as_ref() { - let blob = blobs.get(blob_index).unwrap(); - self.network_beacon_processor - .send_gossip_blob_sidecar( - junk_message_id(), - junk_peer_id(), - Client::default(), - blob.index, - blob.clone(), - Duration::from_secs(0), - ) - .unwrap(); - } - } - pub fn enqueue_gossip_data_columns(&self, col_index: usize) { if let Some(data_columns) = self.next_data_columns.as_ref() { let data_column = data_columns.get(col_index).unwrap(); @@ -386,7 +372,7 @@ impl TestRig { .send_gossip_data_column_sidecar( junk_message_id(), junk_peer_id(), - DataColumnSubnetId::from_column_index(data_column.index, &self.chain.spec), + DataColumnSubnetId::from_column_index(*data_column.index(), &self.chain.spec), data_column.clone(), Duration::from_secs(0), ) @@ -1022,13 +1008,6 @@ async fn import_gossip_block_acceptably_early() { rig.assert_event_journal_completes(&[WorkType::GossipBlock]) .await; - let num_blobs = rig.next_blobs.as_ref().map(|b| b.len()).unwrap_or(0); - for i in 0..num_blobs { - rig.enqueue_gossip_blob(i); - rig.assert_event_journal_completes(&[WorkType::GossipBlobSidecar]) - .await; - } - let num_data_columns = rig.next_data_columns.as_ref().map(|c| c.len()).unwrap_or(0); for i in 0..num_data_columns { rig.enqueue_gossip_data_columns(i); @@ -1115,8 +1094,8 @@ async fn accept_processed_gossip_data_columns_without_import() { .into_iter() .map(|data_column| { let subnet_id = - DataColumnSubnetId::from_column_index(data_column.index, &rig.chain.spec); - validate_data_column_sidecar_for_gossip::<_, DoNotObserve>( + DataColumnSubnetId::from_column_index(*data_column.index(), &rig.chain.spec); + validate_data_column_sidecar_for_gossip_fulu::<_, DoNotObserve>( data_column, subnet_id, &rig.chain, @@ -1167,13 +1146,6 @@ async fn import_gossip_block_at_current_slot() { rig.assert_event_journal_completes(&[WorkType::GossipBlock]) .await; - let num_blobs = rig.next_blobs.as_ref().map(|b| b.len()).unwrap_or(0); - for i in 0..num_blobs { - rig.enqueue_gossip_blob(i); - rig.assert_event_journal_completes(&[WorkType::GossipBlobSidecar]) - .await; - } - let num_data_columns = rig.next_data_columns.as_ref().map(|c| c.len()).unwrap_or(0); for i in 0..num_data_columns { rig.enqueue_gossip_data_columns(i); @@ -1240,10 +1212,6 @@ async fn attestation_to_unknown_block_processed(import_method: BlockImportMethod BlockImportMethod::Gossip => { rig.enqueue_gossip_block(); events.push(WorkType::GossipBlock); - for i in 0..num_blobs { - rig.enqueue_gossip_blob(i); - events.push(WorkType::GossipBlobSidecar); - } for i in 0..num_data_columns { rig.enqueue_gossip_data_columns(i); events.push(WorkType::GossipDataColumnSidecar); @@ -1326,10 +1294,6 @@ async fn aggregate_attestation_to_unknown_block(import_method: BlockImportMethod BlockImportMethod::Gossip => { rig.enqueue_gossip_block(); events.push(WorkType::GossipBlock); - for i in 0..num_blobs { - rig.enqueue_gossip_blob(i); - events.push(WorkType::GossipBlobSidecar); - } for i in 0..num_data_columns { rig.enqueue_gossip_data_columns(i); events.push(WorkType::GossipDataColumnSidecar) @@ -1948,7 +1912,7 @@ async fn test_data_columns_by_range_request_only_returns_requested_columns() { } = next { if let Some(column) = data_column { - received_columns.push(column.index); + received_columns.push(*column.index()); } else { break; } diff --git a/beacon_node/network/src/router.rs b/beacon_node/network/src/router.rs index 60fe094bb7c..fb043d0be5f 100644 --- a/beacon_node/network/src/router.rs +++ b/beacon_node/network/src/router.rs @@ -358,19 +358,6 @@ impl Router { timestamp_now(), ), ), - PubsubMessage::BlobSidecar(data) => { - let (blob_index, blob_sidecar) = *data; - self.handle_beacon_processor_send_result( - self.network_beacon_processor.send_gossip_blob_sidecar( - message_id, - peer_id, - self.network_globals.client(&peer_id), - blob_index, - blob_sidecar, - timestamp_now(), - ), - ) - } PubsubMessage::DataColumnSidecar(data) => { let (subnet_id, column_sidecar) = *data; self.handle_beacon_processor_send_result( diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index f8ffd298caf..8579b31b319 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -45,7 +45,7 @@ use std::sync::Arc; use std::time::Duration; use store::Hash256; use tracing::{debug, error, warn}; -use types::{BlobSidecar, DataColumnSidecar, EthSpec, SignedBeaconBlock}; +use types::{DataColumnSidecar, EthSpec, SignedBeaconBlock}; pub mod common; pub mod parent_chain; @@ -79,7 +79,6 @@ const MAX_LOOKUPS: usize = 200; pub enum BlockComponent { Block(DownloadResult>>), - Blob(DownloadResult>>), DataColumn(DownloadResult>>), } @@ -87,14 +86,16 @@ impl BlockComponent { fn parent_root(&self) -> Hash256 { match self { BlockComponent::Block(block) => block.value.parent_root(), - BlockComponent::Blob(blob) => blob.value.block_parent_root(), - BlockComponent::DataColumn(column) => column.value.block_parent_root(), + BlockComponent::DataColumn(column) => match column.value.as_ref() { + DataColumnSidecar::Fulu(column) => column.block_parent_root(), + // TODO(gloas) we don't have a parent root post gloas, not sure what to do here + DataColumnSidecar::Gloas(column) => column.beacon_block_root, + }, } } fn get_type(&self) -> &'static str { match self { BlockComponent::Block(_) => "block", - BlockComponent::Blob(_) => "blob", BlockComponent::DataColumn(_) => "data_column", } } diff --git a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs index fea8794b2e1..550ddd9b3c3 100644 --- a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs @@ -151,10 +151,10 @@ impl SingleBlockLookup { .block_request_state .state .insert_verified_response(block), - BlockComponent::Blob(_) | BlockComponent::DataColumn(_) => { - // For now ignore single blobs and columns, as the blob request state assumes all blobs are - // attributed to the same peer = the peer serving the remaining blobs. Ignoring this - // block component has a minor effect, causing the node to re-request this blob + BlockComponent::DataColumn(_) => { + // For now ignore single columns, as the column request state assumes all columns are + // attributed to the same peer = the peer serving the remaining columns. Ignoring this + // block component has a minor effect, causing the node to re-request this column // once the parent chain is successfully resolved false } diff --git a/beacon_node/network/src/sync/block_sidecar_coupling.rs b/beacon_node/network/src/sync/block_sidecar_coupling.rs index 6f563820f7c..27e334fa104 100644 --- a/beacon_node/network/src/sync/block_sidecar_coupling.rs +++ b/beacon_node/network/src/sync/block_sidecar_coupling.rs @@ -346,7 +346,7 @@ impl RangeBlockComponentsRequest { for column in data_columns { let block_root = column.block_root(); - let index = column.index; + let index = *column.index(); if data_columns_by_block .entry(block_root) .or_default() @@ -624,7 +624,7 @@ mod tests { *req, blocks .iter() - .flat_map(|b| b.1.iter().filter(|d| d.index == column_index).cloned()) + .flat_map(|b| b.1.iter().filter(|d| *d.index() == column_index).cloned()) .collect(), ) .unwrap(); @@ -707,7 +707,7 @@ mod tests { .iter() .flat_map(|b| { b.1.iter() - .filter(|d| column_indices.contains(&d.index)) + .filter(|d| column_indices.contains(d.index())) .cloned() }) .collect::>(), @@ -779,7 +779,7 @@ mod tests { *req, blocks .iter() - .flat_map(|b| b.1.iter().filter(|d| d.index == column_index).cloned()) + .flat_map(|b| b.1.iter().filter(|d| *d.index() == column_index).cloned()) .collect(), ) .unwrap(); @@ -864,7 +864,7 @@ mod tests { *req1, blocks .iter() - .flat_map(|b| b.1.iter().filter(|d| d.index == 1).cloned()) + .flat_map(|b| b.1.iter().filter(|d| *d.index() == 1).cloned()) .collect(), ) .unwrap(); @@ -891,7 +891,7 @@ mod tests { new_columns_req_id, blocks .iter() - .flat_map(|b| b.1.iter().filter(|d| d.index == 2).cloned()) + .flat_map(|b| b.1.iter().filter(|d| *d.index() == 2).cloned()) .collect(), ) .unwrap(); @@ -957,7 +957,7 @@ mod tests { *req1, blocks .iter() - .flat_map(|b| b.1.iter().filter(|d| d.index == 1).cloned()) + .flat_map(|b| b.1.iter().filter(|d| *d.index() == 1).cloned()) .collect(), ) .unwrap(); diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 338f21ce987..c82f3be3087 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -135,9 +135,6 @@ pub enum SyncMessage { /// A block with an unknown parent has been received. UnknownParentBlock(PeerId, Arc>, Hash256), - /// A blob with an unknown parent has been received. - UnknownParentBlob(PeerId, Arc>), - /// A data column with an unknown parent has been received. UnknownParentDataColumn(PeerId, Arc>), @@ -850,41 +847,29 @@ impl SyncManager { }), ); } - SyncMessage::UnknownParentBlob(peer_id, blob) => { - let blob_slot = blob.slot(); - let block_root = blob.block_root(); - let parent_root = blob.block_parent_root(); - debug!(%block_root, %parent_root, "Received unknown parent blob message"); - self.handle_unknown_parent( - peer_id, - block_root, - parent_root, - blob_slot, - BlockComponent::Blob(DownloadResult { - value: blob, - block_root, - seen_timestamp: timestamp_now(), - peer_group: PeerGroup::from_single(peer_id), - }), - ); - } SyncMessage::UnknownParentDataColumn(peer_id, data_column) => { let data_column_slot = data_column.slot(); let block_root = data_column.block_root(); - let parent_root = data_column.block_parent_root(); - debug!(%block_root, %parent_root, "Received unknown parent data column message"); - self.handle_unknown_parent( - peer_id, - block_root, - parent_root, - data_column_slot, - BlockComponent::DataColumn(DownloadResult { - value: data_column, - block_root, - seen_timestamp: timestamp_now(), - peer_group: PeerGroup::from_single(peer_id), - }), - ); + match data_column.as_ref() { + DataColumnSidecar::Fulu(column) => { + let parent_root = column.block_parent_root(); + debug!(%block_root, %parent_root, "Received unknown parent data column message"); + self.handle_unknown_parent( + peer_id, + block_root, + parent_root, + data_column_slot, + BlockComponent::DataColumn(DownloadResult { + value: data_column, + block_root, + seen_timestamp: timestamp_now(), + peer_group: PeerGroup::from_single(peer_id), + }), + ); + } + // TODO(gloas) + DataColumnSidecar::Gloas(_) => todo!(), + } } SyncMessage::UnknownBlockHashFromAttestation(peer_id, block_root) => { if !self.notified_unknown_roots.contains(&(peer_id, block_root)) { diff --git a/beacon_node/network/src/sync/network_context/custody.rs b/beacon_node/network/src/sync/network_context/custody.rs index fe0924fae86..b91028a8f6d 100644 --- a/beacon_node/network/src/sync/network_context/custody.rs +++ b/beacon_node/network/src/sync/network_context/custody.rs @@ -128,7 +128,7 @@ impl ActiveCustodyRequest { // requested index. The worse case is 128 loops over a 128 item vec + mutation to // drop the consumed columns. let mut data_columns = HashMap::::from_iter( - data_columns.into_iter().map(|d| (d.index, d)), + data_columns.into_iter().map(|d| (*d.index(), d)), ); // Accumulate columns that the peer does not have to issue a single log per request let mut missing_column_indexes = vec![]; @@ -210,7 +210,7 @@ impl ActiveCustodyRequest { peers .entry(peer) .or_default() - .push(data_column.index as usize); + .push(*data_column.index() as usize); seen_timestamps.push(seen_timestamp); Ok(data_column) }) diff --git a/beacon_node/network/src/sync/network_context/requests/blobs_by_range.rs b/beacon_node/network/src/sync/network_context/requests/blobs_by_range.rs index 9c6f516199c..1da0fb52f74 100644 --- a/beacon_node/network/src/sync/network_context/requests/blobs_by_range.rs +++ b/beacon_node/network/src/sync/network_context/requests/blobs_by_range.rs @@ -33,6 +33,7 @@ impl ActiveRequestItems for BlobsByRangeRequestItems { if blob.index >= self.max_blobs_per_block { return Err(LookupVerifyError::UnrequestedIndex(blob.index)); } + if !blob.verify_blob_sidecar_inclusion_proof() { return Err(LookupVerifyError::InvalidInclusionProof); } diff --git a/beacon_node/network/src/sync/network_context/requests/blobs_by_root.rs b/beacon_node/network/src/sync/network_context/requests/blobs_by_root.rs index 556985c2b48..f0ff99867b3 100644 --- a/beacon_node/network/src/sync/network_context/requests/blobs_by_root.rs +++ b/beacon_node/network/src/sync/network_context/requests/blobs_by_root.rs @@ -50,9 +50,11 @@ impl ActiveRequestItems for BlobsByRootRequestItems { if self.request.block_root != block_root { return Err(LookupVerifyError::UnrequestedBlockRoot(block_root)); } + if !blob.verify_blob_sidecar_inclusion_proof() { return Err(LookupVerifyError::InvalidInclusionProof); } + if !self.request.indices.contains(&blob.index) { return Err(LookupVerifyError::UnrequestedIndex(blob.index)); } diff --git a/beacon_node/network/src/sync/network_context/requests/data_columns_by_range.rs b/beacon_node/network/src/sync/network_context/requests/data_columns_by_range.rs index 9dabb2defa0..74bdd0a1143 100644 --- a/beacon_node/network/src/sync/network_context/requests/data_columns_by_range.rs +++ b/beacon_node/network/src/sync/network_context/requests/data_columns_by_range.rs @@ -28,18 +28,22 @@ impl ActiveRequestItems for DataColumnsByRangeRequestItems { { return Err(LookupVerifyError::UnrequestedSlot(data_column.slot())); } - if !self.request.columns.contains(&data_column.index) { - return Err(LookupVerifyError::UnrequestedIndex(data_column.index)); + if !self.request.columns.contains(data_column.index()) { + return Err(LookupVerifyError::UnrequestedIndex(*data_column.index())); } - if !data_column.verify_inclusion_proof() { + + if let DataColumnSidecar::Fulu(data_column) = data_column.as_ref() + && !data_column.verify_inclusion_proof() + { return Err(LookupVerifyError::InvalidInclusionProof); } + if self.items.iter().any(|existing| { - existing.slot() == data_column.slot() && existing.index == data_column.index + existing.slot() == data_column.slot() && *existing.index() == *data_column.index() }) { return Err(LookupVerifyError::DuplicatedData( data_column.slot(), - data_column.index, + *data_column.index(), )); } diff --git a/beacon_node/network/src/sync/network_context/requests/data_columns_by_root.rs b/beacon_node/network/src/sync/network_context/requests/data_columns_by_root.rs index 34df801eaa8..5ad0f377c10 100644 --- a/beacon_node/network/src/sync/network_context/requests/data_columns_by_root.rs +++ b/beacon_node/network/src/sync/network_context/requests/data_columns_by_root.rs @@ -56,16 +56,24 @@ impl ActiveRequestItems for DataColumnsByRootRequestItems { if self.request.block_root != block_root { return Err(LookupVerifyError::UnrequestedBlockRoot(block_root)); } - if !data_column.verify_inclusion_proof() { + + if let DataColumnSidecar::Fulu(data_column) = data_column.as_ref() + && !data_column.verify_inclusion_proof() + { return Err(LookupVerifyError::InvalidInclusionProof); } - if !self.request.indices.contains(&data_column.index) { - return Err(LookupVerifyError::UnrequestedIndex(data_column.index)); + + if !self.request.indices.contains(data_column.index()) { + return Err(LookupVerifyError::UnrequestedIndex(*data_column.index())); } - if self.items.iter().any(|d| d.index == data_column.index) { + if self + .items + .iter() + .any(|d| *d.index() == *data_column.index()) + { return Err(LookupVerifyError::DuplicatedData( data_column.slot(), - data_column.index, + *data_column.index(), )); } diff --git a/beacon_node/network/src/sync/range_data_column_batch_request.rs b/beacon_node/network/src/sync/range_data_column_batch_request.rs index b912a6badc9..b3d963657ee 100644 --- a/beacon_node/network/src/sync/range_data_column_batch_request.rs +++ b/beacon_node/network/src/sync/range_data_column_batch_request.rs @@ -191,7 +191,12 @@ impl RangeDataColumnBatchRequest { let column_block_signatures = columns .iter() - .map(|column| column.signed_block_header.signature.clone()) + .filter_map(|column| match column.as_ref() { + DataColumnSidecar::Fulu(column) => { + Some(column.signed_block_header.signature.clone()) + } + _ => None, + }) .unique() .collect::>(); @@ -201,8 +206,8 @@ impl RangeDataColumnBatchRequest { // If there are no block roots, penalize all peers [] => { for column in &columns { - if let Some(naughty_peer) = column_to_peer.get(&column.index) { - naughty_peers.push((column.index, *naughty_peer)); + if let Some(naughty_peer) = column_to_peer.get(column.index()) { + naughty_peers.push((*column.index(), *naughty_peer)); } } continue; @@ -212,9 +217,9 @@ impl RangeDataColumnBatchRequest { for column in columns { if column_block_roots.contains(&column.block_root()) && block_root != column.block_root() - && let Some(naughty_peer) = column_to_peer.get(&column.index) + && let Some(naughty_peer) = column_to_peer.get(column.index()) { - naughty_peers.push((column.index, *naughty_peer)); + naughty_peers.push((*column.index(), *naughty_peer)); } } continue; @@ -227,17 +232,19 @@ impl RangeDataColumnBatchRequest { // If there are no block signatures, penalize all peers [] => { for column in &columns { - if let Some(naughty_peer) = column_to_peer.get(&column.index) { - naughty_peers.push((column.index, *naughty_peer)); + if let Some(naughty_peer) = column_to_peer.get(column.index()) { + naughty_peers.push((*column.index(), *naughty_peer)); } } continue; } // If theres more than one unique block signature, penalize the peers serving the - // invalid block signatures. + // invalid block signatures. This check is only relevant for Fulu. column_block_signatures => { for column in columns { - if column_block_signatures.contains(&column.signed_block_header.signature) + if let DataColumnSidecar::Fulu(column) = column.as_ref() + && column_block_signatures + .contains(&column.signed_block_header.signature) && block.signature() != &column.signed_block_header.signature && let Some(naughty_peer) = column_to_peer.get(&column.index) { @@ -251,8 +258,8 @@ impl RangeDataColumnBatchRequest { // if the block root doesn't match the columns block root, penalize the peers if block_root != column_block_root { for column in &columns { - if let Some(naughty_peer) = column_to_peer.get(&column.index) { - naughty_peers.push((column.index, *naughty_peer)); + if let Some(naughty_peer) = column_to_peer.get(column.index()) { + naughty_peers.push((*column.index(), *naughty_peer)); } } } @@ -260,13 +267,13 @@ impl RangeDataColumnBatchRequest { // If the block signature doesn't match the columns block signature, penalize the peers if block.signature() != column_block_signature { for column in &columns { - if let Some(naughty_peer) = column_to_peer.get(&column.index) { - naughty_peers.push((column.index, *naughty_peer)); + if let Some(naughty_peer) = column_to_peer.get(column.index()) { + naughty_peers.push((*column.index(), *naughty_peer)); } } } - let received_columns = columns.iter().map(|c| c.index).collect::>(); + let received_columns = columns.iter().map(|c| *c.index()).collect::>(); let missing_columns = expected_custody_columns .difference(&received_columns) diff --git a/beacon_node/network/src/sync/tests/lookups.rs b/beacon_node/network/src/sync/tests/lookups.rs index 715928906ee..86ac8e2e7a6 100644 --- a/beacon_node/network/src/sync/tests/lookups.rs +++ b/beacon_node/network/src/sync/tests/lookups.rs @@ -13,15 +13,16 @@ use std::time::Duration; use super::*; use crate::sync::block_lookups::common::ResponseType; +use beacon_chain::data_column_verification::GossipVerifiedDataColumn; use beacon_chain::observed_data_sidecars::Observe; use beacon_chain::{ AvailabilityPendingExecutedBlock, AvailabilityProcessingStatus, BlockError, PayloadVerificationOutcome, PayloadVerificationStatus, - blob_verification::GossipVerifiedBlob, block_verification_types::{AsBlock, BlockImportData}, data_availability_checker::Availability, test_utils::{ - BeaconChainHarness, EphemeralHarnessType, NumBlobs, generate_rand_block_and_blobs, + BeaconChainHarness, EphemeralHarnessType, NumBlobs, + generate_data_column_sidecars_from_block, generate_rand_block_and_blobs, generate_rand_block_and_data_columns, test_spec, }, validator_monitor::timestamp_now, @@ -40,9 +41,10 @@ use lighthouse_network::{ use slot_clock::{SlotClock, TestingSlotClock}; use tokio::sync::mpsc; use tracing::info; +use types::BlobSidecar; use types::{ - BeaconState, BeaconStateBase, BlobSidecar, BlockImportSource, DataColumnSidecar, EthSpec, - ForkContext, ForkName, Hash256, MinimalEthSpec as E, SignedBeaconBlock, Slot, + BeaconState, BeaconStateBase, BlockImportSource, DataColumnSidecar, EthSpec, ForkContext, + ForkName, Hash256, MinimalEthSpec as E, SignedBeaconBlock, Slot, data::ColumnIndex, test_utils::{SeedableRng, TestRandom, XorShiftRng}, }; @@ -101,8 +103,6 @@ impl TestRig { .network_globals .set_sync_state(SyncState::Synced); - let spec = chain.spec.clone(); - // deterministic seed let rng_08 = ::from_seed([0u8; 32]); let rng = ChaCha20Rng::from_seed([0u8; 32]); @@ -128,7 +128,6 @@ impl TestRig { ), harness, fork_name, - spec, } } @@ -167,8 +166,12 @@ impl TestRig { self.send_sync_message(SyncMessage::UnknownParentBlock(peer_id, block, block_root)) } - fn trigger_unknown_parent_blob(&mut self, peer_id: PeerId, blob: BlobSidecar) { - self.send_sync_message(SyncMessage::UnknownParentBlob(peer_id, blob.into())); + fn trigger_unknown_parent_data_column( + &mut self, + peer_id: PeerId, + data_column: Arc>, + ) { + self.send_sync_message(SyncMessage::UnknownParentDataColumn(peer_id, data_column)); } fn trigger_unknown_block_from_attestation(&mut self, block_root: Hash256, peer_id: PeerId) { @@ -396,9 +399,9 @@ impl TestRig { self.single_block_component_processed(id, result); } - fn parent_blob_processed(&mut self, chain_hash: Hash256, result: BlockProcessingResult) { + fn parent_column_processed(&mut self, chain_hash: Hash256, result: BlockProcessingResult) { let id = self.find_single_lookup_for(self.find_oldest_parent_lookup(chain_hash)); - self.single_blob_component_processed(id, result); + self.single_custody_column_component_processed(id, result); } fn parent_block_processed_imported(&mut self, chain_hash: Hash256) { @@ -430,6 +433,13 @@ impl TestRig { }) } + fn single_custody_column_component_processed(&mut self, id: Id, result: BlockProcessingResult) { + self.send_sync_message(SyncMessage::BlockComponentProcessed { + process_type: BlockProcessType::SingleCustodyColumn(id), + result, + }) + } + fn parent_lookup_block_response( &mut self, id: SingleLookupReqId, @@ -460,24 +470,6 @@ impl TestRig { }); } - fn parent_lookup_blob_response( - &mut self, - id: SingleLookupReqId, - peer_id: PeerId, - blob_sidecar: Option>>, - ) { - self.log(&format!( - "parent_lookup_blob_response {:?}", - blob_sidecar.as_ref().map(|b| b.index) - )); - self.send_sync_message(SyncMessage::RpcBlob { - sync_request_id: SyncRequestId::SingleBlob { id }, - peer_id, - blob_sidecar, - seen_timestamp: D, - }); - } - fn single_lookup_blob_response( &mut self, id: SingleLookupReqId, @@ -840,26 +832,6 @@ impl TestRig { } } - #[track_caller] - fn expect_blob_parent_request(&mut self, for_block: Hash256) -> SingleLookupReqId { - self.pop_received_network_event(|ev| match ev { - NetworkMessage::SendRequest { - peer_id: _, - request: RequestType::BlobsByRoot(request), - app_request_id: AppRequestId::Sync(SyncRequestId::SingleBlob { id }), - } if request - .blob_ids - .to_vec() - .iter() - .all(|r| r.block_root == for_block) => - { - Some(*id) - } - _ => None, - }) - .unwrap_or_else(|e| panic!("Expected blob parent request for {for_block:?}: {e}")) - } - /// Retrieves an unknown number of requests for data columns of `block_root`. Because peer ENRs /// are random, and peer selection is random, the total number of batched requests is unknown. fn expect_data_columns_by_root_requests( @@ -1021,17 +993,14 @@ impl TestRig { self.log(&format!("Found expected penalty {penalty_msg}")); } - pub fn block_with_parent_and_blobs( + pub fn block_with_parent_and_data_columns( &mut self, parent_root: Hash256, - num_blobs: NumBlobs, - ) -> (SignedBeaconBlock, Vec>) { - let (mut block, mut blobs) = self.rand_block_and_blobs(num_blobs); + ) -> (SignedBeaconBlock, Vec>>) { + let (mut block, _) = self.rand_block_and_data_columns(); *block.message_mut().parent_root_mut() = parent_root; - blobs.iter_mut().for_each(|blob| { - blob.signed_block_header = block.signed_block_header(); - }); - (block, blobs) + let data_columns = generate_data_column_sidecars_from_block(&block, &self.harness.spec); + (block, data_columns) } pub fn rand_blockchain(&mut self, depth: usize) -> Vec>> { @@ -1089,22 +1058,23 @@ impl TestRig { }; } - fn insert_blob_to_da_checker(&mut self, blob: BlobSidecar) { + fn insert_column_to_da_checker(&mut self, column: Arc>) { match self .harness .chain .data_availability_checker - .put_gossip_verified_blobs( - blob.block_root(), - std::iter::once(GossipVerifiedBlob::<_, Observe>::__assumed_valid( - blob.into(), + .put_gossip_verified_data_columns( + column.block_root(), + column.slot(), + std::iter::once(GossipVerifiedDataColumn::<_, Observe>::__assumed_valid( + column, )), ) .unwrap() { - Availability::Available(_) => panic!("blob removed from da_checker, available"), + Availability::Available(_) => panic!("column removed from da_checker, available"), Availability::MissingComponents(block_root) => { - self.log(&format!("inserted blob to da_checker {block_root:?}")) + self.log(&format!("inserted column to da_checker {block_root:?}")) } }; } @@ -1877,15 +1847,15 @@ fn block_in_processing_cache_becomes_valid_imported() { // IGNORE: wait for change that delays blob fetching to knowing the block #[ignore] #[test] -fn blobs_in_da_checker_skip_download() { +fn columns_in_da_checker_skip_download() { let Some(mut r) = TestRig::test_setup_after_deneb_before_fulu() else { return; }; - let (block, blobs) = r.rand_block_and_blobs(NumBlobs::Number(1)); + let (block, columns) = r.rand_block_and_data_columns(); let block_root = block.canonical_root(); let peer_id = r.new_connected_peer(); - for blob in blobs { - r.insert_blob_to_da_checker(blob); + for column in columns { + r.insert_column_to_da_checker(column); } r.trigger_unknown_block_from_attestation(block_root, peer_id); // Should download and process the block @@ -1923,37 +1893,34 @@ fn custody_lookup_happy_path() { // - Respond with stream terminator // ^ The stream terminator should be ignored and not close the next retry -mod deneb_only { +mod fulu_only { use super::*; - use beacon_chain::{ - block_verification_types::{AsBlock, RpcBlock}, - data_availability_checker::AvailabilityCheckError, - }; - use ssz_types::RuntimeVariableList; + use beacon_chain::data_availability_checker::AvailabilityCheckError; use std::collections::VecDeque; - struct DenebTester { + struct FuluTester { rig: TestRig, block: Arc>, - blobs: Vec>>, + data_columns: Vec>>, parent_block_roots: Vec, parent_block: VecDeque>>, - parent_blobs: VecDeque>>>, + parent_data_columns: VecDeque>>>, unknown_parent_block: Option>>, - unknown_parent_blobs: Option>>>, + unknown_parent_data_columns: Option>>>, peer_id: PeerId, block_req_id: Option, parent_block_req_id: Option, - blob_req_id: Option, - parent_blob_req_id: Option, + column_req_ids: Option, + parent_column_req_ids: Option, slot: Slot, block_root: Hash256, + sample_column_count: usize, } enum RequestTrigger { AttestationUnknownBlock, GossipUnknownParentBlock(usize), - GossipUnknownParentBlob(usize), + GossipUnknownParentDataColumn(usize), } impl RequestTrigger { @@ -1961,97 +1928,103 @@ mod deneb_only { match self { RequestTrigger::AttestationUnknownBlock => 0, RequestTrigger::GossipUnknownParentBlock(num_parents) => *num_parents, - RequestTrigger::GossipUnknownParentBlob(num_parents) => *num_parents, + RequestTrigger::GossipUnknownParentDataColumn(num_parents) => *num_parents, } } } - impl DenebTester { + impl FuluTester { fn new(request_trigger: RequestTrigger) -> Option { - let Some(mut rig) = TestRig::test_setup_after_deneb_before_fulu() else { + let Some(mut rig) = TestRig::test_setup_after_fulu() else { return None; }; - let (block, blobs) = rig.rand_block_and_blobs(NumBlobs::Random); + rig.new_connected_peers_for_peerdas(); + + let spec = E::default_spec(); + let sample_column_count = + (spec.samples_per_slot * spec.data_columns_per_group::()) as usize; + + let (block, data_columns) = rig.rand_block_and_data_columns(); let mut block = Arc::new(block); - let mut blobs = blobs.into_iter().map(Arc::new).collect::>(); + let mut data_columns = data_columns; let slot = block.slot(); let num_parents = request_trigger.num_parents(); let mut parent_block_chain = VecDeque::with_capacity(num_parents); - let mut parent_blobs_chain = VecDeque::with_capacity(num_parents); + let mut parent_data_columns_chain = VecDeque::with_capacity(num_parents); let mut parent_block_roots = vec![]; for _ in 0..num_parents { - // Set the current block as the parent. + // Set the current block as the parent. let parent_root = block.canonical_root(); let parent_block = block.clone(); - let parent_blobs = blobs.clone(); + let parent_columns = data_columns.clone(); parent_block_chain.push_front(parent_block); - parent_blobs_chain.push_front(parent_blobs); + parent_data_columns_chain.push_front(parent_columns); parent_block_roots.push(parent_root); // Create the next block. - let (child_block, child_blobs) = - rig.block_with_parent_and_blobs(parent_root, NumBlobs::Random); + let (child_block, child_data_columns) = + rig.block_with_parent_and_data_columns(parent_root); let mut child_block = Arc::new(child_block); - let mut child_blobs = child_blobs.into_iter().map(Arc::new).collect::>(); + let mut child_data_columns = child_data_columns; // Update the new block to the current block. std::mem::swap(&mut child_block, &mut block); - std::mem::swap(&mut child_blobs, &mut blobs); + std::mem::swap(&mut child_data_columns, &mut data_columns); } let block_root = block.canonical_root(); let peer_id = rig.new_connected_peer(); // Trigger the request - let (block_req_id, blob_req_id, parent_block_req_id, parent_blob_req_id) = - match request_trigger { - RequestTrigger::AttestationUnknownBlock => { - rig.send_sync_message(SyncMessage::UnknownBlockHashFromAttestation( - peer_id, block_root, - )); - let block_req_id = rig.expect_block_lookup_request(block_root); - (Some(block_req_id), None, None, None) - } - RequestTrigger::GossipUnknownParentBlock { .. } => { - rig.send_sync_message(SyncMessage::UnknownParentBlock( - peer_id, - block.clone(), - block_root, - )); - - let parent_root = block.parent_root(); - let parent_block_req_id = rig.expect_block_parent_request(parent_root); - rig.expect_empty_network(); // expect no more requests - (None, None, Some(parent_block_req_id), None) - } - RequestTrigger::GossipUnknownParentBlob { .. } => { - let single_blob = blobs.first().cloned().unwrap(); - let parent_root = single_blob.block_parent_root(); - rig.send_sync_message(SyncMessage::UnknownParentBlob(peer_id, single_blob)); - - let parent_block_req_id = rig.expect_block_parent_request(parent_root); - rig.expect_empty_network(); // expect no more requests - (None, None, Some(parent_block_req_id), None) - } - }; + let (block_req_id, parent_block_req_id) = match request_trigger { + RequestTrigger::AttestationUnknownBlock => { + rig.send_sync_message(SyncMessage::UnknownBlockHashFromAttestation( + peer_id, block_root, + )); + let block_req_id = rig.expect_block_lookup_request(block_root); + (Some(block_req_id), None) + } + RequestTrigger::GossipUnknownParentBlock { .. } => { + rig.send_sync_message(SyncMessage::UnknownParentBlock( + peer_id, + block.clone(), + block_root, + )); + + let parent_root = block.parent_root(); + let parent_block_req_id = rig.expect_block_parent_request(parent_root); + rig.expect_empty_network(); // expect no more requests + (None, Some(parent_block_req_id)) + } + RequestTrigger::GossipUnknownParentDataColumn { .. } => { + let single_column = data_columns.first().cloned().unwrap(); + let parent_root = single_column.as_fulu().unwrap().block_parent_root(); + rig.trigger_unknown_parent_data_column(peer_id, single_column); + + let parent_block_req_id = rig.expect_block_parent_request(parent_root); + rig.expect_empty_network(); // expect no more requests + (None, Some(parent_block_req_id)) + } + }; Some(Self { rig, block, - blobs, + data_columns, parent_block: parent_block_chain, - parent_blobs: parent_blobs_chain, + parent_data_columns: parent_data_columns_chain, parent_block_roots, unknown_parent_block: None, - unknown_parent_blobs: None, + unknown_parent_data_columns: None, peer_id, block_req_id, parent_block_req_id, - blob_req_id, - parent_blob_req_id, + column_req_ids: None, + parent_column_req_ids: None, slot, block_root, + sample_column_count, }) } @@ -2076,9 +2049,10 @@ mod deneb_only { self } - fn parent_block_response_expect_blobs(mut self) -> Self { + fn parent_block_response_expect_columns(mut self) -> Self { self.rig.expect_empty_network(); let block = self.parent_block.pop_front().unwrap().clone(); + let block_root = block.canonical_root(); let _ = self.unknown_parent_block.insert(block.clone()); self.rig.parent_lookup_block_response( self.parent_block_req_id.expect("parent request id"), @@ -2086,35 +2060,40 @@ mod deneb_only { Some(block), ); - // Expect blobs request after sending block - let s = self.expect_parent_blobs_request(); + // Expect data columns request after sending block + let s = self.expect_parent_columns_request(block_root); s.rig.assert_parent_lookups_count(1); s } - fn parent_blob_response(mut self) -> Self { - let blobs = self.parent_blobs.pop_front().unwrap(); - let _ = self.unknown_parent_blobs.insert(blobs.clone()); - for blob in &blobs { - self.rig.parent_lookup_blob_response( - self.parent_blob_req_id.expect("parent blob request id"), - self.peer_id, - Some(blob.clone()), - ); - assert_eq!(self.rig.active_parent_lookups_count(), 1); + fn parent_column_response(mut self) -> Self { + let data_columns = self.parent_data_columns.pop_front().unwrap(); + let _ = self + .unknown_parent_data_columns + .insert(data_columns.clone()); + + let column_req_ids = self + .parent_column_req_ids + .take() + .expect("parent column request ids"); + + // Send responses for each custody request + for id in column_req_ids { + let indices = &id.1; + let columns_to_send: Vec<_> = indices + .iter() + .map(|&i| data_columns[i as usize].clone()) + .collect(); + self.rig + .complete_data_columns_by_root_request(id, &columns_to_send); } - self.rig.parent_lookup_blob_response( - self.parent_blob_req_id.expect("parent blob request id"), - self.peer_id, - None, - ); self } fn block_response_triggering_process(self) -> Self { - let mut me = self.block_response_and_expect_blob_request(); + let mut me = self.block_response_and_expect_column_request(); me.rig.expect_block_process(ResponseType::Block); // The request should still be active. @@ -2122,48 +2101,47 @@ mod deneb_only { me } - fn block_response_and_expect_blob_request(mut self) -> Self { - // The peer provides the correct block, should not be penalized. Now the block should be sent - // for processing. + fn block_response_and_expect_column_request(mut self) -> Self { + // The peer provides the correct block, should not be penalized. self.rig.single_lookup_block_response( self.block_req_id.expect("block request id"), self.peer_id, Some(self.block.clone()), ); - // After responding with block the node will issue a blob request - let mut s = self.expect_blobs_request(); - - s.rig.expect_empty_network(); + // After responding with block the node will issue data column requests + let s = self.expect_columns_request(); // The request should still be active. s.rig.assert_lookup_is_active(s.block.canonical_root()); s } - fn blobs_response(mut self) -> Self { - self.rig - .log(&format!("blobs response {}", self.blobs.len())); - for blob in &self.blobs { - self.rig.single_lookup_blob_response( - self.blob_req_id.expect("blob request id"), - self.peer_id, - Some(blob.clone()), - ); + fn data_columns_response(mut self) -> Self { + self.rig.log(&format!( + "data columns response {}", + self.data_columns.len() + )); + + let column_req_ids = self.column_req_ids.take().expect("column request ids"); + + // Send responses for each custody request + for id in column_req_ids { + let indices = &id.1; + let columns_to_send: Vec<_> = indices + .iter() + .map(|&i| self.data_columns[i as usize].clone()) + .collect(); self.rig - .assert_lookup_is_active(self.block.canonical_root()); + .complete_data_columns_by_root_request(id, &columns_to_send); } - self.rig.single_lookup_blob_response( - self.blob_req_id.expect("blob request id"), - self.peer_id, - None, - ); + self } - fn blobs_response_was_valid(mut self) -> Self { + fn data_columns_response_was_valid(mut self) -> Self { self.rig.expect_empty_network(); - if !self.blobs.is_empty() { - self.rig.expect_block_process(ResponseType::Blob); + if !self.data_columns.is_empty() { + self.rig.expect_block_process(ResponseType::CustodyColumn); } self } @@ -2182,21 +2160,37 @@ mod deneb_only { self } - fn empty_blobs_response(mut self) -> Self { - self.rig.single_lookup_blob_response( - self.blob_req_id.expect("blob request id"), - self.peer_id, - None, - ); + fn empty_data_columns_response(mut self) -> Self { + let column_req_ids = self.column_req_ids.take().expect("column request ids"); + + // Send empty responses for each custody request + for (sync_request_id, _) in column_req_ids { + // Send stream termination with no data + self.rig.send_sync_message(SyncMessage::RpcDataColumn { + sync_request_id, + peer_id: self.peer_id, + data_column: None, + seen_timestamp: D, + }); + } self } - fn empty_parent_blobs_response(mut self) -> Self { - self.rig.parent_lookup_blob_response( - self.parent_blob_req_id.expect("blob request id"), - self.peer_id, - None, - ); + fn empty_parent_columns_response(mut self) -> Self { + let column_req_ids = self + .parent_column_req_ids + .take() + .expect("parent column request ids"); + + // Send empty responses for each custody request + for (sync_request_id, _) in column_req_ids { + self.rig.send_sync_message(SyncMessage::RpcDataColumn { + sync_request_id, + peer_id: self.peer_id, + data_column: None, + seen_timestamp: D, + }); + } self } @@ -2213,24 +2207,39 @@ mod deneb_only { self } - fn blob_imported(mut self) -> Self { - self.rig.single_blob_component_processed( - self.blob_req_id.expect("blob request id").lookup_id, - BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(self.block_root)), - ); + fn custody_columns_imported(mut self) -> Self { + let lookup_id = self.get_custody_lookup_id(); + self.rig + .send_sync_message(SyncMessage::BlockComponentProcessed { + process_type: BlockProcessType::SingleCustodyColumn(lookup_id), + result: BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported( + self.block_root, + )), + }); self.rig.expect_empty_network(); self.rig.assert_single_lookups_count(0); self } + fn get_custody_lookup_id(&self) -> Id { + if let Some(ref ids) = self.column_req_ids + && let SyncRequestId::DataColumnsByRoot(DataColumnsByRootRequestId { + requester: DataColumnsByRootRequester::Custody(id), + .. + }) = ids.first().unwrap().0 + { + return id.requester.0.lookup_id; + } + + if let Some(id) = self.block_req_id { + return id.lookup_id; + } + panic!("No custody lookup id found") + } + fn block_imported(mut self) -> Self { - // Missing blobs should be the request is not removed, the outstanding blobs request should - // mean we do not send a new request. self.rig.single_block_component_processed( - self.block_req_id - .or(self.blob_req_id) - .expect("block request id") - .lookup_id, + self.block_req_id.expect("block request id").lookup_id, BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(self.block_root)), ); self.rig.expect_empty_network(); @@ -2266,11 +2275,12 @@ mod deneb_only { self } - fn parent_blob_imported(mut self) -> Self { + fn parent_columns_imported(mut self) -> Self { let parent_root = *self.parent_block_roots.first().unwrap(); self.rig - .log(&format!("parent_blob_imported {parent_root:?}")); - self.rig.parent_blob_processed( + .log(&format!("parent_columns_imported {parent_root:?}")); + + self.rig.parent_column_processed( self.block_root, BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(parent_root)), ); @@ -2283,17 +2293,8 @@ mod deneb_only { fn parent_block_unknown_parent(mut self) -> Self { self.rig.log("parent_block_unknown_parent"); let block = self.unknown_parent_block.take().unwrap(); - let max_len = self.rig.spec.max_blobs_per_block(block.epoch()) as usize; // Now this block is the one we expect requests from self.block = block.clone(); - let block = RpcBlock::new( - Some(block.canonical_root()), - block, - self.unknown_parent_blobs - .take() - .map(|vec| RuntimeVariableList::new(vec, max_len).unwrap()), - ) - .unwrap(); self.rig.parent_block_processed( self.block_root, BlockProcessingResult::Err(BlockError::ParentUnknown { @@ -2322,14 +2323,19 @@ mod deneb_only { self } - fn invalid_blob_processed(mut self) -> Self { - self.rig.log("invalid_blob_processed"); - self.rig.single_blob_component_processed( - self.blob_req_id.expect("blob request id").lookup_id, - BlockProcessingResult::Err(BlockError::AvailabilityCheck( - AvailabilityCheckError::InvalidBlobs(kzg::Error::KzgVerificationFailed), - )), - ); + fn invalid_column_processed(mut self) -> Self { + self.rig.log("invalid_column_processed"); + let lookup_id = self.get_custody_lookup_id(); + self.rig + .send_sync_message(SyncMessage::BlockComponentProcessed { + process_type: BlockProcessType::SingleCustodyColumn(lookup_id), + result: BlockProcessingResult::Err(BlockError::AvailabilityCheck( + AvailabilityCheckError::InvalidColumn(( + None, + kzg::Error::KzgVerificationFailed, + )), + )), + }); self.rig.assert_single_lookups_count(1); self } @@ -2342,18 +2348,17 @@ mod deneb_only { self.block_root, )), ); - // Add block to da_checker so blobs request can continue + // Add block to da_checker so columns request can continue self.rig.insert_block_to_da_checker(self.block.clone()); self.rig.assert_single_lookups_count(1); self } - fn complete_current_block_and_blobs_lookup(self) -> Self { + fn complete_current_block_and_columns_lookup(self) -> Self { self.expect_block_request() - .block_response_and_expect_blob_request() - .blobs_response() - // TODO: Should send blobs for processing + .block_response_and_expect_column_request() + .data_columns_response() .expect_block_process() .block_imported() } @@ -2363,22 +2368,22 @@ mod deneb_only { self } - fn parent_block_then_empty_parent_blobs(self) -> Self { + fn parent_block_then_empty_parent_columns(self) -> Self { self.log( - " Return empty blobs for parent, block errors with missing components, downscore", + " Return empty columns for parent, block errors with missing components, downscore", ) .parent_block_response() - .expect_parent_blobs_request() - .empty_parent_blobs_response() + .expect_parent_columns_request_for_unknown_parent() + .empty_parent_columns_response() .expect_penalty("NotEnoughResponsesReturned") - .log("Re-request parent blobs, succeed and import parent") - .expect_parent_blobs_request() - .parent_blob_response() + .log("Re-request parent columns, succeed and import parent") + .expect_parent_columns_request_for_unknown_parent() + .parent_column_response() .expect_block_process() .parent_block_missing_components() // Insert new peer into child request before completing parent .trigger_unknown_block_from_attestation() - .parent_blob_imported() + .parent_columns_imported() } fn expect_penalty(mut self, expect_penalty_msg: &'static str) -> Self { @@ -2400,11 +2405,12 @@ mod deneb_only { self.block_req_id = Some(id); self } - fn expect_blobs_request(mut self) -> Self { - let id = self - .rig - .expect_blob_lookup_request(self.block.canonical_root()); - self.blob_req_id = Some(id); + fn expect_columns_request(mut self) -> Self { + let ids = self.rig.expect_data_columns_by_root_requests( + self.block.canonical_root(), + self.sample_column_count, + ); + self.column_req_ids = Some(ids); self } fn expect_parent_block_request(mut self) -> Self { @@ -2414,14 +2420,26 @@ mod deneb_only { self.parent_block_req_id = Some(id); self } - fn expect_parent_blobs_request(mut self) -> Self { - let id = self + fn expect_parent_columns_request(mut self, block_root: Hash256) -> Self { + let ids = self + .rig + .expect_data_columns_by_root_requests(block_root, self.sample_column_count); + self.parent_column_req_ids = Some(ids); + self + } + fn expect_parent_columns_request_for_unknown_parent(mut self) -> Self { + let block_root = self + .unknown_parent_block + .as_ref() + .map(|b| b.canonical_root()) + .unwrap(); + let ids = self .rig - .expect_blob_parent_request(self.block.parent_root()); - self.parent_blob_req_id = Some(id); + .expect_data_columns_by_root_requests(block_root, self.sample_column_count); + self.parent_column_req_ids = Some(ids); self } - fn expect_no_blobs_request(mut self) -> Self { + fn expect_no_columns_request(mut self) -> Self { self.rig.expect_empty_network(); self } @@ -2429,8 +2447,8 @@ mod deneb_only { self.rig.expect_empty_network(); self } - fn invalidate_blobs_too_few(mut self) -> Self { - self.blobs.pop().expect("blobs"); + fn invalidate_columns_too_few(mut self) -> Self { + self.data_columns.pop().expect("data_columns"); self } fn expect_block_process(mut self) -> Self { @@ -2449,35 +2467,35 @@ mod deneb_only { } #[test] - fn single_block_and_blob_lookup_block_returned_first_attestation() { - let Some(tester) = DenebTester::new(RequestTrigger::AttestationUnknownBlock) else { + fn single_block_and_column_lookup_block_returned_first_attestation() { + let Some(tester) = FuluTester::new(RequestTrigger::AttestationUnknownBlock) else { return; }; tester - .block_response_and_expect_blob_request() - .blobs_response() - .block_missing_components() // blobs not yet imported - .blobs_response_was_valid() - .blob_imported(); // now blobs resolve as imported + .block_response_and_expect_column_request() + .data_columns_response() + .block_missing_components() // columns not yet imported + .data_columns_response_was_valid() + .custody_columns_imported(); // now columns resolve as imported } #[test] - fn single_block_response_then_empty_blob_response_attestation() { - let Some(tester) = DenebTester::new(RequestTrigger::AttestationUnknownBlock) else { + fn single_block_response_then_empty_column_response_attestation() { + let Some(tester) = FuluTester::new(RequestTrigger::AttestationUnknownBlock) else { return; }; tester - .block_response_and_expect_blob_request() + .block_response_and_expect_column_request() .missing_components_from_block_request() - .empty_blobs_response() + .empty_data_columns_response() .expect_penalty("NotEnoughResponsesReturned") - .expect_blobs_request() + .expect_columns_request() .expect_no_block_request(); } #[test] - fn single_invalid_block_response_then_blob_response_attestation() { - let Some(tester) = DenebTester::new(RequestTrigger::AttestationUnknownBlock) else { + fn single_invalid_block_response_then_column_response_attestation() { + let Some(tester) = FuluTester::new(RequestTrigger::AttestationUnknownBlock) else { return; }; tester @@ -2485,52 +2503,52 @@ mod deneb_only { .invalid_block_processed() .expect_penalty("lookup_block_processing_failure") .expect_block_request() - .expect_no_blobs_request() - .blobs_response() - // blobs not sent for processing until the block is processed + .expect_no_columns_request() + .data_columns_response() + // columns not sent for processing until the block is processed .expect_no_penalty_and_no_requests(); } #[test] - fn single_block_response_then_invalid_blob_response_attestation() { - let Some(tester) = DenebTester::new(RequestTrigger::AttestationUnknownBlock) else { + fn single_block_response_then_invalid_column_response_attestation() { + let Some(tester) = FuluTester::new(RequestTrigger::AttestationUnknownBlock) else { return; }; tester .block_response_triggering_process() .missing_components_from_block_request() - .blobs_response() - .invalid_blob_processed() - .expect_penalty("lookup_blobs_processing_failure") - .expect_blobs_request() + .data_columns_response() + .invalid_column_processed() + .expect_penalty("lookup_custody_column_processing_failure") + .expect_columns_request() .expect_no_block_request(); } #[test] - fn single_block_response_then_too_few_blobs_response_attestation() { - let Some(tester) = DenebTester::new(RequestTrigger::AttestationUnknownBlock) else { + fn single_block_response_then_too_few_columns_response_attestation() { + let Some(tester) = FuluTester::new(RequestTrigger::AttestationUnknownBlock) else { return; }; tester .block_response_triggering_process() .missing_components_from_block_request() - .invalidate_blobs_too_few() - .blobs_response() + .invalidate_columns_too_few() + .data_columns_response() .expect_penalty("NotEnoughResponsesReturned") - .expect_blobs_request() + .expect_columns_request() .expect_no_block_request(); } // Test peer returning block that has unknown parent, and a new lookup is created #[test] fn parent_block_unknown_parent() { - let Some(tester) = DenebTester::new(RequestTrigger::GossipUnknownParentBlock(1)) else { + let Some(tester) = FuluTester::new(RequestTrigger::GossipUnknownParentBlock(1)) else { return; }; tester .expect_empty_beacon_processor() - .parent_block_response_expect_blobs() - .parent_blob_response() + .parent_block_response_expect_columns() + .parent_column_response() .expect_block_process() .parent_block_unknown_parent() .expect_parent_block_request() @@ -2540,12 +2558,12 @@ mod deneb_only { // Test peer returning invalid (processing) block, expect retry #[test] fn parent_block_invalid_parent() { - let Some(tester) = DenebTester::new(RequestTrigger::GossipUnknownParentBlock(1)) else { + let Some(tester) = FuluTester::new(RequestTrigger::GossipUnknownParentBlock(1)) else { return; }; tester - .parent_block_response_expect_blobs() - .parent_blob_response() + .parent_block_response_expect_columns() + .parent_column_response() .expect_block_process() .invalid_parent_processed() .expect_penalty("lookup_block_processing_failure") @@ -2556,44 +2574,44 @@ mod deneb_only { // Tests that if a peer does not respond with a block, we downscore and retry the block only #[test] fn empty_block_is_retried() { - let Some(tester) = DenebTester::new(RequestTrigger::AttestationUnknownBlock) else { + let Some(tester) = FuluTester::new(RequestTrigger::AttestationUnknownBlock) else { return; }; tester .empty_block_response() .expect_penalty("NotEnoughResponsesReturned") .expect_block_request() - .expect_no_blobs_request() - .block_response_and_expect_blob_request() - .blobs_response() + .expect_no_columns_request() + .block_response_and_expect_column_request() + .data_columns_response() .block_imported() .expect_no_active_lookups(); } #[test] - fn parent_block_then_empty_parent_blobs() { - let Some(tester) = DenebTester::new(RequestTrigger::GossipUnknownParentBlock(1)) else { + fn parent_block_then_empty_parent_columns() { + let Some(tester) = FuluTester::new(RequestTrigger::GossipUnknownParentBlock(1)) else { return; }; tester - .parent_block_then_empty_parent_blobs() - .log("resolve original block trigger blobs request and import") + .parent_block_then_empty_parent_columns() + .log("resolve original block trigger columns request and import") // Should not have block request, it is cached - .expect_blobs_request() - // TODO: Should send blobs for processing + .expect_columns_request() + // TODO: Should send columns for processing .block_imported() .expect_no_active_lookups(); } #[test] - fn parent_blob_unknown_parent() { - let Some(tester) = DenebTester::new(RequestTrigger::GossipUnknownParentBlob(1)) else { + fn parent_column_unknown_parent() { + let Some(tester) = FuluTester::new(RequestTrigger::GossipUnknownParentDataColumn(1)) else { return; }; tester .expect_empty_beacon_processor() - .parent_block_response_expect_blobs() - .parent_blob_response() + .parent_block_response_expect_columns() + .parent_column_response() .expect_block_process() .parent_block_unknown_parent() .expect_parent_block_request() @@ -2601,118 +2619,90 @@ mod deneb_only { } #[test] - fn parent_blob_invalid_parent() { - let Some(tester) = DenebTester::new(RequestTrigger::GossipUnknownParentBlob(1)) else { + fn parent_column_invalid_parent() { + let Some(tester) = FuluTester::new(RequestTrigger::GossipUnknownParentDataColumn(1)) else { return; }; tester .expect_empty_beacon_processor() - .parent_block_response_expect_blobs() - .parent_blob_response() + .parent_block_response_expect_columns() + .parent_column_response() .expect_block_process() .invalid_parent_processed() .expect_penalty("lookup_block_processing_failure") .expect_parent_block_request() - // blobs are not sent until block is processed + // columns are not sent until block is processed .expect_empty_beacon_processor(); } #[test] - fn parent_block_and_blob_lookup_parent_returned_first_blob_trigger() { - let Some(tester) = DenebTester::new(RequestTrigger::GossipUnknownParentBlob(1)) else { + fn parent_block_and_column_lookup_parent_returned_first_column_trigger() { + let Some(tester) = FuluTester::new(RequestTrigger::GossipUnknownParentDataColumn(1)) else { return; }; tester .parent_block_response() - .expect_parent_blobs_request() - .parent_blob_response() + .expect_parent_columns_request_for_unknown_parent() + .parent_column_response() .expect_block_process() .trigger_unknown_block_from_attestation() .parent_block_imported() - .complete_current_block_and_blobs_lookup() + .complete_current_block_and_columns_lookup() .expect_no_active_lookups(); } #[test] - fn parent_block_then_empty_parent_blobs_blob_trigger() { - let Some(tester) = DenebTester::new(RequestTrigger::GossipUnknownParentBlob(1)) else { + fn parent_block_then_empty_parent_columns_column_trigger() { + let Some(tester) = FuluTester::new(RequestTrigger::GossipUnknownParentDataColumn(1)) else { return; }; tester - .parent_block_then_empty_parent_blobs() - .log("resolve original block trigger blobs request and import") - .complete_current_block_and_blobs_lookup() + .parent_block_then_empty_parent_columns() + .log("resolve original block trigger columns request and import") + .complete_current_block_and_columns_lookup() .expect_no_active_lookups(); } #[test] - fn parent_blob_unknown_parent_chain() { - let Some(tester) = DenebTester::new(RequestTrigger::GossipUnknownParentBlob(2)) else { + fn parent_column_unknown_parent_chain() { + let Some(tester) = FuluTester::new(RequestTrigger::GossipUnknownParentDataColumn(2)) else { return; }; tester .expect_empty_beacon_processor() - .parent_block_response_expect_blobs() - .parent_blob_response() + .parent_block_response_expect_columns() + .parent_column_response() .expect_no_penalty() .expect_block_process() .parent_block_unknown_parent() .expect_parent_block_request() .expect_empty_beacon_processor() .parent_block_response() - .expect_parent_blobs_request() - .parent_blob_response() + .expect_parent_columns_request_for_unknown_parent() + .parent_column_response() .expect_no_penalty() .expect_block_process(); } #[test] fn unknown_parent_block_dup() { - let Some(tester) = DenebTester::new(RequestTrigger::GossipUnknownParentBlock(1)) else { + let Some(tester) = FuluTester::new(RequestTrigger::GossipUnknownParentBlock(1)) else { return; }; tester .search_parent_dup() - .expect_no_blobs_request() + .expect_no_columns_request() .expect_no_block_request(); } #[test] - fn unknown_parent_blob_dup() { - let Some(tester) = DenebTester::new(RequestTrigger::GossipUnknownParentBlob(1)) else { + fn unknown_parent_column_dup() { + let Some(tester) = FuluTester::new(RequestTrigger::GossipUnknownParentDataColumn(1)) else { return; }; tester .search_parent_dup() - .expect_no_blobs_request() + .expect_no_columns_request() .expect_no_block_request(); } - - // This test no longer applies, we don't issue requests for child lookups - // Keep for after updating rules on fetching blocks only first - #[ignore] - #[test] - fn no_peer_penalty_when_rpc_response_already_known_from_gossip() { - let Some(mut r) = TestRig::test_setup_after_deneb_before_fulu() else { - return; - }; - let (block, blobs) = r.rand_block_and_blobs(NumBlobs::Number(2)); - let block_root = block.canonical_root(); - let blob_0 = blobs[0].clone(); - let blob_1 = blobs[1].clone(); - let peer_a = r.new_connected_peer(); - let peer_b = r.new_connected_peer(); - // Send unknown parent block lookup - r.trigger_unknown_parent_block(peer_a, block.into()); - // Expect network request for blobs - let id = r.expect_blob_lookup_request(block_root); - // Peer responses with blob 0 - r.single_lookup_blob_response(id, peer_a, Some(blob_0.into())); - // Blob 1 is received via gossip unknown parent blob from a different peer - r.trigger_unknown_parent_blob(peer_b, blob_1.clone()); - // Original peer sends blob 1 via RPC - r.single_lookup_blob_response(id, peer_a, Some(blob_1.into())); - // Assert no downscore event for original peer - r.expect_no_penalty_for(peer_a); - } } diff --git a/beacon_node/network/src/sync/tests/mod.rs b/beacon_node/network/src/sync/tests/mod.rs index 23c14ff63ef..dcc7e3e49df 100644 --- a/beacon_node/network/src/sync/tests/mod.rs +++ b/beacon_node/network/src/sync/tests/mod.rs @@ -16,7 +16,7 @@ use tokio::sync::mpsc; use tracing_subscriber::fmt::MakeWriter; use tracing_subscriber::layer::SubscriberExt; use tracing_subscriber::util::SubscriberInitExt; -use types::{ChainSpec, ForkName, MinimalEthSpec as E}; +use types::{ForkName, MinimalEthSpec as E}; mod lookups; mod range; @@ -68,7 +68,6 @@ struct TestRig { rng_08: rand_chacha_03::ChaCha20Rng, rng: ChaCha20Rng, fork_name: ForkName, - spec: Arc, } // Environment variable to read if `fork_from_env` feature is enabled. diff --git a/beacon_node/network/src/sync/tests/range.rs b/beacon_node/network/src/sync/tests/range.rs index cb728a90c1b..9cda9fec95c 100644 --- a/beacon_node/network/src/sync/tests/range.rs +++ b/beacon_node/network/src/sync/tests/range.rs @@ -393,7 +393,7 @@ impl TestRig { let data_sidecars = if fork.fulu_enabled() { store - .get_data_columns(&block_root) + .get_data_columns(&block_root, fork) .unwrap() .map(|columns| { columns diff --git a/beacon_node/store/src/hot_cold_store.rs b/beacon_node/store/src/hot_cold_store.rs index 8eec4d5eceb..fd6190d8e7f 100644 --- a/beacon_node/store/src/hot_cold_store.rs +++ b/beacon_node/store/src/hot_cold_store.rs @@ -114,7 +114,7 @@ impl BlockCache { pub fn put_data_column(&mut self, block_root: Hash256, data_column: Arc>) { self.data_column_cache .get_or_insert_mut(block_root, Default::default) - .insert(data_column.index, data_column); + .insert(*data_column.index(), data_column); } pub fn put_data_column_custody_info( &mut self, @@ -969,7 +969,7 @@ impl, Cold: ItemStore> HotColdDB ) { ops.push(KeyValueStoreOp::PutKeyValue( DBColumn::BeaconDataColumn, - get_data_column_key(block_root, &data_column.index), + get_data_column_key(block_root, data_column.index()), data_column.as_ssz_bytes(), )); } @@ -1002,7 +1002,7 @@ impl, Cold: ItemStore> HotColdDB for data_column in data_columns { self.blobs_db.put_bytes( DBColumn::BeaconDataColumn, - &get_data_column_key(block_root, &data_column.index), + &get_data_column_key(block_root, data_column.index()), &data_column.as_ssz_bytes(), )?; self.block_cache @@ -1021,7 +1021,7 @@ impl, Cold: ItemStore> HotColdDB for data_column in data_columns { ops.push(KeyValueStoreOp::PutKeyValue( DBColumn::BeaconDataColumn, - get_data_column_key(block_root, &data_column.index), + get_data_column_key(block_root, data_column.index()), data_column.as_ssz_bytes(), )); } @@ -1301,7 +1301,7 @@ impl, Cold: ItemStore> HotColdDB )); } - StoreOp::DeleteDataColumns(block_root, column_indices) => { + StoreOp::DeleteDataColumns(block_root, column_indices, _) => { for index in column_indices { let key = get_data_column_key(&block_root, &index); key_value_batch @@ -1415,10 +1415,10 @@ impl, Cold: ItemStore> HotColdDB } true } - StoreOp::DeleteDataColumns(block_root, indices) => { + StoreOp::DeleteDataColumns(block_root, indices, fork_name) => { match indices .iter() - .map(|index| self.get_data_column(block_root, index)) + .map(|index| self.get_data_column(block_root, index, *fork_name)) .collect::, _>>() { Ok(data_column_sidecar_list_opt) => { @@ -1471,14 +1471,22 @@ impl, Cold: ItemStore> HotColdDB let reverse_op = match op { StoreOp::PutBlobs(block_root, _) => StoreOp::DeleteBlobs(*block_root), StoreOp::PutDataColumns(block_root, data_columns) => { - let indices = data_columns.iter().map(|c| c.index).collect(); - StoreOp::DeleteDataColumns(*block_root, indices) + let indices = data_columns.iter().map(|c| *c.index()).collect(); + if let Some(column) = data_columns.first() { + let slot = column.slot(); + let fork_name = self.spec.fork_name_at_slot::(slot); + StoreOp::DeleteDataColumns(*block_root, indices, fork_name) + } else { + return Err(Error::DBError { + message: "Failed to rollback data columns".to_owned(), + }); + } } StoreOp::DeleteBlobs(_) => match blobs_to_delete.pop() { Some((block_root, blobs)) => StoreOp::PutBlobs(block_root, blobs), None => return Err(HotColdDBError::Rollback.into()), }, - StoreOp::DeleteDataColumns(_, _) => match data_columns_to_delete.pop() { + StoreOp::DeleteDataColumns(_, _, _) => match data_columns_to_delete.pop() { Some((block_root, data_columns)) => { StoreOp::PutDataColumns(block_root, data_columns) } @@ -1530,7 +1538,7 @@ impl, Cold: ItemStore> HotColdDB StoreOp::DeleteBlobs(_) => (), - StoreOp::DeleteDataColumns(_, _) => (), + StoreOp::DeleteDataColumns(_, _, _) => (), StoreOp::DeleteExecutionPayload(_) => (), @@ -2506,12 +2514,16 @@ impl, Cold: ItemStore> HotColdDB pub fn get_data_columns( &self, block_root: &Hash256, + fork_name: ForkName, ) -> Result>, Error> { let column_indices = self.get_data_column_keys(*block_root)?; let columns: DataColumnSidecarList = column_indices .into_iter() - .filter_map(|col_index| self.get_data_column(block_root, &col_index).transpose()) + .filter_map(|col_index| { + self.get_data_column(block_root, &col_index, fork_name) + .transpose() + }) .collect::>()?; Ok((!columns.is_empty()).then_some(columns)) @@ -2585,6 +2597,7 @@ impl, Cold: ItemStore> HotColdDB &self, block_root: &Hash256, column_index: &ColumnIndex, + fork_name: ForkName, ) -> Result>>, Error> { // Check the cache. if let Some(data_column) = self @@ -2601,7 +2614,10 @@ impl, Cold: ItemStore> HotColdDB &get_data_column_key(block_root, column_index), )? { Some(ref data_column_bytes) => { - let data_column = Arc::new(DataColumnSidecar::from_ssz_bytes(data_column_bytes)?); + let data_column = Arc::new(DataColumnSidecar::from_ssz_bytes_for_fork( + data_column_bytes, + fork_name, + )?); self.block_cache.as_ref().inspect(|cache| { cache .lock() diff --git a/beacon_node/store/src/lib.rs b/beacon_node/store/src/lib.rs index ae5b2e1e571..83ca43ebaa5 100644 --- a/beacon_node/store/src/lib.rs +++ b/beacon_node/store/src/lib.rs @@ -237,7 +237,7 @@ pub enum StoreOp<'a, E: EthSpec> { PutStateSummary(Hash256, HotStateSummary), DeleteBlock(Hash256), DeleteBlobs(Hash256), - DeleteDataColumns(Hash256, Vec), + DeleteDataColumns(Hash256, Vec, ForkName), DeleteState(Hash256, Option), DeleteExecutionPayload(Hash256), DeleteSyncCommitteeBranch(Hash256), diff --git a/common/eth2/src/types.rs b/common/eth2/src/types.rs index b1a61ce00cc..c1572ca354c 100644 --- a/common/eth2/src/types.rs +++ b/common/eth2/src/types.rs @@ -1016,14 +1016,14 @@ impl SseDataColumnSidecar { pub fn from_data_column_sidecar( data_column_sidecar: &DataColumnSidecar, ) -> SseDataColumnSidecar { - let kzg_commitments = data_column_sidecar.kzg_commitments.to_vec(); + let kzg_commitments = data_column_sidecar.kzg_commitments().to_vec(); let versioned_hashes = kzg_commitments .iter() .map(|c| c.calculate_versioned_hash()) .collect(); SseDataColumnSidecar { block_root: data_column_sidecar.block_root(), - index: data_column_sidecar.index, + index: *data_column_sidecar.index(), slot: data_column_sidecar.slot(), kzg_commitments, versioned_hashes, diff --git a/consensus/types/src/builder/mod.rs b/consensus/types/src/builder/mod.rs index 54d0ae4eb73..4d6e1b9e63b 100644 --- a/consensus/types/src/builder/mod.rs +++ b/consensus/types/src/builder/mod.rs @@ -1,6 +1,7 @@ mod builder_bid; mod builder_pending_payment; mod builder_pending_withdrawal; +mod proposer_preference; pub use builder_bid::{ BuilderBid, BuilderBidBellatrix, BuilderBidCapella, BuilderBidDeneb, BuilderBidElectra, diff --git a/consensus/types/src/builder/proposer_preference.rs b/consensus/types/src/builder/proposer_preference.rs new file mode 100644 index 00000000000..8b137891791 --- /dev/null +++ b/consensus/types/src/builder/proposer_preference.rs @@ -0,0 +1 @@ + diff --git a/consensus/types/src/data/data_column_sidecar.rs b/consensus/types/src/data/data_column_sidecar.rs index 71d821f83ef..bb18e0a180b 100644 --- a/consensus/types/src/data/data_column_sidecar.rs +++ b/consensus/types/src/data/data_column_sidecar.rs @@ -7,10 +7,11 @@ use kzg::{KzgCommitment, KzgProof}; use merkle_proof::verify_merkle_proof; use safe_arith::ArithError; use serde::{Deserialize, Serialize}; -use ssz::Encode; +use ssz::{Decode, Encode}; use ssz_derive::{Decode, Encode}; use ssz_types::Error as SszError; use ssz_types::{FixedVector, VariableList}; +use superstruct::superstruct; use test_random_derive::TestRandom; use tree_hash::TreeHash; use tree_hash_derive::TreeHash; @@ -38,15 +39,49 @@ pub struct DataColumnsByRootIdentifier { pub type DataColumnSidecarList = Vec>>; +#[derive(Debug, PartialEq, Clone)] +pub enum Error { + IncorrectStateVariant, +} + +#[superstruct( + variants(Fulu, Gloas), + variant_attributes( + derive( + Debug, + Clone, + Serialize, + Deserialize, + Decode, + Encode, + TestRandom, + Educe, + TreeHash, + ), + context_deserialize(ForkName), + educe(PartialEq, Hash(bound(E: EthSpec))), + serde(bound = "E: EthSpec", deny_unknown_fields), + cfg_attr( + feature = "arbitrary", + derive(arbitrary::Arbitrary), + arbitrary(bound = "E: EthSpec") + ) + ), + ref_attributes(derive(TreeHash), tree_hash(enum_behaviour = "transparent")), + cast_error(ty = "Error", expr = "Error::IncorrectStateVariant"), + partial_getter_error(ty = "Error", expr = "Error::IncorrectStateVariant") +)] #[cfg_attr( feature = "arbitrary", derive(arbitrary::Arbitrary), arbitrary(bound = "E: EthSpec") )] -#[derive(Debug, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom, Educe)] -#[serde(bound = "E: EthSpec")] -#[educe(PartialEq, Eq, Hash(bound(E: EthSpec)))] -#[context_deserialize(ForkName)] +#[derive(Debug, Clone, Serialize, TreeHash, Encode, Decode, Educe, Deserialize)] +#[educe(PartialEq, Hash(bound(E: EthSpec)))] +#[serde(untagged)] +#[tree_hash(enum_behaviour = "transparent")] +#[ssz(enum_behaviour = "transparent")] +#[serde(bound = "E: EthSpec", deny_unknown_fields)] pub struct DataColumnSidecar { #[serde(with = "serde_utils::quoted_u64")] pub index: ColumnIndex, @@ -55,20 +90,58 @@ pub struct DataColumnSidecar { /// All the KZG commitments and proofs associated with the block, used for verifying sample cells. pub kzg_commitments: KzgCommitments, pub kzg_proofs: VariableList, + #[superstruct(only(Fulu))] pub signed_block_header: SignedBeaconBlockHeader, /// An inclusion proof, proving the inclusion of `blob_kzg_commitments` in `BeaconBlockBody`. + #[superstruct(only(Fulu))] pub kzg_commitments_inclusion_proof: FixedVector, + #[superstruct(only(Gloas), partial_getter(rename = "slot_gloas"))] + pub slot: Slot, + #[superstruct(only(Gloas))] + pub beacon_block_root: Hash256, } impl DataColumnSidecar { pub fn slot(&self) -> Slot { - self.signed_block_header.message.slot + match self { + DataColumnSidecar::Fulu(column) => column.slot(), + DataColumnSidecar::Gloas(column) => column.slot, + } } pub fn epoch(&self) -> Epoch { self.slot().epoch(E::slots_per_epoch()) } + pub fn block_root(&self) -> Hash256 { + match self { + DataColumnSidecar::Fulu(column) => column.block_root(), + DataColumnSidecar::Gloas(column) => column.beacon_block_root, + } + } + + /// Custom SSZ decoder that takes a `ForkName` as context. + pub fn from_ssz_bytes_for_fork( + bytes: &[u8], + fork_name: ForkName, + ) -> Result { + if fork_name.gloas_enabled() { + Ok(DataColumnSidecar::Gloas( + DataColumnSidecarGloas::from_ssz_bytes(bytes)?, + )) + } else { + Ok(DataColumnSidecar::Fulu( + DataColumnSidecarFulu::from_ssz_bytes(bytes)?, + )) + } + } +} + +impl DataColumnSidecarFulu { + pub fn slot(&self) -> Slot { + self.signed_block_header.message.slot + } + pub fn block_root(&self) -> Hash256 { self.signed_block_header.message.tree_hash_root() } @@ -132,6 +205,39 @@ impl DataColumnSidecar { } } +impl DataColumnSidecarGloas { + pub fn min_size() -> usize { + // min size is one cell + Self { + index: 0, + column: VariableList::new(vec![Cell::::default()]).unwrap(), + kzg_commitments: VariableList::new(vec![KzgCommitment::empty_for_testing()]).unwrap(), + kzg_proofs: VariableList::new(vec![KzgProof::empty()]).unwrap(), + slot: Slot::new(0), + beacon_block_root: Hash256::ZERO, + } + .as_ssz_bytes() + .len() + } + + pub fn max_size(max_blobs_per_block: usize) -> usize { + Self { + index: 0, + column: VariableList::new(vec![Cell::::default(); max_blobs_per_block]).unwrap(), + kzg_commitments: VariableList::new(vec![ + KzgCommitment::empty_for_testing(); + max_blobs_per_block + ]) + .unwrap(), + kzg_proofs: VariableList::new(vec![KzgProof::empty(); max_blobs_per_block]).unwrap(), + slot: Slot::new(0), + beacon_block_root: Hash256::ZERO, + } + .as_ssz_bytes() + .len() + } +} + #[derive(Debug)] pub enum DataColumnSidecarError { ArithError(ArithError), diff --git a/consensus/types/src/data/mod.rs b/consensus/types/src/data/mod.rs index 10d062bada9..eab16896496 100644 --- a/consensus/types/src/data/mod.rs +++ b/consensus/types/src/data/mod.rs @@ -13,7 +13,8 @@ pub use data_column_custody_group::{ }; pub use data_column_sidecar::{ Cell, ColumnIndex, DataColumn, DataColumnSidecar, DataColumnSidecarError, - DataColumnSidecarList, DataColumnsByRootIdentifier, + DataColumnSidecarFulu, DataColumnSidecarGloas, DataColumnSidecarList, + DataColumnsByRootIdentifier, }; pub use data_column_subnet_id::DataColumnSubnetId; diff --git a/consensus/types/src/test_utils/generate_random_block_and_blobs.rs b/consensus/types/src/test_utils/generate_random_block_and_blobs.rs index cf7b5df891a..e8814bfcb2e 100644 --- a/consensus/types/src/test_utils/generate_random_block_and_blobs.rs +++ b/consensus/types/src/test_utils/generate_random_block_and_blobs.rs @@ -93,12 +93,7 @@ mod test { fn test_verify_blob_inclusion_proof_from_existing_proof() { let (block, mut blob_sidecars) = generate_rand_block_and_blobs::(ForkName::Deneb, 1, &mut rng()); - let BlobSidecar { - index, - blob, - kzg_proof, - .. - } = blob_sidecars.pop().unwrap(); + let blob = blob_sidecars.pop().unwrap(); // Compute the commitments inclusion proof and use it for building blob sidecar. let (signed_block_header, kzg_commitments_inclusion_proof) = block @@ -106,12 +101,12 @@ mod test { .unwrap(); let blob_sidecar = BlobSidecar::new_with_existing_proof( - index as usize, - blob, + blob.index as usize, + blob.blob.clone(), &block, signed_block_header, &kzg_commitments_inclusion_proof, - kzg_proof, + blob.kzg_proof, ) .unwrap(); diff --git a/testing/ef_tests/src/cases/fork_choice.rs b/testing/ef_tests/src/cases/fork_choice.rs index 8e9d438a243..82d8f9eca93 100644 --- a/testing/ef_tests/src/cases/fork_choice.rs +++ b/testing/ef_tests/src/cases/fork_choice.rs @@ -2,7 +2,6 @@ use super::*; use crate::decode::{ssz_decode_file, ssz_decode_file_with, ssz_decode_state, yaml_decode_file}; use ::fork_choice::{PayloadVerificationStatus, ProposerHeadError}; use beacon_chain::beacon_proposer_cache::compute_proposer_duties_from_head; -use beacon_chain::blob_verification::GossipBlobError; use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::chain_config::{ DEFAULT_RE_ORG_HEAD_THRESHOLD, DEFAULT_RE_ORG_MAX_EPOCHS_SINCE_FINALIZATION, @@ -15,7 +14,6 @@ use beacon_chain::{ attestation_verification::{ VerifiedAttestation, obtain_indexed_attestation_and_committees_per_slot, }, - blob_verification::GossipVerifiedBlob, custody_context::NodeCustodyType, test_utils::{BeaconChainHarness, EphemeralHarnessType}, }; @@ -28,9 +26,9 @@ use std::sync::Arc; use std::time::Duration; use types::{ Attestation, AttestationRef, AttesterSlashing, AttesterSlashingRef, BeaconBlock, BeaconState, - BlobSidecar, BlobsList, BlockImportSource, Checkpoint, DataColumnSidecarList, - DataColumnSubnetId, ExecutionBlockHash, Hash256, IndexedAttestation, KzgProof, - ProposerPreparationData, SignedBeaconBlock, Slot, Uint256, + BlobsList, BlockImportSource, Checkpoint, DataColumnSidecarList, DataColumnSubnetId, + ExecutionBlockHash, Hash256, IndexedAttestation, KzgProof, ProposerPreparationData, + SignedBeaconBlock, Slot, Uint256, }; // When set to true, cache any states fetched from the db. @@ -521,7 +519,8 @@ impl Tester { let gossip_verified_data_columns = columns .into_iter() .map(|column| { - let subnet_id = DataColumnSubnetId::from_column_index(column.index, &self.spec); + let subnet_id = + DataColumnSubnetId::from_column_index(*column.index(), &self.spec); GossipVerifiedDataColumn::new(column.clone(), subnet_id, &self.harness.chain) .unwrap_or_else(|_| { data_column_success = false; @@ -572,63 +571,52 @@ impl Tester { &self, block: SignedBeaconBlock, blobs: Option>, - kzg_proofs: Option>, + _kzg_proofs: Option>, valid: bool, ) -> Result<(), Error> { let block_root = block.canonical_root(); - let mut blob_success = true; + // let mut blob_success = true; + let blob_success = true; // Convert blobs and kzg_proofs into sidecars, then plumb them into the availability tracker - if let Some(blobs) = blobs.clone() { - let proofs = kzg_proofs.unwrap(); - let commitments = block - .message() - .body() - .blob_kzg_commitments() - .unwrap() - .clone(); - - // Zipping will stop when any of the zipped lists runs out, which is what we want. Some - // of the tests don't provide enough proofs/blobs, and should fail the availability - // check. - for (i, ((blob, kzg_proof), kzg_commitment)) in blobs - .into_iter() - .zip(proofs) - .zip(commitments.into_iter()) - .enumerate() - { - let blob_sidecar = Arc::new(BlobSidecar { - index: i as u64, - blob, - kzg_commitment, - kzg_proof, - signed_block_header: block.signed_block_header(), - kzg_commitment_inclusion_proof: block - .message() - .body() - .kzg_commitment_merkle_proof(i) - .unwrap(), - }); - - let chain = self.harness.chain.clone(); - let blob = - match GossipVerifiedBlob::new(blob_sidecar.clone(), blob_sidecar.index, &chain) - { - Ok(gossip_verified_blob) => gossip_verified_blob, - Err(GossipBlobError::KzgError(_)) => { - blob_success = false; - GossipVerifiedBlob::__assumed_valid(blob_sidecar) - } - Err(_) => GossipVerifiedBlob::__assumed_valid(blob_sidecar), - }; - let result = - self.block_on_dangerous(self.harness.chain.process_gossip_blob(blob))?; - if valid { - assert!(result.is_ok()); - } - } - }; + // if let Some(blobs) = blobs.clone() { + // let proofs = kzg_proofs.unwrap(); + // let commitments = block + // .message() + // .body() + // .blob_kzg_commitments() + // .unwrap() + // .clone(); + + // // Zipping will stop when any of the zipped lists runs out, which is what we want. Some + // // of the tests don't provide enough proofs/blobs, and should fail the availability + // // check. + // for (i, ((blob, kzg_proof), kzg_commitment)) in blobs + // .into_iter() + // .zip(proofs) + // .zip(commitments.into_iter()) + // .enumerate() + // { + + // let chain = self.harness.chain.clone(); + // let blob = + // match KzgVerifiedBlob::new(blob_sidecar.clone(), *blob_sidecar.index(), &chain) + // { + // Ok(gossip_verified_blob) => gossip_verified_blob, + // Err(GossipBlobError::KzgError(_)) => { + // blob_success = false; + // GossipVerifiedBlob::__assumed_valid(blob_sidecar) + // } + // Err(_) => GossipVerifiedBlob::__assumed_valid(blob_sidecar), + // }; + // let result = + // self.block_on_dangerous(self.harness.chain.process_gossip_blob(blob))?; + // if valid { + // assert!(result.is_ok()); + // } + // } + // }; let block = Arc::new(block); let result: Result, _> = self