From 662d6cf931eb2dca8df90f47924a13c2cc4a60c0 Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Thu, 29 Aug 2024 10:55:33 +1000 Subject: [PATCH 01/25] Get blobs from EL. Co-authored-by: Michael Sproul --- Cargo.lock | 5 +- beacon_node/beacon_chain/src/beacon_chain.rs | 255 ++++++++++++------ .../beacon_chain/src/blob_verification.rs | 19 ++ .../beacon_chain/src/block_verification.rs | 4 +- .../src/data_availability_checker.rs | 33 ++- .../src/data_availability_checker/error.rs | 5 +- .../overflow_lru_cache.rs | 138 ++++------ .../state_lru_cache.rs | 5 + beacon_node/beacon_chain/src/fetch_blobs.rs | 225 ++++++++++++++++ beacon_node/beacon_chain/src/lib.rs | 1 + beacon_node/beacon_chain/src/metrics.rs | 6 + beacon_node/beacon_chain/src/test_utils.rs | 3 +- .../beacon_chain/tests/block_verification.rs | 2 +- beacon_node/execution_layer/src/engine_api.rs | 6 +- .../execution_layer/src/engine_api/http.rs | 19 ++ .../src/engine_api/json_structures.rs | 13 +- beacon_node/execution_layer/src/lib.rs | 20 +- .../execution_layer/src/test_utils/mod.rs | 1 + .../gossip_methods.rs | 11 +- .../src/network_beacon_processor/mod.rs | 55 +++- .../network_beacon_processor/sync_methods.rs | 44 +-- consensus/types/src/beacon_block_body.rs | 108 ++++---- consensus/types/src/blob_sidecar.rs | 31 +++ consensus/types/src/signed_beacon_block.rs | 31 +++ 24 files changed, 784 insertions(+), 256 deletions(-) create mode 100644 beacon_node/beacon_chain/src/fetch_blobs.rs diff --git a/Cargo.lock b/Cargo.lock index cb25eb92be7..88d1cd85b46 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8641,9 +8641,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.11" +version = "0.7.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9cf6b47b3771c49ac75ad09a6162f53ad4b8088b76ac60e8ec1455b31a189fe1" +checksum = "5419f34732d9eb6ee4c3578b7989078579b7f039cbbb9ca2c4da015749371e15" dependencies = [ "bytes", "futures-core", @@ -8652,6 +8652,7 @@ dependencies = [ "pin-project-lite", "slab", "tokio", + "tracing", ] [[package]] diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 9d744003360..756fe70bb1a 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -89,7 +89,7 @@ use kzg::Kzg; use operation_pool::{ CompactAttestationRef, OperationPool, PersistedOperationPool, ReceivedPreCapella, }; -use parking_lot::{Mutex, RwLock}; +use parking_lot::{Mutex, RwLock, RwLockWriteGuard}; use proto_array::{DoNotReOrg, ProposerHeadError}; use safe_arith::SafeArith; use slasher::Slasher; @@ -121,6 +121,7 @@ use store::{ DatabaseBlock, Error as DBError, HotColdDB, KeyValueStore, KeyValueStoreOp, StoreItem, StoreOp, }; use task_executor::{ShutdownReason, TaskExecutor}; +use tokio::sync::mpsc::Receiver; use tokio_stream::Stream; use tree_hash::TreeHash; use types::blob_sidecar::FixedBlobSidecarList; @@ -3004,13 +3005,7 @@ impl BeaconChain { return Err(BlockError::BlobNotRequired(blob.slot())); } - if let Some(event_handler) = self.event_handler.as_ref() { - if event_handler.has_blob_sidecar_subscribers() { - event_handler.register(EventKind::BlobSidecar(SseBlobSidecar::from_blob_sidecar( - blob.as_blob(), - ))); - } - } + self.emit_sse_blob_sidecar_events(&block_root, std::iter::once(blob.as_blob())); let r = self.check_gossip_blob_availability_and_import(blob).await; self.remove_notified(&block_root, r) @@ -3073,20 +3068,58 @@ impl BeaconChain { return Err(BlockError::BlockIsAlreadyKnown(block_root)); } + self.emit_sse_blob_sidecar_events(&block_root, blobs.iter().flatten().map(Arc::as_ref)); + + let r = self + .check_rpc_blob_availability_and_import(slot, block_root, blobs) + .await; + self.remove_notified(&block_root, r) + } + + pub async fn process_engine_blobs( + self: &Arc, + slot: Slot, + block_root: Hash256, + blobs: FixedBlobSidecarList, + data_column_recv: Option>>, + ) -> Result> { + // 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::BlockIsAlreadyKnown(block_root)); + } + + self.emit_sse_blob_sidecar_events(&block_root, blobs.iter().flatten().map(Arc::as_ref)); + + let r = self + .check_engine_blob_availability_and_import(slot, block_root, blobs, data_column_recv) + .await; + self.remove_notified(&block_root, r) + } + + fn emit_sse_blob_sidecar_events<'a, I>(self: &Arc, block_root: &Hash256, blobs_iter: I) + where + I: Iterator>, + { if let Some(event_handler) = self.event_handler.as_ref() { if event_handler.has_blob_sidecar_subscribers() { - for blob in blobs.iter().filter_map(|maybe_blob| maybe_blob.as_ref()) { + let imported_blobs = self + .data_availability_checker + .imported_blob_indexes(block_root) + .unwrap_or_default(); + let new_blobs = blobs_iter.filter(|b| !imported_blobs.contains(&b.index)); + + for blob in new_blobs { event_handler.register(EventKind::BlobSidecar( SseBlobSidecar::from_blob_sidecar(blob), )); } } } - - let r = self - .check_rpc_blob_availability_and_import(slot, block_root, blobs) - .await; - self.remove_notified(&block_root, r) } /// Cache the columns in the processing cache, process it, then evict it from the cache if it was @@ -3254,7 +3287,7 @@ impl BeaconChain { match executed_block { ExecutedBlock::Available(block) => { - self.import_available_block(Box::new(block)).await + self.import_available_block(Box::new(block), None).await } ExecutedBlock::AvailabilityPending(block) => { self.check_block_availability_and_import(block).await @@ -3386,7 +3419,7 @@ impl BeaconChain { let availability = self .data_availability_checker .put_pending_executed_block(block)?; - self.process_availability(slot, availability).await + self.process_availability(slot, availability, None).await } /// Checks if the provided blob can make any cached blocks available, and imports immediately @@ -3401,7 +3434,7 @@ impl BeaconChain { } let availability = self.data_availability_checker.put_gossip_blob(blob)?; - self.process_availability(slot, availability).await + self.process_availability(slot, availability, None).await } /// Checks if the provided data column can make any cached blocks available, and imports immediately @@ -3428,11 +3461,38 @@ impl BeaconChain { .data_availability_checker .put_gossip_data_columns(slot, block_root, data_columns)?; - self.process_availability(slot, availability) + self.process_availability(slot, availability, None) .await .map(|result| (result, data_columns_to_publish)) } + fn check_blobs_for_slashability( + self: &Arc, + block_root: Hash256, + blobs: &FixedBlobSidecarList, + ) -> Result<(), BlockError> { + let mut slashable_cache = self.observed_slashable.write(); + for header in blobs + .into_iter() + .filter_map(|b| b.as_ref().map(|b| b.signed_block_header.clone())) + .unique() + { + if verify_header_signature::>(self, &header).is_ok() { + slashable_cache + .observe_slashable( + header.message.slot, + header.message.proposer_index, + block_root, + ) + .map_err(|e| BlockError::BeaconChainError(e.into()))?; + if let Some(slasher) = self.slasher.as_ref() { + slasher.accept_block_header(header); + } + } + } + Ok(()) + } + /// Checks if the provided blobs can make any cached blocks available, and imports immediately /// if so, otherwise caches the blob in the data availability checker. async fn check_rpc_blob_availability_and_import( @@ -3441,35 +3501,28 @@ impl BeaconChain { block_root: Hash256, blobs: FixedBlobSidecarList, ) -> Result> { - // Need to scope this to ensure the lock is dropped before calling `process_availability` - // Even an explicit drop is not enough to convince the borrow checker. - { - let mut slashable_cache = self.observed_slashable.write(); - for header in blobs - .into_iter() - .filter_map(|b| b.as_ref().map(|b| b.signed_block_header.clone())) - .unique() - { - if verify_header_signature::>(self, &header).is_ok() { - slashable_cache - .observe_slashable( - header.message.slot, - header.message.proposer_index, - block_root, - ) - .map_err(|e| BlockError::BeaconChainError(e.into()))?; - if let Some(slasher) = self.slasher.as_ref() { - slasher.accept_block_header(header); - } - } - } - } - let epoch = slot.epoch(T::EthSpec::slots_per_epoch()); + self.check_blobs_for_slashability(block_root, &blobs)?; let availability = self .data_availability_checker - .put_rpc_blobs(block_root, epoch, blobs)?; + .put_rpc_blobs(block_root, blobs)?; - self.process_availability(slot, availability).await + self.process_availability(slot, availability, None).await + } + + async fn check_engine_blob_availability_and_import( + self: &Arc, + slot: Slot, + block_root: Hash256, + blobs: FixedBlobSidecarList, + data_column_recv: Option>>, + ) -> Result> { + self.check_blobs_for_slashability(block_root, &blobs)?; + let availability = self + .data_availability_checker + .put_engine_blobs(block_root, blobs)?; + + self.process_availability(slot, availability, data_column_recv) + .await } /// Checks if the provided columns can make any cached blocks available, and imports immediately @@ -3517,7 +3570,7 @@ impl BeaconChain { custody_columns, )?; - self.process_availability(slot, availability) + self.process_availability(slot, availability, None) .await .map(|result| (result, data_columns_to_publish)) } @@ -3530,11 +3583,12 @@ impl BeaconChain { self: &Arc, slot: Slot, availability: Availability, + recv: Option>>, ) -> Result> { match availability { Availability::Available(block) => { // Block is fully available, import into fork choice - self.import_available_block(block).await + self.import_available_block(block, recv).await } Availability::MissingComponents(block_root) => Ok( AvailabilityProcessingStatus::MissingComponents(slot, block_root), @@ -3545,6 +3599,7 @@ impl BeaconChain { pub async fn import_available_block( self: &Arc, block: Box>, + data_column_recv: Option>>, ) -> Result> { let AvailableExecutedBlock { block, @@ -3586,6 +3641,7 @@ impl BeaconChain { parent_block, parent_eth1_finalization_data, consensus_context, + data_column_recv, ) }, "payload_verification_handle", @@ -3624,6 +3680,7 @@ impl BeaconChain { parent_block: SignedBlindedBeaconBlock, parent_eth1_finalization_data: Eth1FinalizationData, mut consensus_context: ConsensusContext, + data_column_recv: Option>>, ) -> Result> { // ----------------------------- BLOCK NOT YET ATTESTABLE ---------------------------------- // Everything in this initial section is on the hot path between processing the block and @@ -3769,7 +3826,6 @@ impl BeaconChain { // state if we returned early without committing. In other words, an error here would // corrupt the node's database permanently. // ----------------------------------------------------------------------------------------- - self.import_block_update_shuffling_cache(block_root, &mut state); self.import_block_observe_attestations( block, @@ -3786,15 +3842,50 @@ impl BeaconChain { ); self.import_block_update_slasher(block, &state, &mut consensus_context); - let db_write_timer = metrics::start_timer(&metrics::BLOCK_PROCESSING_DB_WRITE); - // Store the block and its state, and execute the confirmation batch for the intermediate // states, which will delete their temporary flags. // If the write fails, revert fork choice to the version from disk, else we can // end up with blocks in fork choice that are missing from disk. // See https://github.com/sigp/lighthouse/issues/2028 let (_, signed_block, blobs, data_columns) = signed_block.deconstruct(); + let custody_columns_count = self.data_availability_checker.get_custody_columns_count(); + let data_columns = data_columns.filter(|columns| columns.len() >= custody_columns_count); + + let data_columns = match (data_columns, data_column_recv) { + // If the block was made available via custody columns received from gossip / rpc, use them + // since we already have them. + (Some(columns), _) => Some(columns), + // Otherwise, it means blobs were likely available via fetching from EL, in this case we + // wait for the data columns to be computed (blocking). + (None, Some(mut data_column_recv)) => { + let _column_recv_timer = + metrics::start_timer(&metrics::BLOCK_PROCESSING_DATA_COLUMNS_WAIT); + // Unable to receive data columns from sender, sender is either dropped or + // failed to compute data columns from blobs. We restore fork choice here and + // return to avoid inconsistency in database. + if let Some(columns) = data_column_recv.blocking_recv() { + Some(columns) + } else { + let err_msg = "Did not receive data columns from sender"; + error!( + self.log, + "Failed to store data columns into the database"; + "msg" => "Restoring fork choice from disk", + "error" => err_msg, + ); + return Err(self + .handle_import_block_db_write_error(fork_choice) + .err() + .unwrap_or(BlockError::InternalError(err_msg.to_string()))); + } + } + // Non data columns present and compute data columns task was not spawned. + // Could either be no blobs in the block or before PeerDAS activation. + (None, None) => None, + }; + let block = signed_block.message(); + let db_write_timer = metrics::start_timer(&metrics::BLOCK_PROCESSING_DB_WRITE); ops.extend( confirmed_state_roots .into_iter() @@ -3834,33 +3925,10 @@ impl BeaconChain { "msg" => "Restoring fork choice from disk", "error" => ?e, ); - - // Clear the early attester cache to prevent attestations which we would later be unable - // to verify due to the failure. - self.early_attester_cache.clear(); - - // Since the write failed, try to revert the canonical head back to what was stored - // in the database. This attempts to prevent inconsistency between the database and - // fork choice. - if let Err(e) = self.canonical_head.restore_from_store( - fork_choice, - ResetPayloadStatuses::always_reset_conditionally( - self.config.always_reset_payload_statuses, - ), - &self.store, - &self.spec, - &self.log, - ) { - crit!( - self.log, - "No stored fork choice found to restore from"; - "error" => ?e, - "warning" => "The database is likely corrupt now, consider --purge-db" - ); - return Err(BlockError::BeaconChainError(e)); - } - - return Err(e.into()); + return Err(self + .handle_import_block_db_write_error(fork_choice) + .err() + .unwrap_or(e.into())); } drop(txn_lock); @@ -3928,6 +3996,41 @@ impl BeaconChain { Ok(block_root) } + fn handle_import_block_db_write_error( + &self, + // We don't actually need this value, however it's always present when we call this function + // and it needs to be dropped to prevent a dead-lock. Requiring it to be passed here is + // defensive programming. + fork_choice_write_lock: RwLockWriteGuard>, + ) -> Result<(), BlockError> { + // Clear the early attester cache to prevent attestations which we would later be unable + // to verify due to the failure. + self.early_attester_cache.clear(); + + // Since the write failed, try to revert the canonical head back to what was stored + // in the database. This attempts to prevent inconsistency between the database and + // fork choice. + if let Err(e) = self.canonical_head.restore_from_store( + fork_choice_write_lock, + ResetPayloadStatuses::always_reset_conditionally( + self.config.always_reset_payload_statuses, + ), + &self.store, + &self.spec, + &self.log, + ) { + crit!( + self.log, + "No stored fork choice found to restore from"; + "error" => ?e, + "warning" => "The database is likely corrupt now, consider --purge-db" + ); + Err(BlockError::BeaconChainError(e)) + } else { + Ok(()) + } + } + /// Check block's consistentency with any configured weak subjectivity checkpoint. fn check_block_against_weak_subjectivity_checkpoint( &self, diff --git a/beacon_node/beacon_chain/src/blob_verification.rs b/beacon_node/beacon_chain/src/blob_verification.rs index 99fc5d9d0c0..528514d3b8a 100644 --- a/beacon_node/beacon_chain/src/blob_verification.rs +++ b/beacon_node/beacon_chain/src/blob_verification.rs @@ -351,6 +351,25 @@ impl KzgVerifiedBlobList { verified_blobs: blobs, }) } + + /// Create a `KzgVerifiedBlobList` from `blobs` that are already KZG verified. + /// + /// This should be used with caution, as used incorrectly it could result in KZG verification + /// being skipped and invalid blobs being deemed valid. + pub fn from_verified>>>( + blobs: I, + seen_timestamp: Duration, + ) -> Self { + Self { + verified_blobs: blobs + .into_iter() + .map(|blob| KzgVerifiedBlob { + blob, + seen_timestamp, + }) + .collect(), + } + } } impl IntoIterator for KzgVerifiedBlobList { diff --git a/beacon_node/beacon_chain/src/block_verification.rs b/beacon_node/beacon_chain/src/block_verification.rs index d9662d59f9e..5eda8c1d1bf 100644 --- a/beacon_node/beacon_chain/src/block_verification.rs +++ b/beacon_node/beacon_chain/src/block_verification.rs @@ -688,7 +688,7 @@ pub struct SignatureVerifiedBlock { consensus_context: ConsensusContext, } -/// Used to await the result of executing payload with a remote EE. +/// Used to await the result of executing payload with an EE. type PayloadVerificationHandle = JoinHandle>>>; @@ -803,6 +803,7 @@ fn build_gossip_verified_data_columns( ))?; let timer = metrics::start_timer(&metrics::DATA_COLUMN_SIDECAR_COMPUTATION); + let sidecars = blobs_to_data_column_sidecars(&blobs, block, kzg, &chain.spec)?; drop(timer); let mut gossip_verified_data_columns = vec![]; @@ -1410,7 +1411,6 @@ impl ExecutionPendingBlock { /* * Perform cursory checks to see if the block is even worth processing. */ - check_block_relevancy(block.as_block(), block_root, chain)?; // Define a future that will verify the execution payload with an execution engine. diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index 470cee713fa..d21c7da4eae 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -187,7 +187,6 @@ impl DataAvailabilityChecker { pub fn put_rpc_blobs( &self, block_root: Hash256, - epoch: Epoch, blobs: FixedBlobSidecarList, ) -> Result, AvailabilityCheckError> { let Some(kzg) = self.kzg.as_ref() else { @@ -200,11 +199,11 @@ impl DataAvailabilityChecker { .ok_or(AvailabilityCheckError::SlotClockError)?; let verified_blobs = - KzgVerifiedBlobList::new(Vec::from(blobs).into_iter().flatten(), kzg, seen_timestamp) + KzgVerifiedBlobList::new(blobs.iter().flatten().cloned(), kzg, seen_timestamp) .map_err(AvailabilityCheckError::Kzg)?; self.availability_cache - .put_kzg_verified_blobs(block_root, epoch, verified_blobs) + .put_kzg_verified_blobs(block_root, verified_blobs) } /// Put a list of custody columns received via RPC into the availability cache. This performs KZG @@ -240,6 +239,27 @@ impl DataAvailabilityChecker { ) } + /// Put a list of blobs received from the EL pool into the availability cache. + /// + /// This DOES NOT perform KZG verification because the KZG proofs should have been constructed + /// immediately prior to calling this function so they are assumed to be valid. + pub fn put_engine_blobs( + &self, + block_root: Hash256, + blobs: FixedBlobSidecarList, + ) -> Result, AvailabilityCheckError> { + let seen_timestamp = self + .slot_clock + .now_duration() + .ok_or(AvailabilityCheckError::SlotClockError)?; + + let verified_blobs = + KzgVerifiedBlobList::from_verified(blobs.iter().flatten().cloned(), seen_timestamp); + + self.availability_cache + .put_kzg_verified_blobs(block_root, verified_blobs) + } + /// 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. @@ -249,11 +269,8 @@ impl DataAvailabilityChecker { &self, gossip_blob: GossipVerifiedBlob, ) -> Result, AvailabilityCheckError> { - self.availability_cache.put_kzg_verified_blobs( - gossip_blob.block_root(), - gossip_blob.epoch(), - vec![gossip_blob.into_inner()], - ) + self.availability_cache + .put_kzg_verified_blobs(gossip_blob.block_root(), vec![gossip_blob.into_inner()]) } /// Check if we've cached other data columns for this block. If it satisfies the custody requirement and we also diff --git a/beacon_node/beacon_chain/src/data_availability_checker/error.rs b/beacon_node/beacon_chain/src/data_availability_checker/error.rs index 79793d6dc29..3a3ab035861 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/error.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/error.rs @@ -1,4 +1,5 @@ use kzg::{Error as KzgError, KzgCommitment}; +use types::data_column_sidecar::DataColumnSidecarError; use types::{BeaconStateError, Hash256}; #[derive(Debug)] @@ -23,6 +24,7 @@ pub enum Error { BlockReplayError(state_processing::BlockReplayError), RebuildingStateCaches(BeaconStateError), SlotClockError, + DataColumnSidecarError(DataColumnSidecarError), } #[derive(PartialEq, Eq)] @@ -47,7 +49,8 @@ impl Error { | Error::BlockReplayError(_) | Error::UnableToDetermineImportRequirement | Error::RebuildingStateCaches(_) - | Error::SlotClockError => ErrorCategory::Internal, + | Error::SlotClockError + | Error::DataColumnSidecarError(_) => ErrorCategory::Internal, Error::Kzg(_) | Error::BlobIndexInvalid(_) | Error::DataColumnIndexInvalid(_) 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 4863982b552..61b81d9bc5d 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 @@ -11,7 +11,7 @@ use crate::BeaconChainTypes; use kzg::Kzg; use lru::LruCache; use parking_lot::RwLock; -use ssz_types::{FixedVector, VariableList}; +use ssz_types::FixedVector; use std::collections::HashSet; use std::num::NonZeroUsize; use std::sync::Arc; @@ -38,11 +38,6 @@ pub struct PendingComponents { pub reconstruction_started: bool, } -pub enum BlockImportRequirement { - AllBlobs, - CustodyColumns(usize), -} - impl PendingComponents { /// Returns an immutable reference to the cached block. pub fn get_cached_block(&self) -> &Option> { @@ -203,23 +198,14 @@ impl PendingComponents { /// /// Returns `true` if both the block exists and the number of received blobs / custody columns /// matches the number of expected blobs / custody columns. - pub fn is_available(&self, block_import_requirement: &BlockImportRequirement) -> bool { - match block_import_requirement { - BlockImportRequirement::AllBlobs => self - .num_expected_blobs() - .map_or(false, |num_expected_blobs| { - num_expected_blobs == self.num_received_blobs() - }), - BlockImportRequirement::CustodyColumns(num_expected_columns) => { - let num_received_data_columns = self.num_received_data_columns(); - // No data columns when there are 0 blobs - self.num_expected_blobs() - .map_or(false, |num_expected_blobs| { - num_expected_blobs == 0 - || *num_expected_columns == num_received_data_columns - }) - } - } + pub fn is_available(&self, custody_column_count: usize) -> bool { + self.num_expected_blobs() + .map_or(false, |num_expected_blobs| { + let all_blobs_received = num_expected_blobs == self.num_received_blobs(); + let all_columns_received = num_expected_blobs == 0 + || custody_column_count == self.num_received_data_columns(); + all_blobs_received || all_columns_received + }) } /// Returns an empty `PendingComponents` object with the given block root. @@ -241,7 +227,6 @@ impl PendingComponents { /// reconstructed from disk. Ensure you are not holding any write locks while calling this. pub fn make_available( self, - block_import_requirement: BlockImportRequirement, spec: &Arc, recover: R, ) -> Result, AvailabilityCheckError> @@ -268,27 +253,26 @@ impl PendingComponents { return Err(AvailabilityCheckError::Unexpected); }; - let (blobs, data_columns) = match block_import_requirement { - BlockImportRequirement::AllBlobs => { - let num_blobs_expected = diet_executed_block.num_blobs_expected(); - let Some(verified_blobs) = verified_blobs - .into_iter() - .cloned() - .map(|b| b.map(|b| b.to_blob())) - .take(num_blobs_expected) - .collect::>>() - else { - return Err(AvailabilityCheckError::Unexpected); - }; - (Some(VariableList::new(verified_blobs)?), None) - } - BlockImportRequirement::CustodyColumns(_) => { - let verified_data_columns = verified_data_columns - .into_iter() - .map(|d| d.into_inner()) - .collect(); - (None, Some(verified_data_columns)) - } + let is_peer_das_enabled = spec.is_peer_das_enabled_for_epoch(diet_executed_block.epoch()); + let (blobs, data_columns) = if is_peer_das_enabled { + let data_columns = verified_data_columns + .into_iter() + .map(|d| d.into_inner()) + .collect::>(); + (None, Some(data_columns)) + } else { + let num_blobs_expected = diet_executed_block.num_blobs_expected(); + let Some(verified_blobs) = verified_blobs + .into_iter() + .cloned() + .map(|b| b.map(|b| b.to_blob())) + .take(num_blobs_expected) + .collect::>>() + .map(Into::into) + else { + return Err(AvailabilityCheckError::Unexpected); + }; + (Some(verified_blobs), None) }; let executed_block = recover(diet_executed_block)?; @@ -435,46 +419,24 @@ impl DataAvailabilityCheckerInner { f(self.critical.read().peek(block_root)) } - fn block_import_requirement( - &self, - epoch: Epoch, - ) -> Result { - let peer_das_enabled = self.spec.is_peer_das_enabled_for_epoch(epoch); - if peer_das_enabled { - Ok(BlockImportRequirement::CustodyColumns( - self.custody_column_count, - )) - } else { - Ok(BlockImportRequirement::AllBlobs) - } - } - /// Potentially trigger reconstruction if: /// - Our custody requirement is all columns /// - We >= 50% of columns, but not all columns - fn should_reconstruct( - &self, - block_import_requirement: &BlockImportRequirement, - pending_components: &PendingComponents, - ) -> bool { - let BlockImportRequirement::CustodyColumns(num_expected_columns) = block_import_requirement - else { - return false; - }; - + fn should_reconstruct(&self, pending_components: &PendingComponents) -> bool { let num_of_columns = self.spec.number_of_columns; let has_missing_columns = pending_components.verified_data_columns.len() < num_of_columns; - - has_missing_columns + let is_super_node = self.custody_column_count == num_of_columns; + let is_reconstruction_possible = + pending_components.verified_data_columns.len() >= num_of_columns / 2; + is_super_node + && has_missing_columns + && is_reconstruction_possible && !pending_components.reconstruction_started - && *num_expected_columns == num_of_columns - && pending_components.verified_data_columns.len() >= num_of_columns / 2 } pub fn put_kzg_verified_blobs>>( &self, block_root: Hash256, - epoch: Epoch, kzg_verified_blobs: I, ) -> Result, AvailabilityCheckError> { let mut fixed_blobs = FixedVector::default(); @@ -496,12 +458,11 @@ impl DataAvailabilityCheckerInner { // Merge in the blobs. pending_components.merge_blobs(fixed_blobs); - let block_import_requirement = self.block_import_requirement(epoch)?; - if pending_components.is_available(&block_import_requirement) { + if pending_components.is_available(self.custody_column_count) { write_lock.put(block_root, pending_components.clone()); // No need to hold the write lock anymore drop(write_lock); - pending_components.make_available(block_import_requirement, &self.spec, |diet_block| { + pending_components.make_available(&self.spec, |diet_block| { self.state_cache.recover_pending_executed_block(diet_block) }) } else { @@ -532,13 +493,12 @@ impl DataAvailabilityCheckerInner { // Merge in the data columns. pending_components.merge_data_columns(kzg_verified_data_columns)?; - let block_import_requirement = self.block_import_requirement(epoch)?; - // Potentially trigger reconstruction if: // - Our custody requirement is all columns // - We >= 50% of columns + let is_peer_das_enabled = self.spec.is_peer_das_enabled_for_epoch(epoch); let data_columns_to_publish = - if self.should_reconstruct(&block_import_requirement, &pending_components) { + if is_peer_das_enabled && self.should_reconstruct(&pending_components) { pending_components.reconstruction_started(); let timer = metrics::start_timer(&metrics::DATA_AVAILABILITY_RECONSTRUCTION_TIME); @@ -577,12 +537,12 @@ impl DataAvailabilityCheckerInner { None }; - if pending_components.is_available(&block_import_requirement) { + if pending_components.is_available(self.custody_column_count) { write_lock.put(block_root, pending_components.clone()); // No need to hold the write lock anymore drop(write_lock); pending_components - .make_available(block_import_requirement, &self.spec, |diet_block| { + .make_available(&self.spec, |diet_block| { self.state_cache.recover_pending_executed_block(diet_block) }) .map(|availability| (availability, data_columns_to_publish)) @@ -603,7 +563,6 @@ impl DataAvailabilityCheckerInner { ) -> Result, AvailabilityCheckError> { let mut write_lock = self.critical.write(); let block_root = executed_block.import_data.block_root; - let epoch = executed_block.block.epoch(); // register the block to get the diet block let diet_executed_block = self @@ -620,12 +579,11 @@ impl DataAvailabilityCheckerInner { pending_components.merge_block(diet_executed_block); // Check if we have all components and entire set is consistent. - let block_import_requirement = self.block_import_requirement(epoch)?; - if pending_components.is_available(&block_import_requirement) { + if pending_components.is_available(self.custody_column_count) { write_lock.put(block_root, pending_components.clone()); // No need to hold the write lock anymore drop(write_lock); - pending_components.make_available(block_import_requirement, &self.spec, |diet_block| { + pending_components.make_available(&self.spec, |diet_block| { self.state_cache.recover_pending_executed_block(diet_block) }) } else { @@ -681,6 +639,7 @@ impl DataAvailabilityCheckerInner { #[cfg(test)] mod test { use super::*; + use crate::{ blob_verification::GossipVerifiedBlob, block_verification::PayloadVerificationOutcome, @@ -690,6 +649,7 @@ mod test { test_utils::{BaseHarnessType, BeaconChainHarness, DiskHarnessType}, }; use fork_choice::PayloadVerificationStatus; + use logging::test_logger; use slog::{info, Logger}; use state_processing::ConsensusContext; @@ -908,7 +868,6 @@ mod test { let (pending_block, blobs) = 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(); assert_eq!( @@ -957,7 +916,7 @@ mod test { for (blob_index, gossip_blob) in blobs.into_iter().enumerate() { kzg_verified_blobs.push(gossip_blob.into_inner()); let availability = cache - .put_kzg_verified_blobs(root, epoch, kzg_verified_blobs.clone()) + .put_kzg_verified_blobs(root, kzg_verified_blobs.clone()) .expect("should put blob"); if blob_index == blobs_expected - 1 { assert!(matches!(availability, Availability::Available(_))); @@ -979,12 +938,11 @@ mod test { "should have expected number of blobs" ); let root = pending_block.import_data.block_root; - let epoch = pending_block.block.epoch(); let mut kzg_verified_blobs = vec![]; for gossip_blob in blobs { kzg_verified_blobs.push(gossip_blob.into_inner()); let availability = cache - .put_kzg_verified_blobs(root, epoch, kzg_verified_blobs.clone()) + .put_kzg_verified_blobs(root, kzg_verified_blobs.clone()) .expect("should put blob"); assert_eq!( availability, diff --git a/beacon_node/beacon_chain/src/data_availability_checker/state_lru_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/state_lru_cache.rs index 03e3289118d..5b9b7c70233 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/state_lru_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/state_lru_cache.rs @@ -57,6 +57,11 @@ impl DietAvailabilityPendingExecutedBlock { .cloned() .unwrap_or_default() } + + /// Returns the epoch corresponding to `self.slot()`. + pub fn epoch(&self) -> Epoch { + self.block.slot().epoch(E::slots_per_epoch()) + } } /// This LRU cache holds BeaconStates used for block import. If the cache overflows, diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs new file mode 100644 index 00000000000..c14bb4c08cb --- /dev/null +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -0,0 +1,225 @@ +//! This module implements an optimisation to fetch blobs via JSON-RPC from the EL. +//! If a blob has already been seen in the public mempool, then it is often unnecessary to wait for +//! it to arrive on P2P gossip. This PR uses a new JSON-RPC method (`engine_getBlobsV1`) which +//! allows the CL to load the blobs quickly from the EL's blob pool. +//! +//! Once the node fetches the blobs from EL, it then publishes the remaining blobs that hasn't seen +//! on P2P gossip to the network. From PeerDAS onwards, together with the increase in blob count, +//! broadcasting blobs requires a much higher bandwidth, and is only done by high capacity +//! supernodes. +use crate::kzg_utils::blobs_to_data_column_sidecars; +use crate::observed_data_sidecars::ObservableDataSidecar; +use crate::{BeaconChain, BeaconChainError, BeaconChainTypes, BlockError, ExecutionPayloadError}; +use itertools::Either; +use slog::{debug, error, warn}; +use state_processing::per_block_processing::deneb::kzg_commitment_to_versioned_hash; +use std::sync::Arc; +use types::blob_sidecar::FixedBlobSidecarList; +use types::{ + BlobSidecar, DataColumnSidecarList, EthSpec, FullPayload, Hash256, SignedBeaconBlock, + VariableList, +}; + +pub enum BlobsOrDataColumns { + Blobs(Vec>>), + DataColumns(DataColumnSidecarList), +} + +/// Fetches blobs from the EL mempool and processes them. It also broadcasts unseen blobs or +/// data columns (PeerDAS onwards) to the network, using the supplied `publish_fn`. +pub async fn fetch_and_process_engine_blobs( + chain: Arc>, + block_root: Hash256, + block: Arc>>, + publish_fn: impl FnOnce(BlobsOrDataColumns) + Send + 'static, +) -> Result<(), BlockError> { + let versioned_hashes = + if let Ok(kzg_commitments) = block.message().body().blob_kzg_commitments() { + kzg_commitments + .iter() + .map(kzg_commitment_to_versioned_hash) + .collect() + } else { + vec![] + }; + let num_blobs = versioned_hashes.len(); + + if versioned_hashes.is_empty() { + debug!(chain.log, "Blobs from EL - none required"); + return Ok(()); + } + + let execution_layer = chain + .execution_layer + .as_ref() + .ok_or(BeaconChainError::ExecutionLayerMissing)?; + + debug!( + chain.log, + "Blobs from EL - start request"; + "num_blobs" => num_blobs, + ); + let response = execution_layer + .get_blobs(versioned_hashes) + .await + .map_err(|e| BlockError::ExecutionPayloadError(ExecutionPayloadError::RequestFailed(e)))?; + let num_fetched_blobs = response.iter().filter(|b| b.is_some()).count(); + let mut all_blobs_fetched = false; + if num_fetched_blobs == 0 { + debug!(chain.log, "Blobs from EL - response with none"); + return Ok(()); + } else if num_fetched_blobs < num_blobs { + debug!( + chain.log, + "Blobs from EL - response with some"; + "fetched" => num_fetched_blobs, + "total" => num_blobs, + ); + } else { + all_blobs_fetched = true; + debug!( + chain.log, + "Blobs from EL - response with all"; + "num_blobs" => num_blobs + ); + } + + let (signed_block_header, kzg_commitments_proof) = + block.signed_block_header_and_kzg_commitments_proof()?; + + let mut fixed_blob_sidecar_list = FixedBlobSidecarList::default(); + for (i, blob_and_proof) in response + .into_iter() + .enumerate() + .filter_map(|(i, opt_blob)| Some((i, opt_blob?))) + { + match BlobSidecar::new_with_existing_proof( + i, + blob_and_proof.blob, + &block, + signed_block_header.clone(), + &kzg_commitments_proof, + blob_and_proof.proof, + ) { + Ok(blob) => { + if let Some(blob_mut) = fixed_blob_sidecar_list.get_mut(i) { + *blob_mut = Some(Arc::new(blob)); + } else { + error!( + chain.log, + "Blobs from EL - out of bounds"; + "i" => i + ); + } + } + Err(e) => { + warn!( + chain.log, + "Blobs from EL - error"; + "error" => ?e + ); + } + } + } + + // Spawn an async task here for long computation tasks, so it doesn't block processing, and it + // allows blobs / data columns to propagate without waiting for processing. + // + // An `mpsc::Sender` is then used to send the produced data columns to the `beacon_chain` for it + // to be persisted, **after** the block is made attestable. + // + // The reason for doing this is to make the block available and attestable as soon as possible, + // while maintaining the invariant that block and data columns are persisted atomically. + let (data_columns_sender, data_columns_receiver) = tokio::sync::mpsc::channel(1); + let peer_das_enabled = chain.spec.is_peer_das_enabled_for_epoch(block.epoch()); + + // Partial blobs response isn't useful for PeerDAS, so we don't bother building and publishing data columns. + if peer_das_enabled && all_blobs_fetched { + let logger = chain.log.clone(); + let block_cloned = block.clone(); + let kzg = chain.kzg.clone().expect("KZG not initialized"); + let spec = chain.spec.clone(); + let blobs_cloned = fixed_blob_sidecar_list.clone(); + let chain_cloned = chain.clone(); + chain + .task_executor + .spawn_handle( + async move { + // TODO(das): inefficient allocation + let blob_vec = blobs_cloned.iter() + .filter_map(|b| b.as_ref().map(|b| b.blob.clone())) + .collect::>(); + let blob_list = VariableList::from(blob_vec); + let data_columns_result = + blobs_to_data_column_sidecars(&blob_list, &block_cloned, &kzg, &spec); + + let all_data_columns = match data_columns_result { + Ok(d) => d, + Err(e) => { + error!(logger, "Failed to build data column sidecars from blobs"; "error" => ?e); + return; + } + }; + + // Check indices from cache before sending the columns, to make sure we don't + // publish components already seen on gossip. + let all_data_columns_iter = all_data_columns.clone().into_iter(); + let data_columns_to_publish = match chain_cloned.data_availability_checker.imported_custody_column_indexes(&block_root) { + None => Either::Left(all_data_columns_iter), + Some(imported_columns_indices) => Either::Right(all_data_columns_iter.filter(move |d| !imported_columns_indices.contains(&d.index()))), + }.collect::>(); + + if let Err(e) = data_columns_sender.try_send(all_data_columns) { + error!(logger, "Failed to send computed data columns"; "error" => ?e); + }; + + let is_supernode = + chain_cloned.data_availability_checker.get_custody_columns_count() == spec.number_of_columns; + if is_supernode && !data_columns_to_publish.is_empty() { + publish_fn(BlobsOrDataColumns::DataColumns(data_columns_to_publish)); + } + }, + "compute_data_columns", + ) + .ok_or(BeaconChainError::RuntimeShutdown)?; + } else { + let all_blobs = fixed_blob_sidecar_list.clone(); + let all_blobs_iter = all_blobs.into_iter().flat_map(|b| b.clone()); + + let blobs_to_publish = match chain + .data_availability_checker + .imported_blob_indexes(&block_root) + { + None => Either::Left(all_blobs_iter), + Some(imported_blob_indices) => Either::Right( + all_blobs_iter.filter(move |b| !imported_blob_indices.contains(&b.index())), + ), + }; + + publish_fn(BlobsOrDataColumns::Blobs( + blobs_to_publish.collect::>(), + )); + }; + + debug!( + chain.log, + "Blobs from EL - start processing"; + "num_blobs" => num_blobs, + ); + + chain + .process_engine_blobs( + block.slot(), + block_root, + fixed_blob_sidecar_list.clone(), + peer_das_enabled.then_some(data_columns_receiver), + ) + .await + .map(|_| debug!(chain.log, "Blobs from EL - processed")) + .map_err(|e| { + warn!(chain.log, "Blobs from EL - error"; "error" => ?e); + e + })?; + + Ok(()) +} diff --git a/beacon_node/beacon_chain/src/lib.rs b/beacon_node/beacon_chain/src/lib.rs index 7bfb5b08beb..8ad62268979 100644 --- a/beacon_node/beacon_chain/src/lib.rs +++ b/beacon_node/beacon_chain/src/lib.rs @@ -28,6 +28,7 @@ pub mod eth1_chain; mod eth1_finalization_cache; pub mod events; pub mod execution_payload; +pub mod fetch_blobs; pub mod fork_choice_signal; pub mod fork_revert; pub mod graffiti_calculator; diff --git a/beacon_node/beacon_chain/src/metrics.rs b/beacon_node/beacon_chain/src/metrics.rs index 3394946255f..d315c2d1f5f 100644 --- a/beacon_node/beacon_chain/src/metrics.rs +++ b/beacon_node/beacon_chain/src/metrics.rs @@ -110,6 +110,12 @@ pub static BLOCK_PROCESSING_POST_EXEC_PROCESSING: LazyLock> = linear_buckets(5e-3, 5e-3, 10), ) }); +pub static BLOCK_PROCESSING_DATA_COLUMNS_WAIT: LazyLock> = LazyLock::new(|| { + try_create_histogram( + "beacon_block_processing_data_columns_wait_seconds", + "Time spent waiting for data columns to be computed before starting database write", + ) +}); pub static BLOCK_PROCESSING_DB_WRITE: LazyLock> = LazyLock::new(|| { try_create_histogram( "beacon_block_processing_db_write_seconds", diff --git a/beacon_node/beacon_chain/src/test_utils.rs b/beacon_node/beacon_chain/src/test_utils.rs index b28d221da7e..df914594356 100644 --- a/beacon_node/beacon_chain/src/test_utils.rs +++ b/beacon_node/beacon_chain/src/test_utils.rs @@ -2700,7 +2700,6 @@ pub fn generate_rand_block_and_blobs( (block, blob_sidecars) } -#[allow(clippy::type_complexity)] pub fn generate_rand_block_and_data_columns( fork_name: ForkName, num_blobs: NumBlobs, @@ -2708,7 +2707,7 @@ pub fn generate_rand_block_and_data_columns( spec: &ChainSpec, ) -> ( SignedBeaconBlock>, - Vec>>, + DataColumnSidecarList, ) { let (block, blobs) = generate_rand_block_and_blobs(fork_name, num_blobs, rng); let blob: BlobsList = blobs.into_iter().map(|b| b.blob).collect::>().into(); diff --git a/beacon_node/beacon_chain/tests/block_verification.rs b/beacon_node/beacon_chain/tests/block_verification.rs index 1c494d99bf5..88540fa3bdc 100644 --- a/beacon_node/beacon_chain/tests/block_verification.rs +++ b/beacon_node/beacon_chain/tests/block_verification.rs @@ -1726,7 +1726,7 @@ async fn import_execution_pending_block( .unwrap() { ExecutedBlock::Available(block) => chain - .import_available_block(Box::from(block)) + .import_available_block(Box::from(block), None) .await .map_err(|e| format!("{e:?}")), ExecutedBlock::AvailabilityPending(_) => { diff --git a/beacon_node/execution_layer/src/engine_api.rs b/beacon_node/execution_layer/src/engine_api.rs index 316b0dfa86a..843fa1b52d4 100644 --- a/beacon_node/execution_layer/src/engine_api.rs +++ b/beacon_node/execution_layer/src/engine_api.rs @@ -1,7 +1,7 @@ use crate::engines::ForkchoiceState; use crate::http::{ ENGINE_FORKCHOICE_UPDATED_V1, ENGINE_FORKCHOICE_UPDATED_V2, ENGINE_FORKCHOICE_UPDATED_V3, - ENGINE_GET_CLIENT_VERSION_V1, ENGINE_GET_PAYLOAD_BODIES_BY_HASH_V1, + ENGINE_GET_BLOBS_V1, ENGINE_GET_CLIENT_VERSION_V1, ENGINE_GET_PAYLOAD_BODIES_BY_HASH_V1, ENGINE_GET_PAYLOAD_BODIES_BY_HASH_V2, ENGINE_GET_PAYLOAD_BODIES_BY_RANGE_V1, ENGINE_GET_PAYLOAD_BODIES_BY_RANGE_V2, ENGINE_GET_PAYLOAD_V1, ENGINE_GET_PAYLOAD_V2, ENGINE_GET_PAYLOAD_V3, ENGINE_GET_PAYLOAD_V4, ENGINE_NEW_PAYLOAD_V1, ENGINE_NEW_PAYLOAD_V2, @@ -789,6 +789,7 @@ pub struct EngineCapabilities { pub get_payload_v3: bool, pub get_payload_v4: bool, pub get_client_version_v1: bool, + pub get_blobs_v1: bool, } impl EngineCapabilities { @@ -842,6 +843,9 @@ impl EngineCapabilities { if self.get_client_version_v1 { response.push(ENGINE_GET_CLIENT_VERSION_V1); } + if self.get_blobs_v1 { + response.push(ENGINE_GET_BLOBS_V1); + } response } diff --git a/beacon_node/execution_layer/src/engine_api/http.rs b/beacon_node/execution_layer/src/engine_api/http.rs index 1838d13bb06..6719e16d0ed 100644 --- a/beacon_node/execution_layer/src/engine_api/http.rs +++ b/beacon_node/execution_layer/src/engine_api/http.rs @@ -60,6 +60,9 @@ pub const ENGINE_EXCHANGE_CAPABILITIES_TIMEOUT: Duration = Duration::from_secs(1 pub const ENGINE_GET_CLIENT_VERSION_V1: &str = "engine_getClientVersionV1"; pub const ENGINE_GET_CLIENT_VERSION_TIMEOUT: Duration = Duration::from_secs(1); +pub const ENGINE_GET_BLOBS_V1: &str = "engine_getBlobsV1"; +pub const ENGINE_GET_BLOBS_TIMEOUT: Duration = Duration::from_secs(1); + /// This error is returned during a `chainId` call by Geth. pub const EIP155_ERROR_STR: &str = "chain not synced beyond EIP-155 replay-protection fork block"; /// This code is returned by all clients when a method is not supported @@ -83,6 +86,7 @@ pub static LIGHTHOUSE_CAPABILITIES: &[&str] = &[ ENGINE_GET_PAYLOAD_BODIES_BY_HASH_V2, ENGINE_GET_PAYLOAD_BODIES_BY_RANGE_V2, ENGINE_GET_CLIENT_VERSION_V1, + ENGINE_GET_BLOBS_V1, ]; /// We opt to initialize the JsonClientVersionV1 rather than the ClientVersionV1 @@ -706,6 +710,20 @@ impl HttpJsonRpc { } } + pub async fn get_blobs( + &self, + versioned_hashes: Vec, + ) -> Result>>, Error> { + let params = json!([versioned_hashes]); + + self.rpc_request( + ENGINE_GET_BLOBS_V1, + params, + ENGINE_GET_BLOBS_TIMEOUT * self.execution_timeout_multiplier, + ) + .await + } + pub async fn get_block_by_number<'a>( &self, query: BlockByNumberQuery<'a>, @@ -1156,6 +1174,7 @@ impl HttpJsonRpc { get_payload_v3: capabilities.contains(ENGINE_GET_PAYLOAD_V3), get_payload_v4: capabilities.contains(ENGINE_GET_PAYLOAD_V4), get_client_version_v1: capabilities.contains(ENGINE_GET_CLIENT_VERSION_V1), + get_blobs_v1: capabilities.contains(ENGINE_GET_BLOBS_V1), }) } diff --git a/beacon_node/execution_layer/src/engine_api/json_structures.rs b/beacon_node/execution_layer/src/engine_api/json_structures.rs index 00572150e05..8256db574dc 100644 --- a/beacon_node/execution_layer/src/engine_api/json_structures.rs +++ b/beacon_node/execution_layer/src/engine_api/json_structures.rs @@ -5,7 +5,10 @@ use strum::EnumString; use superstruct::superstruct; use types::beacon_block_body::KzgCommitments; use types::blob_sidecar::BlobsList; -use types::{DepositRequest, FixedVector, PublicKeyBytes, Signature, Unsigned, WithdrawalRequest}; +use types::{ + Blob, DepositRequest, FixedVector, KzgProof, PublicKeyBytes, Signature, Unsigned, + WithdrawalRequest, +}; #[derive(Debug, PartialEq, Serialize, Deserialize)] #[serde(rename_all = "camelCase")] @@ -594,6 +597,14 @@ impl From> for BlobsBundle { } } +#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] +#[serde(bound = "E: EthSpec", rename_all = "camelCase")] +pub struct BlobAndProofV1 { + #[serde(with = "ssz_types::serde_utils::hex_fixed_vec")] + pub blob: Blob, + pub proof: KzgProof, +} + #[derive(Debug, PartialEq, Clone, Serialize, Deserialize)] #[serde(rename_all = "camelCase")] pub struct JsonForkchoiceStateV1 { diff --git a/beacon_node/execution_layer/src/lib.rs b/beacon_node/execution_layer/src/lib.rs index 0da9225173c..bb1bdc16569 100644 --- a/beacon_node/execution_layer/src/lib.rs +++ b/beacon_node/execution_layer/src/lib.rs @@ -4,6 +4,7 @@ //! This crate only provides useful functionality for "The Merge", it does not provide any of the //! deposit-contract functionality that the `beacon_node/eth1` crate already provides. +use crate::json_structures::BlobAndProofV1; use crate::payload_cache::PayloadCache; use arc_swap::ArcSwapOption; use auth::{strip_prefix, Auth, JwtKey}; @@ -63,7 +64,7 @@ mod metrics; pub mod payload_cache; mod payload_status; pub mod test_utils; -mod versioned_hashes; +pub mod versioned_hashes; /// Indicates the default jwt authenticated execution endpoint. pub const DEFAULT_EXECUTION_ENDPOINT: &str = "http://localhost:8551/"; @@ -1843,6 +1844,23 @@ impl ExecutionLayer { } } + pub async fn get_blobs( + &self, + query: Vec, + ) -> Result>>, Error> { + let capabilities = self.get_engine_capabilities(None).await?; + + if capabilities.get_blobs_v1 { + self.engine() + .request(|engine| async move { engine.api.get_blobs(query).await }) + .await + .map_err(Box::new) + .map_err(Error::EngineError) + } else { + Ok(vec![None; query.len()]) + } + } + pub async fn get_block_by_number( &self, query: BlockByNumberQuery<'_>, diff --git a/beacon_node/execution_layer/src/test_utils/mod.rs b/beacon_node/execution_layer/src/test_utils/mod.rs index c6bcdf4e444..8286dc1edaa 100644 --- a/beacon_node/execution_layer/src/test_utils/mod.rs +++ b/beacon_node/execution_layer/src/test_utils/mod.rs @@ -55,6 +55,7 @@ pub const DEFAULT_ENGINE_CAPABILITIES: EngineCapabilities = EngineCapabilities { get_payload_v3: true, get_payload_v4: true, get_client_version_v1: true, + get_blobs_v1: true, }; pub static DEFAULT_CLIENT_VERSION: LazyLock = 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 d5d83d540a0..463c2555ade 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -944,7 +944,7 @@ impl NetworkBeaconProcessor { metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL); info!( self.log, - "Gossipsub blob processed, imported fully available block"; + "Gossipsub blob processed - imported fully available block"; "block_root" => %block_root ); self.chain.recompute_head_at_current_slot().await; @@ -955,9 +955,9 @@ impl NetworkBeaconProcessor { ); } Ok(AvailabilityProcessingStatus::MissingComponents(slot, block_root)) => { - trace!( + debug!( self.log, - "Processed blob, waiting for other components"; + "Processed gossip blob - waiting for other components"; "slot" => %slot, "blob_index" => %blob_index, "block_root" => %block_root, @@ -1100,7 +1100,7 @@ impl NetworkBeaconProcessor { message_id, peer_id, peer_client, - block, + block.clone(), reprocess_tx.clone(), seen_duration, ) @@ -1515,6 +1515,9 @@ impl NetworkBeaconProcessor { "slot" => slot, "block_root" => %block_root, ); + + self.fetch_engine_blobs_and_publish(block.clone(), *block_root) + .await; } Err(BlockError::ParentUnknown(_)) => { // This should not occur. It should be checked by `should_forward_block`. diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 7f551c544c7..4640b973ac7 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -2,6 +2,7 @@ use crate::sync::manager::BlockProcessType; use crate::sync::SamplingId; use crate::{service::NetworkMessage, sync::manager::SyncMessage}; use beacon_chain::block_verification_types::RpcBlock; +use beacon_chain::fetch_blobs::{fetch_and_process_engine_blobs, BlobsOrDataColumns}; use beacon_chain::{builder::Witness, eth1_chain::CachingEth1Backend, BeaconChain}; use beacon_chain::{BeaconChainTypes, NotifyExecutionLayer}; use beacon_processor::{ @@ -14,9 +15,9 @@ use lighthouse_network::rpc::methods::{ }; use lighthouse_network::{ rpc::{BlocksByRangeRequest, BlocksByRootRequest, LightClientBootstrapRequest, StatusMessage}, - Client, MessageId, NetworkGlobals, PeerId, PeerRequestId, + Client, MessageId, NetworkGlobals, PeerId, PeerRequestId, PubsubMessage, }; -use slog::{debug, Logger}; +use slog::{debug, error, Logger}; use slot_clock::ManualSlotClock; use std::path::PathBuf; use std::sync::Arc; @@ -767,6 +768,56 @@ impl NetworkBeaconProcessor { "error" => %e) }); } + + fn publish_blobs_or_data_column(&self, blobs_or_data_column: BlobsOrDataColumns) { + let messages = match blobs_or_data_column { + BlobsOrDataColumns::Blobs(blobs) => blobs + .into_iter() + .map(|blob| PubsubMessage::BlobSidecar(Box::new((blob.index, blob)))) + .collect(), + BlobsOrDataColumns::DataColumns(columns) => columns + .into_iter() + .map(|column| { + let subnet = DataColumnSubnetId::from_column_index::( + column.index as usize, + &self.chain.spec, + ); + PubsubMessage::DataColumnSidecar(Box::new((subnet, column))) + }) + .collect(), + }; + self.send_network_message(NetworkMessage::Publish { messages }) + } + + pub async fn fetch_engine_blobs_and_publish( + self: &Arc, + block: Arc>>, + block_root: Hash256, + ) { + let self_cloned = self.clone(); + let log_cloned = self.log.clone(); + let publish_fn = move |blobs_or_data_column| { + match &blobs_or_data_column { + BlobsOrDataColumns::Blobs(b) => { + debug!(log_cloned, "Publishing blobs from EL"; "count" => b.len()); + } + BlobsOrDataColumns::DataColumns(d) => { + debug!(log_cloned, "Publishing data columns built from EL blobs"; "count" => d.len()); + } + } + self_cloned.publish_blobs_or_data_column(blobs_or_data_column) + }; + if let Err(e) = fetch_and_process_engine_blobs( + self.chain.clone(), + block_root, + block.clone(), + publish_fn, + ) + .await + { + error!(self.log, "Error fetching or processing blobs from EL"; "error" => ?e); + } + } } type TestBeaconChainType = 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 508576d9f52..02ec255f054 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -153,6 +153,7 @@ impl NetworkBeaconProcessor { "process_type" => ?process_type, ); + let signed_beacon_block = block.block_cloned(); let result = self .chain .process_block_with_early_caching( @@ -166,26 +167,33 @@ impl NetworkBeaconProcessor { metrics::inc_counter(&metrics::BEACON_PROCESSOR_RPC_BLOCK_IMPORTED_TOTAL); // RPC block imported, regardless of process type - if let &Ok(AvailabilityProcessingStatus::Imported(hash)) = &result { - info!(self.log, "New RPC block received"; "slot" => slot, "hash" => %hash); + match result.as_ref() { + Ok(AvailabilityProcessingStatus::Imported(hash)) => { + info!(self.log, "New RPC block received"; "slot" => slot, "hash" => %hash); - // Trigger processing for work referencing this block. - let reprocess_msg = ReprocessQueueMessage::BlockImported { - block_root: hash, - parent_root, - }; - if reprocess_tx.try_send(reprocess_msg).is_err() { - error!(self.log, "Failed to inform block import"; "source" => "rpc", "block_root" => %hash) - }; - self.chain.block_times_cache.write().set_time_observed( - hash, - slot, - seen_timestamp, - None, - None, - ); + // Trigger processing for work referencing this block. + let reprocess_msg = ReprocessQueueMessage::BlockImported { + block_root: *hash, + parent_root, + }; + if reprocess_tx.try_send(reprocess_msg).is_err() { + error!(self.log, "Failed to inform block import"; "source" => "rpc", "block_root" => %hash) + }; + self.chain.block_times_cache.write().set_time_observed( + *hash, + slot, + seen_timestamp, + None, + None, + ); - self.chain.recompute_head_at_current_slot().await; + self.chain.recompute_head_at_current_slot().await; + } + Ok(AvailabilityProcessingStatus::MissingComponents(..)) => { + self.fetch_engine_blobs_and_publish(signed_beacon_block, block_root) + .await + } + _ => {} } // RPC block imported or execution validated. If the block was already imported by gossip we diff --git a/consensus/types/src/beacon_block_body.rs b/consensus/types/src/beacon_block_body.rs index 305ef105445..d1de036d661 100644 --- a/consensus/types/src/beacon_block_body.rs +++ b/consensus/types/src/beacon_block_body.rs @@ -176,57 +176,71 @@ impl<'a, E: EthSpec, Payload: AbstractExecPayload> BeaconBlockBodyRef<'a, E, leaves } - /// Produces the proof of inclusion for a `KzgCommitment` in `self.blob_kzg_commitments` - /// at `index`. + /// Calculate a KZG commitment merkle proof slowly. + /// + /// Prefer to use `complete_kzg_commitment_merkle_proof` with a reused proof for the + /// `blob_kzg_commitments` field. pub fn kzg_commitment_merkle_proof( &self, index: usize, ) -> Result, Error> { - // We compute the branches by generating 2 merkle trees: - // 1. Merkle tree for the `blob_kzg_commitments` List object - // 2. Merkle tree for the `BeaconBlockBody` container - // We then merge the branches for both the trees all the way up to the root. - - // Part1 (Branches for the subtree rooted at `blob_kzg_commitments`) - // - // Branches for `blob_kzg_commitments` without length mix-in - let blob_leaves = self - .blob_kzg_commitments()? - .iter() - .map(|commitment| commitment.tree_hash_root()) - .collect::>(); - let depth = E::max_blob_commitments_per_block() - .next_power_of_two() - .ilog2(); - let tree = MerkleTree::create(&blob_leaves, depth as usize); - let (_, mut proof) = tree - .generate_proof(index, depth as usize) - .map_err(Error::MerkleTreeError)?; - - // Add the branch corresponding to the length mix-in. - let length = blob_leaves.len(); - let usize_len = std::mem::size_of::(); - let mut length_bytes = [0; BYTES_PER_CHUNK]; - length_bytes - .get_mut(0..usize_len) - .ok_or(Error::MerkleTreeError(MerkleTreeError::PleaseNotifyTheDevs))? - .copy_from_slice(&length.to_le_bytes()); - let length_root = Hash256::from_slice(length_bytes.as_slice()); - proof.push(length_root); - - // Part 2 - // Branches for `BeaconBlockBody` container - let body_leaves = self.body_merkle_leaves(); - let beacon_block_body_depth = body_leaves.len().next_power_of_two().ilog2() as usize; - let tree = MerkleTree::create(&body_leaves, beacon_block_body_depth); - let (_, mut proof_body) = tree - .generate_proof(BLOB_KZG_COMMITMENTS_INDEX, beacon_block_body_depth) - .map_err(Error::MerkleTreeError)?; - // Join the proofs for the subtree and the main tree - proof.append(&mut proof_body); - debug_assert_eq!(proof.len(), E::kzg_proof_inclusion_proof_depth()); + let kzg_commitments_proof = self.kzg_commitments_merkle_proof()?; + let proof = self.complete_kzg_commitment_merkle_proof(index, &kzg_commitments_proof)?; + Ok(proof) + } - Ok(proof.into()) + /// Produces the proof of inclusion for a `KzgCommitment` in `self.blob_kzg_commitments` + /// at `index` using an existing proof for the `blob_kzg_commitments` field. + pub fn complete_kzg_commitment_merkle_proof( + &self, + index: usize, + kzg_commitments_proof: &[Hash256], + ) -> Result, Error> { + match self { + Self::Base(_) | Self::Altair(_) | Self::Bellatrix(_) | Self::Capella(_) => { + Err(Error::IncorrectStateVariant) + } + Self::Deneb(_) | Self::Electra(_) => { + // We compute the branches by generating 2 merkle trees: + // 1. Merkle tree for the `blob_kzg_commitments` List object + // 2. Merkle tree for the `BeaconBlockBody` container + // We then merge the branches for both the trees all the way up to the root. + + // Part1 (Branches for the subtree rooted at `blob_kzg_commitments`) + // + // Branches for `blob_kzg_commitments` without length mix-in + let blob_leaves = self + .blob_kzg_commitments()? + .iter() + .map(|commitment| commitment.tree_hash_root()) + .collect::>(); + let depth = E::max_blob_commitments_per_block() + .next_power_of_two() + .ilog2(); + let tree = MerkleTree::create(&blob_leaves, depth as usize); + let (_, mut proof) = tree + .generate_proof(index, depth as usize) + .map_err(Error::MerkleTreeError)?; + + // Add the branch corresponding to the length mix-in. + let length = blob_leaves.len(); + let usize_len = std::mem::size_of::(); + let mut length_bytes = [0; BYTES_PER_CHUNK]; + length_bytes + .get_mut(0..usize_len) + .ok_or(Error::MerkleTreeError(MerkleTreeError::PleaseNotifyTheDevs))? + .copy_from_slice(&length.to_le_bytes()); + let length_root = Hash256::from_slice(length_bytes.as_slice()); + proof.push(length_root); + + // Part 2 + // Branches for `BeaconBlockBody` container + // Join the proofs for the subtree and the main tree + proof.extend_from_slice(kzg_commitments_proof); + + Ok(FixedVector::new(proof)?) + } + } } /// Produces the proof of inclusion for `self.blob_kzg_commitments`. @@ -239,7 +253,7 @@ impl<'a, E: EthSpec, Payload: AbstractExecPayload> BeaconBlockBodyRef<'a, E, let (_, proof) = tree .generate_proof(BLOB_KZG_COMMITMENTS_INDEX, beacon_block_body_depth) .map_err(Error::MerkleTreeError)?; - Ok(proof.into()) + Ok(FixedVector::new(proof)?) } pub fn block_body_merkle_proof(&self, generalized_index: usize) -> Result, Error> { diff --git a/consensus/types/src/blob_sidecar.rs b/consensus/types/src/blob_sidecar.rs index 0f7dbb2673c..5a330388cce 100644 --- a/consensus/types/src/blob_sidecar.rs +++ b/consensus/types/src/blob_sidecar.rs @@ -150,6 +150,37 @@ impl BlobSidecar { }) } + pub fn new_with_existing_proof( + index: usize, + blob: Blob, + signed_block: &SignedBeaconBlock, + signed_block_header: SignedBeaconBlockHeader, + kzg_commitments_inclusion_proof: &[Hash256], + kzg_proof: KzgProof, + ) -> Result { + let expected_kzg_commitments = signed_block + .message() + .body() + .blob_kzg_commitments() + .map_err(|_e| BlobSidecarError::PreDeneb)?; + let kzg_commitment = *expected_kzg_commitments + .get(index) + .ok_or(BlobSidecarError::MissingKzgCommitment)?; + let kzg_commitment_inclusion_proof = signed_block + .message() + .body() + .complete_kzg_commitment_merkle_proof(index, kzg_commitments_inclusion_proof)?; + + Ok(Self { + index: index as u64, + blob, + kzg_commitment, + kzg_proof, + signed_block_header, + kzg_commitment_inclusion_proof, + }) + } + pub fn id(&self) -> BlobIdentifier { BlobIdentifier { block_root: self.block_root(), diff --git a/consensus/types/src/signed_beacon_block.rs b/consensus/types/src/signed_beacon_block.rs index 4d3279a7f77..04068a53259 100644 --- a/consensus/types/src/signed_beacon_block.rs +++ b/consensus/types/src/signed_beacon_block.rs @@ -239,6 +239,37 @@ impl> SignedBeaconBlock } } + /// Produce a signed beacon block header AND a merkle proof for the KZG commitments. + /// + /// This method is more efficient than generating each part separately as it reuses hashing. + pub fn signed_block_header_and_kzg_commitments_proof( + &self, + ) -> Result< + ( + SignedBeaconBlockHeader, + FixedVector, + ), + Error, + > { + let proof = self.message().body().kzg_commitments_merkle_proof()?; + let body_root = *proof.last().unwrap(); + + let block_header = BeaconBlockHeader { + slot: self.slot(), + proposer_index: self.message().proposer_index(), + parent_root: self.parent_root(), + state_root: self.state_root(), + body_root, + }; + + let signed_header = SignedBeaconBlockHeader { + message: block_header, + signature: self.signature().clone(), + }; + + Ok((signed_header, proof)) + } + /// Convenience accessor for the block's slot. pub fn slot(&self) -> Slot { self.message().slot() From 0c2384830b91d73cb350c426ebe38ee2e1c39ae0 Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Thu, 29 Aug 2024 11:40:20 +1000 Subject: [PATCH 02/25] Avoid cloning blobs after fetching blobs. --- beacon_node/beacon_chain/benches/benches.rs | 12 +++++++---- .../beacon_chain/src/block_verification.rs | 3 ++- beacon_node/beacon_chain/src/fetch_blobs.rs | 13 ++++-------- beacon_node/beacon_chain/src/kzg_utils.rs | 21 +++++++++++-------- beacon_node/beacon_chain/src/test_utils.rs | 5 +++-- 5 files changed, 29 insertions(+), 25 deletions(-) diff --git a/beacon_node/beacon_chain/benches/benches.rs b/beacon_node/beacon_chain/benches/benches.rs index 4a29be90251..4cb50178168 100644 --- a/beacon_node/beacon_chain/benches/benches.rs +++ b/beacon_node/beacon_chain/benches/benches.rs @@ -42,11 +42,15 @@ fn all_benches(c: &mut Criterion) { for blob_count in [1, 2, 3, 6] { let kzg = kzg.clone(); - let (signed_block, blob_sidecars) = create_test_block_and_blobs::(blob_count, &spec); + let (signed_block, blobs) = create_test_block_and_blobs::(blob_count, &spec); - let column_sidecars = - blobs_to_data_column_sidecars(&blob_sidecars, &signed_block, &kzg.clone(), &spec) - .unwrap(); + let column_sidecars = blobs_to_data_column_sidecars( + &blobs.iter().collect::>(), + &signed_block, + &kzg.clone(), + &spec, + ) + .unwrap(); let spec = spec.clone(); diff --git a/beacon_node/beacon_chain/src/block_verification.rs b/beacon_node/beacon_chain/src/block_verification.rs index 5eda8c1d1bf..c4377598783 100644 --- a/beacon_node/beacon_chain/src/block_verification.rs +++ b/beacon_node/beacon_chain/src/block_verification.rs @@ -804,7 +804,8 @@ fn build_gossip_verified_data_columns( let timer = metrics::start_timer(&metrics::DATA_COLUMN_SIDECAR_COMPUTATION); - let sidecars = blobs_to_data_column_sidecars(&blobs, block, kzg, &chain.spec)?; + let blob_refs = blobs.iter().collect::>(); + let sidecars = blobs_to_data_column_sidecars(&blob_refs, block, kzg, &chain.spec)?; drop(timer); let mut gossip_verified_data_columns = vec![]; for sidecar in sidecars { diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index c14bb4c08cb..9ee70ac474d 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -15,10 +15,7 @@ use slog::{debug, error, warn}; use state_processing::per_block_processing::deneb::kzg_commitment_to_versioned_hash; use std::sync::Arc; use types::blob_sidecar::FixedBlobSidecarList; -use types::{ - BlobSidecar, DataColumnSidecarList, EthSpec, FullPayload, Hash256, SignedBeaconBlock, - VariableList, -}; +use types::{BlobSidecar, DataColumnSidecarList, EthSpec, FullPayload, Hash256, SignedBeaconBlock}; pub enum BlobsOrDataColumns { Blobs(Vec>>), @@ -145,13 +142,11 @@ pub async fn fetch_and_process_engine_blobs( .task_executor .spawn_handle( async move { - // TODO(das): inefficient allocation - let blob_vec = blobs_cloned.iter() - .filter_map(|b| b.as_ref().map(|b| b.blob.clone())) + let blob_refs = blobs_cloned.iter() + .filter_map(|b| b.as_ref().map(|b| &b.blob)) .collect::>(); - let blob_list = VariableList::from(blob_vec); let data_columns_result = - blobs_to_data_column_sidecars(&blob_list, &block_cloned, &kzg, &spec); + blobs_to_data_column_sidecars(&blob_refs, &block_cloned, &kzg, &spec); let all_data_columns = match data_columns_result { Ok(d) => d, diff --git a/beacon_node/beacon_chain/src/kzg_utils.rs b/beacon_node/beacon_chain/src/kzg_utils.rs index 55c1ee9e980..7f6293dc641 100644 --- a/beacon_node/beacon_chain/src/kzg_utils.rs +++ b/beacon_node/beacon_chain/src/kzg_utils.rs @@ -7,8 +7,8 @@ use std::sync::Arc; use types::beacon_block_body::KzgCommitments; use types::data_column_sidecar::{Cell, DataColumn, DataColumnSidecarError}; use types::{ - Blob, BlobsList, ChainSpec, ColumnIndex, DataColumnSidecar, DataColumnSidecarList, EthSpec, - Hash256, KzgCommitment, KzgProof, KzgProofs, SignedBeaconBlock, SignedBeaconBlockHeader, + Blob, ChainSpec, ColumnIndex, DataColumnSidecar, DataColumnSidecarList, EthSpec, Hash256, + KzgCommitment, KzgProof, KzgProofs, SignedBeaconBlock, SignedBeaconBlockHeader, }; /// Converts a blob ssz List object to an array to be used with the kzg @@ -142,7 +142,7 @@ pub fn verify_kzg_proof( /// Build data column sidecars from a signed beacon block and its blobs. pub fn blobs_to_data_column_sidecars( - blobs: &BlobsList, + blobs: &[&Blob], block: &SignedBeaconBlock, kzg: &Kzg, spec: &ChainSpec, @@ -309,19 +309,21 @@ mod test { #[track_caller] fn test_build_data_columns_empty(kzg: &Kzg, spec: &ChainSpec) { let num_of_blobs = 0; - let (signed_block, blob_sidecars) = create_test_block_and_blobs::(num_of_blobs, spec); + let (signed_block, blobs) = create_test_block_and_blobs::(num_of_blobs, spec); + let blob_refs = blobs.iter().collect::>(); let column_sidecars = - blobs_to_data_column_sidecars(&blob_sidecars, &signed_block, kzg, spec).unwrap(); + blobs_to_data_column_sidecars(&blob_refs, &signed_block, kzg, spec).unwrap(); assert!(column_sidecars.is_empty()); } #[track_caller] fn test_build_data_columns(kzg: &Kzg, spec: &ChainSpec) { let num_of_blobs = 6; - let (signed_block, blob_sidecars) = create_test_block_and_blobs::(num_of_blobs, spec); + let (signed_block, blobs) = create_test_block_and_blobs::(num_of_blobs, spec); + let blob_refs = blobs.iter().collect::>(); let column_sidecars = - blobs_to_data_column_sidecars(&blob_sidecars, &signed_block, kzg, spec).unwrap(); + blobs_to_data_column_sidecars(&blob_refs, &signed_block, kzg, spec).unwrap(); let block_kzg_commitments = signed_block .message() @@ -355,9 +357,10 @@ mod test { #[track_caller] fn test_reconstruct_data_columns(kzg: &Kzg, spec: &ChainSpec) { let num_of_blobs = 6; - let (signed_block, blob_sidecars) = create_test_block_and_blobs::(num_of_blobs, spec); + let (signed_block, blobs) = create_test_block_and_blobs::(num_of_blobs, spec); + let blob_refs = blobs.iter().collect::>(); let column_sidecars = - blobs_to_data_column_sidecars(&blob_sidecars, &signed_block, kzg, spec).unwrap(); + blobs_to_data_column_sidecars(&blob_refs, &signed_block, kzg, spec).unwrap(); // Now reconstruct let reconstructed_columns = reconstruct_data_columns( diff --git a/beacon_node/beacon_chain/src/test_utils.rs b/beacon_node/beacon_chain/src/test_utils.rs index df914594356..646d94719a9 100644 --- a/beacon_node/beacon_chain/src/test_utils.rs +++ b/beacon_node/beacon_chain/src/test_utils.rs @@ -2710,8 +2710,9 @@ pub fn generate_rand_block_and_data_columns( DataColumnSidecarList, ) { let (block, blobs) = generate_rand_block_and_blobs(fork_name, num_blobs, rng); - let blob: BlobsList = blobs.into_iter().map(|b| b.blob).collect::>().into(); - let data_columns = blobs_to_data_column_sidecars(&blob, &block, &KZG_PEERDAS, spec).unwrap(); + let blob_refs = blobs.iter().map(|b| &b.blob).collect::>(); + let data_columns = + blobs_to_data_column_sidecars(&blob_refs, &block, &KZG_PEERDAS, spec).unwrap(); (block, data_columns) } From 89dfaaac800c93af702f01cd08792ac314f38d4c Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Thu, 29 Aug 2024 12:41:51 +1000 Subject: [PATCH 03/25] Address review comments and refactor code. --- beacon_node/beacon_chain/src/beacon_chain.rs | 2 +- beacon_node/beacon_chain/src/fetch_blobs.rs | 215 +++++++++++------- beacon_node/beacon_chain/src/metrics.rs | 14 ++ .../gossip_methods.rs | 3 + .../src/network_beacon_processor/mod.rs | 29 +-- .../network_beacon_processor/sync_methods.rs | 3 + 6 files changed, 169 insertions(+), 97 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 756fe70bb1a..98b28adc7dc 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -3879,7 +3879,7 @@ impl BeaconChain { .unwrap_or(BlockError::InternalError(err_msg.to_string()))); } } - // Non data columns present and compute data columns task was not spawned. + // No data columns present and compute data columns task was not spawned. // Could either be no blobs in the block or before PeerDAS activation. (None, None) => None, }; diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index 9ee70ac474d..fff31fa50a7 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -7,21 +7,42 @@ //! on P2P gossip to the network. From PeerDAS onwards, together with the increase in blob count, //! broadcasting blobs requires a much higher bandwidth, and is only done by high capacity //! supernodes. -use crate::kzg_utils::blobs_to_data_column_sidecars; -use crate::observed_data_sidecars::ObservableDataSidecar; -use crate::{BeaconChain, BeaconChainError, BeaconChainTypes, BlockError, ExecutionPayloadError}; +use std::sync::Arc; + +use execution_layer::json_structures::BlobAndProofV1; +use execution_layer::Error as ExecutionLayerError; use itertools::Either; use slog::{debug, error, warn}; +use ssz_types::FixedVector; + +use lighthouse_metrics::inc_counter; use state_processing::per_block_processing::deneb::kzg_commitment_to_versioned_hash; -use std::sync::Arc; -use types::blob_sidecar::FixedBlobSidecarList; -use types::{BlobSidecar, DataColumnSidecarList, EthSpec, FullPayload, Hash256, SignedBeaconBlock}; +use types::blob_sidecar::{BlobSidecarError, FixedBlobSidecarList}; +use types::{ + BeaconStateError, BlobSidecar, DataColumnSidecarList, EthSpec, FullPayload, Hash256, + SignedBeaconBlock, SignedBeaconBlockHeader, +}; + +use crate::kzg_utils::blobs_to_data_column_sidecars; +use crate::observed_data_sidecars::ObservableDataSidecar; +use crate::{metrics, BeaconChain, BeaconChainTypes, BlockError}; pub enum BlobsOrDataColumns { Blobs(Vec>>), DataColumns(DataColumnSidecarList), } +#[derive(Debug)] +pub enum FetchEngineBlobError { + BeaconStateError(BeaconStateError), + BlobProcessingError(BlockError), + BlobSidecarError(BlobSidecarError), + ExecutionLayerMissing, + InternalError(String), + RequestFailed(ExecutionLayerError), + RuntimeShutdown, +} + /// Fetches blobs from the EL mempool and processes them. It also broadcasts unseen blobs or /// data columns (PeerDAS onwards) to the network, using the supplied `publish_fn`. pub async fn fetch_and_process_engine_blobs( @@ -29,7 +50,7 @@ pub async fn fetch_and_process_engine_blobs( block_root: Hash256, block: Arc>>, publish_fn: impl FnOnce(BlobsOrDataColumns) + Send + 'static, -) -> Result<(), BlockError> { +) -> Result<(), FetchEngineBlobError> { let versioned_hashes = if let Ok(kzg_commitments) = block.message().body().blob_kzg_commitments() { kzg_commitments @@ -39,7 +60,7 @@ pub async fn fetch_and_process_engine_blobs( } else { vec![] }; - let num_blobs = versioned_hashes.len(); + let num_expected_blobs = versioned_hashes.len(); if versioned_hashes.is_empty() { debug!(chain.log, "Blobs from EL - none required"); @@ -49,75 +70,30 @@ pub async fn fetch_and_process_engine_blobs( let execution_layer = chain .execution_layer .as_ref() - .ok_or(BeaconChainError::ExecutionLayerMissing)?; + .ok_or(FetchEngineBlobError::ExecutionLayerMissing)?; debug!( chain.log, "Blobs from EL - start request"; - "num_blobs" => num_blobs, + "num_expected_blobs" => num_expected_blobs, ); let response = execution_layer .get_blobs(versioned_hashes) .await - .map_err(|e| BlockError::ExecutionPayloadError(ExecutionPayloadError::RequestFailed(e)))?; + .map_err(FetchEngineBlobError::RequestFailed)?; let num_fetched_blobs = response.iter().filter(|b| b.is_some()).count(); - let mut all_blobs_fetched = false; - if num_fetched_blobs == 0 { - debug!(chain.log, "Blobs from EL - response with none"); - return Ok(()); - } else if num_fetched_blobs < num_blobs { - debug!( - chain.log, - "Blobs from EL - response with some"; - "fetched" => num_fetched_blobs, - "total" => num_blobs, - ); - } else { - all_blobs_fetched = true; - debug!( - chain.log, - "Blobs from EL - response with all"; - "num_blobs" => num_blobs - ); - } + let all_blobs_fetched = num_fetched_blobs == num_expected_blobs; - let (signed_block_header, kzg_commitments_proof) = - block.signed_block_header_and_kzg_commitments_proof()?; + let (signed_block_header, kzg_commitments_proof) = block + .signed_block_header_and_kzg_commitments_proof() + .map_err(FetchEngineBlobError::BeaconStateError)?; - let mut fixed_blob_sidecar_list = FixedBlobSidecarList::default(); - for (i, blob_and_proof) in response - .into_iter() - .enumerate() - .filter_map(|(i, opt_blob)| Some((i, opt_blob?))) - { - match BlobSidecar::new_with_existing_proof( - i, - blob_and_proof.blob, - &block, - signed_block_header.clone(), - &kzg_commitments_proof, - blob_and_proof.proof, - ) { - Ok(blob) => { - if let Some(blob_mut) = fixed_blob_sidecar_list.get_mut(i) { - *blob_mut = Some(Arc::new(blob)); - } else { - error!( - chain.log, - "Blobs from EL - out of bounds"; - "i" => i - ); - } - } - Err(e) => { - warn!( - chain.log, - "Blobs from EL - error"; - "error" => ?e - ); - } - } - } + let fixed_blob_sidecar_list = build_blob_sidecars( + &block, + response, + signed_block_header, + &kzg_commitments_proof, + )?; // Spawn an async task here for long computation tasks, so it doesn't block processing, and it // allows blobs / data columns to propagate without waiting for processing. @@ -127,22 +103,36 @@ pub async fn fetch_and_process_engine_blobs( // // The reason for doing this is to make the block available and attestable as soon as possible, // while maintaining the invariant that block and data columns are persisted atomically. - let (data_columns_sender, data_columns_receiver) = tokio::sync::mpsc::channel(1); let peer_das_enabled = chain.spec.is_peer_das_enabled_for_epoch(block.epoch()); // Partial blobs response isn't useful for PeerDAS, so we don't bother building and publishing data columns. - if peer_das_enabled && all_blobs_fetched { + let data_columns_receiver_opt = if peer_das_enabled { + if !all_blobs_fetched { + debug!( + chain.log, + "Not all blobs fetched from the EL"; + "num_fetched_blobs" => num_fetched_blobs, + "num_expected_blobs" => num_expected_blobs, + ); + inc_counter(&metrics::BLOBS_FROM_EL_MISS_TOTAL); + return Ok(()); + } + + inc_counter(&metrics::BLOBS_FROM_EL_HIT_TOTAL); + let logger = chain.log.clone(); let block_cloned = block.clone(); let kzg = chain.kzg.clone().expect("KZG not initialized"); let spec = chain.spec.clone(); let blobs_cloned = fixed_blob_sidecar_list.clone(); let chain_cloned = chain.clone(); + let (data_columns_sender, data_columns_receiver) = tokio::sync::mpsc::channel(1); chain .task_executor .spawn_handle( async move { - let blob_refs = blobs_cloned.iter() + let blob_refs = blobs_cloned + .iter() .filter_map(|b| b.as_ref().map(|b| &b.blob)) .collect::>(); let data_columns_result = @@ -151,7 +141,11 @@ pub async fn fetch_and_process_engine_blobs( let all_data_columns = match data_columns_result { Ok(d) => d, Err(e) => { - error!(logger, "Failed to build data column sidecars from blobs"; "error" => ?e); + error!( + logger, + "Failed to build data column sidecars from blobs"; + "error" => ?e + ); return; } }; @@ -159,25 +153,48 @@ pub async fn fetch_and_process_engine_blobs( // Check indices from cache before sending the columns, to make sure we don't // publish components already seen on gossip. let all_data_columns_iter = all_data_columns.clone().into_iter(); - let data_columns_to_publish = match chain_cloned.data_availability_checker.imported_custody_column_indexes(&block_root) { + let data_columns_to_publish = match chain_cloned + .data_availability_checker + .imported_custody_column_indexes(&block_root) + { None => Either::Left(all_data_columns_iter), - Some(imported_columns_indices) => Either::Right(all_data_columns_iter.filter(move |d| !imported_columns_indices.contains(&d.index()))), - }.collect::>(); + Some(imported_columns_indices) => Either::Right( + all_data_columns_iter + .filter(move |d| !imported_columns_indices.contains(&d.index())), + ), + } + .collect::>(); if let Err(e) = data_columns_sender.try_send(all_data_columns) { error!(logger, "Failed to send computed data columns"; "error" => ?e); }; - let is_supernode = - chain_cloned.data_availability_checker.get_custody_columns_count() == spec.number_of_columns; + let is_supernode = chain_cloned + .data_availability_checker + .get_custody_columns_count() + == spec.number_of_columns; if is_supernode && !data_columns_to_publish.is_empty() { publish_fn(BlobsOrDataColumns::DataColumns(data_columns_to_publish)); } }, "compute_data_columns", ) - .ok_or(BeaconChainError::RuntimeShutdown)?; + .ok_or(FetchEngineBlobError::RuntimeShutdown)?; + + Some(data_columns_receiver) } else { + if num_fetched_blobs == 0 { + debug!( + chain.log, + "No blobs fetched from the EL"; + "num_expected_blobs" => num_expected_blobs, + ); + inc_counter(&metrics::BLOBS_FROM_EL_MISS_TOTAL); + return Ok(()); + } + + inc_counter(&metrics::BLOBS_FROM_EL_HIT_TOTAL); + let all_blobs = fixed_blob_sidecar_list.clone(); let all_blobs_iter = all_blobs.into_iter().flat_map(|b| b.clone()); @@ -194,12 +211,14 @@ pub async fn fetch_and_process_engine_blobs( publish_fn(BlobsOrDataColumns::Blobs( blobs_to_publish.collect::>(), )); + + None }; debug!( chain.log, "Blobs from EL - start processing"; - "num_blobs" => num_blobs, + "num_fetched_blobs" => num_fetched_blobs, ); chain @@ -207,14 +226,52 @@ pub async fn fetch_and_process_engine_blobs( block.slot(), block_root, fixed_blob_sidecar_list.clone(), - peer_das_enabled.then_some(data_columns_receiver), + data_columns_receiver_opt, ) .await .map(|_| debug!(chain.log, "Blobs from EL - processed")) .map_err(|e| { warn!(chain.log, "Blobs from EL - error"; "error" => ?e); - e + FetchEngineBlobError::BlobProcessingError(e) })?; Ok(()) } + +fn build_blob_sidecars( + block: &Arc>>, + response: Vec>>, + signed_block_header: SignedBeaconBlockHeader, + kzg_commitments_proof: &FixedVector, +) -> Result>>, E::MaxBlobsPerBlock>, FetchEngineBlobError> +{ + let mut fixed_blob_sidecar_list = FixedBlobSidecarList::default(); + for (i, blob_and_proof) in response + .into_iter() + .enumerate() + .filter_map(|(i, opt_blob)| Some((i, opt_blob?))) + { + match BlobSidecar::new_with_existing_proof( + i, + blob_and_proof.blob, + block, + signed_block_header.clone(), + kzg_commitments_proof, + blob_and_proof.proof, + ) { + Ok(blob) => { + if let Some(blob_mut) = fixed_blob_sidecar_list.get_mut(i) { + *blob_mut = Some(Arc::new(blob)); + } else { + return Err(FetchEngineBlobError::InternalError( + "Unreachable: Blobs from EL - out of bounds".to_string(), + )); + } + } + Err(e) => { + return Err(FetchEngineBlobError::BlobSidecarError(e)); + } + } + } + Ok(fixed_blob_sidecar_list) +} diff --git a/beacon_node/beacon_chain/src/metrics.rs b/beacon_node/beacon_chain/src/metrics.rs index d315c2d1f5f..ae960271278 100644 --- a/beacon_node/beacon_chain/src/metrics.rs +++ b/beacon_node/beacon_chain/src/metrics.rs @@ -1695,6 +1695,20 @@ pub static DATA_COLUMNS_SIDECAR_PROCESSING_SUCCESSES: LazyLock> = LazyLock::new(|| { + try_create_int_counter( + "beacon_blobs_from_el_hit_total", + "Number of blob batches fetched from the execution layer", + ) +}); + +pub static BLOBS_FROM_EL_MISS_TOTAL: LazyLock> = LazyLock::new(|| { + try_create_int_counter( + "beacon_blobs_from_el_miss_total", + "Number of blob batches failed to fetch from the execution layer", + ) +}); + /* * Light server message verification */ 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 463c2555ade..f7b165e69a7 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -1516,6 +1516,9 @@ impl NetworkBeaconProcessor { "block_root" => %block_root, ); + // Block is valid, we can now attempt fetching blobs from EL using version hashes + // derived from kzg commitments from the block, without having to wait for all blobs + // to be sent from the peers if we already have them. self.fetch_engine_blobs_and_publish(block.clone(), *block_root) .await; } diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 4640b973ac7..56b5e9e01f9 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -771,20 +771,24 @@ impl NetworkBeaconProcessor { fn publish_blobs_or_data_column(&self, blobs_or_data_column: BlobsOrDataColumns) { let messages = match blobs_or_data_column { - BlobsOrDataColumns::Blobs(blobs) => blobs - .into_iter() - .map(|blob| PubsubMessage::BlobSidecar(Box::new((blob.index, blob)))) - .collect(), - BlobsOrDataColumns::DataColumns(columns) => columns - .into_iter() - .map(|column| { + BlobsOrDataColumns::Blobs(blobs) => { + debug!(self.log, "Publishing blobs from EL"; "count" => blobs.len()); + blobs + .into_iter() + .map(|blob| PubsubMessage::BlobSidecar(Box::new((blob.index, blob)))) + .collect() + } + BlobsOrDataColumns::DataColumns(columns) => { + debug!(self.log, "Publishing data columns built from EL blobs"; "count" => columns.len()); + columns.into_iter().map(|column| { let subnet = DataColumnSubnetId::from_column_index::( column.index as usize, &self.chain.spec, ); PubsubMessage::DataColumnSidecar(Box::new((subnet, column))) }) - .collect(), + } + .collect(), }; self.send_network_message(NetworkMessage::Publish { messages }) } @@ -795,16 +799,7 @@ impl NetworkBeaconProcessor { block_root: Hash256, ) { let self_cloned = self.clone(); - let log_cloned = self.log.clone(); let publish_fn = move |blobs_or_data_column| { - match &blobs_or_data_column { - BlobsOrDataColumns::Blobs(b) => { - debug!(log_cloned, "Publishing blobs from EL"; "count" => b.len()); - } - BlobsOrDataColumns::DataColumns(d) => { - debug!(log_cloned, "Publishing data columns built from EL blobs"; "count" => d.len()); - } - } self_cloned.publish_blobs_or_data_column(blobs_or_data_column) }; if let Err(e) = fetch_and_process_engine_blobs( 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 02ec255f054..35b0b88d474 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -190,6 +190,9 @@ impl NetworkBeaconProcessor { self.chain.recompute_head_at_current_slot().await; } Ok(AvailabilityProcessingStatus::MissingComponents(..)) => { + // Block is valid, we can now attempt fetching blobs from EL using version hashes + // derived from kzg commitments from the block, without having to wait for all blobs + // to be sent from the peers if we already have them. self.fetch_engine_blobs_and_publish(signed_beacon_block, block_root) .await } From 401231ba626d2a43cb357f494aacc3d5515f0f27 Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Thu, 29 Aug 2024 14:12:13 +1000 Subject: [PATCH 04/25] Fix lint. --- beacon_node/beacon_chain/src/fetch_blobs.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index fff31fa50a7..0ca99c6bd7a 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -243,8 +243,7 @@ fn build_blob_sidecars( response: Vec>>, signed_block_header: SignedBeaconBlockHeader, kzg_commitments_proof: &FixedVector, -) -> Result>>, E::MaxBlobsPerBlock>, FetchEngineBlobError> -{ +) -> Result, FetchEngineBlobError> { let mut fixed_blob_sidecar_list = FixedBlobSidecarList::default(); for (i, blob_and_proof) in response .into_iter() From 2efc99bedf91b5c2bb79b778c7b074fc026a8bb4 Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Thu, 29 Aug 2024 15:21:18 +1000 Subject: [PATCH 05/25] Move blob computation metric to the right spot. --- .../beacon_chain/src/block_verification.rs | 3 -- beacon_node/beacon_chain/src/kzg_utils.rs | 15 ++++++++-- common/lighthouse_metrics/src/lib.rs | 28 +++++++++++++++++++ 3 files changed, 41 insertions(+), 5 deletions(-) diff --git a/beacon_node/beacon_chain/src/block_verification.rs b/beacon_node/beacon_chain/src/block_verification.rs index c4377598783..aa1735b5448 100644 --- a/beacon_node/beacon_chain/src/block_verification.rs +++ b/beacon_node/beacon_chain/src/block_verification.rs @@ -802,11 +802,8 @@ fn build_gossip_verified_data_columns( GossipDataColumnError::KzgNotInitialized, ))?; - let timer = metrics::start_timer(&metrics::DATA_COLUMN_SIDECAR_COMPUTATION); - let blob_refs = blobs.iter().collect::>(); let sidecars = blobs_to_data_column_sidecars(&blob_refs, block, kzg, &chain.spec)?; - drop(timer); let mut gossip_verified_data_columns = vec![]; for sidecar in sidecars { let subnet = DataColumnSubnetId::from_column_index::( diff --git a/beacon_node/beacon_chain/src/kzg_utils.rs b/beacon_node/beacon_chain/src/kzg_utils.rs index 7f6293dc641..988de88ed4d 100644 --- a/beacon_node/beacon_chain/src/kzg_utils.rs +++ b/beacon_node/beacon_chain/src/kzg_utils.rs @@ -1,6 +1,8 @@ +use crate::metrics; use kzg::{ Blob as KzgBlob, Bytes48, CellRef as KzgCellRef, CellsAndKzgProofs, Error as KzgError, Kzg, }; +use lighthouse_metrics::TryExt; use rayon::prelude::*; use ssz_types::FixedVector; use std::sync::Arc; @@ -150,12 +152,19 @@ pub fn blobs_to_data_column_sidecars( if blobs.is_empty() { return Ok(vec![]); } + + let mut timer = metrics::start_timer(&metrics::DATA_COLUMN_SIDECAR_COMPUTATION); + let kzg_commitments = block .message() .body() .blob_kzg_commitments() .map_err(|_err| DataColumnSidecarError::PreDeneb)?; - let kzg_commitments_inclusion_proof = block.message().body().kzg_commitments_merkle_proof()?; + let kzg_commitments_inclusion_proof = block + .message() + .body() + .kzg_commitments_merkle_proof() + .discard_timer_on_break(&mut timer)?; let signed_block_header = block.signed_block_header(); // NOTE: assumes blob sidecars are ordered by index @@ -168,7 +177,8 @@ pub fn blobs_to_data_column_sidecars( .expect("blob should have a guaranteed size due to FixedVector"); kzg.compute_cells_and_proofs(blob) }) - .collect::, KzgError>>()?; + .collect::, KzgError>>() + .discard_timer_on_break(&mut timer)?; build_data_column_sidecars( kzg_commitments.clone(), @@ -177,6 +187,7 @@ pub fn blobs_to_data_column_sidecars( blob_cells_and_proofs_vec, spec, ) + .discard_timer_on_break(&mut timer) .map_err(DataColumnSidecarError::BuildSidecarFailed) } diff --git a/common/lighthouse_metrics/src/lib.rs b/common/lighthouse_metrics/src/lib.rs index fa8f47e364a..f52913dd001 100644 --- a/common/lighthouse_metrics/src/lib.rs +++ b/common/lighthouse_metrics/src/lib.rs @@ -400,3 +400,31 @@ pub fn decimal_buckets(min_power: i32, max_power: i32) -> Result> { } Ok(buckets) } + +/// Would be nice to use the `Try` trait bound and have a single implementation, but try_trait_v2 +/// is not a stable feature yet. +pub trait TryExt { + fn discard_timer_on_break(self, timer: &mut Option) -> Self; +} + +impl TryExt for std::result::Result { + fn discard_timer_on_break(self, timer_opt: &mut Option) -> Self { + if self.is_err() { + if let Some(timer) = timer_opt.take() { + timer.stop_and_discard(); + } + } + self + } +} + +impl TryExt for Option { + fn discard_timer_on_break(self, timer_opt: &mut Option) -> Self { + if self.is_none() { + if let Some(timer) = timer_opt.take() { + timer.stop_and_discard(); + } + } + self + } +} From 6bff4abede078f82591a6c4696a31692ed3944b4 Mon Sep 17 00:00:00 2001 From: Michael Sproul Date: Fri, 13 Sep 2024 11:28:43 +1000 Subject: [PATCH 06/25] Gradual publication of data columns for supernodes. --- beacon_node/beacon_chain/src/fetch_blobs.rs | 81 ++++++++++++------ .../gossip_methods.rs | 82 +++++++++++++++---- .../network_beacon_processor/sync_methods.rs | 2 +- 3 files changed, 120 insertions(+), 45 deletions(-) diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index f978b41cc4f..6808292a092 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -7,25 +7,30 @@ //! on P2P gossip to the network. From PeerDAS onwards, together with the increase in blob count, //! broadcasting blobs requires a much higher bandwidth, and is only done by high capacity //! supernodes. -use std::sync::Arc; - +use crate::kzg_utils::blobs_to_data_column_sidecars; +use crate::observed_data_sidecars::ObservableDataSidecar; +use crate::{metrics, BeaconChain, BeaconChainTypes, BlockError}; use execution_layer::json_structures::BlobAndProofV1; use execution_layer::Error as ExecutionLayerError; use itertools::Either; +use lighthouse_metrics::{inc_counter, TryExt}; +use rand::prelude::SliceRandom; use slog::{debug, error, warn}; use ssz_types::FixedVector; - -use lighthouse_metrics::{inc_counter, TryExt}; use state_processing::per_block_processing::deneb::kzg_commitment_to_versioned_hash; +use std::sync::Arc; +use std::time::Duration; use types::blob_sidecar::{BlobSidecarError, FixedBlobSidecarList}; use types::{ BeaconStateError, BlobSidecar, DataColumnSidecarList, EthSpec, FullPayload, Hash256, SignedBeaconBlock, SignedBeaconBlockHeader, }; -use crate::kzg_utils::blobs_to_data_column_sidecars; -use crate::observed_data_sidecars::ObservableDataSidecar; -use crate::{metrics, BeaconChain, BeaconChainTypes, BlockError}; +/// Number of batches that supernodes split data columns into during publishing. +pub const SUPERNODE_DATA_COLUMN_PUBLICATION_BATCHES: usize = 4; + +/// The delay applied by supernodes between the sending of each data column batch. +pub const SUPERNODE_DATA_COLUMN_PUBLICATION_BATCH_INTERVAL: Duration = Duration::from_millis(200); pub enum BlobsOrDataColumns { Blobs(Vec>>), @@ -49,7 +54,7 @@ pub async fn fetch_and_process_engine_blobs( chain: Arc>, block_root: Hash256, block: Arc>>, - publish_fn: impl FnOnce(BlobsOrDataColumns) + Send + 'static, + publish_fn: impl Fn(BlobsOrDataColumns) + Send + 'static, ) -> Result<(), FetchEngineBlobError> { let versioned_hashes = if let Ok(kzg_commitments) = block.message().body().blob_kzg_commitments() { @@ -144,7 +149,7 @@ pub async fn fetch_and_process_engine_blobs( .discard_timer_on_break(&mut timer); drop(timer); - let all_data_columns = match data_columns_result { + let mut all_data_columns = match data_columns_result { Ok(d) => d, Err(e) => { error!( @@ -156,31 +161,53 @@ pub async fn fetch_and_process_engine_blobs( } }; - // Check indices from cache before sending the columns, to make sure we don't - // publish components already seen on gossip. - let all_data_columns_iter = all_data_columns.clone().into_iter(); - let data_columns_to_publish = match chain_cloned - .data_availability_checker - .imported_custody_column_indexes(&block_root) - { - None => Either::Left(all_data_columns_iter), - Some(imported_columns_indices) => Either::Right( - all_data_columns_iter - .filter(move |d| !imported_columns_indices.contains(&d.index())), - ), - } - .collect::>(); - - if let Err(e) = data_columns_sender.try_send(all_data_columns) { + if let Err(e) = data_columns_sender.try_send(all_data_columns.clone()) { error!(logger, "Failed to send computed data columns"; "error" => ?e); }; + // Check indices from cache before sending the columns, to make sure we don't + // publish components already seen on gossip. let is_supernode = chain_cloned .data_availability_checker .get_custody_columns_count() == spec.number_of_columns; - if is_supernode && !data_columns_to_publish.is_empty() { - publish_fn(BlobsOrDataColumns::DataColumns(data_columns_to_publish)); + + // At the moment non supernodes are not required to publish any columns. + // TODO(das): we could experiment with having full nodes publish their custodied + // columns here. + if !is_supernode { + return; + } + + // To reduce bandwidth for supernodes: permute the columns to publish and + // publish them in batches. Our hope is that some columns arrive from + // other supernodes in the meantime, obviating the need for us to publish + // them. If no other publisher exists for a column, it will eventually get + // published here. + // FIXME(das): deduplicate this wrt to gossip/sync methods + all_data_columns.shuffle(&mut rand::thread_rng()); + + let batch_size = + all_data_columns.len() / SUPERNODE_DATA_COLUMN_PUBLICATION_BATCHES; + for batch in all_data_columns.chunks(batch_size) { + let already_seen = chain_cloned + .data_availability_checker + .imported_custody_column_indexes(&block_root) + .unwrap_or_default(); + let publishable = batch + .iter() + .filter(|col| !already_seen.contains(&col.index())) + .cloned() + .collect::>(); + if !publishable.is_empty() { + debug!( + chain_cloned.log, + "Publishing data columns from EL"; + "count" => publishable.len() + ); + publish_fn(BlobsOrDataColumns::DataColumns(publishable)); + } + tokio::time::sleep(SUPERNODE_DATA_COLUMN_PUBLICATION_BATCH_INTERVAL).await; } }, "compute_data_columns", 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 91397685fe0..f841a510e0d 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -26,6 +26,7 @@ use lighthouse_network::{ Client, MessageAcceptance, MessageId, PeerAction, PeerId, PubsubMessage, ReportSource, }; use operation_pool::ReceivedPreCapella; +use rand::prelude::SliceRandom; use slog::{crit, debug, error, info, trace, warn, Logger}; use slot_clock::SlotClock; use ssz::Encode; @@ -38,8 +39,8 @@ use store::hot_cold_store::HotColdDBError; use tokio::sync::mpsc; use types::{ beacon_block::BlockImportSource, Attestation, AttestationRef, AttesterSlashing, BlobSidecar, - DataColumnSidecar, DataColumnSubnetId, EthSpec, Hash256, IndexedAttestation, - LightClientFinalityUpdate, LightClientOptimisticUpdate, ProposerSlashing, + DataColumnSidecar, DataColumnSidecarList, DataColumnSubnetId, EthSpec, Hash256, + IndexedAttestation, LightClientFinalityUpdate, LightClientOptimisticUpdate, ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedBlsToExecutionChange, SignedContributionAndProof, SignedVoluntaryExit, Slot, SubnetId, SyncCommitteeMessage, SyncSubnetId, @@ -57,6 +58,12 @@ use beacon_processor::{ /// messages. const STRICT_LATE_MESSAGE_PENALTIES: bool = false; +/// Number of batches that supernodes split data columns into during publishing. +pub const SUPERNODE_DATA_COLUMN_PUBLICATION_BATCHES: usize = 4; + +/// The delay applied by supernodes between the sending of each data column batch. +pub const SUPERNODE_DATA_COLUMN_PUBLICATION_BATCH_INTERVAL: Duration = Duration::from_millis(200); + /// An attestation that has been validated by the `BeaconChain`. /// /// Since this struct implements `beacon_chain::VerifiedAttestation`, it would be a logic error to @@ -172,23 +179,64 @@ impl NetworkBeaconProcessor { } pub(crate) fn handle_data_columns_to_publish( - &self, + self: &Arc, data_columns_to_publish: DataColumnsToPublish, + block_root: Hash256, ) { - if let Some(data_columns_to_publish) = data_columns_to_publish { - self.send_network_message(NetworkMessage::Publish { - messages: data_columns_to_publish - .iter() - .map(|d| { - let subnet = DataColumnSubnetId::from_column_index::( - d.index as usize, - &self.chain.spec, + let Some(mut data_columns_to_publish) = data_columns_to_publish else { + return; + }; + // let self_executor= self.clone(); + let self_clone = self.clone(); + self.executor.spawn( + async move { + let publish_fn = |columns: DataColumnSidecarList| { + self_clone.send_network_message(NetworkMessage::Publish { + messages: columns + .into_iter() + .map(|d| { + let subnet = DataColumnSubnetId::from_column_index::( + d.index as usize, + &self_clone.chain.spec, + ); + PubsubMessage::DataColumnSidecar(Box::new((subnet, d))) + }) + .collect(), + }); + }; + + // If this node is a super node, permute the columns and split them into batches. + // The hope is that we won't need to publish some columns because we will receive them + // on gossip from other supernodes. + data_columns_to_publish.shuffle(&mut rand::thread_rng()); + let batch_size = + data_columns_to_publish.len() / SUPERNODE_DATA_COLUMN_PUBLICATION_BATCHES; + + for batch in data_columns_to_publish.chunks(batch_size) { + let already_seen = self_clone + .chain + .data_availability_checker + .imported_custody_column_indexes(&block_root) + .unwrap_or_default(); + let publishable = batch + .iter() + .filter(|col| !already_seen.contains(&col.index)) + .cloned() + .collect::>(); + + if !publishable.is_empty() { + debug!( + self_clone.chain.logger(), + "Publishing data column batch"; + "count" => publishable.len() ); - PubsubMessage::DataColumnSidecar(Box::new((subnet, d.clone()))) - }) - .collect(), - }); - } + publish_fn(publishable); + } + tokio::time::sleep(SUPERNODE_DATA_COLUMN_PUBLICATION_BATCH_INTERVAL).await; + } + }, + "handle_data_columns_publish", + ); } /// Send a message on `message_tx` that the `message_id` sent by `peer_id` should be propagated on @@ -1020,7 +1068,7 @@ impl NetworkBeaconProcessor { .await { Ok((availability, data_columns_to_publish)) => { - self.handle_data_columns_to_publish(data_columns_to_publish); + self.handle_data_columns_to_publish(data_columns_to_publish, block_root); match availability { AvailabilityProcessingStatus::Imported(block_root) => { 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 d47582169ac..dcff6b7ec25 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -345,7 +345,7 @@ impl NetworkBeaconProcessor { match &result { Ok((availability, data_columns_to_publish)) => { - self.handle_data_columns_to_publish(data_columns_to_publish.clone()); + self.handle_data_columns_to_publish(data_columns_to_publish.clone(), block_root); match availability { AvailabilityProcessingStatus::Imported(hash) => { From 7977999b4a74db5fd9942d0463bcc642550feed2 Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Fri, 13 Sep 2024 14:10:45 +1000 Subject: [PATCH 07/25] Recompute head after importing block with blobs from the EL. --- beacon_node/beacon_chain/src/fetch_blobs.rs | 20 +++++------ .../src/network_beacon_processor/mod.rs | 35 +++++++++++++++++-- 2 files changed, 40 insertions(+), 15 deletions(-) diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index 6808292a092..e653afeeb67 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -9,7 +9,7 @@ //! supernodes. use crate::kzg_utils::blobs_to_data_column_sidecars; use crate::observed_data_sidecars::ObservableDataSidecar; -use crate::{metrics, BeaconChain, BeaconChainTypes, BlockError}; +use crate::{metrics, AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes, BlockError}; use execution_layer::json_structures::BlobAndProofV1; use execution_layer::Error as ExecutionLayerError; use itertools::Either; @@ -55,7 +55,7 @@ pub async fn fetch_and_process_engine_blobs( block_root: Hash256, block: Arc>>, publish_fn: impl Fn(BlobsOrDataColumns) + Send + 'static, -) -> Result<(), FetchEngineBlobError> { +) -> Result, FetchEngineBlobError> { let versioned_hashes = if let Ok(kzg_commitments) = block.message().body().blob_kzg_commitments() { kzg_commitments @@ -69,7 +69,7 @@ pub async fn fetch_and_process_engine_blobs( if versioned_hashes.is_empty() { debug!(chain.log, "Blobs from EL - none required"); - return Ok(()); + return Ok(None); } let execution_layer = chain @@ -120,7 +120,7 @@ pub async fn fetch_and_process_engine_blobs( "num_expected_blobs" => num_expected_blobs, ); inc_counter(&metrics::BLOBS_FROM_EL_MISS_TOTAL); - return Ok(()); + return Ok(None); } inc_counter(&metrics::BLOBS_FROM_EL_HIT_TOTAL); @@ -223,7 +223,7 @@ pub async fn fetch_and_process_engine_blobs( "num_expected_blobs" => num_expected_blobs, ); inc_counter(&metrics::BLOBS_FROM_EL_MISS_TOTAL); - return Ok(()); + return Ok(None); } inc_counter(&metrics::BLOBS_FROM_EL_HIT_TOTAL); @@ -254,7 +254,7 @@ pub async fn fetch_and_process_engine_blobs( "num_fetched_blobs" => num_fetched_blobs, ); - chain + let availability_processing_status = chain .process_engine_blobs( block.slot(), block_root, @@ -262,13 +262,9 @@ pub async fn fetch_and_process_engine_blobs( data_columns_receiver_opt, ) .await - .map(|_| debug!(chain.log, "Blobs from EL - processed")) - .map_err(|e| { - warn!(chain.log, "Blobs from EL - error"; "error" => ?e); - FetchEngineBlobError::BlobProcessingError(e) - })?; + .map_err(|e| FetchEngineBlobError::BlobProcessingError(e))?; - Ok(()) + Ok(Some(availability_processing_status)) } fn build_blob_sidecars( diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 56b5e9e01f9..c2b81f5df5b 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -3,7 +3,9 @@ use crate::sync::SamplingId; use crate::{service::NetworkMessage, sync::manager::SyncMessage}; use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::fetch_blobs::{fetch_and_process_engine_blobs, BlobsOrDataColumns}; -use beacon_chain::{builder::Witness, eth1_chain::CachingEth1Backend, BeaconChain}; +use beacon_chain::{ + builder::Witness, eth1_chain::CachingEth1Backend, AvailabilityProcessingStatus, BeaconChain, +}; use beacon_chain::{BeaconChainTypes, NotifyExecutionLayer}; use beacon_processor::{ work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessorChannels, BeaconProcessorSend, @@ -802,7 +804,8 @@ impl NetworkBeaconProcessor { let publish_fn = move |blobs_or_data_column| { self_cloned.publish_blobs_or_data_column(blobs_or_data_column) }; - if let Err(e) = fetch_and_process_engine_blobs( + + match fetch_and_process_engine_blobs( self.chain.clone(), block_root, block.clone(), @@ -810,7 +813,33 @@ impl NetworkBeaconProcessor { ) .await { - error!(self.log, "Error fetching or processing blobs from EL"; "error" => ?e); + Ok(Some(availability)) => match availability { + AvailabilityProcessingStatus::Imported(hash) => { + debug!( + self.log, + "Block components retrieved from EL"; + "result" => "imported block and custody columns", + "block_hash" => %block_root, + ); + self.chain.recompute_head_at_current_slot().await; + } + AvailabilityProcessingStatus::MissingComponents(_, _) => { + error!( + self.log, + "MissingComponents is not expected after engine blobs processed successfully"; + "block_hash" => %block_root, + ); + } + }, + Ok(None) => { /* No blobs fetched from the EL. Reasons logged separately. */ } + Err(e) => { + error!( + self.log, + "Error fetching or processing blobs from EL"; + "error" => ?e, + "block_hash" => %block_root, + ); + } } } } From 5e75527eb38c0af098192bdc1b0c677abdc42605 Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Tue, 17 Sep 2024 14:40:28 +1000 Subject: [PATCH 08/25] Fix lint --- beacon_node/beacon_chain/src/fetch_blobs.rs | 4 ++-- beacon_node/network/src/network_beacon_processor/mod.rs | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index e653afeeb67..61113f26da7 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -15,7 +15,7 @@ use execution_layer::Error as ExecutionLayerError; use itertools::Either; use lighthouse_metrics::{inc_counter, TryExt}; use rand::prelude::SliceRandom; -use slog::{debug, error, warn}; +use slog::{debug, error}; use ssz_types::FixedVector; use state_processing::per_block_processing::deneb::kzg_commitment_to_versioned_hash; use std::sync::Arc; @@ -262,7 +262,7 @@ pub async fn fetch_and_process_engine_blobs( data_columns_receiver_opt, ) .await - .map_err(|e| FetchEngineBlobError::BlobProcessingError(e))?; + .map_err(FetchEngineBlobError::BlobProcessingError)?; Ok(Some(availability_processing_status)) } diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index c2b81f5df5b..3ef3d63a468 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -814,7 +814,7 @@ impl NetworkBeaconProcessor { .await { Ok(Some(availability)) => match availability { - AvailabilityProcessingStatus::Imported(hash) => { + AvailabilityProcessingStatus::Imported(_) => { debug!( self.log, "Block components retrieved from EL"; From e76d21f90b7cf7db11abd18a1dbc898eda128e6e Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Thu, 19 Sep 2024 15:03:34 +1000 Subject: [PATCH 09/25] Use blocking task instead of async when computing cells. --- beacon_node/beacon_chain/src/fetch_blobs.rs | 144 ++++++++++---------- 1 file changed, 70 insertions(+), 74 deletions(-) diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index 61113f26da7..a41fe4ff2ec 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -132,87 +132,83 @@ pub async fn fetch_and_process_engine_blobs( let blobs_cloned = fixed_blob_sidecar_list.clone(); let chain_cloned = chain.clone(); let (data_columns_sender, data_columns_receiver) = tokio::sync::mpsc::channel(1); - chain - .task_executor - .spawn_handle( - async move { - let mut timer = metrics::start_timer_vec( - &metrics::DATA_COLUMN_SIDECAR_COMPUTATION, - &[&blobs_cloned.len().to_string()], - ); - let blob_refs = blobs_cloned - .iter() - .filter_map(|b| b.as_ref().map(|b| &b.blob)) - .collect::>(); - let data_columns_result = - blobs_to_data_column_sidecars(&blob_refs, &block_cloned, &kzg, &spec) - .discard_timer_on_break(&mut timer); - drop(timer); + chain.task_executor.spawn_blocking( + move || { + let mut timer = metrics::start_timer_vec( + &metrics::DATA_COLUMN_SIDECAR_COMPUTATION, + &[&blobs_cloned.len().to_string()], + ); + let blob_refs = blobs_cloned + .iter() + .filter_map(|b| b.as_ref().map(|b| &b.blob)) + .collect::>(); + let data_columns_result = + blobs_to_data_column_sidecars(&blob_refs, &block_cloned, &kzg, &spec) + .discard_timer_on_break(&mut timer); + drop(timer); - let mut all_data_columns = match data_columns_result { - Ok(d) => d, - Err(e) => { - error!( - logger, - "Failed to build data column sidecars from blobs"; - "error" => ?e - ); - return; - } - }; + let mut all_data_columns = match data_columns_result { + Ok(d) => d, + Err(e) => { + error!( + logger, + "Failed to build data column sidecars from blobs"; + "error" => ?e + ); + return; + } + }; - if let Err(e) = data_columns_sender.try_send(all_data_columns.clone()) { - error!(logger, "Failed to send computed data columns"; "error" => ?e); - }; + if let Err(e) = data_columns_sender.try_send(all_data_columns.clone()) { + error!(logger, "Failed to send computed data columns"; "error" => ?e); + }; - // Check indices from cache before sending the columns, to make sure we don't - // publish components already seen on gossip. - let is_supernode = chain_cloned - .data_availability_checker - .get_custody_columns_count() - == spec.number_of_columns; + // Check indices from cache before sending the columns, to make sure we don't + // publish components already seen on gossip. + let is_supernode = chain_cloned + .data_availability_checker + .get_custody_columns_count() + == spec.number_of_columns; - // At the moment non supernodes are not required to publish any columns. - // TODO(das): we could experiment with having full nodes publish their custodied - // columns here. - if !is_supernode { - return; - } + // At the moment non supernodes are not required to publish any columns. + // TODO(das): we could experiment with having full nodes publish their custodied + // columns here. + if !is_supernode { + return; + } - // To reduce bandwidth for supernodes: permute the columns to publish and - // publish them in batches. Our hope is that some columns arrive from - // other supernodes in the meantime, obviating the need for us to publish - // them. If no other publisher exists for a column, it will eventually get - // published here. - // FIXME(das): deduplicate this wrt to gossip/sync methods - all_data_columns.shuffle(&mut rand::thread_rng()); + // To reduce bandwidth for supernodes: permute the columns to publish and + // publish them in batches. Our hope is that some columns arrive from + // other supernodes in the meantime, obviating the need for us to publish + // them. If no other publisher exists for a column, it will eventually get + // published here. + // FIXME(das): deduplicate this wrt to gossip/sync methods + all_data_columns.shuffle(&mut rand::thread_rng()); - let batch_size = - all_data_columns.len() / SUPERNODE_DATA_COLUMN_PUBLICATION_BATCHES; - for batch in all_data_columns.chunks(batch_size) { - let already_seen = chain_cloned - .data_availability_checker - .imported_custody_column_indexes(&block_root) - .unwrap_or_default(); - let publishable = batch - .iter() - .filter(|col| !already_seen.contains(&col.index())) - .cloned() - .collect::>(); - if !publishable.is_empty() { - debug!( - chain_cloned.log, - "Publishing data columns from EL"; - "count" => publishable.len() - ); - publish_fn(BlobsOrDataColumns::DataColumns(publishable)); - } - tokio::time::sleep(SUPERNODE_DATA_COLUMN_PUBLICATION_BATCH_INTERVAL).await; + let batch_size = all_data_columns.len() / SUPERNODE_DATA_COLUMN_PUBLICATION_BATCHES; + for batch in all_data_columns.chunks(batch_size) { + let already_seen = chain_cloned + .data_availability_checker + .imported_custody_column_indexes(&block_root) + .unwrap_or_default(); + let publishable = batch + .iter() + .filter(|col| !already_seen.contains(&col.index())) + .cloned() + .collect::>(); + if !publishable.is_empty() { + debug!( + chain_cloned.log, + "Publishing data columns from EL"; + "count" => publishable.len() + ); + publish_fn(BlobsOrDataColumns::DataColumns(publishable)); } - }, - "compute_data_columns", - ) - .ok_or(FetchEngineBlobError::RuntimeShutdown)?; + std::thread::sleep(SUPERNODE_DATA_COLUMN_PUBLICATION_BATCH_INTERVAL); + } + }, + "compute_data_columns", + ); Some(data_columns_receiver) } else { From 4639146dfe16107aac74f2d42115c3e64e67fb37 Mon Sep 17 00:00:00 2001 From: Michael Sproul Date: Fri, 27 Sep 2024 12:50:07 +1000 Subject: [PATCH 10/25] Fix semantic conflicts --- beacon_node/beacon_chain/src/beacon_chain.rs | 2 +- beacon_node/beacon_chain/src/block_verification.rs | 3 ++- beacon_node/beacon_chain/src/fetch_blobs.rs | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 22a68f60634..74cafefb672 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -3110,7 +3110,7 @@ impl BeaconChain { .fork_choice_read_lock() .contains_block(&block_root) { - return Err(BlockError::BlockIsAlreadyKnown(block_root)); + return Err(BlockError::DuplicateFullyImported(block_root)); } self.emit_sse_blob_sidecar_events(&block_root, blobs.iter().flatten().map(Arc::as_ref)); diff --git a/beacon_node/beacon_chain/src/block_verification.rs b/beacon_node/beacon_chain/src/block_verification.rs index d481c6ca860..d19887beb63 100644 --- a/beacon_node/beacon_chain/src/block_verification.rs +++ b/beacon_node/beacon_chain/src/block_verification.rs @@ -750,7 +750,8 @@ pub fn build_blob_data_column_sidecars( &metrics::DATA_COLUMN_SIDECAR_COMPUTATION, &[&blobs.len().to_string()], ); - let sidecars = blobs_to_data_column_sidecars(&blobs, block, &chain.kzg, &chain.spec) + let blob_refs = blobs.iter().collect::>(); + let sidecars = blobs_to_data_column_sidecars(&blob_refs, block, &chain.kzg, &chain.spec) .discard_timer_on_break(&mut timer)?; drop(timer); Ok(sidecars) diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index a41fe4ff2ec..331cbc26a26 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -127,7 +127,7 @@ pub async fn fetch_and_process_engine_blobs( let logger = chain.log.clone(); let block_cloned = block.clone(); - let kzg = chain.kzg.clone().expect("KZG not initialized"); + let kzg = chain.kzg.clone(); let spec = chain.spec.clone(); let blobs_cloned = fixed_blob_sidecar_list.clone(); let chain_cloned = chain.clone(); From 41c2e7d0f6f0ffc918ba0870f7ef9ab0d37ddf1c Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Fri, 4 Oct 2024 10:54:34 +1000 Subject: [PATCH 11/25] Downgrade error log. --- beacon_node/network/src/network_beacon_processor/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 3ef3d63a468..bd465a5a0b0 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -824,9 +824,9 @@ impl NetworkBeaconProcessor { self.chain.recompute_head_at_current_slot().await; } AvailabilityProcessingStatus::MissingComponents(_, _) => { - error!( + debug!( self.log, - "MissingComponents is not expected after engine blobs processed successfully"; + "Still missing blobs after engine blobs processed successfully"; "block_hash" => %block_root, ); } From 0a2c6f76c790e2117ed4a225ef312795eb26f240 Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Fri, 18 Oct 2024 14:27:48 +1100 Subject: [PATCH 12/25] Publish block without waiting for blob and column proof computation. --- beacon_node/beacon_chain/src/beacon_chain.rs | 8 +- .../beacon_chain/src/blob_verification.rs | 6 - .../src/data_column_verification.rs | 2 - .../beacon_chain/tests/block_verification.rs | 4 +- beacon_node/beacon_chain/tests/events.rs | 2 +- beacon_node/http_api/src/publish_blocks.rs | 485 ++++++++++-------- .../tests/broadcast_validation_tests.rs | 8 +- .../gossip_methods.rs | 5 +- testing/ef_tests/src/cases/fork_choice.rs | 4 +- 9 files changed, 275 insertions(+), 249 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index dd70ba8c9a1..0d1609bd6f4 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -2977,7 +2977,6 @@ impl BeaconChain { pub async fn process_gossip_blob( self: &Arc, blob: GossipVerifiedBlob, - publish_fn: impl FnOnce() -> Result<(), BlockError>, ) -> Result { let block_root = blob.block_root(); @@ -2998,9 +2997,7 @@ impl BeaconChain { self.emit_sse_blob_sidecar_events(&block_root, std::iter::once(blob.as_blob())); - let r = self - .check_gossip_blob_availability_and_import(blob, publish_fn) - .await; + let r = self.check_gossip_blob_availability_and_import(blob).await; self.remove_notified(&block_root, r) } @@ -3488,7 +3485,6 @@ impl BeaconChain { async fn check_gossip_blob_availability_and_import( self: &Arc, blob: GossipVerifiedBlob, - publish_fn: impl FnOnce() -> Result<(), BlockError>, ) -> Result { let slot = blob.slot(); if let Some(slasher) = self.slasher.as_ref() { @@ -3496,7 +3492,7 @@ impl BeaconChain { } let availability = self.data_availability_checker.put_gossip_blob(blob)?; - self.process_availability(slot, availability, None, publish_fn) + self.process_availability(slot, availability, None, || Ok(())) .await } diff --git a/beacon_node/beacon_chain/src/blob_verification.rs b/beacon_node/beacon_chain/src/blob_verification.rs index c9c4f6dceb2..f7de26a6ca7 100644 --- a/beacon_node/beacon_chain/src/blob_verification.rs +++ b/beacon_node/beacon_chain/src/blob_verification.rs @@ -12,7 +12,6 @@ use crate::{metrics, BeaconChainError}; use kzg::{Error as KzgError, Kzg, KzgCommitment}; use slog::debug; use ssz_derive::{Decode, Encode}; -use ssz_types::VariableList; use std::time::Duration; use tree_hash::TreeHash; use types::blob_sidecar::BlobIdentifier; @@ -156,11 +155,6 @@ impl From for GossipBlobError { } } -pub type GossipVerifiedBlobList = VariableList< - GossipVerifiedBlob, - <::EthSpec as EthSpec>::MaxBlobsPerBlock, ->; - /// A wrapper around a `BlobSidecar` that indicates it has been approved for re-gossiping on /// the p2p network. #[derive(Debug)] diff --git a/beacon_node/beacon_chain/src/data_column_verification.rs b/beacon_node/beacon_chain/src/data_column_verification.rs index a4e83b27514..62536e67c68 100644 --- a/beacon_node/beacon_chain/src/data_column_verification.rs +++ b/beacon_node/beacon_chain/src/data_column_verification.rs @@ -160,8 +160,6 @@ impl From for GossipDataColumnError { } } -pub type GossipVerifiedDataColumnList = RuntimeVariableList>; - /// A wrapper around a `DataColumnSidecar` that indicates it has been approved for re-gossiping on /// the p2p network. #[derive(Debug)] diff --git a/beacon_node/beacon_chain/tests/block_verification.rs b/beacon_node/beacon_chain/tests/block_verification.rs index ceb1b51fe3c..f094a173eec 100644 --- a/beacon_node/beacon_chain/tests/block_verification.rs +++ b/beacon_node/beacon_chain/tests/block_verification.rs @@ -976,7 +976,7 @@ async fn block_gossip_verification() { harness .chain - .process_gossip_blob(gossip_verified, || Ok(())) + .process_gossip_blob(gossip_verified) .await .expect("should import valid gossip verified blob"); } @@ -1247,7 +1247,7 @@ async fn verify_block_for_gossip_slashing_detection() { .unwrap(); harness .chain - .process_gossip_blob(verified_blob, || Ok(())) + .process_gossip_blob(verified_blob) .await .unwrap(); } diff --git a/beacon_node/beacon_chain/tests/events.rs b/beacon_node/beacon_chain/tests/events.rs index 31e69f0524e..ab784d3be45 100644 --- a/beacon_node/beacon_chain/tests/events.rs +++ b/beacon_node/beacon_chain/tests/events.rs @@ -35,7 +35,7 @@ async fn blob_sidecar_event_on_process_gossip_blob() { let _ = harness .chain - .process_gossip_blob(gossip_verified_blob, || Ok(())) + .process_gossip_blob(gossip_verified_blob) .await .unwrap(); diff --git a/beacon_node/http_api/src/publish_blocks.rs b/beacon_node/http_api/src/publish_blocks.rs index fceeb2dd231..11effe2845b 100644 --- a/beacon_node/http_api/src/publish_blocks.rs +++ b/beacon_node/http_api/src/publish_blocks.rs @@ -1,4 +1,5 @@ use crate::metrics; +use std::future::Future; use beacon_chain::blob_verification::{GossipBlobError, GossipVerifiedBlob}; use beacon_chain::block_verification_types::AsBlock; @@ -13,9 +14,10 @@ use eth2::types::{ PublishBlockRequest, SignedBlockContents, }; use execution_layer::ProvenancedPayload; +use futures::TryFutureExt; use lighthouse_network::{NetworkGlobals, PubsubMessage}; use network::NetworkMessage; -use rand::seq::SliceRandom; +use rand::prelude::SliceRandom; use slog::{debug, error, info, warn, Logger}; use slot_clock::SlotClock; use std::marker::PhantomData; @@ -26,9 +28,8 @@ use tokio::sync::mpsc::UnboundedSender; use tree_hash::TreeHash; use types::{ AbstractExecPayload, BeaconBlockRef, BlobSidecar, BlobsList, BlockImportSource, - DataColumnSidecarList, DataColumnSubnetId, EthSpec, ExecPayload, ExecutionBlockHash, ForkName, - FullPayload, FullPayloadBellatrix, Hash256, KzgProofs, SignedBeaconBlock, - SignedBlindedBeaconBlock, + DataColumnSubnetId, EthSpec, ExecPayload, ExecutionBlockHash, ForkName, FullPayload, + FullPayloadBellatrix, Hash256, KzgProofs, SignedBeaconBlock, SignedBlindedBeaconBlock, }; use warp::http::StatusCode; use warp::{reply::Response, Rejection, Reply}; @@ -97,14 +98,9 @@ pub async fn publish_block>( }; let block = unverified_block.inner_block(); debug!(log, "Signed block received in HTTP API"; "slot" => block.slot()); - let malicious_withhold_count = chain.config.malicious_withhold_count; - let chain_cloned = chain.clone(); /* actually publish a block */ let publish_block_p2p = move |block: Arc>, - should_publish_block: bool, - blob_sidecars: Vec>>, - mut data_column_sidecars: DataColumnSidecarList, sender, log, seen_timestamp| @@ -120,53 +116,16 @@ pub async fn publish_block>( publish_delay, ); - let mut pubsub_messages = if should_publish_block { - info!( - log, - "Signed block published to network via HTTP API"; - "slot" => block.slot(), - "blobs_published" => blob_sidecars.len(), - "publish_delay_ms" => publish_delay.as_millis(), - ); - vec![PubsubMessage::BeaconBlock(block.clone())] - } else { - vec![] - }; + info!( + log, + "Signed block published to network via HTTP API"; + "slot" => block.slot(), + "publish_delay_ms" => publish_delay.as_millis(), + ); - match block.as_ref() { - SignedBeaconBlock::Base(_) - | SignedBeaconBlock::Altair(_) - | SignedBeaconBlock::Bellatrix(_) - | SignedBeaconBlock::Capella(_) => { - crate::publish_pubsub_messages(&sender, pubsub_messages) - .map_err(|_| BlockError::BeaconChainError(BeaconChainError::UnableToPublish))?; - } - SignedBeaconBlock::Deneb(_) | SignedBeaconBlock::Electra(_) => { - for blob in blob_sidecars.into_iter() { - pubsub_messages.push(PubsubMessage::BlobSidecar(Box::new((blob.index, blob)))); - } - if malicious_withhold_count > 0 { - let columns_to_keep = data_column_sidecars - .len() - .saturating_sub(malicious_withhold_count); - // Randomize columns before dropping the last malicious_withhold_count items - data_column_sidecars.shuffle(&mut rand::thread_rng()); - drop(data_column_sidecars.drain(columns_to_keep..)); - } + crate::publish_pubsub_message(&sender, PubsubMessage::BeaconBlock(block.clone())) + .map_err(|_| BlockError::BeaconChainError(BeaconChainError::UnableToPublish))?; - for data_col in data_column_sidecars { - let subnet = DataColumnSubnetId::from_column_index::( - data_col.index as usize, - &chain_cloned.spec, - ); - pubsub_messages.push(PubsubMessage::DataColumnSidecar(Box::new(( - subnet, data_col, - )))); - } - crate::publish_pubsub_messages(&sender, pubsub_messages) - .map_err(|_| BlockError::BeaconChainError(BeaconChainError::UnableToPublish))?; - } - }; Ok(()) }; @@ -174,145 +133,11 @@ pub async fn publish_block>( let slot = block.message().slot(); let sender_clone = network_tx.clone(); - // Convert blobs to either: - // - // 1. Blob sidecars if prior to peer DAS, or - // 2. Data column sidecars if post peer DAS. - let peer_das_enabled = chain.spec.is_peer_das_enabled_for_epoch(block.epoch()); - - let (blob_sidecars, data_column_sidecars) = match unverified_blobs { - // Pre-PeerDAS: construct blob sidecars for the network. - Some((kzg_proofs, blobs)) if !peer_das_enabled => { - let blob_sidecars = 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); - blob_sidecar.map_err(|e| { - error!( - log, - "Invalid blob - not publishing block"; - "error" => ?e, - "blob_index" => i, - "slot" => slot, - ); - warp_utils::reject::custom_bad_request(format!("{e:?}")) - }) - }) - .collect::, Rejection>>()?; - (blob_sidecars, vec![]) - } - // Post PeerDAS: construct data columns. - Some((_, blobs)) => { - // TODO(das): this is sub-optimal and should likely not be happening prior to gossip - // block publishing. - let data_column_sidecars = build_blob_data_column_sidecars(&chain, &block, blobs) - .map_err(|e| { - error!( - log, - "Invalid data column - not publishing block"; - "error" => ?e, - "slot" => slot - ); - warp_utils::reject::custom_bad_request(format!("{e:?}")) - })?; - (vec![], data_column_sidecars) - } - None => (vec![], vec![]), - }; + let build_sidecar_task_handle = + spawn_build_data_sidecar_task(chain.clone(), block.clone(), unverified_blobs, log.clone())?; // Gossip verify the block and blobs/data columns separately. let gossip_verified_block_result = unverified_block.into_gossip_verified_block(&chain); - let gossip_verified_blobs = blob_sidecars - .into_iter() - .map(|blob_sidecar| { - 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!( - log, - "Blob for publication already known"; - "blob_index" => blob_sidecar.index, - "slot" => slot, - "proposer" => proposer, - ); - Ok(None) - } - Err(e) => { - error!( - log, - "Blob for publication is gossip-invalid"; - "blob_index" => blob_sidecar.index, - "slot" => slot, - "error" => ?e, - ); - Err(warp_utils::reject::custom_bad_request(e.to_string())) - } - } - }) - .collect::, Rejection>>()?; - - let gossip_verified_data_columns = data_column_sidecars - .into_iter() - .map(|data_column_sidecar| { - let column_index = data_column_sidecar.index as usize; - let subnet = - DataColumnSubnetId::from_column_index::(column_index, &chain.spec); - let gossip_verified_column = - GossipVerifiedDataColumn::new(data_column_sidecar, subnet.into(), &chain); - - match gossip_verified_column { - Ok(blob) => Ok(Some(blob)), - Err(GossipDataColumnError::PriorKnown { proposer, .. }) => { - // Log the error but do not abort publication, we may need to publish the block - // or some of the other data columns if the block & data columns are only - // partially published by the other publisher. - debug!( - log, - "Data column for publication already known"; - "column_index" => column_index, - "slot" => slot, - "proposer" => proposer, - ); - Ok(None) - } - Err(e) => { - error!( - log, - "Data column for publication is gossip-invalid"; - "column_index" => column_index, - "slot" => slot, - "error" => ?e, - ); - Err(warp_utils::reject::custom_bad_request(format!("{e:?}"))) - } - } - }) - .collect::, Rejection>>()?; - - let publishable_blobs = gossip_verified_blobs - .iter() - .flatten() - .map(|b| b.clone_blob()) - .collect::>(); - - let publishable_data_columns = gossip_verified_data_columns - .iter() - .flatten() - .map(|b| b.clone_data_column()) - .collect::>(); - let block_root = block_root.unwrap_or_else(|| { gossip_verified_block_result.as_ref().map_or_else( |_| block.canonical_root(), @@ -321,12 +146,9 @@ pub async fn publish_block>( }); let should_publish_block = gossip_verified_block_result.is_ok(); - if let BroadcastValidation::Gossip = validation_level { + if BroadcastValidation::Gossip == validation_level && should_publish_block { publish_block_p2p( block.clone(), - should_publish_block, - publishable_blobs.clone(), - publishable_data_columns.clone(), sender_clone.clone(), log.clone(), seen_timestamp, @@ -337,38 +159,41 @@ pub async fn publish_block>( let publish_fn_completed = Arc::new(AtomicBool::new(false)); let block_to_publish = block.clone(); let publish_fn = || { - match validation_level { - BroadcastValidation::Gossip => (), - BroadcastValidation::Consensus => publish_block_p2p( - block_to_publish.clone(), - should_publish_block, - publishable_blobs.clone(), - publishable_data_columns.clone(), - sender_clone.clone(), - log.clone(), - seen_timestamp, - )?, - BroadcastValidation::ConsensusAndEquivocation => { - check_slashable(&chain, block_root, &block_to_publish, &log)?; - publish_block_p2p( + if should_publish_block { + match validation_level { + BroadcastValidation::Gossip => (), + BroadcastValidation::Consensus => publish_block_p2p( block_to_publish.clone(), - should_publish_block, - publishable_blobs.clone(), - publishable_data_columns.clone(), sender_clone.clone(), log.clone(), seen_timestamp, - )?; - } - }; + )?, + BroadcastValidation::ConsensusAndEquivocation => { + check_slashable(&chain, block_root, &block_to_publish, &log)?; + publish_block_p2p( + block_to_publish.clone(), + sender_clone.clone(), + log.clone(), + seen_timestamp, + )?; + } + }; + } + publish_fn_completed.store(true, Ordering::SeqCst); 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()) + })?; // Importing the blobs could trigger block import and network publication in the case // where the block was already seen on gossip. - if let Err(e) = Box::pin(chain.process_gossip_blob(blob, &publish_fn)).await { + 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)) @@ -383,14 +208,12 @@ pub async fn publish_block>( } } - if gossip_verified_data_columns - .iter() - .map(Option::is_some) - .count() - > 0 - { + if gossip_verified_columns.iter().map(Option::is_some).count() > 0 { + publish_column_sidecars(network_tx, &gossip_verified_columns, &chain).map_err(|_| { + warp_utils::reject::custom_server_error("unable to publish data column sidecars".into()) + })?; let sampling_columns_indices = &network_globals.sampling_columns; - let sampling_columns = gossip_verified_data_columns + let sampling_columns = gossip_verified_columns .into_iter() .flatten() .filter(|data_column| sampling_columns_indices.contains(&data_column.index())) @@ -501,6 +324,224 @@ pub async fn publish_block>( } } +type BuildDataSidecarTaskResult = Result< + ( + Vec>>, + Vec>>, + ), + Rejection, +>; + +/// Convert blobs to either: +/// +/// 1. Blob sidecars if prior to peer DAS, or +/// 2. Data column sidecars if post peer DAS. +fn spawn_build_data_sidecar_task( + chain: Arc>, + block: Arc>>, + proofs_and_blobs: UnverifiedBlobs, + log: Logger, +) -> Result>, Rejection> { + chain + .clone() + .task_executor + .spawn_blocking_handle( + move || { + let Some((kzg_proofs, blobs)) = proofs_and_blobs else { + return Ok((vec![], vec![])); + }; + + 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, &log)?; + Ok((gossip_verified_blobs, vec![])) + } else { + // Post PeerDAS: construct data columns. + let gossip_verified_data_columns = + build_gossip_verified_data_columns(&chain, &block, blobs, &log)?; + Ok((vec![], gossip_verified_data_columns)) + } + }, + "build_data_sidecars", + ) + .ok_or(warp_utils::reject::custom_server_error( + "runtime shutdown".to_string(), + )) + .map(|r| { + r.map_err(|_| warp_utils::reject::custom_server_error("join error".to_string())) + .and_then(|output| async move { output }) + }) +} + +fn build_gossip_verified_data_columns( + chain: &BeaconChain, + block: &SignedBeaconBlock>, + blobs: BlobsList, + log: &Logger, +) -> Result>>, Rejection> { + let slot = block.slot(); + let data_column_sidecars = + build_blob_data_column_sidecars(chain, block, blobs).map_err(|e| { + error!( + log, + "Invalid data column - not publishing block"; + "error" => ?e, + "slot" => slot + ); + warp_utils::reject::custom_bad_request(format!("{e:?}")) + })?; + + let slot = block.slot(); + let gossip_verified_data_columns = data_column_sidecars + .into_iter() + .map(|data_column_sidecar| { + let column_index = data_column_sidecar.index as usize; + let subnet = + DataColumnSubnetId::from_column_index::(column_index, &chain.spec); + let gossip_verified_column = + GossipVerifiedDataColumn::new(data_column_sidecar, subnet.into(), chain); + + match gossip_verified_column { + Ok(blob) => Ok(Some(blob)), + Err(GossipDataColumnError::PriorKnown { proposer, .. }) => { + // Log the error but do not abort publication, we may need to publish the block + // or some of the other data columns if the block & data columns are only + // partially published by the other publisher. + debug!( + log, + "Data column for publication already known"; + "column_index" => column_index, + "slot" => slot, + "proposer" => proposer, + ); + Ok(None) + } + Err(e) => { + error!( + log, + "Data column for publication is gossip-invalid"; + "column_index" => column_index, + "slot" => slot, + "error" => ?e, + ); + Err(warp_utils::reject::custom_bad_request(format!("{e:?}"))) + } + } + }) + .collect::, Rejection>>()?; + + Ok(gossip_verified_data_columns) +} + +fn build_gossip_verified_blobs( + chain: &BeaconChain, + block: &SignedBeaconBlock>, + blobs: BlobsList, + kzg_proofs: KzgProofs, + log: &Logger, +) -> 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!( + log, + "Invalid blob - not publishing block"; + "error" => ?e, + "blob_index" => i, + "slot" => slot, + ); + 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!( + log, + "Blob for publication already known"; + "blob_index" => blob_sidecar.index, + "slot" => slot, + "proposer" => proposer, + ); + Ok(None) + } + Err(e) => { + error!( + log, + "Blob for publication is gossip-invalid"; + "blob_index" => blob_sidecar.index, + "slot" => slot, + "error" => ?e, + ); + Err(warp_utils::reject::custom_bad_request(e.to_string())) + } + } + }) + .collect::, Rejection>>()?; + + Ok(gossip_verified_blobs) +} + +fn publish_column_sidecars( + sender_clone: &UnboundedSender>, + data_column_sidecars: &[Option>], + chain: &BeaconChain, +) -> Result<(), BlockError> { + let malicious_withhold_count = chain.config.malicious_withhold_count; + let mut data_column_sidecars = data_column_sidecars + .iter() + .flatten() + .map(|d| d.clone_data_column()) + .collect::>(); + if malicious_withhold_count > 0 { + let columns_to_keep = data_column_sidecars + .len() + .saturating_sub(malicious_withhold_count); + // Randomize columns before dropping the last malicious_withhold_count items + data_column_sidecars.shuffle(&mut rand::thread_rng()); + data_column_sidecars.truncate(columns_to_keep); + } + let pubsub_messages = data_column_sidecars + .into_iter() + .map(|data_col| { + let subnet = DataColumnSubnetId::from_column_index::( + data_col.index as usize, + &chain.spec, + ); + PubsubMessage::DataColumnSidecar(Box::new((subnet, data_col))) + }) + .collect::>(); + crate::publish_pubsub_messages(sender_clone, pubsub_messages) + .map_err(|_| BlockError::BeaconChainError(BeaconChainError::UnableToPublish)) +} + +fn publish_blob_sidecars( + sender_clone: &UnboundedSender>, + blob: &GossipVerifiedBlob, +) -> Result<(), BlockError> { + let pubsub_message = PubsubMessage::BlobSidecar(Box::new((blob.index(), blob.clone_blob()))); + crate::publish_pubsub_message(sender_clone, pubsub_message) + .map_err(|_| BlockError::BeaconChainError(BeaconChainError::UnableToPublish)) +} + async fn post_block_import_logging_and_response( result: Result, validation_level: BroadcastValidation, diff --git a/beacon_node/http_api/tests/broadcast_validation_tests.rs b/beacon_node/http_api/tests/broadcast_validation_tests.rs index f55983ec66a..1338f4f1802 100644 --- a/beacon_node/http_api/tests/broadcast_validation_tests.rs +++ b/beacon_node/http_api/tests/broadcast_validation_tests.rs @@ -1486,7 +1486,7 @@ pub async fn block_seen_on_gossip_with_some_blobs() { tester .harness .chain - .process_gossip_blob(gossip_blob, || panic!("should not publish block yet")) + .process_gossip_blob(gossip_blob) .await .unwrap(); } @@ -1559,7 +1559,7 @@ pub async fn blobs_seen_on_gossip_without_block() { tester .harness .chain - .process_gossip_blob(gossip_blob, || panic!("should not publish block yet")) + .process_gossip_blob(gossip_blob) .await .unwrap(); } @@ -1633,7 +1633,7 @@ pub async fn blobs_seen_on_gossip_without_block_and_no_http_blobs() { tester .harness .chain - .process_gossip_blob(gossip_blob, || panic!("should not publish block yet")) + .process_gossip_blob(gossip_blob) .await .unwrap(); } @@ -1705,7 +1705,7 @@ pub async fn slashable_blobs_seen_on_gossip_cause_failure() { tester .harness .chain - .process_gossip_blob(gossip_blob, || panic!("should not publish block yet")) + .process_gossip_blob(gossip_blob) .await .unwrap(); } 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 b5da8e82a77..aa7194bb056 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -914,10 +914,7 @@ impl NetworkBeaconProcessor { let blob_slot = verified_blob.slot(); let blob_index = verified_blob.id().index; - let result = self - .chain - .process_gossip_blob(verified_blob, || Ok(())) - .await; + let result = self.chain.process_gossip_blob(verified_blob).await; match &result { Ok(AvailabilityProcessingStatus::Imported(block_root)) => { diff --git a/testing/ef_tests/src/cases/fork_choice.rs b/testing/ef_tests/src/cases/fork_choice.rs index 8d933a6fcd5..33ae132e8a2 100644 --- a/testing/ef_tests/src/cases/fork_choice.rs +++ b/testing/ef_tests/src/cases/fork_choice.rs @@ -505,8 +505,8 @@ impl Tester { } Err(_) => GossipVerifiedBlob::__assumed_valid(blob_sidecar), }; - let result = self - .block_on_dangerous(self.harness.chain.process_gossip_blob(blob, || Ok(())))?; + let result = + self.block_on_dangerous(self.harness.chain.process_gossip_blob(blob))?; if valid { assert!(result.is_ok()); } From 0a9d5a0f61b4114dff93fe88c46ca4b97dde75f2 Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Fri, 18 Oct 2024 17:49:27 +1100 Subject: [PATCH 13/25] Address review comments and refactor. --- beacon_node/beacon_chain/src/beacon_chain.rs | 8 +- beacon_node/beacon_chain/src/chain_config.rs | 6 + beacon_node/beacon_chain/src/fetch_blobs.rs | 278 ++++++++++-------- beacon_node/beacon_chain/src/metrics.rs | 17 +- .../src/network_beacon_processor/mod.rs | 32 +- beacon_node/src/cli.rs | 16 + beacon_node/src/config.rs | 16 + lighthouse/tests/beacon_node.rs | 28 ++ 8 files changed, 264 insertions(+), 137 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 0d1609bd6f4..94b0675a320 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -3083,6 +3083,11 @@ impl BeaconChain { self.remove_notified(&block_root, r) } + /// Process blobs retrieved from the EL and returns the `AvailabilityProcessingStatus`. + /// + /// `data_column_recv`: An optional receiver for `DataColumnSidecarList`. + /// If PeerDAS is enabled, this receiver will be provided and used to send + /// the `DataColumnSidecar`s once they have been successfully computed. pub async fn process_engine_blobs( self: &Arc, slot: Slot, @@ -3896,7 +3901,8 @@ impl BeaconChain { let (_, signed_block, blobs, data_columns) = signed_block.deconstruct(); // TODO(das) we currently store all subnet sampled columns. Tracking issue to exclude non custody columns: https://github.com/sigp/lighthouse/issues/6465 let custody_columns_count = self.data_availability_checker.get_sampling_column_count(); - let data_columns = data_columns.filter(|columns| columns.len() >= custody_columns_count); + // if block is made available via blobs, dropped the data columns. + let data_columns = data_columns.filter(|columns| columns.len() == custody_columns_count); let data_columns = match (data_columns, data_column_recv) { // If the block was made available via custody columns received from gossip / rpc, use them diff --git a/beacon_node/beacon_chain/src/chain_config.rs b/beacon_node/beacon_chain/src/chain_config.rs index 20edfbf31a4..5cae99484b8 100644 --- a/beacon_node/beacon_chain/src/chain_config.rs +++ b/beacon_node/beacon_chain/src/chain_config.rs @@ -88,6 +88,10 @@ pub struct ChainConfig { pub malicious_withhold_count: usize, /// Enable peer sampling on blocks. pub enable_sampling: bool, + /// Number of batches that supernodes split data columns into during publishing by a non-proposer. + pub supernode_data_column_publication_batches: usize, + /// The delay applied by supernodes between the sending of each data column batch. + pub supernode_data_column_publication_batch_interval: Duration, } impl Default for ChainConfig { @@ -121,6 +125,8 @@ impl Default for ChainConfig { enable_light_client_server: false, malicious_withhold_count: 0, enable_sampling: false, + supernode_data_column_publication_batches: 4, + supernode_data_column_publication_batch_interval: Duration::from_millis(200), } } } diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index 5ddb310eb06..ce91f2dd025 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -13,25 +13,19 @@ use crate::{metrics, AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes use execution_layer::json_structures::BlobAndProofV1; use execution_layer::Error as ExecutionLayerError; use itertools::Either; -use lighthouse_metrics::{inc_counter, TryExt}; +use lighthouse_metrics::{inc_counter, inc_counter_by, TryExt}; use rand::prelude::SliceRandom; -use slog::{debug, error}; +use slog::{debug, error, o, Logger}; use ssz_types::FixedVector; use state_processing::per_block_processing::deneb::kzg_commitment_to_versioned_hash; use std::sync::Arc; -use std::time::Duration; +use tokio::sync::mpsc::Receiver; use types::blob_sidecar::{BlobSidecarError, FixedBlobSidecarList}; use types::{ - BeaconStateError, BlobSidecar, DataColumnSidecarList, EthSpec, FullPayload, Hash256, - SignedBeaconBlock, SignedBeaconBlockHeader, + BeaconStateError, BlobSidecar, DataColumnSidecar, DataColumnSidecarList, EthSpec, FullPayload, + Hash256, SignedBeaconBlock, SignedBeaconBlockHeader, }; -/// Number of batches that supernodes split data columns into during publishing. -pub const SUPERNODE_DATA_COLUMN_PUBLICATION_BATCHES: usize = 4; - -/// The delay applied by supernodes between the sending of each data column batch. -pub const SUPERNODE_DATA_COLUMN_PUBLICATION_BATCH_INTERVAL: Duration = Duration::from_millis(200); - pub enum BlobsOrDataColumns { Blobs(Vec>>), DataColumns(DataColumnSidecarList), @@ -56,21 +50,26 @@ pub async fn fetch_and_process_engine_blobs( block: Arc>>, publish_fn: impl Fn(BlobsOrDataColumns) + Send + 'static, ) -> Result, FetchEngineBlobError> { + let block_root_str = format!("{:?}", block_root); + let log = chain + .log + .new(o!("service" => "fetch_engine_blobs", "block_root" => block_root_str)); + let versioned_hashes = if let Ok(kzg_commitments) = block.message().body().blob_kzg_commitments() { kzg_commitments .iter() .map(kzg_commitment_to_versioned_hash) - .collect() + .collect::>() } else { - vec![] + debug!( + log, + "Fetch blobs not triggered - none required"; + ); + return Ok(None); }; - let num_expected_blobs = versioned_hashes.len(); - if versioned_hashes.is_empty() { - debug!(chain.log, "Blobs from EL - none required"); - return Ok(None); - } + let num_expected_blobs = versioned_hashes.len(); let execution_layer = chain .execution_layer @@ -78,8 +77,8 @@ pub async fn fetch_and_process_engine_blobs( .ok_or(FetchEngineBlobError::ExecutionLayerMissing)?; debug!( - chain.log, - "Blobs from EL - start request"; + log, + "Fetching blobs from the EL"; "num_expected_blobs" => num_expected_blobs, ); let response = execution_layer @@ -87,7 +86,27 @@ pub async fn fetch_and_process_engine_blobs( .await .map_err(FetchEngineBlobError::RequestFailed)?; let num_fetched_blobs = response.iter().filter(|b| b.is_some()).count(); - let all_blobs_fetched = num_fetched_blobs == num_expected_blobs; + + inc_counter_by( + &metrics::BLOBS_FROM_EL_EXPECTED_TOTAL, + num_expected_blobs as u64, + ); + inc_counter_by( + &metrics::BLOBS_FROM_EL_RECEIVED_TOTAL, + num_fetched_blobs as u64, + ); + + if num_fetched_blobs == 0 { + debug!( + chain.log, + "No blobs fetched from the EL"; + "num_expected_blobs" => num_expected_blobs, + ); + inc_counter(&metrics::BLOBS_FROM_EL_MISS_TOTAL); + return Ok(None); + } else { + inc_counter(&metrics::BLOBS_FROM_EL_HIT_TOTAL); + } let (signed_block_header, kzg_commitments_proof) = block .signed_block_header_and_kzg_commitments_proof() @@ -112,118 +131,28 @@ pub async fn fetch_and_process_engine_blobs( // Partial blobs response isn't useful for PeerDAS, so we don't bother building and publishing data columns. let data_columns_receiver_opt = if peer_das_enabled { - if !all_blobs_fetched { + if !num_fetched_blobs == num_expected_blobs { debug!( - chain.log, + log, "Not all blobs fetched from the EL"; + "info" => "Unable to compute data columns", "num_fetched_blobs" => num_fetched_blobs, "num_expected_blobs" => num_expected_blobs, ); - inc_counter(&metrics::BLOBS_FROM_EL_MISS_TOTAL); return Ok(None); } - inc_counter(&metrics::BLOBS_FROM_EL_HIT_TOTAL); - - let logger = chain.log.clone(); - let block_cloned = block.clone(); - let kzg = chain.kzg.clone(); - let spec = chain.spec.clone(); - let blobs_cloned = fixed_blob_sidecar_list.clone(); - let chain_cloned = chain.clone(); - let (data_columns_sender, data_columns_receiver) = tokio::sync::mpsc::channel(1); - chain.task_executor.spawn_blocking( - move || { - let mut timer = metrics::start_timer_vec( - &metrics::DATA_COLUMN_SIDECAR_COMPUTATION, - &[&blobs_cloned.len().to_string()], - ); - let blob_refs = blobs_cloned - .iter() - .filter_map(|b| b.as_ref().map(|b| &b.blob)) - .collect::>(); - let data_columns_result = - blobs_to_data_column_sidecars(&blob_refs, &block_cloned, &kzg, &spec) - .discard_timer_on_break(&mut timer); - drop(timer); - - let mut all_data_columns = match data_columns_result { - Ok(d) => d, - Err(e) => { - error!( - logger, - "Failed to build data column sidecars from blobs"; - "error" => ?e - ); - return; - } - }; - - if let Err(e) = data_columns_sender.try_send(all_data_columns.clone()) { - error!(logger, "Failed to send computed data columns"; "error" => ?e); - }; - - // Check indices from cache before sending the columns, to make sure we don't - // publish components already seen on gossip. - let is_supernode = chain_cloned - .data_availability_checker - .get_sampling_column_count() - == spec.number_of_columns; - - // At the moment non supernodes are not required to publish any columns. - // TODO(das): we could experiment with having full nodes publish their custodied - // columns here. - if !is_supernode { - return; - } - - // To reduce bandwidth for supernodes: permute the columns to publish and - // publish them in batches. Our hope is that some columns arrive from - // other supernodes in the meantime, obviating the need for us to publish - // them. If no other publisher exists for a column, it will eventually get - // published here. - // FIXME(das): deduplicate this wrt to gossip/sync methods - all_data_columns.shuffle(&mut rand::thread_rng()); - - let batch_size = all_data_columns.len() / SUPERNODE_DATA_COLUMN_PUBLICATION_BATCHES; - for batch in all_data_columns.chunks(batch_size) { - let already_seen = chain_cloned - .data_availability_checker - .cached_data_column_indexes(&block_root) - .unwrap_or_default(); - let publishable = batch - .iter() - .filter(|col| !already_seen.contains(&col.index())) - .cloned() - .collect::>(); - if !publishable.is_empty() { - debug!( - chain_cloned.log, - "Publishing data columns from EL"; - "count" => publishable.len() - ); - publish_fn(BlobsOrDataColumns::DataColumns(publishable)); - } - std::thread::sleep(SUPERNODE_DATA_COLUMN_PUBLICATION_BATCH_INTERVAL); - } - }, - "compute_data_columns", + let data_columns_receiver = spawn_compute_and_publish_data_columns_task( + &chain, + block_root, + block.clone(), + fixed_blob_sidecar_list.clone(), + publish_fn, + log.clone(), ); Some(data_columns_receiver) } else { - if num_fetched_blobs == 0 { - debug!( - chain.log, - "No blobs fetched from the EL"; - "num_expected_blobs" => num_expected_blobs, - ); - inc_counter(&metrics::BLOBS_FROM_EL_MISS_TOTAL); - return Ok(None); - } - - inc_counter(&metrics::BLOBS_FROM_EL_HIT_TOTAL); - let all_blobs = fixed_blob_sidecar_list.clone(); let all_blobs_iter = all_blobs.into_iter().flat_map(|b| b.clone()); @@ -245,8 +174,8 @@ pub async fn fetch_and_process_engine_blobs( }; debug!( - chain.log, - "Blobs from EL - start processing"; + log, + "Processing engine blobs"; "num_fetched_blobs" => num_fetched_blobs, ); @@ -263,6 +192,109 @@ pub async fn fetch_and_process_engine_blobs( Ok(Some(availability_processing_status)) } +fn spawn_compute_and_publish_data_columns_task( + chain: &Arc>, + block_root: Hash256, + block: Arc>>, + blobs: FixedBlobSidecarList, + publish_fn: impl Fn(BlobsOrDataColumns) + Send + 'static, + log: Logger, +) -> Receiver>>> { + let chain_cloned = chain.clone(); + let (data_columns_sender, data_columns_receiver) = tokio::sync::mpsc::channel(1); + + chain.task_executor.spawn_blocking( + move || { + let mut timer = metrics::start_timer_vec( + &metrics::DATA_COLUMN_SIDECAR_COMPUTATION, + &[&blobs.len().to_string()], + ); + let blob_refs = blobs + .iter() + .filter_map(|b| b.as_ref().map(|b| &b.blob)) + .collect::>(); + let data_columns_result = blobs_to_data_column_sidecars( + &blob_refs, + &block, + &chain_cloned.kzg, + &chain_cloned.spec, + ) + .discard_timer_on_break(&mut timer); + drop(timer); + + let mut all_data_columns = match data_columns_result { + Ok(d) => d, + Err(e) => { + error!( + log, + "Failed to build data column sidecars from blobs"; + "error" => ?e + ); + return; + } + }; + + if let Err(e) = data_columns_sender.try_send(all_data_columns.clone()) { + error!(log, "Failed to send computed data columns"; "error" => ?e); + }; + + // Check indices from cache before sending the columns, to make sure we don't + // publish components already seen on gossip. + let is_supernode = chain_cloned + .data_availability_checker + .get_sampling_column_count() + == chain_cloned.spec.number_of_columns; + + // At the moment non supernodes are not required to publish any columns. + // TODO(das): we could experiment with having full nodes publish their custodied + // columns here. + if !is_supernode { + return; + } + + // To reduce bandwidth for supernodes: permute the columns to publish and + // publish them in batches. Our hope is that some columns arrive from + // other supernodes in the meantime, obviating the need for us to publish + // them. If no other publisher exists for a column, it will eventually get + // published here. + // FIXME(das): deduplicate this wrt to gossip/sync methods + all_data_columns.shuffle(&mut rand::thread_rng()); + + let supernode_data_column_publication_batches = chain_cloned + .config + .supernode_data_column_publication_batches; + let supernode_data_column_publication_batch_interval = chain_cloned + .config + .supernode_data_column_publication_batch_interval; + + let batch_size = all_data_columns.len() / supernode_data_column_publication_batches; + for batch in all_data_columns.chunks(batch_size) { + let already_seen = chain_cloned + .data_availability_checker + .cached_data_column_indexes(&block_root) + .unwrap_or_default(); + let publishable = batch + .iter() + .filter(|col| !already_seen.contains(&col.index())) + .cloned() + .collect::>(); + if !publishable.is_empty() { + debug!( + log, + "Publishing data columns from EL"; + "count" => publishable.len() + ); + publish_fn(BlobsOrDataColumns::DataColumns(publishable)); + } + std::thread::sleep(supernode_data_column_publication_batch_interval); + } + }, + "compute_and_publish_data_columns", + ); + + data_columns_receiver +} + fn build_blob_sidecars( block: &Arc>>, response: Vec>>, diff --git a/beacon_node/beacon_chain/src/metrics.rs b/beacon_node/beacon_chain/src/metrics.rs index e260726f654..163fa9082c6 100644 --- a/beacon_node/beacon_chain/src/metrics.rs +++ b/beacon_node/beacon_chain/src/metrics.rs @@ -112,9 +112,10 @@ pub static BLOCK_PROCESSING_POST_EXEC_PROCESSING: LazyLock> = ) }); pub static BLOCK_PROCESSING_DATA_COLUMNS_WAIT: LazyLock> = LazyLock::new(|| { - try_create_histogram( + try_create_histogram_with_buckets( "beacon_block_processing_data_columns_wait_seconds", "Time spent waiting for data columns to be computed before starting database write", + exponential_buckets(0.01, 2.0, 10), ) }); pub static BLOCK_PROCESSING_DB_WRITE: LazyLock> = LazyLock::new(|| { @@ -1711,6 +1712,20 @@ pub static BLOBS_FROM_EL_MISS_TOTAL: LazyLock> = LazyLock::ne ) }); +pub static BLOBS_FROM_EL_EXPECTED_TOTAL: LazyLock> = LazyLock::new(|| { + try_create_int_counter( + "beacon_blobs_from_el_expected_total", + "Number of blob expected from the execution layer", + ) +}); + +pub static BLOBS_FROM_EL_RECEIVED_TOTAL: LazyLock> = LazyLock::new(|| { + try_create_int_counter( + "beacon_blobs_from_el_received_total", + "Number of blob fetched from the execution layer", + ) +}); + /* * Light server message verification */ diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 00c480b53c8..cedd3e6f147 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -45,12 +45,6 @@ mod tests; pub(crate) const FUTURE_SLOT_TOLERANCE: u64 = 1; -/// Number of batches that supernodes split data columns into during publishing. -const SUPERNODE_DATA_COLUMN_PUBLICATION_BATCHES: usize = 4; - -/// The delay applied by supernodes between the sending of each data column batch. -const SUPERNODE_DATA_COLUMN_PUBLICATION_BATCH_INTERVAL: Duration = Duration::from_millis(200); - /// Defines if and where we will store the SSZ files of invalid blocks. #[derive(Clone)] pub enum InvalidBlockStorage { @@ -919,7 +913,13 @@ impl NetworkBeaconProcessor { ); } }, - Ok(None) => { /* No blobs fetched from the EL. Reasons logged separately. */ } + Ok(None) => { + debug!( + self.log, + "Fetch blobs completed without import"; + "block_hash" => %block_root, + ); + } Err(e) => { error!( self.log, @@ -994,8 +994,10 @@ impl NetworkBeaconProcessor { block_root: Hash256, ) { let self_clone = self.clone(); + self.executor.spawn( async move { + let chain = self_clone.chain.clone(); let publish_fn = |columns: DataColumnSidecarList| { self_clone.send_network_message(NetworkMessage::Publish { messages: columns @@ -1003,7 +1005,7 @@ impl NetworkBeaconProcessor { .map(|d| { let subnet = DataColumnSubnetId::from_column_index::( d.index as usize, - &self_clone.chain.spec, + &chain.spec, ); PubsubMessage::DataColumnSidecar(Box::new((subnet, d))) }) @@ -1015,12 +1017,17 @@ impl NetworkBeaconProcessor { // The hope is that we won't need to publish some columns because we will receive them // on gossip from other supernodes. data_columns_to_publish.shuffle(&mut rand::thread_rng()); + + let supernode_data_column_publication_batch_interval = chain + .config + .supernode_data_column_publication_batch_interval; + let supernode_data_column_publication_batches = + chain.config.supernode_data_column_publication_batches; let batch_size = - data_columns_to_publish.len() / SUPERNODE_DATA_COLUMN_PUBLICATION_BATCHES; + data_columns_to_publish.len() / supernode_data_column_publication_batches; for batch in data_columns_to_publish.chunks(batch_size) { - let already_seen = self_clone - .chain + let already_seen = chain .data_availability_checker .cached_data_column_indexes(&block_root) .unwrap_or_default(); @@ -1038,7 +1045,8 @@ impl NetworkBeaconProcessor { ); publish_fn(publishable); } - tokio::time::sleep(SUPERNODE_DATA_COLUMN_PUBLICATION_BATCH_INTERVAL).await; + + tokio::time::sleep(supernode_data_column_publication_batch_interval).await; } }, "handle_data_columns_publish", diff --git a/beacon_node/src/cli.rs b/beacon_node/src/cli.rs index d6ed1068036..8f5afd9f565 100644 --- a/beacon_node/src/cli.rs +++ b/beacon_node/src/cli.rs @@ -86,6 +86,22 @@ pub fn cli_app() -> Command { .hide(true) .display_order(0) ) + .arg( + Arg::new("supernode-data-column-publication-batches") + .long("supernode-data-column-publication-batches") + .action(ArgAction::Set) + .help_heading(FLAG_HEADER) + .help("Number of batches that supernodes split data columns into during publishing by a non-proposer. For PeerDAS only.") + .display_order(0) + ) + .arg( + Arg::new("supernode-data-column-publication-batch-interval") + .long("supernode-data-column-publication-batch-interval") + .action(ArgAction::Set) + .help_heading(FLAG_HEADER) + .help("The delay applied by supernodes between the sending of each data column batch. For PeerDAS only.") + .display_order(0) + ) .arg( Arg::new("subscribe-all-subnets") .long("subscribe-all-subnets") diff --git a/beacon_node/src/config.rs b/beacon_node/src/config.rs index f62ccfe3ed9..f886409c69c 100644 --- a/beacon_node/src/config.rs +++ b/beacon_node/src/config.rs @@ -208,6 +208,22 @@ pub fn get_config( client_config.chain.enable_sampling = true; } + if let Some(batches) = + clap_utils::parse_optional(cli_args, "supernode-data-column-publication-batches")? + { + client_config + .chain + .supernode_data_column_publication_batches = batches; + } + + if let Some(interval) = + clap_utils::parse_optional(cli_args, "supernode-data-column-publication-batch-interval")? + { + client_config + .chain + .supernode_data_column_publication_batch_interval = Duration::from_millis(interval); + } + /* * Prometheus metrics HTTP server */ diff --git a/lighthouse/tests/beacon_node.rs b/lighthouse/tests/beacon_node.rs index f3832a1a1e5..881dcb579fc 100644 --- a/lighthouse/tests/beacon_node.rs +++ b/lighthouse/tests/beacon_node.rs @@ -838,6 +838,34 @@ fn network_enable_sampling_flag() { .run_with_zero_port() .with_config(|config| assert!(config.chain.enable_sampling)); } +#[test] +fn supernode_data_column_publication_batches() { + CommandLineTest::new() + .flag("supernode-data-column-publication-batches", Some("3")) + .run_with_zero_port() + .with_config(|config| { + assert_eq!(config.chain.supernode_data_column_publication_batches, 3) + }); +} + +#[test] +fn supernode_data_column_publication_batch_interval() { + CommandLineTest::new() + .flag( + "supernode-data-column-publication-batch-interval", + Some("300"), + ) + .run_with_zero_port() + .with_config(|config| { + assert_eq!( + config + .chain + .supernode_data_column_publication_batch_interval, + 300 + ) + }); +} + #[test] fn network_enable_sampling_flag_default() { CommandLineTest::new() From 7939888430f017f41701ad2c90f749cbc75fb92f Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Fri, 18 Oct 2024 20:07:19 +1100 Subject: [PATCH 14/25] Fix test and docs. --- beacon_node/src/cli.rs | 2 +- book/src/help_bn.md | 6 ++++++ lighthouse/tests/beacon_node.rs | 2 +- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/beacon_node/src/cli.rs b/beacon_node/src/cli.rs index 8b60e53e376..59f0e1dbcbe 100644 --- a/beacon_node/src/cli.rs +++ b/beacon_node/src/cli.rs @@ -99,7 +99,7 @@ pub fn cli_app() -> Command { .long("supernode-data-column-publication-batch-interval") .action(ArgAction::Set) .help_heading(FLAG_HEADER) - .help("The delay applied by supernodes between the sending of each data column batch. For PeerDAS only.") + .help("The delay in milliseconds applied by supernodes between the sending of each data column batch. For PeerDAS only.") .display_order(0) ) .arg( diff --git a/book/src/help_bn.md b/book/src/help_bn.md index 338905a4fbf..fbe0242e7dc 100644 --- a/book/src/help_bn.md +++ b/book/src/help_bn.md @@ -586,6 +586,12 @@ Flags: Subscribe to all subnets regardless of validator count. This will also advertise the beacon node as being long-lived subscribed to all subnets. + --supernode-data-column-publication-batch-interval + The delay in milliseconds applied by supernodes between the sending of + each data column batch. For PeerDAS only. + --supernode-data-column-publication-batches + Number of batches that supernodes split data columns into during + publishing by a non-proposer. For PeerDAS only. --validator-monitor-auto Enables the automatic detection and monitoring of validators connected to the HTTP API and using the subnet subscription endpoint. This diff --git a/lighthouse/tests/beacon_node.rs b/lighthouse/tests/beacon_node.rs index 3762c337a6a..274454dd931 100644 --- a/lighthouse/tests/beacon_node.rs +++ b/lighthouse/tests/beacon_node.rs @@ -854,7 +854,7 @@ fn supernode_data_column_publication_batch_interval() { config .chain .supernode_data_column_publication_batch_interval, - 300 + Duration::from_millis(300) ) }); } From 5ec97561a5bae6efc390719614dbfbd115c6175f Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Fri, 18 Oct 2024 23:03:20 +1100 Subject: [PATCH 15/25] Comment cleanups. --- beacon_node/beacon_chain/benches/benches.rs | 3 +-- beacon_node/beacon_chain/src/fetch_blobs.rs | 18 +++++++++--------- beacon_node/beacon_chain/src/metrics.rs | 4 ++-- beacon_node/http_api/src/publish_blocks.rs | 2 -- consensus/types/src/beacon_block_body.rs | 2 +- 5 files changed, 13 insertions(+), 16 deletions(-) diff --git a/beacon_node/beacon_chain/benches/benches.rs b/beacon_node/beacon_chain/benches/benches.rs index 89bdea8ead3..c09af00be68 100644 --- a/beacon_node/beacon_chain/benches/benches.rs +++ b/beacon_node/beacon_chain/benches/benches.rs @@ -37,13 +37,12 @@ fn all_benches(c: &mut Criterion) { let kzg = get_kzg(&spec); for blob_count in [1, 2, 3, 6] { - let kzg = kzg.clone(); let (signed_block, blobs) = create_test_block_and_blobs::(blob_count, &spec); let column_sidecars = blobs_to_data_column_sidecars( &blobs.iter().collect::>(), &signed_block, - &kzg.clone(), + &kzg, &spec, ) .unwrap(); diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index ce91f2dd025..a3b5063f488 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -119,18 +119,10 @@ pub async fn fetch_and_process_engine_blobs( &kzg_commitments_proof, )?; - // Spawn an async task here for long computation tasks, so it doesn't block processing, and it - // allows blobs / data columns to propagate without waiting for processing. - // - // An `mpsc::Sender` is then used to send the produced data columns to the `beacon_chain` for it - // to be persisted, **after** the block is made attestable. - // - // The reason for doing this is to make the block available and attestable as soon as possible, - // while maintaining the invariant that block and data columns are persisted atomically. let peer_das_enabled = chain.spec.is_peer_das_enabled_for_epoch(block.epoch()); - // Partial blobs response isn't useful for PeerDAS, so we don't bother building and publishing data columns. let data_columns_receiver_opt = if peer_das_enabled { + // Partial blobs response isn't useful for PeerDAS, so we don't bother building and publishing data columns. if !num_fetched_blobs == num_expected_blobs { debug!( log, @@ -192,6 +184,14 @@ pub async fn fetch_and_process_engine_blobs( Ok(Some(availability_processing_status)) } +/// Spawn a blocking task here for long computation tasks, so it doesn't block processing, and it +/// allows blobs / data columns to propagate without waiting for processing. +/// +/// An `mpsc::Sender` is then used to send the produced data columns to the `beacon_chain` for it +/// to be persisted, **after** the block is made attestable. +/// +/// The reason for doing this is to make the block available and attestable as soon as possible, +/// while maintaining the invariant that block and data columns are persisted atomically. fn spawn_compute_and_publish_data_columns_task( chain: &Arc>, block_root: Hash256, diff --git a/beacon_node/beacon_chain/src/metrics.rs b/beacon_node/beacon_chain/src/metrics.rs index 163fa9082c6..aedb9d6efb4 100644 --- a/beacon_node/beacon_chain/src/metrics.rs +++ b/beacon_node/beacon_chain/src/metrics.rs @@ -1715,14 +1715,14 @@ pub static BLOBS_FROM_EL_MISS_TOTAL: LazyLock> = LazyLock::ne pub static BLOBS_FROM_EL_EXPECTED_TOTAL: LazyLock> = LazyLock::new(|| { try_create_int_counter( "beacon_blobs_from_el_expected_total", - "Number of blob expected from the execution layer", + "Number of blobs expected from the execution layer", ) }); pub static BLOBS_FROM_EL_RECEIVED_TOTAL: LazyLock> = LazyLock::new(|| { try_create_int_counter( "beacon_blobs_from_el_received_total", - "Number of blob fetched from the execution layer", + "Number of blobs fetched from the execution layer", ) }); diff --git a/beacon_node/http_api/src/publish_blocks.rs b/beacon_node/http_api/src/publish_blocks.rs index 11effe2845b..b5aa23acf8e 100644 --- a/beacon_node/http_api/src/publish_blocks.rs +++ b/beacon_node/http_api/src/publish_blocks.rs @@ -191,8 +191,6 @@ pub async fn publish_block>( publish_blob_sidecars(network_tx, &blob).map_err(|_| { warp_utils::reject::custom_server_error("unable to publish blob sidecars".into()) })?; - // Importing the blobs could trigger block import and network publication in the case - // where the block was already seen on gossip. 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 { diff --git a/consensus/types/src/beacon_block_body.rs b/consensus/types/src/beacon_block_body.rs index 39fdf9a4f5e..b970980c861 100644 --- a/consensus/types/src/beacon_block_body.rs +++ b/consensus/types/src/beacon_block_body.rs @@ -178,7 +178,7 @@ impl<'a, E: EthSpec, Payload: AbstractExecPayload> BeaconBlockBodyRef<'a, E, leaves } - /// Calculate a KZG commitment merkle proof slowly. + /// Calculate a KZG commitment merkle proof. /// /// Prefer to use `complete_kzg_commitment_merkle_proof` with a reused proof for the /// `blob_kzg_commitments` field. From bf19769e0d7bf425c3b9850216ef10e7a611491e Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Wed, 23 Oct 2024 10:29:22 +1100 Subject: [PATCH 16/25] Address review comments and cleanup --- beacon_node/beacon_chain/src/beacon_chain.rs | 3 ++- .../beacon_chain/src/data_availability_checker/error.rs | 7 +------ beacon_node/beacon_chain/src/fetch_blobs.rs | 2 +- beacon_node/src/cli.rs | 2 ++ book/src/help_bn.md | 6 ------ 5 files changed, 6 insertions(+), 14 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 94b0675a320..9289356507e 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -3899,7 +3899,8 @@ impl BeaconChain { // end up with blocks in fork choice that are missing from disk. // See https://github.com/sigp/lighthouse/issues/2028 let (_, signed_block, blobs, data_columns) = signed_block.deconstruct(); - // TODO(das) we currently store all subnet sampled columns. Tracking issue to exclude non custody columns: https://github.com/sigp/lighthouse/issues/6465 + // TODO(das) we currently store all subnet sampled columns. Tracking issue to exclude non + // custody columns: https://github.com/sigp/lighthouse/issues/6465 let custody_columns_count = self.data_availability_checker.get_sampling_column_count(); // if block is made available via blobs, dropped the data columns. let data_columns = data_columns.filter(|columns| columns.len() == custody_columns_count); diff --git a/beacon_node/beacon_chain/src/data_availability_checker/error.rs b/beacon_node/beacon_chain/src/data_availability_checker/error.rs index 1c1e953949d..cfdb3cfe91e 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/error.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/error.rs @@ -1,5 +1,4 @@ use kzg::{Error as KzgError, KzgCommitment}; -use types::data_column_sidecar::DataColumnSidecarError; use types::{BeaconStateError, ColumnIndex, Hash256}; #[derive(Debug)] @@ -11,7 +10,6 @@ pub enum Error { blob_commitment: KzgCommitment, block_commitment: KzgCommitment, }, - UnableToDetermineImportRequirement, Unexpected, SszTypes(ssz_types::Error), MissingBlobs, @@ -24,7 +22,6 @@ pub enum Error { BlockReplayError(state_processing::BlockReplayError), RebuildingStateCaches(BeaconStateError), SlotClockError, - DataColumnSidecarError(DataColumnSidecarError), } #[derive(PartialEq, Eq)] @@ -46,10 +43,8 @@ impl Error { | Error::Unexpected | Error::ParentStateMissing(_) | Error::BlockReplayError(_) - | Error::UnableToDetermineImportRequirement | Error::RebuildingStateCaches(_) - | Error::SlotClockError - | Error::DataColumnSidecarError(_) => ErrorCategory::Internal, + | Error::SlotClockError => ErrorCategory::Internal, Error::InvalidBlobs { .. } | Error::InvalidColumn { .. } | Error::ReconstructColumnsError { .. } diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index a3b5063f488..046f301fb66 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -98,7 +98,7 @@ pub async fn fetch_and_process_engine_blobs( if num_fetched_blobs == 0 { debug!( - chain.log, + log, "No blobs fetched from the EL"; "num_expected_blobs" => num_expected_blobs, ); diff --git a/beacon_node/src/cli.rs b/beacon_node/src/cli.rs index 59f0e1dbcbe..4800d3e1b97 100644 --- a/beacon_node/src/cli.rs +++ b/beacon_node/src/cli.rs @@ -93,6 +93,7 @@ pub fn cli_app() -> Command { .help_heading(FLAG_HEADER) .help("Number of batches that supernodes split data columns into during publishing by a non-proposer. For PeerDAS only.") .display_order(0) + .hide(true) ) .arg( Arg::new("supernode-data-column-publication-batch-interval") @@ -101,6 +102,7 @@ pub fn cli_app() -> Command { .help_heading(FLAG_HEADER) .help("The delay in milliseconds applied by supernodes between the sending of each data column batch. For PeerDAS only.") .display_order(0) + .hide(true) ) .arg( Arg::new("subscribe-all-subnets") diff --git a/book/src/help_bn.md b/book/src/help_bn.md index 56ab0ce7854..69701a3ad93 100644 --- a/book/src/help_bn.md +++ b/book/src/help_bn.md @@ -557,12 +557,6 @@ Flags: Subscribe to all subnets regardless of validator count. This will also advertise the beacon node as being long-lived subscribed to all subnets. - --supernode-data-column-publication-batch-interval - The delay in milliseconds applied by supernodes between the sending of - each data column batch. For PeerDAS only. - --supernode-data-column-publication-batches - Number of batches that supernodes split data columns into during - publishing by a non-proposer. For PeerDAS only. --validator-monitor-auto Enables the automatic detection and monitoring of validators connected to the HTTP API and using the subnet subscription endpoint. This From 6e44763b26fa9fa3e5960a1d19ab738097abe24d Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Wed, 23 Oct 2024 10:58:43 +1100 Subject: [PATCH 17/25] Address review comments and cleanup --- .../src/data_availability_checker.rs | 4 ++++ beacon_node/beacon_chain/src/fetch_blobs.rs | 17 +++++++---------- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index df914628657..72806a74d27 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -146,6 +146,10 @@ impl DataAvailabilityChecker { self.availability_cache.sampling_column_count() } + pub(crate) fn is_supernode(&self) -> bool { + self.get_sampling_column_count() == self.spec.number_of_columns + } + /// Checks if the block root is currenlty in the availability cache awaiting import because /// of missing components. pub fn get_execution_valid_block( diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index 046f301fb66..4d15502bd4d 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -240,10 +240,7 @@ fn spawn_compute_and_publish_data_columns_task( // Check indices from cache before sending the columns, to make sure we don't // publish components already seen on gossip. - let is_supernode = chain_cloned - .data_availability_checker - .get_sampling_column_count() - == chain_cloned.spec.number_of_columns; + let is_supernode = chain_cloned.data_availability_checker.is_supernode(); // At the moment non supernodes are not required to publish any columns. // TODO(das): we could experiment with having full nodes publish their custodied @@ -302,13 +299,13 @@ fn build_blob_sidecars( kzg_commitments_proof: &FixedVector, ) -> Result, FetchEngineBlobError> { let mut fixed_blob_sidecar_list = FixedBlobSidecarList::default(); - for (i, blob_and_proof) in response + for (index, blob_and_proof) in response .into_iter() .enumerate() .filter_map(|(i, opt_blob)| Some((i, opt_blob?))) { match BlobSidecar::new_with_existing_proof( - i, + index, blob_and_proof.blob, block, signed_block_header.clone(), @@ -316,12 +313,12 @@ fn build_blob_sidecars( blob_and_proof.proof, ) { Ok(blob) => { - if let Some(blob_mut) = fixed_blob_sidecar_list.get_mut(i) { + if let Some(blob_mut) = fixed_blob_sidecar_list.get_mut(index) { *blob_mut = Some(Arc::new(blob)); } else { - return Err(FetchEngineBlobError::InternalError( - "Unreachable: Blobs from EL - out of bounds".to_string(), - )); + return Err(FetchEngineBlobError::InternalError(format!( + "Blobs from EL contains blob with invalid index {index}" + ))); } } Err(e) => { From 1872ae53b80d654f0764f17595f79ee1bc45d634 Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Mon, 28 Oct 2024 17:30:10 +1100 Subject: [PATCH 18/25] Refactor to de-duplicate gradual publication logic. --- beacon_node/beacon_chain/src/fetch_blobs.rs | 42 +-------------- .../src/network_beacon_processor/mod.rs | 54 ++++++++++++------- 2 files changed, 36 insertions(+), 60 deletions(-) diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index 4d15502bd4d..17a23d1553f 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -14,7 +14,6 @@ use execution_layer::json_structures::BlobAndProofV1; use execution_layer::Error as ExecutionLayerError; use itertools::Either; use lighthouse_metrics::{inc_counter, inc_counter_by, TryExt}; -use rand::prelude::SliceRandom; use slog::{debug, error, o, Logger}; use ssz_types::FixedVector; use state_processing::per_block_processing::deneb::kzg_commitment_to_versioned_hash; @@ -136,7 +135,6 @@ pub async fn fetch_and_process_engine_blobs( let data_columns_receiver = spawn_compute_and_publish_data_columns_task( &chain, - block_root, block.clone(), fixed_blob_sidecar_list.clone(), publish_fn, @@ -194,7 +192,6 @@ pub async fn fetch_and_process_engine_blobs( /// while maintaining the invariant that block and data columns are persisted atomically. fn spawn_compute_and_publish_data_columns_task( chain: &Arc>, - block_root: Hash256, block: Arc>>, blobs: FixedBlobSidecarList, publish_fn: impl Fn(BlobsOrDataColumns) + Send + 'static, @@ -222,7 +219,7 @@ fn spawn_compute_and_publish_data_columns_task( .discard_timer_on_break(&mut timer); drop(timer); - let mut all_data_columns = match data_columns_result { + let all_data_columns = match data_columns_result { Ok(d) => d, Err(e) => { error!( @@ -249,42 +246,7 @@ fn spawn_compute_and_publish_data_columns_task( return; } - // To reduce bandwidth for supernodes: permute the columns to publish and - // publish them in batches. Our hope is that some columns arrive from - // other supernodes in the meantime, obviating the need for us to publish - // them. If no other publisher exists for a column, it will eventually get - // published here. - // FIXME(das): deduplicate this wrt to gossip/sync methods - all_data_columns.shuffle(&mut rand::thread_rng()); - - let supernode_data_column_publication_batches = chain_cloned - .config - .supernode_data_column_publication_batches; - let supernode_data_column_publication_batch_interval = chain_cloned - .config - .supernode_data_column_publication_batch_interval; - - let batch_size = all_data_columns.len() / supernode_data_column_publication_batches; - for batch in all_data_columns.chunks(batch_size) { - let already_seen = chain_cloned - .data_availability_checker - .cached_data_column_indexes(&block_root) - .unwrap_or_default(); - let publishable = batch - .iter() - .filter(|col| !already_seen.contains(&col.index())) - .cloned() - .collect::>(); - if !publishable.is_empty() { - debug!( - log, - "Publishing data columns from EL"; - "count" => publishable.len() - ); - publish_fn(BlobsOrDataColumns::DataColumns(publishable)); - } - std::thread::sleep(supernode_data_column_publication_batch_interval); - } + publish_fn(BlobsOrDataColumns::DataColumns(all_data_columns)); }, "compute_and_publish_data_columns", ); diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 998222ebc36..dc65ad7b27d 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -880,28 +880,35 @@ impl NetworkBeaconProcessor { }); } - fn publish_blobs_or_data_column(&self, blobs_or_data_column: BlobsOrDataColumns) { - let messages = match blobs_or_data_column { + fn publish_blobs_or_data_column( + self: &Arc, + blobs_or_data_column: BlobsOrDataColumns, + block_root: Hash256, + ) { + match blobs_or_data_column { BlobsOrDataColumns::Blobs(blobs) => { - debug!(self.log, "Publishing blobs from EL"; "count" => blobs.len()); - blobs + debug!( + self.log, + "Publishing blobs from EL"; + "count" => blobs.len(), + "block_root" => ?block_root, + ); + let messages = blobs .into_iter() .map(|blob| PubsubMessage::BlobSidecar(Box::new((blob.index, blob)))) - .collect() + .collect(); + self.send_network_message(NetworkMessage::Publish { messages }); } BlobsOrDataColumns::DataColumns(columns) => { - debug!(self.log, "Publishing data columns built from EL blobs"; "count" => columns.len()); - columns.into_iter().map(|column| { - let subnet = DataColumnSubnetId::from_column_index::( - column.index as usize, - &self.chain.spec, - ); - PubsubMessage::DataColumnSidecar(Box::new((subnet, column))) - }) + debug!( + self.log, + "Publishing data columns built from EL blobs"; + "count" => columns.len(), + "block_root" => ?block_root, + ); + self.publish_data_columns_gradually(columns, block_root); } - .collect(), }; - self.send_network_message(NetworkMessage::Publish { messages }) } pub async fn fetch_engine_blobs_and_publish( @@ -911,7 +918,7 @@ impl NetworkBeaconProcessor { ) { let self_cloned = self.clone(); let publish_fn = move |blobs_or_data_column| { - self_cloned.publish_blobs_or_data_column(blobs_or_data_column) + self_cloned.publish_blobs_or_data_column(blobs_or_data_column, block_root) }; match fetch_and_process_engine_blobs( @@ -971,7 +978,7 @@ impl NetworkBeaconProcessor { let result = self.chain.reconstruct_data_columns(block_root).await; match result { Ok(Some((availability_processing_status, data_columns_to_publish))) => { - self.handle_data_columns_to_publish(data_columns_to_publish, block_root); + self.publish_data_columns_gradually(data_columns_to_publish, block_root); match &availability_processing_status { AvailabilityProcessingStatus::Imported(hash) => { debug!( @@ -1015,7 +1022,13 @@ impl NetworkBeaconProcessor { } } - fn handle_data_columns_to_publish( + /// 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 + /// by some nodes on the network as soon as possible. Our hope is that some columns arrive from + /// other supernodes in the meantime, obviating the need for us to publish them. If no other + /// publisher exists for a column, it will eventually get published here. + fn publish_data_columns_gradually( self: &Arc, mut data_columns_to_publish: DataColumnSidecarList, block_root: Hash256, @@ -1068,7 +1081,8 @@ impl NetworkBeaconProcessor { debug!( self_clone.chain.logger(), "Publishing data column batch"; - "count" => publishable.len() + "count" => publishable.len(), + "block_root" => ?block_root, ); publish_fn(publishable); } @@ -1076,7 +1090,7 @@ impl NetworkBeaconProcessor { tokio::time::sleep(supernode_data_column_publication_batch_interval).await; } }, - "handle_data_columns_publish", + "publish_data_columns_gradually", ); } } From 81493f5e67da03aabbffbbf3c0d11f754e27ec21 Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Mon, 28 Oct 2024 18:01:52 +1100 Subject: [PATCH 19/25] Add more logging. --- .../src/network_beacon_processor/mod.rs | 25 ++++++++++++------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index dc65ad7b27d..30d834a4e85 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -900,12 +900,6 @@ impl NetworkBeaconProcessor { self.send_network_message(NetworkMessage::Publish { messages }); } BlobsOrDataColumns::DataColumns(columns) => { - debug!( - self.log, - "Publishing data columns built from EL blobs"; - "count" => columns.len(), - "block_root" => ?block_root, - ); self.publish_data_columns_gradually(columns, block_root); } }; @@ -1038,6 +1032,7 @@ impl NetworkBeaconProcessor { self.executor.spawn( async move { let chain = self_clone.chain.clone(); + let log = self_clone.chain.logger(); let publish_fn = |columns: DataColumnSidecarList| { self_clone.send_network_message(NetworkMessage::Publish { messages: columns @@ -1065,6 +1060,7 @@ impl NetworkBeaconProcessor { chain.config.supernode_data_column_publication_batches; let batch_size = data_columns_to_publish.len() / supernode_data_column_publication_batches; + let mut publish_count = 0usize; for batch in data_columns_to_publish.chunks(batch_size) { let already_seen = chain @@ -1079,18 +1075,29 @@ impl NetworkBeaconProcessor { if !publishable.is_empty() { debug!( - self_clone.chain.logger(), + log, "Publishing data column batch"; - "count" => publishable.len(), + "publish_count" => publishable.len(), "block_root" => ?block_root, ); + publish_count += publishable.len(); publish_fn(publishable); } tokio::time::sleep(supernode_data_column_publication_batch_interval).await; } + + debug!( + log, + "Batch data column publishing complete"; + "batch_size" => batch_size, + "batch_interval" => supernode_data_column_publication_batch_interval.as_millis(), + "data_columns_to_publish_count" => data_columns_to_publish.len(), + "published_count" => publish_count, + "block_root" => ?block_root, + ) }, - "publish_data_columns_gradually", + "gradual_data_column_publication", ); } } From 756230e9589b6f15ef89495d7070c25adf3f97d3 Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Tue, 29 Oct 2024 15:45:10 +1100 Subject: [PATCH 20/25] Fix incorrect comparison on `num_fetched_blobs`. --- beacon_node/beacon_chain/src/fetch_blobs.rs | 25 +++++++++++---------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index 17a23d1553f..9118d630959 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -84,18 +84,8 @@ pub async fn fetch_and_process_engine_blobs( .get_blobs(versioned_hashes) .await .map_err(FetchEngineBlobError::RequestFailed)?; - let num_fetched_blobs = response.iter().filter(|b| b.is_some()).count(); - inc_counter_by( - &metrics::BLOBS_FROM_EL_EXPECTED_TOTAL, - num_expected_blobs as u64, - ); - inc_counter_by( - &metrics::BLOBS_FROM_EL_RECEIVED_TOTAL, - num_fetched_blobs as u64, - ); - - if num_fetched_blobs == 0 { + if response.is_empty() { debug!( log, "No blobs fetched from the EL"; @@ -118,11 +108,22 @@ pub async fn fetch_and_process_engine_blobs( &kzg_commitments_proof, )?; + let num_fetched_blobs = fixed_blob_sidecar_list.filter(|b| b.is_some()).count(); + + inc_counter_by( + &metrics::BLOBS_FROM_EL_EXPECTED_TOTAL, + num_expected_blobs as u64, + ); + inc_counter_by( + &metrics::BLOBS_FROM_EL_RECEIVED_TOTAL, + num_fetched_blobs as u64, + ); + let peer_das_enabled = chain.spec.is_peer_das_enabled_for_epoch(block.epoch()); let data_columns_receiver_opt = if peer_das_enabled { // Partial blobs response isn't useful for PeerDAS, so we don't bother building and publishing data columns. - if !num_fetched_blobs == num_expected_blobs { + if num_fetched_blobs != num_expected_blobs { debug!( log, "Not all blobs fetched from the EL"; From 0f32b73ebc1505b3495d3719e158ac4630d7020b Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Tue, 29 Oct 2024 16:36:33 +1100 Subject: [PATCH 21/25] Implement gradual blob publication. --- beacon_node/beacon_chain/src/chain_config.rs | 14 ++- beacon_node/beacon_chain/src/fetch_blobs.rs | 7 +- .../src/network_beacon_processor/mod.rs | 104 ++++++++++++++---- beacon_node/src/cli.rs | 12 +- beacon_node/src/config.rs | 15 +-- lighthouse/tests/beacon_node.rs | 21 ++-- 6 files changed, 114 insertions(+), 59 deletions(-) diff --git a/beacon_node/beacon_chain/src/chain_config.rs b/beacon_node/beacon_chain/src/chain_config.rs index 5cae99484b8..b8a607c8864 100644 --- a/beacon_node/beacon_chain/src/chain_config.rs +++ b/beacon_node/beacon_chain/src/chain_config.rs @@ -88,10 +88,12 @@ pub struct ChainConfig { pub malicious_withhold_count: usize, /// Enable peer sampling on blocks. pub enable_sampling: bool, - /// Number of batches that supernodes split data columns into during publishing by a non-proposer. - pub supernode_data_column_publication_batches: usize, - /// The delay applied by supernodes between the sending of each data column batch. - pub supernode_data_column_publication_batch_interval: Duration, + /// Number of batches that the node splits blobs or data columns into during publication. + /// This doesn't apply if the node is the block proposer. For PeerDAS only. + pub blob_publication_batches: usize, + /// The delay in milliseconds applied by the node between sending each blob or data column batch. + /// This doesn't apply if the node is the block proposer. + pub blob_publication_batch_interval: Duration, } impl Default for ChainConfig { @@ -125,8 +127,8 @@ impl Default for ChainConfig { enable_light_client_server: false, malicious_withhold_count: 0, enable_sampling: false, - supernode_data_column_publication_batches: 4, - supernode_data_column_publication_batch_interval: Duration::from_millis(200), + blob_publication_batches: 4, + blob_publication_batch_interval: Duration::from_millis(300), } } } diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index 9118d630959..836d2b85cc5 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -13,7 +13,7 @@ use crate::{metrics, AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes use execution_layer::json_structures::BlobAndProofV1; use execution_layer::Error as ExecutionLayerError; use itertools::Either; -use lighthouse_metrics::{inc_counter, inc_counter_by, TryExt}; +use metrics::{inc_counter, inc_counter_by, TryExt}; use slog::{debug, error, o, Logger}; use ssz_types::FixedVector; use state_processing::per_block_processing::deneb::kzg_commitment_to_versioned_hash; @@ -108,7 +108,10 @@ pub async fn fetch_and_process_engine_blobs( &kzg_commitments_proof, )?; - let num_fetched_blobs = fixed_blob_sidecar_list.filter(|b| b.is_some()).count(); + let num_fetched_blobs = fixed_blob_sidecar_list + .iter() + .filter(|b| b.is_some()) + .count(); inc_counter_by( &metrics::BLOBS_FROM_EL_EXPECTED_TOTAL, diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 30d834a4e85..6ccbca8ef7b 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -69,6 +69,9 @@ pub struct NetworkBeaconProcessor { pub log: Logger, } +// 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 @@ -887,17 +890,7 @@ impl NetworkBeaconProcessor { ) { match blobs_or_data_column { BlobsOrDataColumns::Blobs(blobs) => { - debug!( - self.log, - "Publishing blobs from EL"; - "count" => blobs.len(), - "block_root" => ?block_root, - ); - let messages = blobs - .into_iter() - .map(|blob| PubsubMessage::BlobSidecar(Box::new((blob.index, blob)))) - .collect(); - self.send_network_message(NetworkMessage::Publish { messages }); + self.publish_blobs_gradually(blobs, block_root); } BlobsOrDataColumns::DataColumns(columns) => { self.publish_data_columns_gradually(columns, block_root); @@ -1016,6 +1009,81 @@ 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 log = self_clone.chain.logger(); + 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::thread_rng()); + + let blob_publication_batch_interval = chain.config.blob_publication_batch_interval; + let mut publish_count = 0usize; + let mut blobs_iter = blobs.iter().peekable(); + let mut batch_size = 1usize; + + while blobs_iter.peek().is_some() { + let batch = blobs_iter.by_ref().take(batch_size); + let already_seen = chain + .data_availability_checker + .cached_blob_indexes(&block_root) + .unwrap_or_default(); + let publishable = batch + .filter(|col| !already_seen.contains(&col.index)) + .cloned() + .collect::>(); + + if !publishable.is_empty() { + debug!( + log, + "Publishing blob batch"; + "publish_count" => publishable.len(), + "block_root" => ?block_root, + ); + publish_count += publishable.len(); + publish_fn(publishable); + } + + tokio::time::sleep(blob_publication_batch_interval).await; + batch_size *= BLOB_PUBLICATION_EXP_FACTOR; + } + + debug!( + log, + "Batch blob publication complete"; + "batch_interval" => blob_publication_batch_interval.as_millis(), + "blob_count" => blobs.len(), + "published_count" => publish_count, + "block_root" => ?block_root, + ) + }, + "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 @@ -1053,13 +1121,9 @@ impl NetworkBeaconProcessor { // on gossip from other supernodes. data_columns_to_publish.shuffle(&mut rand::thread_rng()); - let supernode_data_column_publication_batch_interval = chain - .config - .supernode_data_column_publication_batch_interval; - let supernode_data_column_publication_batches = - chain.config.supernode_data_column_publication_batches; - let batch_size = - data_columns_to_publish.len() / supernode_data_column_publication_batches; + let blob_publication_batch_interval = chain.config.blob_publication_batch_interval; + let blob_publication_batches = chain.config.blob_publication_batches; + let batch_size = chain.spec.number_of_columns / blob_publication_batches; let mut publish_count = 0usize; for batch in data_columns_to_publish.chunks(batch_size) { @@ -1084,14 +1148,14 @@ impl NetworkBeaconProcessor { publish_fn(publishable); } - tokio::time::sleep(supernode_data_column_publication_batch_interval).await; + tokio::time::sleep(blob_publication_batch_interval).await; } debug!( log, "Batch data column publishing complete"; "batch_size" => batch_size, - "batch_interval" => supernode_data_column_publication_batch_interval.as_millis(), + "batch_interval" => blob_publication_batch_interval.as_millis(), "data_columns_to_publish_count" => data_columns_to_publish.len(), "published_count" => publish_count, "block_root" => ?block_root, diff --git a/beacon_node/src/cli.rs b/beacon_node/src/cli.rs index 4800d3e1b97..c5427e0ce02 100644 --- a/beacon_node/src/cli.rs +++ b/beacon_node/src/cli.rs @@ -87,20 +87,20 @@ pub fn cli_app() -> Command { .display_order(0) ) .arg( - Arg::new("supernode-data-column-publication-batches") - .long("supernode-data-column-publication-batches") + Arg::new("blob-publication-batches") + .long("blob-publication-batches") .action(ArgAction::Set) .help_heading(FLAG_HEADER) - .help("Number of batches that supernodes split data columns into during publishing by a non-proposer. For PeerDAS only.") + .help("Number of batches that the node splits blobs or data columns into during publication. This doesn't apply if the node is the block proposer. Used in PeerDAS only.") .display_order(0) .hide(true) ) .arg( - Arg::new("supernode-data-column-publication-batch-interval") - .long("supernode-data-column-publication-batch-interval") + Arg::new("blob-publication-batch-interval") + .long("blob-publication-batch-interval") .action(ArgAction::Set) .help_heading(FLAG_HEADER) - .help("The delay in milliseconds applied by supernodes between the sending of each data column batch. For PeerDAS only.") + .help("The delay in milliseconds applied by the node between sending each blob or data column batch. This doesn't apply if the node is the block proposer.") .display_order(0) .hide(true) ) diff --git a/beacon_node/src/config.rs b/beacon_node/src/config.rs index 625e3a8ab2f..0f17c368005 100644 --- a/beacon_node/src/config.rs +++ b/beacon_node/src/config.rs @@ -192,20 +192,13 @@ pub fn get_config( client_config.chain.enable_sampling = true; } - if let Some(batches) = - clap_utils::parse_optional(cli_args, "supernode-data-column-publication-batches")? - { - client_config - .chain - .supernode_data_column_publication_batches = batches; + if let Some(batches) = clap_utils::parse_optional(cli_args, "blob-publication-batches")? { + client_config.chain.blob_publication_batches = batches; } - if let Some(interval) = - clap_utils::parse_optional(cli_args, "supernode-data-column-publication-batch-interval")? + if let Some(interval) = clap_utils::parse_optional(cli_args, "blob-publication-batch-interval")? { - client_config - .chain - .supernode_data_column_publication_batch_interval = Duration::from_millis(interval); + client_config.chain.blob_publication_batch_interval = Duration::from_millis(interval); } /* diff --git a/lighthouse/tests/beacon_node.rs b/lighthouse/tests/beacon_node.rs index 1dc760857bb..83a4367252a 100644 --- a/lighthouse/tests/beacon_node.rs +++ b/lighthouse/tests/beacon_node.rs @@ -801,29 +801,22 @@ fn network_enable_sampling_flag() { .with_config(|config| assert!(config.chain.enable_sampling)); } #[test] -fn supernode_data_column_publication_batches() { +fn blob_publication_batches() { CommandLineTest::new() - .flag("supernode-data-column-publication-batches", Some("3")) + .flag("blob-publication-batches", Some("3")) .run_with_zero_port() - .with_config(|config| { - assert_eq!(config.chain.supernode_data_column_publication_batches, 3) - }); + .with_config(|config| assert_eq!(config.chain.blob_publication_batches, 3)); } #[test] -fn supernode_data_column_publication_batch_interval() { +fn blob_publication_batch_interval() { CommandLineTest::new() - .flag( - "supernode-data-column-publication-batch-interval", - Some("300"), - ) + .flag("blob-publication-batch-interval", Some("400")) .run_with_zero_port() .with_config(|config| { assert_eq!( - config - .chain - .supernode_data_column_publication_batch_interval, - Duration::from_millis(300) + config.chain.blob_publication_batch_interval, + Duration::from_millis(400) ) }); } From 17015658087a50aad8985cb5784841cf4a3f40d2 Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Tue, 29 Oct 2024 17:20:56 +1100 Subject: [PATCH 22/25] Inline `publish_fn`. --- .../src/network_beacon_processor/mod.rs | 24 +++++++------------ 1 file changed, 8 insertions(+), 16 deletions(-) diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 6ccbca8ef7b..35e8b33cec6 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -883,21 +883,6 @@ impl NetworkBeaconProcessor { }); } - fn publish_blobs_or_data_column( - self: &Arc, - blobs_or_data_column: BlobsOrDataColumns, - block_root: Hash256, - ) { - match blobs_or_data_column { - BlobsOrDataColumns::Blobs(blobs) => { - self.publish_blobs_gradually(blobs, block_root); - } - BlobsOrDataColumns::DataColumns(columns) => { - self.publish_data_columns_gradually(columns, block_root); - } - }; - } - pub async fn fetch_engine_blobs_and_publish( self: &Arc, block: Arc>>, @@ -905,7 +890,14 @@ impl NetworkBeaconProcessor { ) { let self_cloned = self.clone(); let publish_fn = move |blobs_or_data_column| { - self_cloned.publish_blobs_or_data_column(blobs_or_data_column, block_root) + match blobs_or_data_column { + BlobsOrDataColumns::Blobs(blobs) => { + self_cloned.publish_blobs_gradually(blobs, block_root); + } + BlobsOrDataColumns::DataColumns(columns) => { + self_cloned.publish_data_columns_gradually(columns, block_root); + } + }; }; match fetch_and_process_engine_blobs( From 1aa125e62b36ff8e745acb9a5c2e9e6f38a3c81f Mon Sep 17 00:00:00 2001 From: Michael Sproul Date: Fri, 1 Nov 2024 15:14:12 +1100 Subject: [PATCH 23/25] Gossip verify blobs before publishing --- .../beacon_chain/src/blob_verification.rs | 71 +++++++++++++------ beacon_node/beacon_chain/src/fetch_blobs.rs | 55 +++++++------- beacon_node/beacon_chain/src/lib.rs | 2 +- .../src/observed_data_sidecars.rs | 25 +++++++ .../src/network_beacon_processor/mod.rs | 29 +++++--- 5 files changed, 126 insertions(+), 56 deletions(-) diff --git a/beacon_node/beacon_chain/src/blob_verification.rs b/beacon_node/beacon_chain/src/blob_verification.rs index f7de26a6ca7..6c87deb8260 100644 --- a/beacon_node/beacon_chain/src/blob_verification.rs +++ b/beacon_node/beacon_chain/src/blob_verification.rs @@ -1,5 +1,6 @@ use derivative::Derivative; use slot_clock::SlotClock; +use std::marker::PhantomData; use std::sync::Arc; use crate::beacon_chain::{BeaconChain, BeaconChainTypes}; @@ -8,6 +9,7 @@ use crate::block_verification::{ BlockSlashInfo, }; use crate::kzg_utils::{validate_blob, validate_blobs}; +use crate::observed_data_sidecars::{DoNotObserve, ObservationStrategy, Observe}; use crate::{metrics, BeaconChainError}; use kzg::{Error as KzgError, Kzg, KzgCommitment}; use slog::debug; @@ -158,12 +160,13 @@ 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 { +pub struct GossipVerifiedBlob { block_root: Hash256, blob: KzgVerifiedBlob, + _phantom: PhantomData, } -impl GossipVerifiedBlob { +impl GossipVerifiedBlob { pub fn new( blob: Arc>, subnet_id: u64, @@ -172,7 +175,7 @@ impl GossipVerifiedBlob { 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| { + 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), @@ -189,6 +192,7 @@ impl GossipVerifiedBlob { blob, seen_timestamp: Duration::from_secs(0), }, + _phantom: PhantomData, } } pub fn id(&self) -> BlobIdentifier { @@ -377,11 +381,11 @@ where validate_blobs::(kzg, commitments.as_slice(), blobs, proofs.as_slice()) } -pub fn validate_blob_sidecar_for_gossip( +pub fn validate_blob_sidecar_for_gossip( blob_sidecar: Arc>, subnet: u64, chain: &BeaconChain, -) -> Result, GossipBlobError> { +) -> Result, GossipBlobError> { let blob_slot = blob_sidecar.slot(); let blob_index = blob_sidecar.index; let block_parent_root = blob_sidecar.block_parent_root(); @@ -581,16 +585,45 @@ pub fn validate_blob_sidecar_for_gossip( ) .map_err(|e| GossipBlobError::BeaconChainError(e.into()))?; + if O::observe() { + observe_gossip_blob(&kzg_verified_blob.blob, chain)?; + } + + Ok(GossipVerifiedBlob { + block_root, + blob: kzg_verified_blob, + _phantom: PhantomData, + }) +} + +impl GossipVerifiedBlob { + pub fn observe( + self, + chain: &BeaconChain, + ) -> Result, GossipBlobError> { + observe_gossip_blob(&self.blob.blob, chain)?; + Ok(GossipVerifiedBlob { + block_root: self.block_root, + blob: self.blob, + _phantom: PhantomData, + }) + } +} + +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. + // 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 + // 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 @@ -599,16 +632,12 @@ pub fn validate_blob_sidecar_for_gossip( .map_err(|e| GossipBlobError::BeaconChainError(e.into()))? { return Err(GossipBlobError::RepeatBlob { - proposer: proposer_index as u64, - slot: blob_slot, - index: blob_index, + proposer: blob_sidecar.block_proposer_index(), + slot: blob_sidecar.slot(), + index: blob_sidecar.index, }); } - - Ok(GossipVerifiedBlob { - block_root, - blob: kzg_verified_blob, - }) + Ok(()) } /// Returns the canonical root of the given `blob`. diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index 836d2b85cc5..e075e000fc8 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -7,12 +7,12 @@ //! on P2P gossip to the network. From PeerDAS onwards, together with the increase in blob count, //! broadcasting blobs requires a much higher bandwidth, and is only done by high capacity //! supernodes. +use crate::blob_verification::{GossipBlobError, GossipVerifiedBlob}; use crate::kzg_utils::blobs_to_data_column_sidecars; -use crate::observed_data_sidecars::ObservableDataSidecar; +use crate::observed_data_sidecars::DoNotObserve; use crate::{metrics, AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes, BlockError}; use execution_layer::json_structures::BlobAndProofV1; use execution_layer::Error as ExecutionLayerError; -use itertools::Either; use metrics::{inc_counter, inc_counter_by, TryExt}; use slog::{debug, error, o, Logger}; use ssz_types::FixedVector; @@ -25,9 +25,9 @@ use types::{ Hash256, SignedBeaconBlock, SignedBeaconBlockHeader, }; -pub enum BlobsOrDataColumns { - Blobs(Vec>>), - DataColumns(DataColumnSidecarList), +pub enum BlobsOrDataColumns { + Blobs(Vec>), + DataColumns(DataColumnSidecarList), } #[derive(Debug)] @@ -37,6 +37,7 @@ pub enum FetchEngineBlobError { BlobSidecarError(BlobSidecarError), ExecutionLayerMissing, InternalError(String), + GossipBlob(GossipBlobError), RequestFailed(ExecutionLayerError), RuntimeShutdown, } @@ -47,7 +48,7 @@ pub async fn fetch_and_process_engine_blobs( chain: Arc>, block_root: Hash256, block: Arc>>, - publish_fn: impl Fn(BlobsOrDataColumns) + Send + 'static, + publish_fn: impl Fn(BlobsOrDataColumns) + Send + 'static, ) -> Result, FetchEngineBlobError> { let block_root_str = format!("{:?}", block_root); let log = chain @@ -147,22 +148,28 @@ pub async fn fetch_and_process_engine_blobs( Some(data_columns_receiver) } else { - let all_blobs = fixed_blob_sidecar_list.clone(); - let all_blobs_iter = all_blobs.into_iter().flat_map(|b| b.clone()); - - let blobs_to_publish = match chain - .data_availability_checker - .cached_blob_indexes(&block_root) - { - None => Either::Left(all_blobs_iter), - Some(imported_blob_indices) => Either::Right( - all_blobs_iter.filter(move |b| !imported_blob_indices.contains(&b.index())), - ), - }; + // Gossip verify blobs before publishing. This prevents blobs with invalid KZG proofs from + // the EL making it into the data availability checker. We do not immediately add these + // blobs to the observed blobs cache because we want to allow blobs to arrive on gossip + // and be accepted (and propagated) while we are waiting to publish. Just before publishing + // we will observe the blobs and only proceed with publishing if they are not yet seen. + let blobs_to_publish = fixed_blob_sidecar_list + .iter() + .filter_map(|opt_blob| { + let blob = opt_blob.as_ref()?; + match GossipVerifiedBlob::::new(blob.clone(), blob.index, &chain) { + Ok(verified) => Some(Ok(verified)), + // Ignore already seen blobs. + Err(GossipBlobError::RepeatBlob { .. }) => None, + Err(e) => Some(Err(e)), + } + }) + .collect::, _>>() + .map_err(FetchEngineBlobError::GossipBlob)?; - publish_fn(BlobsOrDataColumns::Blobs( - blobs_to_publish.collect::>(), - )); + if !blobs_to_publish.is_empty() { + publish_fn(BlobsOrDataColumns::Blobs(blobs_to_publish)); + } None }; @@ -198,7 +205,7 @@ fn spawn_compute_and_publish_data_columns_task( chain: &Arc>, block: Arc>>, blobs: FixedBlobSidecarList, - publish_fn: impl Fn(BlobsOrDataColumns) + Send + 'static, + publish_fn: impl Fn(BlobsOrDataColumns) + Send + 'static, log: Logger, ) -> Receiver>>> { let chain_cloned = chain.clone(); @@ -262,7 +269,7 @@ fn build_blob_sidecars( block: &Arc>>, response: Vec>>, signed_block_header: SignedBeaconBlockHeader, - kzg_commitments_proof: &FixedVector, + kzg_commitments_inclusion_proof: &FixedVector, ) -> Result, FetchEngineBlobError> { let mut fixed_blob_sidecar_list = FixedBlobSidecarList::default(); for (index, blob_and_proof) in response @@ -275,7 +282,7 @@ fn build_blob_sidecars( blob_and_proof.blob, block, signed_block_header.clone(), - kzg_commitments_proof, + kzg_commitments_inclusion_proof, blob_and_proof.proof, ) { Ok(blob) => { diff --git a/beacon_node/beacon_chain/src/lib.rs b/beacon_node/beacon_chain/src/lib.rs index 974fdf09e57..2953516fb1a 100644 --- a/beacon_node/beacon_chain/src/lib.rs +++ b/beacon_node/beacon_chain/src/lib.rs @@ -44,7 +44,7 @@ mod naive_aggregation_pool; pub mod observed_aggregates; mod observed_attesters; pub mod observed_block_producers; -mod observed_data_sidecars; +pub mod observed_data_sidecars; pub mod observed_operations; mod observed_slashable; pub mod otb_verification_service; diff --git a/beacon_node/beacon_chain/src/observed_data_sidecars.rs b/beacon_node/beacon_chain/src/observed_data_sidecars.rs index 9b59a8f85b1..53f8c71f54e 100644 --- a/beacon_node/beacon_chain/src/observed_data_sidecars.rs +++ b/beacon_node/beacon_chain/src/observed_data_sidecars.rs @@ -148,6 +148,31 @@ impl ObservedDataSidecars { } } +/// Abstraction to control "observation" of gossip messages (currently just blobs and data columns). +/// +/// If a type returns `false` for `observe` then the message will not be immediately added to its +/// respective gossip observation cache. Unobserved messages should usually be observed later. +pub trait ObservationStrategy { + fn observe() -> bool; +} + +/// Type for messages that are observed immediately. +pub struct Observe; +/// Type for messages that have not been observed. +pub struct DoNotObserve; + +impl ObservationStrategy for Observe { + fn observe() -> bool { + true + } +} + +impl ObservationStrategy for DoNotObserve { + fn observe() -> bool { + false + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 35e8b33cec6..02dba625f0c 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -1,8 +1,10 @@ use crate::sync::manager::BlockProcessType; use crate::sync::SamplingId; use crate::{service::NetworkMessage, sync::manager::SyncMessage}; +use beacon_chain::blob_verification::{GossipBlobError, GossipVerifiedBlob}; use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::fetch_blobs::{fetch_and_process_engine_blobs, BlobsOrDataColumns}; +use beacon_chain::observed_data_sidecars::DoNotObserve; use beacon_chain::{ builder::Witness, eth1_chain::CachingEth1Backend, AvailabilityProcessingStatus, BeaconChain, }; @@ -23,7 +25,7 @@ use lighthouse_network::{ Client, MessageId, NetworkGlobals, PeerId, PubsubMessage, }; use rand::prelude::SliceRandom; -use slog::{debug, error, trace, Logger}; +use slog::{debug, error, trace, warn, Logger}; use slot_clock::ManualSlotClock; use std::path::PathBuf; use std::sync::Arc; @@ -1009,7 +1011,7 @@ impl NetworkBeaconProcessor { /// publisher exists for a blob, it will eventually get published here. fn publish_blobs_gradually( self: &Arc, - mut blobs: Vec>>, + mut blobs: Vec>, block_root: Hash256, ) { let self_clone = self.clone(); @@ -1034,18 +1036,25 @@ impl NetworkBeaconProcessor { let blob_publication_batch_interval = chain.config.blob_publication_batch_interval; let mut publish_count = 0usize; - let mut blobs_iter = blobs.iter().peekable(); + 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 already_seen = chain - .data_availability_checker - .cached_blob_indexes(&block_root) - .unwrap_or_default(); let publishable = batch - .filter(|col| !already_seen.contains(&col.index)) - .cloned() + .filter_map(|unobserved| match unobserved.observe(&chain) { + Ok(observed) => Some(observed.clone_blob()), + Err(GossipBlobError::RepeatBlob { .. }) => None, + Err(e) => { + warn!( + log, + "Previously verified blob is invalid"; + "error" => ?e + ); + None + } + }) .collect::>(); if !publishable.is_empty() { @@ -1067,7 +1076,7 @@ impl NetworkBeaconProcessor { log, "Batch blob publication complete"; "batch_interval" => blob_publication_batch_interval.as_millis(), - "blob_count" => blobs.len(), + "blob_count" => blob_count, "published_count" => publish_count, "block_root" => ?block_root, ) From 7fa2f44eb34ac1cd9e36470814d260cf6afacc90 Mon Sep 17 00:00:00 2001 From: Michael Sproul Date: Fri, 1 Nov 2024 15:28:00 +1100 Subject: [PATCH 24/25] Avoid queries for 0 blobs and error for duplicates --- beacon_node/beacon_chain/src/fetch_blobs.rs | 31 +++++++++++-------- .../src/network_beacon_processor/mod.rs | 23 ++++++++++---- 2 files changed, 35 insertions(+), 19 deletions(-) diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index e075e000fc8..3fccd141347 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -55,19 +55,24 @@ pub async fn fetch_and_process_engine_blobs( .log .new(o!("service" => "fetch_engine_blobs", "block_root" => block_root_str)); - let versioned_hashes = - if let Ok(kzg_commitments) = block.message().body().blob_kzg_commitments() { - kzg_commitments - .iter() - .map(kzg_commitment_to_versioned_hash) - .collect::>() - } else { - debug!( - log, - "Fetch blobs not triggered - none required"; - ); - return Ok(None); - }; + let versioned_hashes = if let Some(kzg_commitments) = block + .message() + .body() + .blob_kzg_commitments() + .ok() + .filter(|blobs| !blobs.is_empty()) + { + kzg_commitments + .iter() + .map(kzg_commitment_to_versioned_hash) + .collect::>() + } else { + debug!( + log, + "Fetch blobs not triggered - none required"; + ); + return Ok(None); + }; let num_expected_blobs = versioned_hashes.len(); diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 02dba625f0c..99a60c66b05 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -3,12 +3,14 @@ use crate::sync::SamplingId; use crate::{service::NetworkMessage, sync::manager::SyncMessage}; use beacon_chain::blob_verification::{GossipBlobError, GossipVerifiedBlob}; use beacon_chain::block_verification_types::RpcBlock; -use beacon_chain::fetch_blobs::{fetch_and_process_engine_blobs, BlobsOrDataColumns}; +use beacon_chain::fetch_blobs::{ + fetch_and_process_engine_blobs, BlobsOrDataColumns, FetchEngineBlobError, +}; use beacon_chain::observed_data_sidecars::DoNotObserve; use beacon_chain::{ builder::Witness, eth1_chain::CachingEth1Backend, AvailabilityProcessingStatus, BeaconChain, + BeaconChainTypes, BlockError, NotifyExecutionLayer, }; -use beacon_chain::{BeaconChainTypes, NotifyExecutionLayer}; use beacon_processor::{ work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessorChannels, BeaconProcessorSend, DuplicateCache, GossipAggregatePackage, GossipAttestationPackage, Work, @@ -916,7 +918,7 @@ impl NetworkBeaconProcessor { self.log, "Block components retrieved from EL"; "result" => "imported block and custody columns", - "block_hash" => %block_root, + "block_root" => %block_root, ); self.chain.recompute_head_at_current_slot().await; } @@ -924,7 +926,7 @@ impl NetworkBeaconProcessor { debug!( self.log, "Still missing blobs after engine blobs processed successfully"; - "block_hash" => %block_root, + "block_root" => %block_root, ); } }, @@ -932,7 +934,16 @@ impl NetworkBeaconProcessor { debug!( self.log, "Fetch blobs completed without import"; - "block_hash" => %block_root, + "block_root" => %block_root, + ); + } + Err(FetchEngineBlobError::BlobProcessingError(BlockError::DuplicateFullyImported( + .., + ))) => { + debug!( + self.log, + "Fetch blobs duplicate import"; + "block_root" => %block_root, ); } Err(e) => { @@ -940,7 +951,7 @@ impl NetworkBeaconProcessor { self.log, "Error fetching or processing blobs from EL"; "error" => ?e, - "block_hash" => %block_root, + "block_root" => %block_root, ); } } From ca7a78f0041355c7fa123f0e649acbf73cd7aaf3 Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Fri, 1 Nov 2024 16:16:29 +1100 Subject: [PATCH 25/25] Gossip verified engine blob before processing them, and use observe cache to detect duplicates before publishing. --- .../src/data_column_verification.rs | 68 +++++++++++++++---- beacon_node/beacon_chain/src/fetch_blobs.rs | 42 ++++++------ .../src/network_beacon_processor/mod.rs | 19 ++++-- 3 files changed, 90 insertions(+), 39 deletions(-) diff --git a/beacon_node/beacon_chain/src/data_column_verification.rs b/beacon_node/beacon_chain/src/data_column_verification.rs index 62536e67c68..6cfd26786aa 100644 --- a/beacon_node/beacon_chain/src/data_column_verification.rs +++ b/beacon_node/beacon_chain/src/data_column_verification.rs @@ -3,6 +3,7 @@ use crate::block_verification::{ BlockSlashInfo, }; use crate::kzg_utils::{reconstruct_data_columns, validate_data_columns}; +use crate::observed_data_sidecars::{ObservationStrategy, Observe}; use crate::{metrics, BeaconChain, BeaconChainError, BeaconChainTypes}; use derivative::Derivative; use fork_choice::ProtoBlock; @@ -13,6 +14,7 @@ use slog::debug; use slot_clock::SlotClock; use ssz_derive::{Decode, Encode}; use std::iter; +use std::marker::PhantomData; use std::sync::Arc; use types::data_column_sidecar::{ColumnIndex, DataColumnIdentifier}; use types::{ @@ -163,12 +165,13 @@ impl From for GossipDataColumnError { /// A wrapper around a `DataColumnSidecar` that indicates it has been approved for re-gossiping on /// the p2p network. #[derive(Debug)] -pub struct GossipVerifiedDataColumn { +pub struct GossipVerifiedDataColumn { block_root: Hash256, data_column: KzgVerifiedDataColumn, + _phantom: PhantomData, } -impl GossipVerifiedDataColumn { +impl GossipVerifiedDataColumn { pub fn new( column_sidecar: Arc>, subnet_id: u64, @@ -177,12 +180,14 @@ impl GossipVerifiedDataColumn { 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>( - chain, - BlockSlashInfo::from_early_error_data_column(header, e), - ) - }) + validate_data_column_sidecar_for_gossip::(column_sidecar, subnet_id, chain).map_err( + |e| { + process_block_slash_info::<_, GossipDataColumnError>( + chain, + BlockSlashInfo::from_early_error_data_column(header, e), + ) + }, + ) } pub fn id(&self) -> DataColumnIdentifier { @@ -373,11 +378,11 @@ where Ok(()) } -pub fn validate_data_column_sidecar_for_gossip( +pub fn validate_data_column_sidecar_for_gossip( data_column: Arc>, subnet: u64, chain: &BeaconChain, -) -> Result, GossipDataColumnError> { +) -> Result, GossipDataColumnError> { let column_slot = data_column.slot(); verify_data_column_sidecar(&data_column, &chain.spec)?; verify_index_matches_subnet(&data_column, subnet, &chain.spec)?; @@ -402,9 +407,14 @@ pub fn validate_data_column_sidecar_for_gossip( ) .map_err(|e| GossipDataColumnError::BeaconChainError(e.into()))?; + if O::observe() { + observe_gossip_data_column(&kzg_verified_data_column.data, chain)?; + } + Ok(GossipVerifiedDataColumn { block_root: data_column.block_root(), data_column: kzg_verified_data_column, + _phantom: PhantomData, }) } @@ -646,11 +656,42 @@ fn verify_sidecar_not_from_future_slot( Ok(()) } +pub fn observe_gossip_data_column( + data_column_sidecar: &DataColumnSidecar, + chain: &BeaconChain, +) -> Result<(), GossipDataColumnError> { + // Now the signature is valid, store the proposal so we don't accept another data column sidecar + // with the same `DataColumnIdentifier`. 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 DataColumnSidecar goes on to fail full verification, we do not evict it from the + // seen_cache as alternate data_column_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_column_sidecars + .write() + .observe_sidecar(data_column_sidecar) + .map_err(|e| GossipDataColumnError::BeaconChainError(e.into()))? + { + return Err(GossipDataColumnError::PriorKnown { + proposer: data_column_sidecar.block_proposer_index(), + slot: data_column_sidecar.slot(), + index: data_column_sidecar.index, + }); + } + Ok(()) +} + #[cfg(test)] mod test { use crate::data_column_verification::{ validate_data_column_sidecar_for_gossip, GossipDataColumnError, }; + use crate::observed_data_sidecars::Observe; use crate::test_utils::BeaconChainHarness; use types::{DataColumnSidecar, EthSpec, ForkName, MainnetEthSpec}; @@ -689,8 +730,11 @@ mod test { .unwrap(), }; - let result = - validate_data_column_sidecar_for_gossip(column_sidecar.into(), index, &harness.chain); + let result = validate_data_column_sidecar_for_gossip::<_, Observe>( + column_sidecar.into(), + index, + &harness.chain, + ); assert!(matches!( result.err(), Some(GossipDataColumnError::UnexpectedDataColumn) diff --git a/beacon_node/beacon_chain/src/fetch_blobs.rs b/beacon_node/beacon_chain/src/fetch_blobs.rs index e075e000fc8..6b70edba848 100644 --- a/beacon_node/beacon_chain/src/fetch_blobs.rs +++ b/beacon_node/beacon_chain/src/fetch_blobs.rs @@ -123,6 +123,25 @@ pub async fn fetch_and_process_engine_blobs( num_fetched_blobs as u64, ); + // Gossip verify blobs before publishing. This prevents blobs with invalid KZG proofs from + // the EL making it into the data availability checker. We do not immediately add these + // blobs to the observed blobs/columns cache because we want to allow blobs/columns to arrive on gossip + // and be accepted (and propagated) while we are waiting to publish. Just before publishing + // we will observe the blobs/columns and only proceed with publishing if they are not yet seen. + let blobs_to_import_and_publish = fixed_blob_sidecar_list + .iter() + .filter_map(|opt_blob| { + let blob = opt_blob.as_ref()?; + match GossipVerifiedBlob::::new(blob.clone(), blob.index, &chain) { + Ok(verified) => Some(Ok(verified)), + // Ignore already seen blobs. + Err(GossipBlobError::RepeatBlob { .. }) => None, + Err(e) => Some(Err(e)), + } + }) + .collect::, _>>() + .map_err(FetchEngineBlobError::GossipBlob)?; + let peer_das_enabled = chain.spec.is_peer_das_enabled_for_epoch(block.epoch()); let data_columns_receiver_opt = if peer_das_enabled { @@ -148,27 +167,8 @@ pub async fn fetch_and_process_engine_blobs( Some(data_columns_receiver) } else { - // Gossip verify blobs before publishing. This prevents blobs with invalid KZG proofs from - // the EL making it into the data availability checker. We do not immediately add these - // blobs to the observed blobs cache because we want to allow blobs to arrive on gossip - // and be accepted (and propagated) while we are waiting to publish. Just before publishing - // we will observe the blobs and only proceed with publishing if they are not yet seen. - let blobs_to_publish = fixed_blob_sidecar_list - .iter() - .filter_map(|opt_blob| { - let blob = opt_blob.as_ref()?; - match GossipVerifiedBlob::::new(blob.clone(), blob.index, &chain) { - Ok(verified) => Some(Ok(verified)), - // Ignore already seen blobs. - Err(GossipBlobError::RepeatBlob { .. }) => None, - Err(e) => Some(Err(e)), - } - }) - .collect::, _>>() - .map_err(FetchEngineBlobError::GossipBlob)?; - - if !blobs_to_publish.is_empty() { - publish_fn(BlobsOrDataColumns::Blobs(blobs_to_publish)); + if !blobs_to_import_and_publish.is_empty() { + publish_fn(BlobsOrDataColumns::Blobs(blobs_to_import_and_publish)); } None diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 02dba625f0c..2307941b97a 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -3,6 +3,7 @@ use crate::sync::SamplingId; use crate::{service::NetworkMessage, sync::manager::SyncMessage}; use beacon_chain::blob_verification::{GossipBlobError, GossipVerifiedBlob}; use beacon_chain::block_verification_types::RpcBlock; +use beacon_chain::data_column_verification::{observe_gossip_data_column, GossipDataColumnError}; use beacon_chain::fetch_blobs::{fetch_and_process_engine_blobs, BlobsOrDataColumns}; use beacon_chain::observed_data_sidecars::DoNotObserve; use beacon_chain::{ @@ -1128,14 +1129,20 @@ impl NetworkBeaconProcessor { let mut publish_count = 0usize; for batch in data_columns_to_publish.chunks(batch_size) { - let already_seen = chain - .data_availability_checker - .cached_data_column_indexes(&block_root) - .unwrap_or_default(); let publishable = batch .iter() - .filter(|col| !already_seen.contains(&col.index)) - .cloned() + .filter_map(|col| match observe_gossip_data_column(col, &chain) { + Ok(()) => Some(col.clone()), + Err(GossipDataColumnError::PriorKnown { .. }) => None, + Err(e) => { + warn!( + log, + "Previously verified data column is invalid"; + "error" => ?e + ); + None + } + }) .collect::>(); if !publishable.is_empty() {