From b2246c685c62d25585e506cbcd2b0b1f6fd4070d Mon Sep 17 00:00:00 2001 From: realbigsean Date: Fri, 14 Jul 2023 16:23:36 -0400 Subject: [PATCH 01/18] Revert "fix merge" This reverts commit 405e95b0ce15409f06504f45c8d93071523e9539. --- .../gossip_methods.rs | 2 +- .../src/network_beacon_processor/mod.rs | 26 ++++++++++++++----- .../network_beacon_processor/sync_methods.rs | 22 ++++++++-------- .../src/network_beacon_processor/tests.rs | 18 ++++++------- beacon_node/network/src/router.rs | 16 ++++++++---- beacon_node/network/src/service/tests.rs | 2 +- .../src/sync/block_lookups/delayed_lookup.rs | 6 ++--- .../network/src/sync/block_lookups/mod.rs | 14 ++++------ beacon_node/network/src/sync/manager.rs | 2 +- .../network/src/sync/range_sync/range.rs | 2 +- 10 files changed, 61 insertions(+), 49 deletions(-) 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 000c4d85dc1..2255b401703 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -5,8 +5,8 @@ use crate::{ sync::SyncMessage, }; -use beacon_chain::blob_verification::AsBlock; use beacon_chain::blob_verification::{BlobError, GossipVerifiedBlob}; +use beacon_chain::block_verification_types::AsBlock; use beacon_chain::store::Error; use beacon_chain::{ attestation_verification::{self, Error as AttnError, VerifiedAttestation}, diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index ef4b3daae7a..166417ba93c 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -2,7 +2,7 @@ use crate::{ service::NetworkMessage, sync::{manager::BlockProcessType, SyncMessage}, }; -use beacon_chain::blob_verification::BlockWrapper; +use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::{ builder::Witness, eth1_chain::CachingEth1Backend, test_utils::BeaconChainHarness, BeaconChain, }; @@ -13,7 +13,6 @@ use beacon_processor::{ MAX_SCHEDULED_WORK_QUEUE_LEN, MAX_WORK_EVENT_QUEUE_LEN, }; use environment::null_logger; -use lighthouse_network::rpc::methods::{BlobsByRangeRequest, BlobsByRootRequest}; use lighthouse_network::{ rpc::{BlocksByRangeRequest, BlocksByRootRequest, LightClientBootstrapRequest, StatusMessage}, Client, MessageId, NetworkGlobals, PeerId, PeerRequestId, @@ -27,6 +26,7 @@ use store::MemoryStore; use task_executor::test_utils::TestRuntime; use task_executor::TaskExecutor; use tokio::sync::mpsc::{self, error::TrySendError}; +use lighthouse_network::rpc::methods::{BlobsByRangeRequest, BlobsByRootRequest}; use types::*; pub use sync_methods::ChainSegmentProcessId; @@ -229,7 +229,9 @@ impl NetworkBeaconProcessor { }) } - pub fn send_banana() {} + pub fn send_banana(){ + + } /// Create a new `Work` event for some sync committee signature. pub fn send_gossip_sync_signature( @@ -411,7 +413,7 @@ impl NetworkBeaconProcessor { pub fn send_rpc_beacon_block( self: &Arc, block_root: Hash256, - block: BlockWrapper, + block: RpcBlock, seen_timestamp: Duration, process_type: BlockProcessType, ) -> Result<(), Error> { @@ -452,7 +454,7 @@ impl NetworkBeaconProcessor { pub fn send_chain_segment( self: &Arc, process_id: ChainSegmentProcessId, - blocks: Vec>, + blocks: Vec>, ) -> Result<(), Error> { let is_backfill = matches!(&process_id, ChainSegmentProcessId::BackSyncBatchId { .. }); let processor = self.clone(); @@ -561,7 +563,12 @@ impl NetworkBeaconProcessor { ) -> Result<(), Error> { let processor = self.clone(); let process_fn = move |send_idle_on_drop| { - processor.handle_blobs_by_range_request(send_idle_on_drop, peer_id, request_id, request) + processor.handle_blobs_by_range_request( + send_idle_on_drop, + peer_id, + request_id, + request, + ) }; self.try_send(BeaconWorkEvent { @@ -579,7 +586,12 @@ impl NetworkBeaconProcessor { ) -> Result<(), Error> { let processor = self.clone(); let process_fn = move |send_idle_on_drop| { - processor.handle_blobs_by_root_request(send_idle_on_drop, peer_id, request_id, request) + processor.handle_blobs_by_root_request( + send_idle_on_drop, + peer_id, + request_id, + request, + ) }; self.try_send(BeaconWorkEvent { 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 8d9146e6888..3a09373c56f 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -6,7 +6,8 @@ use crate::sync::{ manager::{BlockProcessType, SyncMessage}, ChainId, }; -use beacon_chain::blob_verification::{AsBlock, BlockWrapper, MaybeAvailableBlock}; +use beacon_chain::data_availability_checker::MaybeAvailableBlock; +use beacon_chain::block_verification_types::{AsBlock, RpcBlock}; use beacon_chain::data_availability_checker::AvailabilityCheckError; use beacon_chain::{ observed_block_producers::Error as ObserveError, validator_monitor::get_block_delay_ms, @@ -54,7 +55,7 @@ impl NetworkBeaconProcessor { pub fn generate_rpc_beacon_block_process_fn( self: Arc, block_root: Hash256, - block: BlockWrapper, + block: RpcBlock, seen_timestamp: Duration, process_type: BlockProcessType, ) -> AsyncFn { @@ -78,7 +79,7 @@ impl NetworkBeaconProcessor { pub fn generate_rpc_beacon_block_fns( self: Arc, block_root: Hash256, - block: BlockWrapper, + block: RpcBlock, seen_timestamp: Duration, process_type: BlockProcessType, ) -> (AsyncFn, BlockingFn) { @@ -106,7 +107,7 @@ impl NetworkBeaconProcessor { pub async fn process_rpc_block( self: Arc>, block_root: Hash256, - block: BlockWrapper, + block: RpcBlock, seen_timestamp: Duration, process_type: BlockProcessType, reprocess_tx: mpsc::Sender, @@ -269,8 +270,7 @@ impl NetworkBeaconProcessor { process_type: BlockProcessType, ) -> AsyncFn { let process_fn = async move { - self.clone() - .process_rpc_blobs(block_root, block, seen_timestamp, process_type) + self.clone().process_rpc_blobs(block_root, block, seen_timestamp, process_type) .await; }; Box::pin(process_fn) @@ -306,7 +306,7 @@ impl NetworkBeaconProcessor { }); } - pub fn send_delayed_lookup(&self, block_root: Hash256) { + pub fn send_delayed_lookup(&self, block_root: Hash256){ self.send_sync_message(SyncMessage::MissingGossipBlockComponentsDelayed(block_root)) } @@ -315,7 +315,7 @@ impl NetworkBeaconProcessor { pub async fn process_chain_segment( &self, sync_type: ChainSegmentProcessId, - downloaded_blocks: Vec>, + downloaded_blocks: Vec>, notify_execution_layer: NotifyExecutionLayer, ) { let result = match sync_type { @@ -440,7 +440,7 @@ impl NetworkBeaconProcessor { /// Helper function to process blocks batches which only consumes the chain and blocks to process. async fn process_blocks<'a>( &self, - downloaded_blocks: impl Iterator>, + downloaded_blocks: impl Iterator>, notify_execution_layer: NotifyExecutionLayer, ) -> (usize, Result<(), ChainSegmentFailed>) { let blocks: Vec<_> = downloaded_blocks.cloned().collect(); @@ -473,7 +473,7 @@ impl NetworkBeaconProcessor { /// Helper function to process backfill block batches which only consumes the chain and blocks to process. fn process_backfill_blocks( &self, - downloaded_blocks: Vec>, + downloaded_blocks: Vec>, ) -> (usize, Result<(), ChainSegmentFailed>) { let total_blocks = downloaded_blocks.len(); let available_blocks = match downloaded_blocks @@ -481,7 +481,7 @@ impl NetworkBeaconProcessor { .map(|block| { self.chain .data_availability_checker - .check_availability(block) + .check_rpc_block_availability(block) }) .collect::, _>>() { diff --git a/beacon_node/network/src/network_beacon_processor/tests.rs b/beacon_node/network/src/network_beacon_processor/tests.rs index 2c37d177aab..39d3575d6e3 100644 --- a/beacon_node/network/src/network_beacon_processor/tests.rs +++ b/beacon_node/network/src/network_beacon_processor/tests.rs @@ -336,16 +336,14 @@ impl TestRig { } pub fn enqueue_blobs_by_range_request(&self, count: u64) { - self.network_beacon_processor - .send_blobs_by_range_request( - PeerId::random(), - (ConnectionId::new(42), SubstreamId::new(24)), - BlobsByRangeRequest { - start_slot: 0, - count, - }, - ) - .unwrap(); + self.network_beacon_processor.send_blobs_by_range_request( + PeerId::random(), + (ConnectionId::new(42), SubstreamId::new(24)), + BlobsByRangeRequest { + start_slot: 0, + count, + }, + ).unwrap(); } pub fn enqueue_backfill_batch(&self) { diff --git a/beacon_node/network/src/router.rs b/beacon_node/network/src/router.rs index 30a75a91052..5a954d05a40 100644 --- a/beacon_node/network/src/router.rs +++ b/beacon_node/network/src/router.rs @@ -209,12 +209,18 @@ impl Router { .send_blocks_by_roots_request(peer_id, request_id, request), ), Request::BlobsByRange(request) => self.handle_beacon_processor_send_result( - self.network_beacon_processor - .send_blobs_by_range_request(peer_id, request_id, request), + self.network_beacon_processor.send_blobs_by_range_request( + peer_id, + request_id, + request, + ), ), Request::BlobsByRoot(request) => self.handle_beacon_processor_send_result( - self.network_beacon_processor - .send_blobs_by_roots_request(peer_id, request_id, request), + self.network_beacon_processor.send_blobs_by_roots_request( + peer_id, + request_id, + request, + ), ), Request::LightClientBootstrap(request) => self.handle_beacon_processor_send_result( self.network_beacon_processor @@ -305,7 +311,7 @@ impl Router { blob_index, signed_blob, timestamp_now(), - ), + ) ) } PubsubMessage::VoluntaryExit(exit) => { diff --git a/beacon_node/network/src/service/tests.rs b/beacon_node/network/src/service/tests.rs index 544c5dd9c7f..10110aa891b 100644 --- a/beacon_node/network/src/service/tests.rs +++ b/beacon_node/network/src/service/tests.rs @@ -3,7 +3,6 @@ mod tests { use crate::persisted_dht::load_dht; use crate::{NetworkConfig, NetworkService}; - use beacon_chain::test_utils::EphemeralHarnessType; use beacon_processor::{ BeaconProcessorSend, MAX_SCHEDULED_WORK_QUEUE_LEN, MAX_WORK_EVENT_QUEUE_LEN, }; @@ -13,6 +12,7 @@ mod tests { use std::str::FromStr; use std::sync::Arc; use tokio::{runtime::Runtime, sync::mpsc}; + use beacon_chain::test_utils::EphemeralHarnessType; use types::MinimalEthSpec as E; type BeaconChainHarness = beacon_chain::test_utils::BeaconChainHarness>; diff --git a/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs b/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs index 55e9e49db30..c492470b4ae 100644 --- a/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs @@ -1,12 +1,12 @@ -use crate::network_beacon_processor::NetworkBeaconProcessor; use beacon_chain::{BeaconChain, BeaconChainTypes}; -use slog::crit; +use slog::{crit, }; use slot_clock::SlotClock; use std::sync::Arc; use tokio::sync::mpsc; use tokio::time::interval_at; use tokio::time::Instant; -use types::Hash256; +use types::{ Hash256}; +use crate::network_beacon_processor::NetworkBeaconProcessor; #[derive(Debug)] pub enum DelayedLookupMessage { diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index dfe960832a5..ff095c719ea 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -10,7 +10,7 @@ use super::{ use crate::metrics; use crate::network_beacon_processor::ChainSegmentProcessId; use crate::sync::block_lookups::single_block_lookup::LookupId; -use beacon_chain::blob_verification::{AsBlock, BlockWrapper}; +use beacon_chain::block_verification_types::{AsBlock, RpcBlock}; use beacon_chain::data_availability_checker::{AvailabilityCheckError, DataAvailabilityChecker}; use beacon_chain::{AvailabilityProcessingStatus, BeaconChainTypes, BlockError}; use lighthouse_network::rpc::RPCError; @@ -34,7 +34,7 @@ mod single_block_lookup; #[cfg(test)] mod tests; -pub type DownloadedBlocks = (Hash256, BlockWrapper); +pub type DownloadedBlocks = (Hash256, RpcBlock); pub type RootBlockTuple = (Hash256, Arc>); pub type RootBlobsTuple = (Hash256, FixedBlobSidecarList); @@ -381,7 +381,7 @@ impl BlockLookups { if !has_pending_parent_request { let rpc_block = request_ref .get_downloaded_block() - .unwrap_or(BlockWrapper::Block(block)); + .unwrap_or(RpcBlock::new_without_blobs(block)); // This is the correct block, send it for processing match self.send_block_for_processing( block_root, @@ -910,11 +910,7 @@ impl BlockLookups { BlockError::ParentUnknown(block) => { let slot = block.slot(); let parent_root = block.parent_root(); - let (block, blobs) = block.deconstruct(); - request_ref.add_unknown_parent_components(UnknownParentComponents::new( - Some(block), - blobs, - )); + request_ref.add_unknown_parent_components(block.into()); self.search_parent(slot, root, parent_root, peer_id.to_peer_id(), cx); ShouldRemoveLookup::False } @@ -1226,7 +1222,7 @@ impl BlockLookups { fn send_block_for_processing( &mut self, block_root: Hash256, - block: BlockWrapper, + block: RpcBlock, duration: Duration, process_type: BlockProcessType, cx: &mut SyncNetworkContext, diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 93b7c9af5bf..8bdf57e2a41 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -127,7 +127,7 @@ pub enum SyncMessage { }, /// A block with an unknown parent has been received. - UnknownParentBlock(PeerId, BlockWrapper, Hash256), + UnknownParentBlock(PeerId, RpcBlock, Hash256), /// A blob with an unknown parent has been received. UnknownParentBlob(PeerId, Arc>), diff --git a/beacon_node/network/src/sync/range_sync/range.rs b/beacon_node/network/src/sync/range_sync/range.rs index 733cbcc9eae..09a85208d96 100644 --- a/beacon_node/network/src/sync/range_sync/range.rs +++ b/beacon_node/network/src/sync/range_sync/range.rs @@ -396,7 +396,7 @@ mod tests { use tokio::sync::mpsc; use beacon_chain::test_utils::{BeaconChainHarness, EphemeralHarnessType}; - use slot_clock::TestingSlotClock; + use slot_clock::{TestingSlotClock, }; use std::collections::HashSet; use std::sync::Arc; use store::MemoryStore; From b19883e63b34a293a24a06a1106d134ed230096c Mon Sep 17 00:00:00 2001 From: realbigsean Date: Fri, 14 Jul 2023 16:38:17 -0400 Subject: [PATCH 02/18] refactor deneb block processing --- beacon_node/beacon_chain/src/beacon_chain.rs | 17 +- .../beacon_chain/src/blob_verification.rs | 240 +--------- .../beacon_chain/src/block_verification.rs | 253 +++------- .../src/block_verification_types.rs | 404 ++++++++++++++++ .../src/data_availability_checker.rs | 442 ++++++------------ .../overflow_lru_cache.rs | 28 +- beacon_node/beacon_chain/src/lib.rs | 9 +- beacon_node/beacon_chain/src/test_utils.rs | 37 +- .../tests/attestation_production.rs | 15 +- .../beacon_chain/tests/block_verification.rs | 83 ++-- beacon_node/beacon_chain/tests/store_tests.rs | 8 +- beacon_node/http_api/src/publish_blocks.rs | 2 +- .../network/src/sync/backfill_sync/mod.rs | 6 +- .../src/sync/block_lookups/parent_lookup.rs | 8 +- .../sync/block_lookups/single_block_lookup.rs | 25 +- .../network/src/sync/block_lookups/tests.rs | 2 +- .../src/sync/block_sidecar_coupling.rs | 44 +- beacon_node/network/src/sync/manager.rs | 12 +- .../network/src/sync/network_context.rs | 4 +- .../network/src/sync/range_sync/batch.rs | 16 +- .../network/src/sync/range_sync/chain.rs | 4 +- .../network/src/sync/range_sync/range.rs | 5 +- consensus/fork_choice/tests/tests.rs | 3 +- .../state_processing/src/consensus_context.rs | 12 - consensus/types/src/lib.rs | 7 +- consensus/types/src/signed_beacon_block.rs | 21 + 26 files changed, 804 insertions(+), 903 deletions(-) create mode 100644 beacon_node/beacon_chain/src/block_verification_types.rs diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 84039e42268..3713f8771ae 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -8,13 +8,16 @@ use crate::beacon_block_streamer::{BeaconBlockStreamer, CheckEarlyAttesterCache} use crate::beacon_proposer_cache::compute_proposer_duties_from_head; use crate::beacon_proposer_cache::BeaconProposerCache; use crate::blob_cache::BlobCache; -use crate::blob_verification::{self, AsBlock, BlobError, BlockWrapper, GossipVerifiedBlob}; +use crate::blob_verification::{self, BlobError, GossipVerifiedBlob}; use crate::block_times_cache::BlockTimesCache; use crate::block_verification::POS_PANDA_BANNER; use crate::block_verification::{ check_block_is_finalized_checkpoint_or_descendant, check_block_relevancy, get_block_root, - signature_verify_chain_segment, AvailableExecutedBlock, BlockError, BlockImportData, - ExecutedBlock, ExecutionPendingBlock, GossipVerifiedBlock, IntoExecutionPendingBlock, + signature_verify_chain_segment, BlockError, ExecutionPendingBlock, GossipVerifiedBlock, + IntoExecutionPendingBlock, +}; +use crate::block_verification_types::{ + AsBlock, AvailableExecutedBlock, BlockImportData, ExecutedBlock, RpcBlock, }; pub use crate::canonical_head::{CanonicalHead, CanonicalHeadRwLock}; use crate::chain_config::ChainConfig; @@ -122,7 +125,7 @@ use types::*; pub type ForkChoiceError = fork_choice::Error; /// Alias to appease clippy. -type HashBlockTuple = (Hash256, BlockWrapper); +type HashBlockTuple = (Hash256, RpcBlock); /// The time-out before failure during an operation to take a read/write RwLock on the block /// processing cache. @@ -2490,7 +2493,7 @@ impl BeaconChain { /// This method is potentially long-running and should not run on the core executor. pub fn filter_chain_segment( self: &Arc, - chain_segment: Vec>, + chain_segment: Vec>, ) -> Result>, ChainSegmentResult> { // This function will never import any blocks. let imported_blocks = 0; @@ -2596,7 +2599,7 @@ impl BeaconChain { /// `Self::process_block`. pub async fn process_chain_segment( self: &Arc, - chain_segment: Vec>, + chain_segment: Vec>, notify_execution_layer: NotifyExecutionLayer, ) -> ChainSegmentResult { let mut imported_blocks = 0; @@ -2773,7 +2776,7 @@ impl BeaconChain { /// /// - `SignedBeaconBlock` /// - `GossipVerifiedBlock` - /// - `BlockWrapper` + /// - `RpcBlock` /// /// ## Errors /// diff --git a/beacon_node/beacon_chain/src/blob_verification.rs b/beacon_node/beacon_chain/src/blob_verification.rs index da3ae2c93b0..78e48f0ed65 100644 --- a/beacon_node/beacon_chain/src/blob_verification.rs +++ b/beacon_node/beacon_chain/src/blob_verification.rs @@ -7,22 +7,18 @@ use crate::beacon_chain::{ BeaconChain, BeaconChainTypes, BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT, MAXIMUM_GOSSIP_CLOCK_DISPARITY, VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT, }; -use crate::data_availability_checker::{ - AvailabilityCheckError, AvailabilityPendingBlock, AvailableBlock, -}; +use crate::data_availability_checker::AvailabilityCheckError; use crate::kzg_utils::{validate_blob, validate_blobs}; use crate::BeaconChainError; -use eth2::types::BlockContentsTuple; use kzg::Kzg; use slog::{debug, warn}; use ssz_derive::{Decode, Encode}; -use ssz_types::{FixedVector, VariableList}; +use ssz_types::VariableList; use std::borrow::Cow; -use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; +use types::blob_sidecar::BlobIdentifier; use types::{ - BeaconBlockRef, BeaconState, BeaconStateError, BlobSidecar, BlobSidecarList, ChainSpec, - CloneConfig, Epoch, EthSpec, FullPayload, Hash256, KzgCommitment, RelativeEpoch, - SignedBeaconBlock, SignedBeaconBlockHeader, SignedBlobSidecar, Slot, + BeaconState, BeaconStateError, BlobSidecar, BlobSidecarList, ChainSpec, CloneConfig, EthSpec, + Hash256, KzgCommitment, RelativeEpoch, SignedBlobSidecar, Slot, }; #[derive(Debug)] @@ -519,13 +515,12 @@ pub fn verify_kzg_for_blob( /// Note: This function should be preferred over calling `verify_kzg_for_blob` /// in a loop since this function kzg verifies a list of blobs more efficiently. pub fn verify_kzg_for_blob_list( - blob_list: Vec>>, + blob_list: &BlobSidecarList, kzg: &Kzg, -) -> Result, AvailabilityCheckError> { +) -> Result<(), AvailabilityCheckError> { let _timer = crate::metrics::start_timer(&crate::metrics::KZG_VERIFICATION_BATCH_TIMES); let (blobs, (commitments, proofs)): (Vec<_>, (Vec<_>, Vec<_>)) = blob_list - .clone() - .into_iter() + .iter() .map(|blob| (blob.blob.clone(), (blob.kzg_commitment, blob.kzg_proof))) .unzip(); if validate_blobs::( @@ -536,225 +531,8 @@ pub fn verify_kzg_for_blob_list( ) .map_err(AvailabilityCheckError::Kzg)? { - Ok(blob_list - .into_iter() - .map(|blob| KzgVerifiedBlob { blob }) - .collect()) + Ok(()) } else { Err(AvailabilityCheckError::KzgVerificationFailed) } } - -pub type KzgVerifiedBlobList = Vec>; - -#[derive(Debug, Clone)] -pub enum MaybeAvailableBlock { - /// This variant is fully available. - /// i.e. for pre-deneb blocks, it contains a (`SignedBeaconBlock`, `Blobs::None`) and for - /// post-4844 blocks, it contains a `SignedBeaconBlock` and a Blobs variant other than `Blobs::None`. - Available(AvailableBlock), - /// This variant is not fully available and requires blobs to become fully available. - AvailabilityPending(AvailabilityPendingBlock), -} - -/// Trait for common block operations. -pub trait AsBlock { - fn slot(&self) -> Slot; - fn epoch(&self) -> Epoch; - fn parent_root(&self) -> Hash256; - fn state_root(&self) -> Hash256; - fn signed_block_header(&self) -> SignedBeaconBlockHeader; - fn message(&self) -> BeaconBlockRef; - fn as_block(&self) -> &SignedBeaconBlock; - fn block_cloned(&self) -> Arc>; - fn canonical_root(&self) -> Hash256; - fn into_block_wrapper(self) -> BlockWrapper; -} - -impl AsBlock for MaybeAvailableBlock { - fn slot(&self) -> Slot { - self.as_block().slot() - } - fn epoch(&self) -> Epoch { - self.as_block().epoch() - } - fn parent_root(&self) -> Hash256 { - self.as_block().parent_root() - } - fn state_root(&self) -> Hash256 { - self.as_block().state_root() - } - fn signed_block_header(&self) -> SignedBeaconBlockHeader { - self.as_block().signed_block_header() - } - fn message(&self) -> BeaconBlockRef { - self.as_block().message() - } - fn as_block(&self) -> &SignedBeaconBlock { - match &self { - MaybeAvailableBlock::Available(block) => block.as_block(), - MaybeAvailableBlock::AvailabilityPending(block) => block.as_block(), - } - } - fn block_cloned(&self) -> Arc> { - match &self { - MaybeAvailableBlock::Available(block) => block.block_cloned(), - MaybeAvailableBlock::AvailabilityPending(block) => block.block_cloned(), - } - } - fn canonical_root(&self) -> Hash256 { - self.as_block().canonical_root() - } - - fn into_block_wrapper(self) -> BlockWrapper { - match self { - MaybeAvailableBlock::Available(available_block) => available_block.into_block_wrapper(), - MaybeAvailableBlock::AvailabilityPending(pending_block) => { - BlockWrapper::Block(pending_block.to_block()) - } - } - } -} - -impl AsBlock for &MaybeAvailableBlock { - fn slot(&self) -> Slot { - self.as_block().slot() - } - fn epoch(&self) -> Epoch { - self.as_block().epoch() - } - fn parent_root(&self) -> Hash256 { - self.as_block().parent_root() - } - fn state_root(&self) -> Hash256 { - self.as_block().state_root() - } - fn signed_block_header(&self) -> SignedBeaconBlockHeader { - self.as_block().signed_block_header() - } - fn message(&self) -> BeaconBlockRef { - self.as_block().message() - } - fn as_block(&self) -> &SignedBeaconBlock { - match &self { - MaybeAvailableBlock::Available(block) => block.as_block(), - MaybeAvailableBlock::AvailabilityPending(block) => block.as_block(), - } - } - fn block_cloned(&self) -> Arc> { - match &self { - MaybeAvailableBlock::Available(block) => block.block_cloned(), - MaybeAvailableBlock::AvailabilityPending(block) => block.block_cloned(), - } - } - fn canonical_root(&self) -> Hash256 { - self.as_block().canonical_root() - } - - fn into_block_wrapper(self) -> BlockWrapper { - self.clone().into_block_wrapper() - } -} - -#[derive(Debug, Clone, Derivative)] -#[derivative(Hash(bound = "E: EthSpec"))] -pub enum BlockWrapper { - Block(Arc>), - BlockAndBlobs(Arc>, FixedBlobSidecarList), -} - -impl BlockWrapper { - pub fn new(block: Arc>, blobs: Option>) -> Self { - match blobs { - Some(blobs) => { - let blobs = FixedVector::from(blobs.into_iter().map(Some).collect::>()); - BlockWrapper::BlockAndBlobs(block, blobs) - } - None => BlockWrapper::Block(block), - } - } - pub fn deconstruct(self) -> (Arc>, Option>) { - match self { - BlockWrapper::Block(block) => (block, None), - BlockWrapper::BlockAndBlobs(block, blobs) => (block, Some(blobs)), - } - } -} - -impl AsBlock for BlockWrapper { - fn slot(&self) -> Slot { - self.as_block().slot() - } - fn epoch(&self) -> Epoch { - self.as_block().epoch() - } - fn parent_root(&self) -> Hash256 { - self.as_block().parent_root() - } - fn state_root(&self) -> Hash256 { - self.as_block().state_root() - } - fn signed_block_header(&self) -> SignedBeaconBlockHeader { - self.as_block().signed_block_header() - } - fn message(&self) -> BeaconBlockRef { - self.as_block().message() - } - fn as_block(&self) -> &SignedBeaconBlock { - match &self { - BlockWrapper::Block(block) => block, - BlockWrapper::BlockAndBlobs(block, _) => block, - } - } - fn block_cloned(&self) -> Arc> { - match &self { - BlockWrapper::Block(block) => block.clone(), - BlockWrapper::BlockAndBlobs(block, _) => block.clone(), - } - } - fn canonical_root(&self) -> Hash256 { - self.as_block().canonical_root() - } - - fn into_block_wrapper(self) -> BlockWrapper { - self - } -} - -impl BlockWrapper { - pub fn n_blobs(&self) -> usize { - match self { - BlockWrapper::Block(_) => 0, - BlockWrapper::BlockAndBlobs(_, blobs) => blobs.len(), - } - } -} - -impl From>> for BlockWrapper { - fn from(value: Arc>) -> Self { - Self::Block(value) - } -} - -impl From> for BlockWrapper { - fn from(value: SignedBeaconBlock) -> Self { - Self::Block(Arc::new(value)) - } -} - -impl From>> for BlockWrapper { - fn from(value: BlockContentsTuple>) -> Self { - match value.1 { - Some(variable_list) => { - let mut blobs = Vec::with_capacity(E::max_blobs_per_block()); - for blob in variable_list { - if blob.message.index < E::max_blobs_per_block() as u64 { - blobs.insert(blob.message.index as usize, Some(blob.message)); - } - } - Self::BlockAndBlobs(Arc::new(value.0), FixedVector::from(blobs)) - } - None => Self::Block(Arc::new(value.0)), - } - } -} diff --git a/beacon_node/beacon_chain/src/block_verification.rs b/beacon_node/beacon_chain/src/block_verification.rs index 796712ed203..dfe388417f9 100644 --- a/beacon_node/beacon_chain/src/block_verification.rs +++ b/beacon_node/beacon_chain/src/block_verification.rs @@ -48,13 +48,11 @@ // returned alongside. #![allow(clippy::result_large_err)] -use crate::blob_verification::{ - AsBlock, BlobError, BlockWrapper, GossipVerifiedBlob, GossipVerifiedBlobList, - MaybeAvailableBlock, -}; -use crate::data_availability_checker::{ - AvailabilityCheckError, AvailabilityPendingBlock, AvailableBlock, +use crate::blob_verification::{BlobError, GossipVerifiedBlob}; +use crate::block_verification_types::{ + AsBlock, BlockImportData, GossipVerifiedBlockContents, RpcBlock, }; +use crate::data_availability_checker::{AvailabilityCheckError, MaybeAvailableBlock}; use crate::eth1_finalization_cache::Eth1FinalizationData; use crate::execution_payload::{ is_optimistic_candidate_block, validate_execution_payload_for_gossip, validate_merge_block, @@ -99,13 +97,11 @@ use std::time::Duration; use store::{Error as DBError, HotStateSummary, KeyValueStore, SignedBlobSidecarList, StoreOp}; use task_executor::JoinHandle; use tree_hash::TreeHash; -use types::blob_sidecar::BlobIdentifier; use types::ExecPayload; -use types::{ssz_tagged_beacon_state, ssz_tagged_signed_beacon_block}; use types::{ - BeaconBlockRef, BeaconState, BeaconStateError, BlindedPayload, ChainSpec, CloneConfig, Epoch, - EthSpec, ExecutionBlockHash, Hash256, InconsistentFork, PublicKey, PublicKeyBytes, - RelativeEpoch, SignedBeaconBlock, SignedBeaconBlockHeader, Slot, + BeaconBlockRef, BeaconState, BeaconStateError, ChainSpec, CloneConfig, Epoch, EthSpec, + ExecutionBlockHash, Hash256, InconsistentFork, PublicKey, PublicKeyBytes, RelativeEpoch, + SignedBeaconBlock, SignedBeaconBlockHeader, Slot, }; pub const POS_PANDA_BANNER: &str = r#" @@ -153,7 +149,7 @@ pub enum BlockError { /// /// It's unclear if this block is valid, but it cannot be processed without already knowing /// its parent. - ParentUnknown(BlockWrapper), + ParentUnknown(RpcBlock), /// The block slot is greater than the present slot. /// /// ## Peer scoring @@ -558,7 +554,7 @@ fn process_block_slash_info( /// The given `chain_segment` must contain only blocks from the same epoch, otherwise an error /// will be returned. pub fn signature_verify_chain_segment( - mut chain_segment: Vec<(Hash256, BlockWrapper)>, + mut chain_segment: Vec<(Hash256, RpcBlock)>, chain: &BeaconChain, ) -> Result>, BlockError> { if chain_segment.is_empty() { @@ -595,7 +591,7 @@ pub fn signature_verify_chain_segment( let maybe_available_block = chain .data_availability_checker - .check_availability(block.clone())?; + .check_rpc_block_availability(block.clone())?; // Save the block and its consensus context. The context will have had its proposer index // and attesting indices filled in, which can be used to accelerate later block processing. @@ -625,19 +621,12 @@ pub fn signature_verify_chain_segment( #[derive(Derivative)] #[derivative(Debug(bound = "T: BeaconChainTypes"))] pub struct GossipVerifiedBlock { - pub block: MaybeAvailableBlock, + pub block: Arc>, pub block_root: Hash256, parent: Option>, consensus_context: ConsensusContext, } -impl GossipVerifiedBlock { - /// Useful for publishing after gossip verification. - pub fn into_block_wrapper(self) -> BlockWrapper { - self.block.into_block_wrapper() - } -} - /// A wrapper around a `SignedBeaconBlock` that indicates that all signatures (except the deposit /// signatures) have been verified. pub struct SignatureVerifiedBlock { @@ -669,147 +658,6 @@ pub struct ExecutionPendingBlock { pub payload_verification_handle: PayloadVerificationHandle, } -pub enum ExecutedBlock { - Available(AvailableExecutedBlock), - AvailabilityPending(AvailabilityPendingExecutedBlock), -} - -impl ExecutedBlock { - pub fn as_block(&self) -> &SignedBeaconBlock { - match self { - Self::Available(available) => available.block.block(), - Self::AvailabilityPending(pending) => pending.block.as_block(), - } - } -} - -impl std::fmt::Debug for ExecutedBlock { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "{:?}", self.as_block()) - } -} - -impl ExecutedBlock { - pub fn new( - block: MaybeAvailableBlock, - import_data: BlockImportData, - payload_verification_outcome: PayloadVerificationOutcome, - ) -> Self { - match block { - MaybeAvailableBlock::Available(available_block) => { - Self::Available(AvailableExecutedBlock::new( - available_block, - import_data, - payload_verification_outcome, - )) - } - MaybeAvailableBlock::AvailabilityPending(pending_block) => { - Self::AvailabilityPending(AvailabilityPendingExecutedBlock::new( - pending_block, - import_data, - payload_verification_outcome, - )) - } - } - } -} - -#[derive(Debug, PartialEq)] -pub struct AvailableExecutedBlock { - pub block: AvailableBlock, - pub import_data: BlockImportData, - pub payload_verification_outcome: PayloadVerificationOutcome, -} - -impl AvailableExecutedBlock { - pub fn new( - block: AvailableBlock, - import_data: BlockImportData, - payload_verification_outcome: PayloadVerificationOutcome, - ) -> Self { - Self { - block, - import_data, - payload_verification_outcome, - } - } - - pub fn get_all_blob_ids(&self) -> Vec { - let num_blobs_expected = self - .block - .message() - .body() - .blob_kzg_commitments() - .map_or(0, |commitments| commitments.len()); - let mut blob_ids = Vec::with_capacity(num_blobs_expected); - for i in 0..num_blobs_expected { - blob_ids.push(BlobIdentifier { - block_root: self.import_data.block_root, - index: i as u64, - }); - } - blob_ids - } -} - -#[derive(Encode, Decode, Clone)] -pub struct AvailabilityPendingExecutedBlock { - pub block: AvailabilityPendingBlock, - pub import_data: BlockImportData, - pub payload_verification_outcome: PayloadVerificationOutcome, -} - -impl AvailabilityPendingExecutedBlock { - pub fn new( - block: AvailabilityPendingBlock, - import_data: BlockImportData, - payload_verification_outcome: PayloadVerificationOutcome, - ) -> Self { - Self { - block, - import_data, - payload_verification_outcome, - } - } - - pub fn num_blobs_expected(&self) -> usize { - self.block - .kzg_commitments() - .map_or(0, |commitments| commitments.len()) - } - - pub fn get_all_blob_ids(&self) -> Vec { - let block_root = self.import_data.block_root; - self.block - .get_filtered_blob_ids(Some(block_root), |_, _| true) - } - - pub fn get_filtered_blob_ids( - &self, - filter: impl Fn(usize, Hash256) -> bool, - ) -> Vec { - self.block - .get_filtered_blob_ids(Some(self.import_data.block_root), filter) - } -} - -#[derive(Debug, PartialEq, Encode, Decode, Clone)] -// TODO (mark): investigate using an Arc / Arc -// here to make this cheaper to clone -pub struct BlockImportData { - pub block_root: Hash256, - #[ssz(with = "ssz_tagged_beacon_state")] - pub state: BeaconState, - #[ssz(with = "ssz_tagged_signed_beacon_block")] - pub parent_block: SignedBeaconBlock>, - pub parent_eth1_finalization_data: Eth1FinalizationData, - pub confirmed_state_roots: Vec, - pub consensus_context: ConsensusContext, -} - -pub type GossipVerifiedBlockContents = - (GossipVerifiedBlock, Option>); - pub trait IntoGossipVerifiedBlockContents: Sized { fn into_gossip_verified_block( self, @@ -911,27 +759,23 @@ impl GossipVerifiedBlock { block: Arc>, chain: &BeaconChain, ) -> Result> { - let maybe_available = chain - .data_availability_checker - .check_availability(block.into())?; // If the block is valid for gossip we don't supply it to the slasher here because // we assume it will be transformed into a fully verified block. We *do* need to supply // it to the slasher if an error occurs, because that's the end of this block's journey, // and it could be a repeat proposal (a likely cause for slashing!). - let header = maybe_available.signed_block_header(); - Self::new_without_slasher_checks(maybe_available, chain).map_err(|e| { + let header = block.signed_block_header(); + Self::new_without_slasher_checks(block, chain).map_err(|e| { process_block_slash_info(chain, BlockSlashInfo::from_early_error(header, e)) }) } /// As for new, but doesn't pass the block to the slasher. fn new_without_slasher_checks( - block: MaybeAvailableBlock, + block: Arc>, chain: &BeaconChain, ) -> Result> { // Ensure the block is the correct structure for the fork at `block.slot()`. block - .as_block() .fork_name(&chain.spec) .map_err(BlockError::InconsistentFork)?; @@ -947,7 +791,7 @@ impl GossipVerifiedBlock { }); } - let block_root = get_block_root(block.as_block()); + let block_root = get_block_root(&block); // Disallow blocks that conflict with the anchor (weak subjectivity checkpoint), if any. check_block_against_anchor_slot(block.message(), chain)?; @@ -1067,7 +911,7 @@ impl GossipVerifiedBlock { let pubkey = pubkey_cache .get(block.message().proposer_index() as usize) .ok_or_else(|| BlockError::UnknownValidator(block.message().proposer_index()))?; - block.as_block().verify_signature( + block.verify_signature( Some(block_root), pubkey, &fork, @@ -1111,8 +955,7 @@ impl GossipVerifiedBlock { // Having checked the proposer index and the block root we can cache them. let consensus_context = ConsensusContext::new(block.slot()) .set_current_block_root(block_root) - .set_proposer_index(block.as_block().message().proposer_index()) - .set_kzg_commitments_consistent(true); + .set_proposer_index(block.as_block().message().proposer_index()); Ok(Self { block, @@ -1155,11 +998,10 @@ impl SignatureVerifiedBlock { /// /// Returns an error if the block is invalid, or if the block was unable to be verified. pub fn new( - block: BlockWrapper, + block: MaybeAvailableBlock, block_root: Hash256, chain: &BeaconChain, ) -> Result> { - let block = chain.data_availability_checker.check_availability(block)?; // Ensure the block is the correct structure for the fork at `block.slot()`. block .as_block() @@ -1182,10 +1024,8 @@ impl SignatureVerifiedBlock { let mut signature_verifier = get_signature_verifier(&state, &pubkey_cache, &chain.spec); - let mut consensus_context = ConsensusContext::new(block.slot()) - .set_current_block_root(block_root) - // An `AvailabileBlock is passed in here, so we know this check has been run.` - .set_kzg_commitments_consistent(true); + let mut consensus_context = + ConsensusContext::new(block.slot()).set_current_block_root(block_root); signature_verifier.include_all_signatures(block.as_block(), &mut consensus_context)?; @@ -1203,7 +1043,7 @@ impl SignatureVerifiedBlock { /// As for `new` above but producing `BlockSlashInfo`. pub fn check_slashable( - block: BlockWrapper, + block: MaybeAvailableBlock, block_root: Hash256, chain: &BeaconChain, ) -> Result>> { @@ -1238,11 +1078,11 @@ impl SignatureVerifiedBlock { // signature. let mut consensus_context = from.consensus_context; signature_verifier - .include_all_signatures_except_proposal(block.as_block(), &mut consensus_context)?; + .include_all_signatures_except_proposal(block.as_ref(), &mut consensus_context)?; if signature_verifier.verify().is_ok() { Ok(Self { - block, + block: MaybeAvailableBlock::AvailabilityPending(block), block_root: from.block_root, parent: Some(parent), consensus_context, @@ -1299,6 +1139,7 @@ impl IntoExecutionPendingBlock for SignatureVerifiedBloc } } +//TODO(sean) can this be deleted impl IntoExecutionPendingBlock for Arc> { /// Verifies the `SignedBeaconBlock` by first transforming it into a `SignatureVerifiedBlock` /// and then using that implementation of `IntoExecutionPendingBlock` to complete verification. @@ -1311,9 +1152,21 @@ impl IntoExecutionPendingBlock for Arc &SignedBeaconBlock { @@ -1321,7 +1174,7 @@ impl IntoExecutionPendingBlock for Arc IntoExecutionPendingBlock for BlockWrapper { +impl IntoExecutionPendingBlock for RpcBlock { /// Verifies the `SignedBeaconBlock` by first transforming it into a `SignatureVerifiedBlock` /// and then using that implementation of `IntoExecutionPendingBlock` to complete verification. fn into_execution_pending_block_slashable( @@ -1333,8 +1186,16 @@ impl IntoExecutionPendingBlock for BlockWrapper ExecutionPendingBlock { // because it will revert finalization. Note that the finalized block is stored in fork // choice, so we will not reject any child of the finalized block (this is relevant during // genesis). - return Err(BlockError::ParentUnknown(block.into_block_wrapper())); + return Err(BlockError::ParentUnknown(block.into_rpc_block())); } /* @@ -1826,7 +1687,7 @@ pub fn check_block_is_finalized_checkpoint_or_descendant< block_parent_root: block.parent_root(), }) } else { - Err(BlockError::ParentUnknown(block.into_block_wrapper())) + Err(BlockError::ParentUnknown(block.into_rpc_block())) } } } @@ -1898,8 +1759,8 @@ pub fn get_block_root(block: &SignedBeaconBlock) -> Hash256 { #[allow(clippy::type_complexity)] fn verify_parent_block_is_known( chain: &BeaconChain, - block: MaybeAvailableBlock, -) -> Result<(ProtoBlock, MaybeAvailableBlock), BlockError> { + block: Arc>, +) -> Result<(ProtoBlock, Arc>), BlockError> { if let Some(proto_block) = chain .canonical_head .fork_choice_read_lock() @@ -1907,7 +1768,9 @@ fn verify_parent_block_is_known( { Ok((proto_block, block)) } else { - Err(BlockError::ParentUnknown(block.into_block_wrapper())) + Err(BlockError::ParentUnknown(RpcBlock::new_without_blobs( + block, + ))) } } @@ -1938,7 +1801,7 @@ fn load_parent>( .fork_choice_read_lock() .contains_block(&block.parent_root()) { - return Err(BlockError::ParentUnknown(block.into_block_wrapper())); + return Err(BlockError::ParentUnknown(block.into_rpc_block())); } let block_delay = chain diff --git a/beacon_node/beacon_chain/src/block_verification_types.rs b/beacon_node/beacon_chain/src/block_verification_types.rs new file mode 100644 index 00000000000..a152b757df3 --- /dev/null +++ b/beacon_node/beacon_chain/src/block_verification_types.rs @@ -0,0 +1,404 @@ +use crate::blob_verification::GossipVerifiedBlobList; +use crate::data_availability_checker::AvailabilityCheckError; +pub use crate::data_availability_checker::{AvailableBlock, MaybeAvailableBlock}; +use crate::{data_availability_checker, GossipVerifiedBlock, PayloadVerificationOutcome}; +use derivative::Derivative; +use ssz_derive::{Decode, Encode}; +use state_processing::ConsensusContext; +use std::sync::Arc; +use types::{ + blob_sidecar::BlobIdentifier, ssz_tagged_beacon_state, ssz_tagged_signed_beacon_block, + ssz_tagged_signed_beacon_block_arc, +}; +use types::{ + BeaconBlockRef, BeaconState, BlindedPayload, BlobSidecarList, Epoch, EthSpec, Hash256, + SignedBeaconBlock, SignedBeaconBlockHeader, Slot, +}; +use crate::eth1_finalization_cache::Eth1FinalizationData; + +#[derive(Debug, Clone, Derivative)] +#[derivative(Hash(bound = "E: EthSpec"))] +pub struct RpcBlock { + block: RpcBlockInner, +} + +#[derive(Debug, Clone, Derivative)] +#[derivative(Hash(bound = "E: EthSpec"))] +enum RpcBlockInner { + /// Single block lookup response. This should potentially hit the data availability cache. + Block(Arc>), + /// This variant is used with parent lookups and by-range responses. It should have all blobs + /// ordered, all block roots matching, and the correct number of blobs for this block. + BlockAndBlobs(Arc>, BlobSidecarList), +} + +impl RpcBlock { + pub fn new_without_blobs(block: Arc>) -> Self { + Self { + block: RpcBlockInner::Block(block), + } + } + + pub fn new( + block: Arc>, + blobs: Option>, + ) -> Result { + if let Some(blobs) = blobs.as_ref() { + data_availability_checker::consistency_checks( &block, blobs)?; + } + let inner = match blobs { + Some(blobs) => RpcBlockInner::BlockAndBlobs(block, blobs), + None => RpcBlockInner::Block(block), + }; + Ok(Self { block: inner }) + } + + pub fn deconstruct(self) -> (Arc>, Option>) { + match self.block { + RpcBlockInner::Block(block) => (block, None), + RpcBlockInner::BlockAndBlobs(block, blobs) => (block, Some(blobs)), + } + } + pub fn n_blobs(&self) -> usize { + match &self.block { + RpcBlockInner::Block(_) => 0, + RpcBlockInner::BlockAndBlobs(_, blobs) => blobs.len(), + } + } +} + +impl From>> for RpcBlock { + fn from(value: Arc>) -> Self { + Self::new_without_blobs(value) + } +} + +impl From> for RpcBlock { + fn from(value: SignedBeaconBlock) -> Self { + Self::new_without_blobs(Arc::new(value)) + } +} + +pub enum ExecutedBlock { + Available(AvailableExecutedBlock), + AvailabilityPending(AvailabilityPendingExecutedBlock), +} + +impl ExecutedBlock { + pub fn new( + block: MaybeAvailableBlock, + import_data: BlockImportData, + payload_verification_outcome: PayloadVerificationOutcome, + ) -> Self { + match block { + MaybeAvailableBlock::Available(available_block) => { + Self::Available(AvailableExecutedBlock::new( + available_block, + import_data, + payload_verification_outcome, + )) + } + MaybeAvailableBlock::AvailabilityPending(pending_block) => { + Self::AvailabilityPending(AvailabilityPendingExecutedBlock::new( + pending_block, + import_data, + payload_verification_outcome, + )) + } + } + } + + pub fn as_block(&self) -> &SignedBeaconBlock { + match self { + Self::Available(available) => available.block.block(), + Self::AvailabilityPending(pending) => &pending.block, + } + } +} + +#[derive(PartialEq)] +pub struct AvailableExecutedBlock { + pub block: AvailableBlock, + pub import_data: BlockImportData, + pub payload_verification_outcome: PayloadVerificationOutcome, +} + +impl AvailableExecutedBlock { + pub fn new( + block: AvailableBlock, + import_data: BlockImportData, + payload_verification_outcome: PayloadVerificationOutcome, + ) -> Self { + Self { + block, + import_data, + payload_verification_outcome, + } + } + + pub fn get_all_blob_ids(&self) -> Vec { + let num_blobs_expected = self + .block + .message() + .body() + .blob_kzg_commitments() + .map_or(0, |commitments| commitments.len()); + let mut blob_ids = Vec::with_capacity(num_blobs_expected); + for i in 0..num_blobs_expected { + blob_ids.push(BlobIdentifier { + block_root: self.import_data.block_root, + index: i as u64, + }); + } + blob_ids + } +} + +#[derive(Encode, Decode, Clone)] +pub struct AvailabilityPendingExecutedBlock { + #[ssz(with = "ssz_tagged_signed_beacon_block_arc")] + pub block: Arc>, + pub import_data: BlockImportData, + pub payload_verification_outcome: PayloadVerificationOutcome, +} + +impl AvailabilityPendingExecutedBlock { + pub fn new( + block: Arc>, + import_data: BlockImportData, + payload_verification_outcome: PayloadVerificationOutcome, + ) -> Self { + Self { + block, + import_data, + payload_verification_outcome, + } + } + + pub fn num_blobs_expected(&self) -> usize { + self.block + .message() + .body() + .blob_kzg_commitments() + .map_or(0, |commitments| commitments.len()) + } + + pub fn get_all_blob_ids(&self) -> Vec { + let block_root = self.import_data.block_root; + self.block + .get_filtered_blob_ids(Some(block_root), |_, _| true) + } + + pub fn get_filtered_blob_ids( + &self, + filter: impl Fn(usize, Hash256) -> bool, + ) -> Vec { + self.block + .get_filtered_blob_ids(Some(self.import_data.block_root), filter) + } +} + +#[derive(Debug, PartialEq, Encode, Decode, Clone)] +// TODO (mark): investigate using an Arc / Arc +// here to make this cheaper to clone +pub struct BlockImportData { + pub block_root: Hash256, + #[ssz(with = "ssz_tagged_beacon_state")] + pub state: BeaconState, + #[ssz(with = "ssz_tagged_signed_beacon_block")] + pub parent_block: SignedBeaconBlock>, + pub parent_eth1_finalization_data: Eth1FinalizationData, + pub confirmed_state_roots: Vec, + pub consensus_context: ConsensusContext, +} + +pub type GossipVerifiedBlockContents = + (GossipVerifiedBlock, Option>); + +/// Trait for common block operations. +pub trait AsBlock { + fn slot(&self) -> Slot; + fn epoch(&self) -> Epoch; + fn parent_root(&self) -> Hash256; + fn state_root(&self) -> Hash256; + fn signed_block_header(&self) -> SignedBeaconBlockHeader; + fn message(&self) -> BeaconBlockRef; + fn as_block(&self) -> &SignedBeaconBlock; + fn block_cloned(&self) -> Arc>; + fn canonical_root(&self) -> Hash256; + fn into_rpc_block(self) -> RpcBlock; +} + +impl AsBlock for Arc> { + fn slot(&self) -> Slot { + SignedBeaconBlock::slot(self) + } + + fn epoch(&self) -> Epoch { + SignedBeaconBlock::epoch(self) + } + + fn parent_root(&self) -> Hash256 { + SignedBeaconBlock::parent_root(self) + } + + fn state_root(&self) -> Hash256 { + SignedBeaconBlock::state_root(self) + } + + fn signed_block_header(&self) -> SignedBeaconBlockHeader { + SignedBeaconBlock::signed_block_header(self) + } + + fn message(&self) -> BeaconBlockRef { + SignedBeaconBlock::message(self) + } + + fn as_block(&self) -> &SignedBeaconBlock { + self + } + + fn block_cloned(&self) -> Arc> { + Arc::>::clone(self) + } + + fn canonical_root(&self) -> Hash256 { + SignedBeaconBlock::canonical_root(self) + } + + fn into_rpc_block(self) -> RpcBlock { + RpcBlock::new_without_blobs(self) + } +} + +impl AsBlock for MaybeAvailableBlock { + fn slot(&self) -> Slot { + self.as_block().slot() + } + fn epoch(&self) -> Epoch { + self.as_block().epoch() + } + fn parent_root(&self) -> Hash256 { + self.as_block().parent_root() + } + fn state_root(&self) -> Hash256 { + self.as_block().state_root() + } + fn signed_block_header(&self) -> SignedBeaconBlockHeader { + self.as_block().signed_block_header() + } + fn message(&self) -> BeaconBlockRef { + self.as_block().message() + } + fn as_block(&self) -> &SignedBeaconBlock { + match &self { + MaybeAvailableBlock::Available(block) => block.as_block(), + MaybeAvailableBlock::AvailabilityPending(block) => block, + } + } + fn block_cloned(&self) -> Arc> { + match &self { + MaybeAvailableBlock::Available(block) => block.block_cloned(), + MaybeAvailableBlock::AvailabilityPending(block) => block.clone(), + } + } + fn canonical_root(&self) -> Hash256 { + self.as_block().canonical_root() + } + + fn into_rpc_block(self) -> RpcBlock { + match self { + MaybeAvailableBlock::Available(available_block) => available_block.into_rpc_block(), + MaybeAvailableBlock::AvailabilityPending(block) => RpcBlock::new_without_blobs(block), + } + } +} + +impl AsBlock for AvailableBlock { + fn slot(&self) -> Slot { + self.block().slot() + } + + fn epoch(&self) -> Epoch { + self.block().epoch() + } + + fn parent_root(&self) -> Hash256 { + self.block().parent_root() + } + + fn state_root(&self) -> Hash256 { + self.block().state_root() + } + + fn signed_block_header(&self) -> SignedBeaconBlockHeader { + self.block().signed_block_header() + } + + fn message(&self) -> BeaconBlockRef { + self.block().message() + } + + fn as_block(&self) -> &SignedBeaconBlock { + self.block() + } + + fn block_cloned(&self) -> Arc> { + AvailableBlock::block_cloned(self) + } + + fn canonical_root(&self) -> Hash256 { + self.block().canonical_root() + } + + fn into_rpc_block(self) -> RpcBlock { + let (block, blobs_opt) = self.deconstruct(); + // Circumvent the constructor here, because an Available block will have already had + // consistency checks performed. + let inner = match blobs_opt { + None => RpcBlockInner::Block(block), + Some(blobs) => RpcBlockInner::BlockAndBlobs(block, blobs), + }; + RpcBlock { block: inner } + } +} + +impl AsBlock for RpcBlock { + fn slot(&self) -> Slot { + self.as_block().slot() + } + fn epoch(&self) -> Epoch { + self.as_block().epoch() + } + fn parent_root(&self) -> Hash256 { + self.as_block().parent_root() + } + fn state_root(&self) -> Hash256 { + self.as_block().state_root() + } + fn signed_block_header(&self) -> SignedBeaconBlockHeader { + self.as_block().signed_block_header() + } + fn message(&self) -> BeaconBlockRef { + self.as_block().message() + } + fn as_block(&self) -> &SignedBeaconBlock { + match &self.block { + RpcBlockInner::Block(block) => block, + RpcBlockInner::BlockAndBlobs(block, _) => block, + } + } + fn block_cloned(&self) -> Arc> { + match &self.block { + RpcBlockInner::Block(block) => block.clone(), + RpcBlockInner::BlockAndBlobs(block, _) => block.clone(), + } + } + fn canonical_root(&self) -> Hash256 { + self.as_block().canonical_root() + } + + fn into_rpc_block(self) -> RpcBlock { + self + } +} \ No newline at end of file diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index fc48ba47ed8..eb33f64e911 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -1,28 +1,23 @@ use crate::blob_verification::{ - verify_kzg_for_blob, verify_kzg_for_blob_list, AsBlock, BlockWrapper, GossipVerifiedBlob, - KzgVerifiedBlob, KzgVerifiedBlobList, MaybeAvailableBlock, + verify_kzg_for_blob, verify_kzg_for_blob_list, GossipVerifiedBlob, KzgVerifiedBlob, +}; +use crate::block_verification_types::{ + AvailabilityPendingExecutedBlock, AvailableExecutedBlock, RpcBlock, }; -use crate::block_verification::{AvailabilityPendingExecutedBlock, AvailableExecutedBlock}; - use crate::data_availability_checker::overflow_lru_cache::OverflowLRUCache; use crate::{BeaconChain, BeaconChainTypes, BeaconStore}; use kzg::Error as KzgError; use kzg::Kzg; use slog::{debug, error}; use slot_clock::SlotClock; -use ssz_types::{Error, FixedVector, VariableList}; +use ssz_types::{Error, VariableList}; use std::collections::HashSet; use std::sync::Arc; use strum::IntoStaticStr; use task_executor::TaskExecutor; -use types::beacon_block_body::KzgCommitments; use types::blob_sidecar::{BlobIdentifier, BlobSidecar, FixedBlobSidecarList}; use types::consts::deneb::MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS; -use types::ssz_tagged_signed_beacon_block; -use types::{ - BeaconBlockRef, BlobSidecarList, ChainSpec, Epoch, EthSpec, FullPayload, Hash256, - SignedBeaconBlock, SignedBeaconBlockHeader, Slot, -}; +use types::{BlobSidecarList, ChainSpec, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot}; mod overflow_lru_cache; @@ -50,12 +45,20 @@ pub enum AvailabilityCheckError { }, IncorrectFork, BlobIndexInvalid(u64), + UnorderedBlobs { + blob_index: u64, + expected_index: u64, + }, StoreError(store::Error), DecodeError(ssz::DecodeError), BlockBlobRootMismatch { block_root: Hash256, blob_block_root: Hash256, }, + BlockBlobSlotMismatch { + block_slot: Slot, + blob_slot: Slot, + }, } impl From for AvailabilityCheckError { @@ -92,7 +95,7 @@ pub struct DataAvailabilityChecker { /// /// Indicates if the block is fully `Available` or if we need blobs or blocks /// to "complete" the requirements for an `AvailableBlock`. -#[derive(Debug, PartialEq)] +#[derive(PartialEq)] pub enum Availability { MissingComponents(Hash256), Available(Box>), @@ -230,86 +233,51 @@ impl DataAvailabilityChecker { /// Checks if a block is available, returns a `MaybeAvailableBlock` that may include the fully /// available block. - pub fn check_availability( + pub fn check_rpc_block_availability( &self, - block: BlockWrapper, + block: RpcBlock, ) -> Result, AvailabilityCheckError> { - match block { - BlockWrapper::Block(block) => self.check_availability_without_blobs(block), - BlockWrapper::BlockAndBlobs(block, blob_list) => { - let kzg = self - .kzg - .as_ref() - .ok_or(AvailabilityCheckError::KzgNotInitialized)?; - let filtered_blobs = blob_list.iter().flatten().cloned().collect(); - let verified_blobs = verify_kzg_for_blob_list(filtered_blobs, kzg)?; - - Ok(MaybeAvailableBlock::Available( - self.check_availability_with_blobs(block, verified_blobs)?, - )) + let (block, blobs) = block.deconstruct(); + match blobs { + None => { + if self.blobs_required_for_block(&block) { + Ok(MaybeAvailableBlock::AvailabilityPending(block)) + } else { + Ok(MaybeAvailableBlock::Available(AvailableBlock { + block, + blobs: None, + })) + } } - } - } - - /// Verifies a block against a set of KZG verified blobs. Returns an AvailableBlock if block's - /// commitments are consistent with the provided verified blob commitments. - pub fn check_availability_with_blobs( - &self, - block: Arc>, - blobs: KzgVerifiedBlobList, - ) -> Result, AvailabilityCheckError> { - match self.check_availability_without_blobs(block)? { - MaybeAvailableBlock::Available(block) => Ok(block), - MaybeAvailableBlock::AvailabilityPending(pending_block) => { - pending_block.make_available(blobs) + Some(blob_list) => { + let verified_blobs = if self.blobs_required_for_block(&block) { + let kzg = self + .kzg + .as_ref() + .ok_or(AvailabilityCheckError::KzgNotInitialized)?; + verify_kzg_for_blob_list(&blob_list, kzg)?; + Some(blob_list) + } else { + None + }; + Ok(MaybeAvailableBlock::Available(AvailableBlock { + block, + blobs: verified_blobs, + })) } } } - /// Verifies a block as much as possible, returning a MaybeAvailableBlock enum that may include - /// an AvailableBlock if no blobs are required. Otherwise this will return an AvailabilityPendingBlock. - pub fn check_availability_without_blobs( - &self, - block: Arc>, - ) -> Result, AvailabilityCheckError> { - let blob_requirements = self.get_blob_requirements(&block)?; - let blobs = match blob_requirements { - BlobRequirements::EmptyBlobs => VerifiedBlobs::EmptyBlobs, - BlobRequirements::NotRequired => VerifiedBlobs::NotRequired, - BlobRequirements::PreDeneb => VerifiedBlobs::PreDeneb, - BlobRequirements::Required => { - return Ok(MaybeAvailableBlock::AvailabilityPending( - AvailabilityPendingBlock { block }, - )) - } - }; - Ok(MaybeAvailableBlock::Available(AvailableBlock { - block, - blobs, - })) - } - /// Determines the blob requirements for a block. Answers the question: "Does this block require /// blobs?". - fn get_blob_requirements( - &self, - block: &Arc>>, - ) -> Result { - let verified_blobs = - if let Ok(block_kzg_commitments) = block.message().body().blob_kzg_commitments() { - if self.da_check_required(block.epoch()) { - if block_kzg_commitments.is_empty() { - BlobRequirements::EmptyBlobs - } else { - BlobRequirements::Required - } - } else { - BlobRequirements::NotRequired - } - } else { - BlobRequirements::PreDeneb - }; - Ok(verified_blobs) + fn blobs_required_for_block(&self, block: &SignedBeaconBlock) -> bool { + let block_within_da_period = self.da_check_required(block.epoch()); + let block_has_kzg_commitments = block + .message() + .body() + .blob_kzg_commitments() + .map_or(false, |commitments| !commitments.is_empty()); + block_within_da_period && block_has_kzg_commitments } /// The epoch at which we require a data availability check in block processing. @@ -340,6 +308,77 @@ impl DataAvailabilityChecker { } } +/// Verifies an `SignedBeaconBlock` against a set of KZG verified blobs. +/// This does not check whether a block *should* have blobs, these checks should have been +/// completed when producing the `AvailabilityPendingBlock`. +pub fn make_available( + block: Arc>, + blobs: Vec>, +) -> Result, AvailabilityCheckError> { + let blobs = VariableList::new(blobs.into_iter().map(|blob| blob.to_blob()).collect())?; + + consistency_checks( &block, &blobs)?; + + Ok(AvailableBlock { + block, + blobs: Some(blobs), + }) +} + +pub fn consistency_checks( + block: &SignedBeaconBlock, + blobs: &[Arc>], +) -> Result<(), AvailabilityCheckError> { + let Ok(block_kzg_commitments) = block + .message() + .body() + .blob_kzg_commitments() else { + return Ok(()) + }; + + if blobs.len() != block_kzg_commitments.len() { + return Err(AvailabilityCheckError::NumBlobsMismatch { + num_kzg_commitments: block_kzg_commitments.len(), + num_blobs: blobs.len(), + }); + } + + if block_kzg_commitments.is_empty() { + return Ok(()); + } + + let block_root = blobs.first().map(|blob|blob.block_root).unwrap_or(block.canonical_root()); + for (index, (block_commitment, blob)) in + block_kzg_commitments.iter().zip(blobs.iter()).enumerate() + { + let index = index as u64; + if index != blob.index { + return Err(AvailabilityCheckError::UnorderedBlobs { + blob_index: blob.index, + expected_index: index, + }); + } + if block_root != blob.block_root { + return Err(AvailabilityCheckError::BlockBlobRootMismatch { + block_root, + blob_block_root: blob.block_root, + }); + } + if block.slot() != blob.slot { + return Err(AvailabilityCheckError::BlockBlobSlotMismatch { + block_slot: block.slot(), + blob_slot: blob.slot, + }); + } + if *block_commitment != blob.kzg_commitment { + return Err(AvailabilityCheckError::KzgCommitmentMismatch { + blob_index: blob.index, + }); + } + } + Ok(()) +} + pub fn start_availability_cache_maintenance_service( executor: TaskExecutor, chain: Arc>, @@ -425,244 +464,37 @@ async fn availability_cache_maintenance_service( } } -pub enum BlobRequirements { - Required, - /// This block is from outside the data availability boundary so doesn't require - /// a data availability check. - NotRequired, - /// The block's `kzg_commitments` field is empty so it does not contain any blobs. - EmptyBlobs, - /// This is a block prior to the 4844 fork, so doesn't require any blobs - PreDeneb, -} - -/// A wrapper over a `SignedBeaconBlock` where we have not verified availability of -/// corresponding `BlobSidecar`s and hence, is not ready for import into fork choice. -/// -/// Note: This wrapper does not necessarily correspond to a pre-deneb block as a pre-deneb -/// block that is ready for import will be of type `AvailableBlock` with its `blobs` field -/// set to `VerifiedBlobs::PreDeneb`. -#[derive(Clone, Debug, PartialEq)] -pub struct AvailabilityPendingBlock { - block: Arc>, -} - -impl AvailabilityPendingBlock { - pub fn slot(&self) -> Slot { - self.block.slot() - } - pub fn num_blobs_expected(&self) -> usize { - self.block.num_expected_blobs() - } - - pub fn get_all_blob_ids(&self, block_root: Option) -> Vec { - self.block.get_expected_blob_ids(block_root) - } - - pub fn get_filtered_blob_ids( - &self, - block_root: Option, - filter: impl Fn(usize, Hash256) -> bool, - ) -> Vec { - self.block.get_filtered_blob_ids(block_root, filter) - } -} - -impl AvailabilityPendingBlock { - pub fn to_block(self) -> Arc> { - self.block - } - pub fn as_block(&self) -> &SignedBeaconBlock { - &self.block - } - pub fn block_cloned(&self) -> Arc> { - self.block.clone() - } - pub fn kzg_commitments(&self) -> Result<&KzgCommitments, AvailabilityCheckError> { - self.block - .message() - .body() - .blob_kzg_commitments() - .map_err(|_| AvailabilityCheckError::IncorrectFork) - } - - /// Verifies an AvailabilityPendingBlock against a set of KZG verified blobs. - /// This does not check whether a block *should* have blobs, these checks should have been - /// completed when producing the `AvailabilityPendingBlock`. - pub fn make_available( - self, - blobs: Vec>, - ) -> Result, AvailabilityCheckError> { - let block_kzg_commitments = self.kzg_commitments()?; - if blobs.len() != block_kzg_commitments.len() { - return Err(AvailabilityCheckError::NumBlobsMismatch { - num_kzg_commitments: block_kzg_commitments.len(), - num_blobs: blobs.len(), - }); - } - - for (block_commitment, blob) in block_kzg_commitments.iter().zip(blobs.iter()) { - if *block_commitment != blob.kzg_commitment() { - return Err(AvailabilityCheckError::KzgCommitmentMismatch { - blob_index: blob.as_blob().index, - }); - } - } - - let blobs = VariableList::new(blobs.into_iter().map(|blob| blob.to_blob()).collect())?; - - Ok(AvailableBlock { - block: self.block, - blobs: VerifiedBlobs::Available(blobs), - }) - } -} - -#[derive(Clone, Debug, PartialEq)] -pub enum VerifiedBlobs { - /// These blobs are available. - Available(BlobSidecarList), - /// This block is from outside the data availability boundary so doesn't require - /// a data availability check. - NotRequired, - /// The block's `kzg_commitments` field is empty so it does not contain any blobs. - EmptyBlobs, - /// This is a block prior to the 4844 fork, so doesn't require any blobs - PreDeneb, -} - -impl VerifiedBlobs { - pub fn to_blobs(self) -> Option> { - match self { - Self::Available(blobs) => Some(blobs), - Self::NotRequired => None, - Self::EmptyBlobs => None, - Self::PreDeneb => None, - } - } -} - /// A fully available block that is ready to be imported into fork choice. #[derive(Clone, Debug, PartialEq)] pub struct AvailableBlock { block: Arc>, - blobs: VerifiedBlobs, + blobs: Option>, } impl AvailableBlock { pub fn block(&self) -> &SignedBeaconBlock { &self.block } - - pub fn da_check_required(&self) -> bool { - match self.blobs { - VerifiedBlobs::PreDeneb | VerifiedBlobs::NotRequired => false, - VerifiedBlobs::EmptyBlobs | VerifiedBlobs::Available(_) => true, - } - } - - pub fn deconstruct(self) -> (Arc>, Option>) { - match self.blobs { - VerifiedBlobs::EmptyBlobs | VerifiedBlobs::NotRequired | VerifiedBlobs::PreDeneb => { - (self.block, None) - } - VerifiedBlobs::Available(blobs) => (self.block, Some(blobs)), - } - } - - pub fn blobs(&self) -> Option<&BlobSidecarList> { - match &self.blobs { - VerifiedBlobs::Available(blobs) => Some(blobs), - _ => None, - } - } -} - -impl AsBlock for AvailableBlock { - fn slot(&self) -> Slot { - self.block.slot() - } - - fn epoch(&self) -> Epoch { - self.block.epoch() - } - - fn parent_root(&self) -> Hash256 { - self.block.parent_root() - } - - fn state_root(&self) -> Hash256 { - self.block.state_root() - } - - fn signed_block_header(&self) -> SignedBeaconBlockHeader { - self.block.signed_block_header() - } - - fn message(&self) -> BeaconBlockRef { - self.block.message() - } - - fn as_block(&self) -> &SignedBeaconBlock { - &self.block - } - - fn block_cloned(&self) -> Arc> { + pub fn block_cloned(&self) -> Arc> { self.block.clone() } - fn canonical_root(&self) -> Hash256 { - self.block.canonical_root() - } - - fn into_block_wrapper(self) -> BlockWrapper { - let (block, blobs_opt) = self.deconstruct(); - if let Some(blobs) = blobs_opt { - let blobs_vec = blobs.iter().cloned().map(Option::Some).collect::>(); - BlockWrapper::BlockAndBlobs(block, FixedVector::from(blobs_vec)) - } else { - BlockWrapper::Block(block) - } - } -} - -// The standard implementation of Encode for SignedBeaconBlock -// requires us to use ssz(enum_behaviour = "transparent"). This -// prevents us from implementing Decode. We need to use a -// custom Encode and Decode in this wrapper object that essentially -// encodes it as if it were ssz(enum_behaviour = "union") -impl ssz::Encode for AvailabilityPendingBlock { - fn is_ssz_fixed_len() -> bool { - ssz_tagged_signed_beacon_block::encode::is_ssz_fixed_len() - } - - fn ssz_append(&self, buf: &mut Vec) { - ssz_tagged_signed_beacon_block::encode::ssz_append(self.block.as_ref(), buf); - } - - fn ssz_bytes_len(&self) -> usize { - ssz_tagged_signed_beacon_block::encode::ssz_bytes_len(self.block.as_ref()) - } -} - -impl ssz::Decode for AvailabilityPendingBlock { - fn is_ssz_fixed_len() -> bool { - ssz_tagged_signed_beacon_block::decode::is_ssz_fixed_len() + pub fn blobs(&self) -> Option<&BlobSidecarList> { + self.blobs.as_ref() } - fn from_ssz_bytes(bytes: &[u8]) -> Result { - Ok(Self { - block: Arc::new(ssz_tagged_signed_beacon_block::decode::from_ssz_bytes( - bytes, - )?), - }) + pub fn deconstruct(self) -> (Arc>, Option>) { + let AvailableBlock { block, blobs } = self; + (block, blobs) } } -#[cfg(test)] -mod test { - #[test] - fn check_encode_decode_availability_pending_block() { - // todo.. (difficult to create default beacon blocks to test) - } +#[derive(Debug, Clone)] +pub enum MaybeAvailableBlock { + /// This variant is fully available. + /// i.e. for pre-deneb blocks, it contains a (`SignedBeaconBlock`, `Blobs::None`) and for + /// post-4844 blocks, it contains a `SignedBeaconBlock` and a Blobs variant other than `Blobs::None`. + Available(AvailableBlock), + /// This variant is not fully available and requires blobs to become fully available. + AvailabilityPending(Arc>), } 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 6b99e62dea2..0b40df73a69 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 @@ -29,8 +29,10 @@ use crate::beacon_chain::BeaconStore; use crate::blob_verification::KzgVerifiedBlob; -use crate::block_verification::{AvailabilityPendingExecutedBlock, AvailableExecutedBlock}; -use crate::data_availability_checker::{Availability, AvailabilityCheckError}; +use crate::block_verification_types::{ + AsBlock, AvailabilityPendingExecutedBlock, AvailableExecutedBlock, +}; +use crate::data_availability_checker::{make_available, Availability, AvailabilityCheckError}; use crate::store::{DBColumn, KeyValueStore}; use crate::BeaconChainTypes; use lru::LruCache; @@ -102,7 +104,7 @@ impl PendingComponents { pub fn epoch(&self) -> Option { self.executed_block .as_ref() - .map(|pending_block| pending_block.block.as_block().epoch()) + .map(|pending_block| pending_block.block.epoch()) .or_else(|| { for maybe_blob in self.verified_blobs.iter() { if maybe_blob.is_some() { @@ -119,7 +121,7 @@ impl PendingComponents { let block_opt = self .executed_block .as_ref() - .map(|block| block.block.block.clone()); + .map(|block| block.block.clone()); let blobs = self .verified_blobs .iter() @@ -538,7 +540,8 @@ impl OverflowLRUCache { import_data, payload_verification_outcome, } = executed_block; - let available_block = block.make_available(vec![])?; + let available_block = + make_available(block, vec![])?; return Ok(Availability::Available(Box::new( AvailableExecutedBlock::new( available_block, @@ -588,7 +591,8 @@ impl OverflowLRUCache { return Ok(Availability::MissingComponents(import_data.block_root)) }; - let available_block = block.make_available(verified_blobs)?; + let available_block = + make_available(block, verified_blobs)?; Ok(Availability::Available(Box::new( AvailableExecutedBlock::new( available_block, @@ -758,7 +762,6 @@ impl OverflowLRUCache { value_bytes.as_slice(), )? .block - .as_block() .epoch() } OverflowKey::Blob(_, _) => { @@ -854,8 +857,7 @@ mod test { blob_verification::{ validate_blob_sidecar_for_gossip, verify_kzg_for_blob, GossipVerifiedBlob, }, - block_verification::{BlockImportData, PayloadVerificationOutcome}, - data_availability_checker::AvailabilityPendingBlock, + block_verification::PayloadVerificationOutcome, eth1_finalization_cache::Eth1FinalizationData, test_utils::{BaseHarnessType, BeaconChainHarness, DiskHarnessType}, }; @@ -1137,10 +1139,6 @@ mod test { }; let slot = block.slot(); - let apb: AvailabilityPendingBlock = AvailabilityPendingBlock { - block: Arc::new(block), - }; - let consensus_context = ConsensusContext::::new(slot); let import_data: BlockImportData = BlockImportData { block_root, @@ -1157,7 +1155,7 @@ mod test { }; let availability_pending_block = AvailabilityPendingExecutedBlock { - block: apb, + block: Arc::new(block), import_data, payload_verification_outcome, }; @@ -1473,7 +1471,7 @@ mod test { // we need blocks with blobs continue; } - let root = pending_block.block.as_block().canonical_root(); + let root = pending_block.block.canonical_root(); let epoch = pending_block .block .as_block() diff --git a/beacon_node/beacon_chain/src/lib.rs b/beacon_node/beacon_chain/src/lib.rs index f394cabe050..f7a9db11af7 100644 --- a/beacon_node/beacon_chain/src/lib.rs +++ b/beacon_node/beacon_chain/src/lib.rs @@ -12,6 +12,7 @@ pub mod blob_verification; pub mod block_reward; mod block_times_cache; mod block_verification; +pub mod block_verification_types; pub mod builder; pub mod canonical_head; pub mod capella_readiness; @@ -69,10 +70,12 @@ pub use self::historical_blocks::HistoricalBlockError; pub use attestation_verification::Error as AttestationError; pub use beacon_fork_choice_store::{BeaconForkChoiceStore, Error as ForkChoiceStoreError}; pub use block_verification::{ - get_block_root, AvailabilityPendingExecutedBlock, BlockError, ExecutedBlock, - ExecutionPayloadError, ExecutionPendingBlock, GossipVerifiedBlock, IntoExecutionPendingBlock, - IntoGossipVerifiedBlockContents, PayloadVerificationOutcome, PayloadVerificationStatus, + get_block_root, BlockError, ExecutionPayloadError, ExecutionPendingBlock, GossipVerifiedBlock, + IntoExecutionPendingBlock, IntoGossipVerifiedBlockContents, PayloadVerificationOutcome, + PayloadVerificationStatus, }; +pub use block_verification_types::AvailabilityPendingExecutedBlock; +pub use block_verification_types::ExecutedBlock; pub use canonical_head::{CachedHead, CanonicalHead, CanonicalHeadRwLock}; pub use eth1_chain::{Eth1Chain, Eth1ChainBackend}; pub use events::ServerSentEventHandler; diff --git a/beacon_node/beacon_chain/src/test_utils.rs b/beacon_node/beacon_chain/src/test_utils.rs index 15331313b95..b2cea41cb45 100644 --- a/beacon_node/beacon_chain/src/test_utils.rs +++ b/beacon_node/beacon_chain/src/test_utils.rs @@ -1,4 +1,4 @@ -use crate::blob_verification::{AsBlock, BlockWrapper}; +use crate::block_verification_types::{AsBlock, RpcBlock}; use crate::observed_operations::ObservationOutcome; pub use crate::persisted_beacon_chain::PersistedBeaconChain; pub use crate::{ @@ -689,18 +689,18 @@ where .execution_block_generator() } - pub fn get_head_block(&self) -> BlockWrapper { + pub fn get_head_block(&self) -> RpcBlock { let block = self.chain.head_beacon_block(); let block_root = block.canonical_root(); let blobs = self.chain.get_blobs(&block_root).unwrap(); - BlockWrapper::new(block, blobs) + RpcBlock::new(block, blobs).unwrap() } - pub fn get_full_block(&self, block_root: &Hash256) -> BlockWrapper { + pub fn get_full_block(&self, block_root: &Hash256) -> RpcBlock { let block = self.chain.get_blinded_block(block_root).unwrap().unwrap(); let full_block = self.chain.store.make_full_block(block_root, block).unwrap(); let blobs = self.chain.get_blobs(block_root).unwrap(); - BlockWrapper::new(Arc::new(full_block), blobs) + RpcBlock::new( Arc::new(full_block), blobs).unwrap() } pub fn get_all_validators(&self) -> Vec { @@ -1868,7 +1868,7 @@ where (deposits, state) } - pub async fn process_block>>( + pub async fn process_block>>( &self, slot: Slot, block_root: Hash256, @@ -1887,7 +1887,7 @@ where Ok(block_hash) } - pub async fn process_block_result>>( + pub async fn process_block_result>>( &self, block: B, ) -> Result> { @@ -1971,11 +1971,28 @@ where BlockError, > { self.set_current_slot(slot); - let (block, new_state) = self.make_block(state, slot).await; + let ((block, blobs), new_state) = self.make_block(state, slot).await; + // Note: we are just dropping signatures here and skipping signature verification. + let blobs_without_signatures = blobs.as_ref().map(|blobs| { + VariableList::from( + blobs + .into_iter() + .map(|blob| blob.message.clone()) + .collect::>(), + ) + }); let block_hash = self - .process_block(slot, block.0.canonical_root(), block.clone()) + .process_block( + slot, + block.canonical_root(), + RpcBlock::new( + Arc::new(block.clone()), + blobs_without_signatures.clone(), + ) + .unwrap(), + ) .await?; - Ok((block_hash, block, new_state)) + Ok((block_hash, (block, blobs), new_state)) } pub fn attest_block( diff --git a/beacon_node/beacon_chain/tests/attestation_production.rs b/beacon_node/beacon_chain/tests/attestation_production.rs index 97122c00043..db3ddf3bf09 100644 --- a/beacon_node/beacon_chain/tests/attestation_production.rs +++ b/beacon_node/beacon_chain/tests/attestation_production.rs @@ -1,6 +1,6 @@ #![cfg(not(debug_assertions))] -use beacon_chain::blob_verification::BlockWrapper; +use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy}; use beacon_chain::{StateSkipConfig, WhenSlotSkipped}; use lazy_static::lazy_static; @@ -133,11 +133,10 @@ async fn produces_attestations() { assert_eq!(data.target.epoch, state.current_epoch(), "bad target epoch"); assert_eq!(data.target.root, target_root, "bad target root"); - let block_wrapper = - BlockWrapper::::new(Arc::new(block.clone()), blobs.clone()); - let beacon_chain::blob_verification::MaybeAvailableBlock::Available(available_block) = chain + let rpc_block = RpcBlock::::new(Arc::new(block.clone()), blobs.clone()).unwrap(); + let beacon_chain::data_availability_checker::MaybeAvailableBlock::Available(available_block) = chain .data_availability_checker - .check_availability(block_wrapper) + .check_rpc_block_availability(rpc_block) .unwrap() else { panic!("block should be available") @@ -209,10 +208,10 @@ async fn early_attester_cache_old_request() { .get_blobs(&head.beacon_block_root) .expect("should get blobs"); - let block_wrapper = BlockWrapper::::new(head.beacon_block.clone(), head_blobs); - let beacon_chain::blob_verification::MaybeAvailableBlock::Available(available_block) = harness.chain + let rpc_block = RpcBlock::::new(head.beacon_block.clone(), head_blobs).unwrap(); + let beacon_chain::data_availability_checker::MaybeAvailableBlock::Available(available_block) = harness.chain .data_availability_checker - .check_availability(block_wrapper) + .check_rpc_block_availability(rpc_block) .unwrap() else { panic!("block should be available") diff --git a/beacon_node/beacon_chain/tests/block_verification.rs b/beacon_node/beacon_chain/tests/block_verification.rs index 37479236459..852e40834c4 100644 --- a/beacon_node/beacon_chain/tests/block_verification.rs +++ b/beacon_node/beacon_chain/tests/block_verification.rs @@ -1,12 +1,10 @@ #![cfg(not(debug_assertions))] -use beacon_chain::blob_verification::BlockWrapper; +use beacon_chain::block_verification_types::{AsBlock, ExecutedBlock, RpcBlock}; use beacon_chain::test_utils::BlobSignatureKey; use beacon_chain::{ - blob_verification::AsBlock, test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType}, - AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes, ExecutedBlock, - ExecutionPendingBlock, + AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes, ExecutionPendingBlock, }; use beacon_chain::{ BeaconSnapshot, BlockError, ChainSegmentResult, IntoExecutionPendingBlock, NotifyExecutionLayer, @@ -156,11 +154,11 @@ fn get_harness(validator_count: usize) -> BeaconChainHarness], blobs: &[Option>], -) -> Vec> { +) -> Vec> { chain_segment .iter() .zip(blobs.into_iter()) - .map(|(snapshot, blobs)| BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone())) + .map(|(snapshot, blobs)| RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()) .collect() } @@ -217,7 +215,7 @@ fn update_parent_roots(snapshots: &mut [BeaconSnapshot]) { async fn chain_segment_full_segment() { let harness = get_harness(VALIDATOR_COUNT); let (chain_segment, chain_segment_blobs) = get_chain_segment().await; - let blocks: Vec> = chain_segment_blocks(&chain_segment, &chain_segment_blobs) + let blocks: Vec> = chain_segment_blocks(&chain_segment, &chain_segment_blobs) .into_iter() .map(|block| block.into()) .collect(); @@ -256,11 +254,10 @@ async fn chain_segment_varying_chunk_size() { for chunk_size in &[1, 2, 3, 5, 31, 32, 33, 42] { let harness = get_harness(VALIDATOR_COUNT); let (chain_segment, chain_segment_blobs) = get_chain_segment().await; - let blocks: Vec> = - chain_segment_blocks(&chain_segment, &chain_segment_blobs) - .into_iter() - .map(|block| block.into()) - .collect(); + let blocks: Vec> = chain_segment_blocks(&chain_segment, &chain_segment_blobs) + .into_iter() + .map(|block| block.into()) + .collect(); harness .chain @@ -299,11 +296,10 @@ async fn chain_segment_non_linear_parent_roots() { /* * Test with a block removed. */ - let mut blocks: Vec> = - chain_segment_blocks(&chain_segment, &chain_segment_blobs) - .into_iter() - .map(|block| block.into()) - .collect(); + let mut blocks: Vec> = chain_segment_blocks(&chain_segment, &chain_segment_blobs) + .into_iter() + .map(|block| block.into()) + .collect(); blocks.remove(2); assert!( @@ -321,11 +317,10 @@ async fn chain_segment_non_linear_parent_roots() { /* * Test with a modified parent root. */ - let mut blocks: Vec> = - chain_segment_blocks(&chain_segment, &chain_segment_blobs) - .into_iter() - .map(|block| block.into()) - .collect(); + let mut blocks: Vec> = chain_segment_blocks(&chain_segment, &chain_segment_blobs) + .into_iter() + .map(|block| block.into()) + .collect(); let (mut block, signature) = blocks[3].as_block().clone().deconstruct(); *block.parent_root_mut() = Hash256::zero(); @@ -357,11 +352,10 @@ async fn chain_segment_non_linear_slots() { * Test where a child is lower than the parent. */ - let mut blocks: Vec> = - chain_segment_blocks(&chain_segment, &chain_segment_blobs) - .into_iter() - .map(|block| block.into()) - .collect(); + let mut blocks: Vec> = chain_segment_blocks(&chain_segment, &chain_segment_blobs) + .into_iter() + .map(|block| block.into()) + .collect(); let (mut block, signature) = blocks[3].as_block().clone().deconstruct(); *block.slot_mut() = Slot::new(0); blocks[3] = Arc::new(SignedBeaconBlock::from_block(block, signature)).into(); @@ -382,11 +376,10 @@ async fn chain_segment_non_linear_slots() { * Test where a child is equal to the parent. */ - let mut blocks: Vec> = - chain_segment_blocks(&chain_segment, &chain_segment_blobs) - .into_iter() - .map(|block| block.into()) - .collect(); + let mut blocks: Vec> = chain_segment_blocks(&chain_segment, &chain_segment_blobs) + .into_iter() + .map(|block| block.into()) + .collect(); let (mut block, signature) = blocks[3].as_block().clone().deconstruct(); *block.slot_mut() = blocks[2].slot(); blocks[3] = Arc::new(SignedBeaconBlock::from_block(block, signature)).into(); @@ -412,10 +405,10 @@ async fn assert_invalid_signature( snapshots: &[BeaconSnapshot], item: &str, ) { - let blocks: Vec> = snapshots + let blocks: Vec> = snapshots .iter() .zip(chain_segment_blobs.iter()) - .map(|(snapshot, blobs)| BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone())) + .map(|(snapshot, blobs)| RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()) .collect(); // Ensure the block will be rejected if imported in a chain segment. @@ -440,7 +433,7 @@ async fn assert_invalid_signature( .iter() .take(block_index) .zip(chain_segment_blobs.iter()) - .map(|(snapshot, blobs)| BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone())) + .map(|(snapshot, blobs)| RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()) .collect(); // We don't care if this fails, we just call this to ensure that all prior blocks have been // imported prior to this test. @@ -454,10 +447,10 @@ async fn assert_invalid_signature( .chain .process_block( snapshots[block_index].beacon_block.canonical_root(), - BlockWrapper::new( + RpcBlock::new( snapshots[block_index].beacon_block.clone(), chain_segment_blobs[block_index].clone(), - ), + ).unwrap(), NotifyExecutionLayer::Yes, || Ok(()), ) @@ -508,9 +501,7 @@ async fn invalid_signature_gossip_block() { .iter() .take(block_index) .zip(chain_segment_blobs.iter()) - .map(|(snapshot, blobs)| { - BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone()) - }) + .map(|(snapshot, blobs)| RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()) .collect(); harness .chain @@ -552,12 +543,10 @@ async fn invalid_signature_block_proposal() { block.clone(), junk_signature(), )); - let blocks: Vec> = snapshots + let blocks: Vec> = snapshots .iter() .zip(chain_segment_blobs.iter()) - .map(|(snapshot, blobs)| { - BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone()) - }) + .map(|(snapshot, blobs)| RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()) .collect::>(); // Ensure the block will be rejected if imported in a chain segment. assert!( @@ -765,12 +754,10 @@ async fn invalid_signature_deposit() { Arc::new(SignedBeaconBlock::from_block(block, signature)); update_parent_roots(&mut snapshots); update_proposal_signatures(&mut snapshots, &harness); - let blocks: Vec> = snapshots + let blocks: Vec> = snapshots .iter() .zip(chain_segment_blobs.iter()) - .map(|(snapshot, blobs)| { - BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone()) - }) + .map(|(snapshot, blobs)| RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()) .collect(); assert!( !matches!( diff --git a/beacon_node/beacon_chain/tests/store_tests.rs b/beacon_node/beacon_chain/tests/store_tests.rs index 7a86b5f9382..d2b6a954520 100644 --- a/beacon_node/beacon_chain/tests/store_tests.rs +++ b/beacon_node/beacon_chain/tests/store_tests.rs @@ -1,7 +1,7 @@ #![cfg(not(debug_assertions))] use beacon_chain::attestation_verification::Error as AttnError; -use beacon_chain::blob_verification::BlockWrapper; +use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::builder::BeaconChainBuilder; use beacon_chain::schema_change::migrate_schema; use beacon_chain::test_utils::{ @@ -10,7 +10,7 @@ use beacon_chain::test_utils::{ }; use beacon_chain::validator_monitor::DEFAULT_INDIVIDUAL_TRACKING_THRESHOLD; use beacon_chain::{ - blob_verification::MaybeAvailableBlock, historical_blocks::HistoricalBlockError, + data_availability_checker::MaybeAvailableBlock, historical_blocks::HistoricalBlockError, migrate::MigratorConfig, BeaconChain, BeaconChainError, BeaconChainTypes, BeaconSnapshot, ChainConfig, NotifyExecutionLayer, ServerSentEventHandler, WhenSlotSkipped, }; @@ -2176,7 +2176,7 @@ async fn weak_subjectivity_sync() { beacon_chain .process_block( full_block.canonical_root(), - BlockWrapper::new(Arc::new(full_block), blobs), + RpcBlock::new(Arc::new(full_block), blobs).unwrap(), NotifyExecutionLayer::Yes, || Ok(()), ) @@ -2236,7 +2236,7 @@ async fn weak_subjectivity_sync() { if let MaybeAvailableBlock::Available(block) = harness .chain .data_availability_checker - .check_availability(BlockWrapper::new(Arc::new(full_block), blobs)) + .check_rpc_block_availability(RpcBlock::new(Arc::new(full_block), blobs).unwrap()) .expect("should check availability") { available_blocks.push(block); diff --git a/beacon_node/http_api/src/publish_blocks.rs b/beacon_node/http_api/src/publish_blocks.rs index 54251608105..38c40b890db 100644 --- a/beacon_node/http_api/src/publish_blocks.rs +++ b/beacon_node/http_api/src/publish_blocks.rs @@ -1,6 +1,6 @@ use crate::metrics; -use beacon_chain::blob_verification::AsBlock; +use beacon_chain::block_verification_types::AsBlock; use beacon_chain::validator_monitor::{get_block_delay_ms, timestamp_now}; use beacon_chain::{ AvailabilityProcessingStatus, BeaconChain, BeaconChainError, BeaconChainTypes, BlockError, diff --git a/beacon_node/network/src/sync/backfill_sync/mod.rs b/beacon_node/network/src/sync/backfill_sync/mod.rs index e5a1428005a..0900e034f00 100644 --- a/beacon_node/network/src/sync/backfill_sync/mod.rs +++ b/beacon_node/network/src/sync/backfill_sync/mod.rs @@ -14,7 +14,7 @@ use crate::sync::network_context::SyncNetworkContext; use crate::sync::range_sync::{ BatchConfig, BatchId, BatchInfo, BatchOperationOutcome, BatchProcessingResult, BatchState, }; -use beacon_chain::blob_verification::BlockWrapper; +use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::{BeaconChain, BeaconChainTypes}; use lighthouse_network::types::{BackFillState, NetworkGlobals}; use lighthouse_network::{PeerAction, PeerId}; @@ -55,7 +55,7 @@ impl BatchConfig for BackFillBatchConfig { fn max_batch_processing_attempts() -> u8 { MAX_BATCH_PROCESSING_ATTEMPTS } - fn batch_attempt_hash(blocks: &[BlockWrapper]) -> u64 { + fn batch_attempt_hash(blocks: &[RpcBlock]) -> u64 { use std::collections::hash_map::DefaultHasher; use std::hash::{Hash, Hasher}; let mut hasher = DefaultHasher::new(); @@ -392,7 +392,7 @@ impl BackFillSync { batch_id: BatchId, peer_id: &PeerId, request_id: Id, - beacon_block: Option>, + beacon_block: Option>, ) -> Result { // check if we have this batch let batch = match self.batches.get_mut(&batch_id) { diff --git a/beacon_node/network/src/sync/block_lookups/parent_lookup.rs b/beacon_node/network/src/sync/block_lookups/parent_lookup.rs index 5175450d9e0..6d870b5aba3 100644 --- a/beacon_node/network/src/sync/block_lookups/parent_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/parent_lookup.rs @@ -3,8 +3,8 @@ use super::{BlobRequestId, BlockRequestId, DownloadedBlocks, PeerShouldHave, Res use crate::sync::block_lookups::single_block_lookup::{State, UnknownParentComponents}; use crate::sync::block_lookups::{RootBlobsTuple, RootBlockTuple}; use crate::sync::{manager::SLOT_IMPORT_TOLERANCE, network_context::SyncNetworkContext}; -use beacon_chain::blob_verification::AsBlock; -use beacon_chain::blob_verification::BlockWrapper; +use beacon_chain::block_verification_types::AsBlock; +use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::data_availability_checker::DataAvailabilityChecker; use beacon_chain::BeaconChainTypes; use lighthouse_network::PeerId; @@ -147,7 +147,7 @@ impl ParentLookup { .check_peer_disconnected(peer_id) } - pub fn add_unknown_parent_block(&mut self, block: BlockWrapper) { + pub fn add_unknown_parent_block(&mut self, block: RpcBlock) { let next_parent = block.parent_root(); // Cache the block. @@ -203,7 +203,7 @@ impl ParentLookup { self, ) -> ( Hash256, - Vec>, + Vec>, Vec, SingleBlockLookup, ) { diff --git a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs index b6e0cef8365..37b5ee0e767 100644 --- a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs @@ -1,6 +1,6 @@ use crate::sync::block_lookups::{BlobRequestId, BlockRequestId, RootBlobsTuple, RootBlockTuple}; use crate::sync::network_context::SyncNetworkContext; -use beacon_chain::blob_verification::BlockWrapper; +use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::data_availability_checker::DataAvailabilityChecker; use beacon_chain::{get_block_root, BeaconChainTypes}; use lighthouse_network::rpc::methods::BlobsByRootRequest; @@ -138,6 +138,16 @@ pub struct UnknownParentComponents { pub downloaded_blobs: FixedBlobSidecarList, } +impl From> for UnknownParentComponents { + fn from(value: RpcBlock) -> Self { + let (block, blobs) = value.deconstruct(); + let fixed_blobs = blobs.map(|blobs| { + FixedBlobSidecarList::from(blobs.into_iter().map(Some).collect::>()) + }); + Self::new(Some(block), fixed_blobs) + } +} + impl UnknownParentComponents { pub fn new( block: Option>>, @@ -284,7 +294,7 @@ impl SingleBlockLookup Option> { + pub fn get_downloaded_block(&mut self) -> Option> { self.unknown_parent_components .as_mut() .and_then(|components| { @@ -302,9 +312,14 @@ impl SingleBlockLookup>(); + let blobs = VariableList::from(filtered); + RpcBlock::new(block.clone(), Some(blobs)).ok() + }) } else { None } diff --git a/beacon_node/network/src/sync/block_lookups/tests.rs b/beacon_node/network/src/sync/block_lookups/tests.rs index c8f19b16981..482254d9ced 100644 --- a/beacon_node/network/src/sync/block_lookups/tests.rs +++ b/beacon_node/network/src/sync/block_lookups/tests.rs @@ -1474,7 +1474,7 @@ mod deneb_only { fn parent_block_unknown_parent(mut self) -> Self { self.bl.parent_block_processed( self.block_root, - BlockProcessingResult::Err(BlockError::ParentUnknown(BlockWrapper::Block( + BlockProcessingResult::Err(BlockError::ParentUnknown(RpcBlock::new_without_blobs( self.parent_block.clone().expect("parent block"), ))), ResponseType::Block, diff --git a/beacon_node/network/src/sync/block_sidecar_coupling.rs b/beacon_node/network/src/sync/block_sidecar_coupling.rs index 7e5362a6f0d..b25dcdf334e 100644 --- a/beacon_node/network/src/sync/block_sidecar_coupling.rs +++ b/beacon_node/network/src/sync/block_sidecar_coupling.rs @@ -1,5 +1,5 @@ -use beacon_chain::blob_verification::BlockWrapper; -use ssz_types::FixedVector; +use beacon_chain::block_verification_types::RpcBlock; +use ssz_types::{VariableList}; use std::{collections::VecDeque, sync::Arc}; use types::{BlobSidecar, EthSpec, SignedBeaconBlock}; @@ -16,28 +16,28 @@ pub struct BlocksAndBlobsRequestInfo { } impl BlocksAndBlobsRequestInfo { - pub fn add_block_response(&mut self, maybe_block: Option>>) { - match maybe_block { + pub fn add_block_response(&mut self, block_opt: Option>>) { + match block_opt { Some(block) => self.accumulated_blocks.push_back(block), None => self.is_blocks_stream_terminated = true, } } - pub fn add_sidecar_response(&mut self, maybe_sidecar: Option>>) { - match maybe_sidecar { + pub fn add_sidecar_response(&mut self, sidecar_opt: Option>>) { + match sidecar_opt { Some(sidecar) => self.accumulated_sidecars.push_back(sidecar), None => self.is_sidecars_stream_terminated = true, } } - pub fn into_responses(self) -> Result>, &'static str> { + pub fn into_responses(self) -> Result>, &'static str> { let BlocksAndBlobsRequestInfo { accumulated_blocks, accumulated_sidecars, .. } = self; - // ASSUMPTION: There can't be more more blobs than blocks. i.e. sending any blob (empty + // There can't be more more blobs than blocks. i.e. sending any blob (empty // included) for a skipped slot is not permitted. let mut responses = Vec::with_capacity(accumulated_blocks.len()); let mut blob_iter = accumulated_sidecars.into_iter().peekable(); @@ -50,29 +50,23 @@ impl BlocksAndBlobsRequestInfo { .unwrap_or(false); pair_next_blob } { - blob_list.push(blob_iter.next().expect("iterator is not empty")); + blob_list.push(blob_iter.next().ok_or("Missing next blob")?); } - if blob_list.is_empty() { - responses.push(BlockWrapper::Block(block)) - } else { - let mut blobs_fixed = vec![None; T::max_blobs_per_block()]; - for blob in blob_list { - let blob_index = blob.index as usize; - let Some(blob_opt) = blobs_fixed.get_mut(blob_index) else { + let mut blobs_buffer = vec![None; T::max_blobs_per_block()]; + for blob in blob_list { + let blob_index = blob.index as usize; + let Some(blob_opt) = blobs_buffer.get_mut(blob_index) else { return Err("Invalid blob index"); }; - if blob_opt.is_some() { - return Err("Repeat blob index"); - } else { - *blob_opt = Some(blob); - } + if blob_opt.is_some() { + return Err("Repeat blob index"); + } else { + *blob_opt = Some(blob); } - responses.push(BlockWrapper::BlockAndBlobs( - block, - FixedVector::from(blobs_fixed), - )) } + let blobs = VariableList::from(blobs_buffer.into_iter().flatten().collect::>()); + responses.push(RpcBlock::new(block, Some(blobs))?) } // if accumulated sidecars is not empty, throw an error. diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 8bdf57e2a41..5e8fc4a4e9e 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -46,8 +46,8 @@ use crate::sync::block_lookups::delayed_lookup::DelayedLookupMessage; pub use crate::sync::block_lookups::ResponseType; use crate::sync::block_lookups::UnknownParentComponents; use crate::sync::range_sync::ByRangeRequestType; -use beacon_chain::blob_verification::AsBlock; -use beacon_chain::blob_verification::BlockWrapper; +use beacon_chain::block_verification_types::AsBlock; +use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::{ AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes, BlockError, EngineState, MAXIMUM_GOSSIP_CLOCK_DISPARITY, @@ -614,15 +614,13 @@ impl SyncManager { } => self.rpc_blob_received(request_id, peer_id, blob_sidecar, seen_timestamp), SyncMessage::UnknownParentBlock(peer_id, block, block_root) => { let block_slot = block.slot(); - let (block, blobs) = block.deconstruct(); let parent_root = block.parent_root(); - let parent_components = UnknownParentComponents::new(Some(block), blobs); self.handle_unknown_parent( peer_id, block_root, parent_root, block_slot, - Some(parent_components), + Some(block.into()), ); } SyncMessage::UnknownParentBlob(peer_id, blob) => { @@ -910,7 +908,7 @@ impl SyncManager { batch_id, &peer_id, id, - block.map(BlockWrapper::Block), + block.map(Into::into), ) { Ok(ProcessResult::SyncCompleted) => self.update_sync_state(), Ok(ProcessResult::Successful) => {} @@ -934,7 +932,7 @@ impl SyncManager { chain_id, batch_id, id, - block.map(BlockWrapper::Block), + block.map(Into::into), ); self.update_sync_state(); } diff --git a/beacon_node/network/src/sync/network_context.rs b/beacon_node/network/src/sync/network_context.rs index 7c162f478c5..d635dd2ea18 100644 --- a/beacon_node/network/src/sync/network_context.rs +++ b/beacon_node/network/src/sync/network_context.rs @@ -8,7 +8,7 @@ use crate::network_beacon_processor::NetworkBeaconProcessor; use crate::service::{NetworkMessage, RequestId}; use crate::status::ToStatusMessage; use crate::sync::block_lookups::{BlobRequestId, BlockRequestId}; -use beacon_chain::blob_verification::BlockWrapper; +use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::{BeaconChain, BeaconChainTypes, EngineState}; use fnv::FnvHashMap; use lighthouse_network::rpc::methods::{BlobsByRangeRequest, BlobsByRootRequest}; @@ -22,7 +22,7 @@ use types::{BlobSidecar, EthSpec, SignedBeaconBlock}; pub struct BlocksAndBlobsByRangeResponse { pub batch_id: BatchId, - pub responses: Result>, &'static str>, + pub responses: Result>, &'static str>, } pub struct BlocksAndBlobsByRangeRequest { diff --git a/beacon_node/network/src/sync/range_sync/batch.rs b/beacon_node/network/src/sync/range_sync/batch.rs index 138d320965e..f5c320cb880 100644 --- a/beacon_node/network/src/sync/range_sync/batch.rs +++ b/beacon_node/network/src/sync/range_sync/batch.rs @@ -1,5 +1,5 @@ use crate::sync::manager::Id; -use beacon_chain::blob_verification::{AsBlock, BlockWrapper}; +use beacon_chain::block_verification_types::{AsBlock, RpcBlock}; use lighthouse_network::rpc::methods::BlocksByRangeRequest; use lighthouse_network::PeerId; use std::collections::HashSet; @@ -56,7 +56,7 @@ pub trait BatchConfig { /// Note that simpler hashing functions considered in the past (hash of first block, hash of last /// block, number of received blocks) are not good enough to differentiate attempts. For this /// reason, we hash the complete set of blocks both in RangeSync and BackFillSync. - fn batch_attempt_hash(blocks: &[BlockWrapper]) -> u64; + fn batch_attempt_hash(blocks: &[RpcBlock]) -> u64; } pub struct RangeSyncBatchConfig {} @@ -68,7 +68,7 @@ impl BatchConfig for RangeSyncBatchConfig { fn max_batch_processing_attempts() -> u8 { MAX_BATCH_PROCESSING_ATTEMPTS } - fn batch_attempt_hash(blocks: &[BlockWrapper]) -> u64 { + fn batch_attempt_hash(blocks: &[RpcBlock]) -> u64 { let mut hasher = std::collections::hash_map::DefaultHasher::new(); blocks.hash(&mut hasher); hasher.finish() @@ -116,9 +116,9 @@ pub enum BatchState { /// The batch has failed either downloading or processing, but can be requested again. AwaitingDownload, /// The batch is being downloaded. - Downloading(PeerId, Vec>, Id), + Downloading(PeerId, Vec>, Id), /// The batch has been completely downloaded and is ready for processing. - AwaitingProcessing(PeerId, Vec>), + AwaitingProcessing(PeerId, Vec>), /// The batch is being processed. Processing(Attempt), /// The batch was successfully processed and is waiting to be validated. @@ -251,7 +251,7 @@ impl BatchInfo { } /// Adds a block to a downloading batch. - pub fn add_block(&mut self, block: BlockWrapper) -> Result<(), WrongState> { + pub fn add_block(&mut self, block: RpcBlock) -> Result<(), WrongState> { match self.state.poison() { BatchState::Downloading(peer, mut blocks, req_id) => { blocks.push(block); @@ -383,7 +383,7 @@ impl BatchInfo { } } - pub fn start_processing(&mut self) -> Result>, WrongState> { + pub fn start_processing(&mut self) -> Result>, WrongState> { match self.state.poison() { BatchState::AwaitingProcessing(peer, blocks) => { self.state = BatchState::Processing(Attempt::new::(peer, &blocks)); @@ -481,7 +481,7 @@ pub struct Attempt { } impl Attempt { - fn new(peer_id: PeerId, blocks: &[BlockWrapper]) -> Self { + fn new(peer_id: PeerId, blocks: &[RpcBlock]) -> Self { let hash = B::batch_attempt_hash(blocks); Attempt { peer_id, hash } } diff --git a/beacon_node/network/src/sync/range_sync/chain.rs b/beacon_node/network/src/sync/range_sync/chain.rs index 3b3cdb6ae3b..4d399b5cb99 100644 --- a/beacon_node/network/src/sync/range_sync/chain.rs +++ b/beacon_node/network/src/sync/range_sync/chain.rs @@ -3,7 +3,7 @@ use crate::network_beacon_processor::ChainSegmentProcessId; use crate::sync::{ manager::Id, network_context::SyncNetworkContext, BatchOperationOutcome, BatchProcessResult, }; -use beacon_chain::blob_verification::BlockWrapper; +use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::BeaconChainTypes; use fnv::FnvHashMap; use lighthouse_network::{PeerAction, PeerId}; @@ -221,7 +221,7 @@ impl SyncingChain { batch_id: BatchId, peer_id: &PeerId, request_id: Id, - beacon_block: Option>, + beacon_block: Option>, ) -> ProcessingResult { // check if we have this batch let batch = match self.batches.get_mut(&batch_id) { diff --git a/beacon_node/network/src/sync/range_sync/range.rs b/beacon_node/network/src/sync/range_sync/range.rs index 09a85208d96..f69a30b3ee0 100644 --- a/beacon_node/network/src/sync/range_sync/range.rs +++ b/beacon_node/network/src/sync/range_sync/range.rs @@ -47,7 +47,7 @@ use crate::status::ToStatusMessage; use crate::sync::manager::Id; use crate::sync::network_context::SyncNetworkContext; use crate::sync::BatchProcessResult; -use beacon_chain::blob_verification::BlockWrapper; +use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::{BeaconChain, BeaconChainTypes}; use lighthouse_network::rpc::GoodbyeReason; use lighthouse_network::PeerId; @@ -210,7 +210,7 @@ where chain_id: ChainId, batch_id: BatchId, request_id: Id, - beacon_block: Option>, + beacon_block: Option>, ) { // check if this chunk removes the chain match self.chains.call_by_id(chain_id, |chain| { @@ -394,7 +394,6 @@ mod tests { use lighthouse_network::{rpc::StatusMessage, NetworkGlobals}; use slog::{o, Drain}; use tokio::sync::mpsc; - use beacon_chain::test_utils::{BeaconChainHarness, EphemeralHarnessType}; use slot_clock::{TestingSlotClock, }; use std::collections::HashSet; diff --git a/consensus/fork_choice/tests/tests.rs b/consensus/fork_choice/tests/tests.rs index 88fb7d8965e..bba7ccdbe67 100644 --- a/consensus/fork_choice/tests/tests.rs +++ b/consensus/fork_choice/tests/tests.rs @@ -11,6 +11,7 @@ use beacon_chain::{ BeaconChain, BeaconChainError, BeaconForkChoiceStore, ChainConfig, ForkChoiceError, StateSkipConfig, WhenSlotSkipped, }; +use beacon_chain::block_verification_types::RpcBlock; use fork_choice::{ ForkChoiceStore, InvalidAttestation, InvalidBlock, PayloadVerificationStatus, QueuedAttestation, }; @@ -200,7 +201,7 @@ impl ForkChoiceTest { if !predicate(block.0.message(), &state) { break; } - if let Ok(block_hash) = self.harness.process_block_result(block.clone()).await { + if let Ok(block_hash) = self.harness.process_block_result(RpcBlock::new(block.0, block.1).unwrap()).await { self.harness.attest_block( &state, block.0.state_root(), diff --git a/consensus/state_processing/src/consensus_context.rs b/consensus/state_processing/src/consensus_context.rs index 78803ab4eb4..8e49a0d4983 100644 --- a/consensus/state_processing/src/consensus_context.rs +++ b/consensus/state_processing/src/consensus_context.rs @@ -21,8 +21,6 @@ pub struct ConsensusContext { #[ssz(skip_serializing, skip_deserializing)] indexed_attestations: HashMap<(AttestationData, BitList), IndexedAttestation>, - /// Whether `verify_kzg_commitments_against_transactions` has successfully passed. - kzg_commitments_consistent: bool, } #[derive(Debug, PartialEq, Clone)] @@ -45,7 +43,6 @@ impl ConsensusContext { proposer_index: None, current_block_root: None, indexed_attestations: HashMap::new(), - kzg_commitments_consistent: false, } } @@ -161,13 +158,4 @@ impl ConsensusContext { pub fn num_cached_indexed_attestations(&self) -> usize { self.indexed_attestations.len() } - - pub fn set_kzg_commitments_consistent(mut self, kzg_commitments_consistent: bool) -> Self { - self.kzg_commitments_consistent = kzg_commitments_consistent; - self - } - - pub fn kzg_commitments_consistent(&self) -> bool { - self.kzg_commitments_consistent - } } diff --git a/consensus/types/src/lib.rs b/consensus/types/src/lib.rs index fe07a9c0ff8..def9f30f8d0 100644 --- a/consensus/types/src/lib.rs +++ b/consensus/types/src/lib.rs @@ -169,9 +169,10 @@ pub use crate::selection_proof::SelectionProof; pub use crate::shuffling_id::AttestationShufflingId; pub use crate::signed_aggregate_and_proof::SignedAggregateAndProof; pub use crate::signed_beacon_block::{ - ssz_tagged_signed_beacon_block, SignedBeaconBlock, SignedBeaconBlockAltair, - SignedBeaconBlockBase, SignedBeaconBlockCapella, SignedBeaconBlockDeneb, SignedBeaconBlockHash, - SignedBeaconBlockMerge, SignedBlindedBeaconBlock, + ssz_tagged_signed_beacon_block, ssz_tagged_signed_beacon_block_arc, SignedBeaconBlock, + SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockCapella, + SignedBeaconBlockDeneb, SignedBeaconBlockHash, SignedBeaconBlockMerge, + SignedBlindedBeaconBlock, }; pub use crate::signed_beacon_block_header::SignedBeaconBlockHeader; pub use crate::signed_blob::*; diff --git a/consensus/types/src/signed_beacon_block.rs b/consensus/types/src/signed_beacon_block.rs index cf7fd6819ea..c52ba11d278 100644 --- a/consensus/types/src/signed_beacon_block.rs +++ b/consensus/types/src/signed_beacon_block.rs @@ -642,6 +642,27 @@ pub mod ssz_tagged_signed_beacon_block { } } +pub mod ssz_tagged_signed_beacon_block_arc { + use super::*; + pub mod encode { + pub use super::ssz_tagged_signed_beacon_block::encode::*; + } + + pub mod decode { + pub use super::ssz_tagged_signed_beacon_block::decode::{is_ssz_fixed_len, ssz_fixed_len}; + use super::*; + #[allow(unused_imports)] + use ssz::*; + use std::sync::Arc; + + pub fn from_ssz_bytes>( + bytes: &[u8], + ) -> Result>, DecodeError> { + ssz_tagged_signed_beacon_block::decode::from_ssz_bytes(bytes).map(Arc::new) + } + } +} + #[cfg(test)] mod test { use super::*; From e3ee0c6cceb22db9684a8e8bef5667f0e4e369f1 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Fri, 14 Jul 2023 17:29:41 -0400 Subject: [PATCH 03/18] cargo fmt --- .../beacon_chain/src/block_verification.rs | 8 ++---- .../src/block_verification_types.rs | 6 ++--- .../src/data_availability_checker.rs | 7 +++-- .../overflow_lru_cache.rs | 6 ++--- beacon_node/beacon_chain/src/test_utils.rs | 8 ++---- .../tests/attestation_production.rs | 3 ++- .../beacon_chain/tests/block_verification.rs | 27 ++++++++++++++----- .../src/network_beacon_processor/mod.rs | 20 +++----------- .../network_beacon_processor/sync_methods.rs | 7 ++--- .../src/network_beacon_processor/tests.rs | 18 +++++++------ beacon_node/network/src/router.rs | 16 ++++------- beacon_node/network/src/service/tests.rs | 2 +- .../src/sync/block_lookups/delayed_lookup.rs | 6 ++--- .../sync/block_lookups/single_block_lookup.rs | 17 +++++++----- .../src/sync/block_sidecar_coupling.rs | 2 +- .../network/src/sync/range_sync/range.rs | 6 ++--- consensus/fork_choice/tests/tests.rs | 8 ++++-- 17 files changed, 83 insertions(+), 84 deletions(-) diff --git a/beacon_node/beacon_chain/src/block_verification.rs b/beacon_node/beacon_chain/src/block_verification.rs index dfe388417f9..c9e0ee4c9d2 100644 --- a/beacon_node/beacon_chain/src/block_verification.rs +++ b/beacon_node/beacon_chain/src/block_verification.rs @@ -1161,12 +1161,8 @@ impl IntoExecutionPendingBlock for Arc &SignedBeaconBlock { diff --git a/beacon_node/beacon_chain/src/block_verification_types.rs b/beacon_node/beacon_chain/src/block_verification_types.rs index a152b757df3..d88275b18ae 100644 --- a/beacon_node/beacon_chain/src/block_verification_types.rs +++ b/beacon_node/beacon_chain/src/block_verification_types.rs @@ -1,6 +1,7 @@ use crate::blob_verification::GossipVerifiedBlobList; use crate::data_availability_checker::AvailabilityCheckError; pub use crate::data_availability_checker::{AvailableBlock, MaybeAvailableBlock}; +use crate::eth1_finalization_cache::Eth1FinalizationData; use crate::{data_availability_checker, GossipVerifiedBlock, PayloadVerificationOutcome}; use derivative::Derivative; use ssz_derive::{Decode, Encode}; @@ -14,7 +15,6 @@ use types::{ BeaconBlockRef, BeaconState, BlindedPayload, BlobSidecarList, Epoch, EthSpec, Hash256, SignedBeaconBlock, SignedBeaconBlockHeader, Slot, }; -use crate::eth1_finalization_cache::Eth1FinalizationData; #[derive(Debug, Clone, Derivative)] #[derivative(Hash(bound = "E: EthSpec"))] @@ -44,7 +44,7 @@ impl RpcBlock { blobs: Option>, ) -> Result { if let Some(blobs) = blobs.as_ref() { - data_availability_checker::consistency_checks( &block, blobs)?; + data_availability_checker::consistency_checks(&block, blobs)?; } let inner = match blobs { Some(blobs) => RpcBlockInner::BlockAndBlobs(block, blobs), @@ -401,4 +401,4 @@ impl AsBlock for RpcBlock { fn into_rpc_block(self) -> RpcBlock { self } -} \ No newline at end of file +} diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index eb33f64e911..d4c4d3c3d4a 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -317,7 +317,7 @@ pub fn make_available( ) -> Result, AvailabilityCheckError> { let blobs = VariableList::new(blobs.into_iter().map(|blob| blob.to_blob()).collect())?; - consistency_checks( &block, &blobs)?; + consistency_checks(&block, &blobs)?; Ok(AvailableBlock { block, @@ -347,7 +347,10 @@ pub fn consistency_checks( return Ok(()); } - let block_root = blobs.first().map(|blob|blob.block_root).unwrap_or(block.canonical_root()); + let block_root = blobs + .first() + .map(|blob| blob.block_root) + .unwrap_or(block.canonical_root()); for (index, (block_commitment, blob)) in block_kzg_commitments.iter().zip(blobs.iter()).enumerate() { 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 0b40df73a69..014fb4597bc 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 @@ -540,8 +540,7 @@ impl OverflowLRUCache { import_data, payload_verification_outcome, } = executed_block; - let available_block = - make_available(block, vec![])?; + let available_block = make_available(block, vec![])?; return Ok(Availability::Available(Box::new( AvailableExecutedBlock::new( available_block, @@ -591,8 +590,7 @@ impl OverflowLRUCache { return Ok(Availability::MissingComponents(import_data.block_root)) }; - let available_block = - make_available(block, verified_blobs)?; + let available_block = make_available(block, verified_blobs)?; Ok(Availability::Available(Box::new( AvailableExecutedBlock::new( available_block, diff --git a/beacon_node/beacon_chain/src/test_utils.rs b/beacon_node/beacon_chain/src/test_utils.rs index b2cea41cb45..ae5f87fe83e 100644 --- a/beacon_node/beacon_chain/src/test_utils.rs +++ b/beacon_node/beacon_chain/src/test_utils.rs @@ -700,7 +700,7 @@ where let block = self.chain.get_blinded_block(block_root).unwrap().unwrap(); let full_block = self.chain.store.make_full_block(block_root, block).unwrap(); let blobs = self.chain.get_blobs(block_root).unwrap(); - RpcBlock::new( Arc::new(full_block), blobs).unwrap() + RpcBlock::new(Arc::new(full_block), blobs).unwrap() } pub fn get_all_validators(&self) -> Vec { @@ -1985,11 +1985,7 @@ where .process_block( slot, block.canonical_root(), - RpcBlock::new( - Arc::new(block.clone()), - blobs_without_signatures.clone(), - ) - .unwrap(), + RpcBlock::new(Arc::new(block.clone()), blobs_without_signatures.clone()).unwrap(), ) .await?; Ok((block_hash, (block, blobs), new_state)) diff --git a/beacon_node/beacon_chain/tests/attestation_production.rs b/beacon_node/beacon_chain/tests/attestation_production.rs index db3ddf3bf09..907e7a40bb5 100644 --- a/beacon_node/beacon_chain/tests/attestation_production.rs +++ b/beacon_node/beacon_chain/tests/attestation_production.rs @@ -133,7 +133,8 @@ async fn produces_attestations() { assert_eq!(data.target.epoch, state.current_epoch(), "bad target epoch"); assert_eq!(data.target.root, target_root, "bad target root"); - let rpc_block = RpcBlock::::new(Arc::new(block.clone()), blobs.clone()).unwrap(); + let rpc_block = + RpcBlock::::new(Arc::new(block.clone()), blobs.clone()).unwrap(); let beacon_chain::data_availability_checker::MaybeAvailableBlock::Available(available_block) = chain .data_availability_checker .check_rpc_block_availability(rpc_block) diff --git a/beacon_node/beacon_chain/tests/block_verification.rs b/beacon_node/beacon_chain/tests/block_verification.rs index 852e40834c4..102707a3892 100644 --- a/beacon_node/beacon_chain/tests/block_verification.rs +++ b/beacon_node/beacon_chain/tests/block_verification.rs @@ -158,7 +158,9 @@ fn chain_segment_blocks( chain_segment .iter() .zip(blobs.into_iter()) - .map(|(snapshot, blobs)| RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()) + .map(|(snapshot, blobs)| { + RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap() + }) .collect() } @@ -408,7 +410,9 @@ async fn assert_invalid_signature( let blocks: Vec> = snapshots .iter() .zip(chain_segment_blobs.iter()) - .map(|(snapshot, blobs)| RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()) + .map(|(snapshot, blobs)| { + RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap() + }) .collect(); // Ensure the block will be rejected if imported in a chain segment. @@ -433,7 +437,9 @@ async fn assert_invalid_signature( .iter() .take(block_index) .zip(chain_segment_blobs.iter()) - .map(|(snapshot, blobs)| RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()) + .map(|(snapshot, blobs)| { + RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap() + }) .collect(); // We don't care if this fails, we just call this to ensure that all prior blocks have been // imported prior to this test. @@ -450,7 +456,8 @@ async fn assert_invalid_signature( RpcBlock::new( snapshots[block_index].beacon_block.clone(), chain_segment_blobs[block_index].clone(), - ).unwrap(), + ) + .unwrap(), NotifyExecutionLayer::Yes, || Ok(()), ) @@ -501,7 +508,9 @@ async fn invalid_signature_gossip_block() { .iter() .take(block_index) .zip(chain_segment_blobs.iter()) - .map(|(snapshot, blobs)| RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()) + .map(|(snapshot, blobs)| { + RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap() + }) .collect(); harness .chain @@ -546,7 +555,9 @@ async fn invalid_signature_block_proposal() { let blocks: Vec> = snapshots .iter() .zip(chain_segment_blobs.iter()) - .map(|(snapshot, blobs)| RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()) + .map(|(snapshot, blobs)| { + RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap() + }) .collect::>(); // Ensure the block will be rejected if imported in a chain segment. assert!( @@ -757,7 +768,9 @@ async fn invalid_signature_deposit() { let blocks: Vec> = snapshots .iter() .zip(chain_segment_blobs.iter()) - .map(|(snapshot, blobs)| RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()) + .map(|(snapshot, blobs)| { + RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap() + }) .collect(); assert!( !matches!( diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 166417ba93c..1d2d62bc1fd 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -13,6 +13,7 @@ use beacon_processor::{ MAX_SCHEDULED_WORK_QUEUE_LEN, MAX_WORK_EVENT_QUEUE_LEN, }; use environment::null_logger; +use lighthouse_network::rpc::methods::{BlobsByRangeRequest, BlobsByRootRequest}; use lighthouse_network::{ rpc::{BlocksByRangeRequest, BlocksByRootRequest, LightClientBootstrapRequest, StatusMessage}, Client, MessageId, NetworkGlobals, PeerId, PeerRequestId, @@ -26,7 +27,6 @@ use store::MemoryStore; use task_executor::test_utils::TestRuntime; use task_executor::TaskExecutor; use tokio::sync::mpsc::{self, error::TrySendError}; -use lighthouse_network::rpc::methods::{BlobsByRangeRequest, BlobsByRootRequest}; use types::*; pub use sync_methods::ChainSegmentProcessId; @@ -229,9 +229,7 @@ impl NetworkBeaconProcessor { }) } - pub fn send_banana(){ - - } + pub fn send_banana() {} /// Create a new `Work` event for some sync committee signature. pub fn send_gossip_sync_signature( @@ -563,12 +561,7 @@ impl NetworkBeaconProcessor { ) -> Result<(), Error> { let processor = self.clone(); let process_fn = move |send_idle_on_drop| { - processor.handle_blobs_by_range_request( - send_idle_on_drop, - peer_id, - request_id, - request, - ) + processor.handle_blobs_by_range_request(send_idle_on_drop, peer_id, request_id, request) }; self.try_send(BeaconWorkEvent { @@ -586,12 +579,7 @@ impl NetworkBeaconProcessor { ) -> Result<(), Error> { let processor = self.clone(); let process_fn = move |send_idle_on_drop| { - processor.handle_blobs_by_root_request( - send_idle_on_drop, - peer_id, - request_id, - request, - ) + processor.handle_blobs_by_root_request(send_idle_on_drop, peer_id, request_id, request) }; self.try_send(BeaconWorkEvent { 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 3a09373c56f..b21bc6abde8 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -6,9 +6,9 @@ use crate::sync::{ manager::{BlockProcessType, SyncMessage}, ChainId, }; -use beacon_chain::data_availability_checker::MaybeAvailableBlock; use beacon_chain::block_verification_types::{AsBlock, RpcBlock}; use beacon_chain::data_availability_checker::AvailabilityCheckError; +use beacon_chain::data_availability_checker::MaybeAvailableBlock; use beacon_chain::{ observed_block_producers::Error as ObserveError, validator_monitor::get_block_delay_ms, AvailabilityProcessingStatus, BeaconChainError, BeaconChainTypes, BlockError, @@ -270,7 +270,8 @@ impl NetworkBeaconProcessor { process_type: BlockProcessType, ) -> AsyncFn { let process_fn = async move { - self.clone().process_rpc_blobs(block_root, block, seen_timestamp, process_type) + self.clone() + .process_rpc_blobs(block_root, block, seen_timestamp, process_type) .await; }; Box::pin(process_fn) @@ -306,7 +307,7 @@ impl NetworkBeaconProcessor { }); } - pub fn send_delayed_lookup(&self, block_root: Hash256){ + pub fn send_delayed_lookup(&self, block_root: Hash256) { self.send_sync_message(SyncMessage::MissingGossipBlockComponentsDelayed(block_root)) } diff --git a/beacon_node/network/src/network_beacon_processor/tests.rs b/beacon_node/network/src/network_beacon_processor/tests.rs index 39d3575d6e3..2c37d177aab 100644 --- a/beacon_node/network/src/network_beacon_processor/tests.rs +++ b/beacon_node/network/src/network_beacon_processor/tests.rs @@ -336,14 +336,16 @@ impl TestRig { } pub fn enqueue_blobs_by_range_request(&self, count: u64) { - self.network_beacon_processor.send_blobs_by_range_request( - PeerId::random(), - (ConnectionId::new(42), SubstreamId::new(24)), - BlobsByRangeRequest { - start_slot: 0, - count, - }, - ).unwrap(); + self.network_beacon_processor + .send_blobs_by_range_request( + PeerId::random(), + (ConnectionId::new(42), SubstreamId::new(24)), + BlobsByRangeRequest { + start_slot: 0, + count, + }, + ) + .unwrap(); } pub fn enqueue_backfill_batch(&self) { diff --git a/beacon_node/network/src/router.rs b/beacon_node/network/src/router.rs index 5a954d05a40..30a75a91052 100644 --- a/beacon_node/network/src/router.rs +++ b/beacon_node/network/src/router.rs @@ -209,18 +209,12 @@ impl Router { .send_blocks_by_roots_request(peer_id, request_id, request), ), Request::BlobsByRange(request) => self.handle_beacon_processor_send_result( - self.network_beacon_processor.send_blobs_by_range_request( - peer_id, - request_id, - request, - ), + self.network_beacon_processor + .send_blobs_by_range_request(peer_id, request_id, request), ), Request::BlobsByRoot(request) => self.handle_beacon_processor_send_result( - self.network_beacon_processor.send_blobs_by_roots_request( - peer_id, - request_id, - request, - ), + self.network_beacon_processor + .send_blobs_by_roots_request(peer_id, request_id, request), ), Request::LightClientBootstrap(request) => self.handle_beacon_processor_send_result( self.network_beacon_processor @@ -311,7 +305,7 @@ impl Router { blob_index, signed_blob, timestamp_now(), - ) + ), ) } PubsubMessage::VoluntaryExit(exit) => { diff --git a/beacon_node/network/src/service/tests.rs b/beacon_node/network/src/service/tests.rs index 10110aa891b..544c5dd9c7f 100644 --- a/beacon_node/network/src/service/tests.rs +++ b/beacon_node/network/src/service/tests.rs @@ -3,6 +3,7 @@ mod tests { use crate::persisted_dht::load_dht; use crate::{NetworkConfig, NetworkService}; + use beacon_chain::test_utils::EphemeralHarnessType; use beacon_processor::{ BeaconProcessorSend, MAX_SCHEDULED_WORK_QUEUE_LEN, MAX_WORK_EVENT_QUEUE_LEN, }; @@ -12,7 +13,6 @@ mod tests { use std::str::FromStr; use std::sync::Arc; use tokio::{runtime::Runtime, sync::mpsc}; - use beacon_chain::test_utils::EphemeralHarnessType; use types::MinimalEthSpec as E; type BeaconChainHarness = beacon_chain::test_utils::BeaconChainHarness>; diff --git a/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs b/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs index c492470b4ae..55e9e49db30 100644 --- a/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs @@ -1,12 +1,12 @@ +use crate::network_beacon_processor::NetworkBeaconProcessor; use beacon_chain::{BeaconChain, BeaconChainTypes}; -use slog::{crit, }; +use slog::crit; use slot_clock::SlotClock; use std::sync::Arc; use tokio::sync::mpsc; use tokio::time::interval_at; use tokio::time::Instant; -use types::{ Hash256}; -use crate::network_beacon_processor::NetworkBeaconProcessor; +use types::Hash256; #[derive(Debug)] pub enum DelayedLookupMessage { diff --git a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs index 37b5ee0e767..90829905b8a 100644 --- a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs @@ -313,13 +313,16 @@ impl SingleBlockLookup>(); - let blobs = VariableList::from(filtered); - RpcBlock::new(block.clone(), Some(blobs)).ok() - }) + //TODO(sean) figure out how to properly deal with a consistency error here, + // should we downscore the peer sending blobs? + let blobs = std::mem::take(downloaded_blobs); + let filtered = blobs + .into_iter() + .filter_map(|b| b.clone()) + .collect::>(); + let blobs = VariableList::from(filtered); + RpcBlock::new(block.clone(), Some(blobs)).ok() + }) } else { None } diff --git a/beacon_node/network/src/sync/block_sidecar_coupling.rs b/beacon_node/network/src/sync/block_sidecar_coupling.rs index b25dcdf334e..fce7a2e30d1 100644 --- a/beacon_node/network/src/sync/block_sidecar_coupling.rs +++ b/beacon_node/network/src/sync/block_sidecar_coupling.rs @@ -1,5 +1,5 @@ use beacon_chain::block_verification_types::RpcBlock; -use ssz_types::{VariableList}; +use ssz_types::VariableList; use std::{collections::VecDeque, sync::Arc}; use types::{BlobSidecar, EthSpec, SignedBeaconBlock}; diff --git a/beacon_node/network/src/sync/range_sync/range.rs b/beacon_node/network/src/sync/range_sync/range.rs index f69a30b3ee0..277606e438c 100644 --- a/beacon_node/network/src/sync/range_sync/range.rs +++ b/beacon_node/network/src/sync/range_sync/range.rs @@ -387,18 +387,18 @@ mod tests { use beacon_chain::builder::Witness; use beacon_chain::eth1_chain::CachingEth1Backend; use beacon_chain::parking_lot::RwLock; + use beacon_chain::test_utils::{BeaconChainHarness, EphemeralHarnessType}; use beacon_chain::EngineState; use beacon_processor::WorkEvent as BeaconWorkEvent; use lighthouse_network::rpc::BlocksByRangeRequest; use lighthouse_network::Request; use lighthouse_network::{rpc::StatusMessage, NetworkGlobals}; use slog::{o, Drain}; - use tokio::sync::mpsc; - use beacon_chain::test_utils::{BeaconChainHarness, EphemeralHarnessType}; - use slot_clock::{TestingSlotClock, }; + use slot_clock::TestingSlotClock; use std::collections::HashSet; use std::sync::Arc; use store::MemoryStore; + use tokio::sync::mpsc; use types::{Hash256, MinimalEthSpec as E}; #[derive(Debug)] diff --git a/consensus/fork_choice/tests/tests.rs b/consensus/fork_choice/tests/tests.rs index bba7ccdbe67..b4981a3eea8 100644 --- a/consensus/fork_choice/tests/tests.rs +++ b/consensus/fork_choice/tests/tests.rs @@ -4,6 +4,7 @@ use std::fmt; use std::sync::Mutex; use std::time::Duration; +use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::test_utils::{ AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType, }; @@ -11,7 +12,6 @@ use beacon_chain::{ BeaconChain, BeaconChainError, BeaconForkChoiceStore, ChainConfig, ForkChoiceError, StateSkipConfig, WhenSlotSkipped, }; -use beacon_chain::block_verification_types::RpcBlock; use fork_choice::{ ForkChoiceStore, InvalidAttestation, InvalidBlock, PayloadVerificationStatus, QueuedAttestation, }; @@ -201,7 +201,11 @@ impl ForkChoiceTest { if !predicate(block.0.message(), &state) { break; } - if let Ok(block_hash) = self.harness.process_block_result(RpcBlock::new(block.0, block.1).unwrap()).await { + if let Ok(block_hash) = self + .harness + .process_block_result(RpcBlock::new(block.0, block.1).unwrap()) + .await + { self.harness.attest_block( &state, block.0.state_root(), From 8a6e8d51b6428efadf7bbaf8427fe7f2a228b0ac Mon Sep 17 00:00:00 2001 From: realbigsean Date: Tue, 25 Jul 2023 10:45:58 -0400 Subject: [PATCH 04/18] make block and blob single lookups generic --- beacon_node/network/src/router.rs | 13 +- .../network/src/sync/block_lookups/mod.rs | 1011 +++++------------ .../src/sync/block_lookups/parent_lookup.rs | 205 ++-- .../sync/block_lookups/single_block_lookup.rs | 895 +++++++++------ .../network/src/sync/block_lookups/tests.rs | 20 +- beacon_node/network/src/sync/manager.rs | 182 ++- .../network/src/sync/network_context.rs | 173 ++- 7 files changed, 1106 insertions(+), 1393 deletions(-) diff --git a/beacon_node/network/src/router.rs b/beacon_node/network/src/router.rs index 30a75a91052..927229e7b7e 100644 --- a/beacon_node/network/src/router.rs +++ b/beacon_node/network/src/router.rs @@ -482,7 +482,10 @@ impl Router { ) { let request_id = match request_id { RequestId::Sync(sync_id) => match sync_id { - SyncId::SingleBlock { .. } | SyncId::ParentLookup { .. } => { + SyncId::SingleBlock { .. } + | SyncId::SingleBlob { .. } + | SyncId::ParentLookup { .. } + | SyncId::ParentLookupBlob { .. } => { unreachable!("Block lookups do not request BBRange requests") } id @ (SyncId::BackFillBlocks { .. } @@ -550,6 +553,9 @@ impl Router { | SyncId::BackFillBlockAndBlobs { .. } => { unreachable!("Batch syncing do not request BBRoot requests") } + SyncId::SingleBlob { .. } | SyncId::ParentLookupBlob { .. } => { + unreachable!("Blob response to block by roots request") + } }, RequestId::Router => unreachable!("All BBRoot requests belong to sync"), }; @@ -576,7 +582,10 @@ impl Router { ) { let request_id = match request_id { RequestId::Sync(sync_id) => match sync_id { - id @ (SyncId::SingleBlock { .. } | SyncId::ParentLookup { .. }) => id, + id @ (SyncId::SingleBlob { .. } | SyncId::ParentLookupBlob { .. }) => id, + SyncId::SingleBlock { .. } | SyncId::ParentLookup { .. } => { + unreachable!("Block response to blobs by roots request") + } SyncId::BackFillBlocks { .. } | SyncId::RangeBlocks { .. } | SyncId::RangeBlockAndBlobs { .. } diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index ff095c719ea..aef88bfc3e7 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -1,22 +1,24 @@ -use self::parent_lookup::PARENT_FAIL_TOLERANCE; -use self::parent_lookup::{ParentLookup, ParentVerifyError}; -use self::single_block_lookup::{LookupVerifyError, SingleBlockLookup}; +use self::parent_lookup::ParentVerifyError; +use self::single_block_lookup::SingleBlockLookup; use super::manager::BlockProcessingResult; use super::BatchProcessResult; -use super::{ - manager::{BlockProcessType, Id}, - network_context::SyncNetworkContext, -}; +use super::{manager::BlockProcessType, network_context::SyncNetworkContext}; use crate::metrics; use crate::network_beacon_processor::ChainSegmentProcessId; -use crate::sync::block_lookups::single_block_lookup::LookupId; +use crate::sync::block_lookups::parent_lookup::ParentLookup; +use crate::sync::block_lookups::single_block_lookup::LookupVerifyError; +use crate::sync::manager::{Id, ResponseType}; use beacon_chain::block_verification_types::{AsBlock, RpcBlock}; use beacon_chain::data_availability_checker::{AvailabilityCheckError, DataAvailabilityChecker}; use beacon_chain::{AvailabilityProcessingStatus, BeaconChainTypes, BlockError}; +use fnv::FnvHashMap; use lighthouse_network::rpc::RPCError; use lighthouse_network::{PeerAction, PeerId}; use lru_cache::LRUTimeCache; pub use single_block_lookup::UnknownParentComponents; +pub use single_block_lookup::{ + BlobRequestState, BlockRequestState, Current, Lookup, Parent, RequestState, +}; use slog::{debug, error, trace, warn, Logger}; use smallvec::SmallVec; use std::collections::HashMap; @@ -26,7 +28,7 @@ use std::time::Duration; use store::{Hash256, SignedBeaconBlock}; use strum::Display; use types::blob_sidecar::FixedBlobSidecarList; -use types::{BlobSidecar, Slot}; +use types::{BlobSidecar, EthSpec, Slot}; pub(crate) mod delayed_lookup; mod parent_lookup; @@ -39,19 +41,18 @@ pub type RootBlockTuple = (Hash256, Arc>); pub type RootBlobsTuple = (Hash256, FixedBlobSidecarList); const FAILED_CHAINS_CACHE_EXPIRY_SECONDS: u64 = 60; -const SINGLE_BLOCK_LOOKUP_MAX_ATTEMPTS: u8 = 3; +pub const SINGLE_BLOCK_LOOKUP_MAX_ATTEMPTS: u8 = 3; -pub(crate) struct BlockLookups { +pub struct BlockLookups { /// Parent chain lookups being downloaded. parent_lookups: SmallVec<[ParentLookup; 3]>, - processing_parent_lookups: - HashMap, SingleBlockLookup)>, + processing_parent_lookups: HashMap, SingleBlockLookup)>, /// A cache of failed chain lookups to prevent duplicate searches. failed_chains: LRUTimeCache, - single_block_lookups: Vec>, + single_block_lookups: FnvHashMap>, da_checker: Arc>, @@ -59,34 +60,6 @@ pub(crate) struct BlockLookups { log: Logger, } -pub type BlockRequestId = Id; -pub type BlobRequestId = Id; - -#[derive(Debug, PartialEq)] -enum StreamTerminator { - True, - False, -} - -impl From for StreamTerminator { - fn from(value: bool) -> Self { - if value { - StreamTerminator::True - } else { - StreamTerminator::False - } - } -} - -/// Used to track block or blob responses in places we want to reduce code duplication in -/// response handling. -// NOTE: a better solution may be to wrap request `Id` in an enum. -#[derive(Debug, Copy, Clone)] -pub enum ResponseType { - Block, - Blob, -} - /// This enum is used to track what a peer *should* be able to respond with respond based on /// other messages we've seen from this peer on the network. This is useful for peer scoring. /// We expect a peer tracked by the `BlockAndBlobs` variant to be able to respond to all @@ -124,13 +97,6 @@ impl PeerShouldHave { } } -/// Tracks the conditions under which we want to drop a parent or single block lookup. -#[derive(Debug, Copy, Clone)] -pub enum ShouldRemoveLookup { - True, - False, -} - impl BlockLookups { pub fn new(da_checker: Arc>, log: Logger) -> Self { Self { @@ -152,9 +118,9 @@ impl BlockLookups { &mut self, block_root: Hash256, peer_source: PeerShouldHave, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, ) { - let lookup = self.search_block_with(block_root, None, &[peer_source]); + let lookup = self.new_current_lookup(block_root, None, &[peer_source], cx); if let Some(lookup) = lookup { self.trigger_single_lookup(lookup, cx); } @@ -163,8 +129,13 @@ impl BlockLookups { /// /// The request is not immediately triggered, and should be triggered by a call to /// `trigger_lookup_by_root`. - pub fn search_block_delayed(&mut self, block_root: Hash256, peer_source: PeerShouldHave) { - let lookup = self.search_block_with(block_root, None, &[peer_source]); + pub fn search_block_delayed( + &mut self, + block_root: Hash256, + peer_source: PeerShouldHave, + cx: &SyncNetworkContext, + ) { + let lookup = self.new_current_lookup(block_root, None, &[peer_source], cx); if let Some(lookup) = lookup { self.add_single_lookup(lookup) } @@ -181,9 +152,9 @@ impl BlockLookups { block_root: Hash256, parent_components: Option>, peer_source: &[PeerShouldHave], - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, ) { - let lookup = self.search_block_with(block_root, parent_components, peer_source); + let lookup = self.new_current_lookup(block_root, parent_components, peer_source, cx); if let Some(lookup) = lookup { self.trigger_single_lookup(lookup, cx); } @@ -201,8 +172,9 @@ impl BlockLookups { block_root: Hash256, parent_components: Option>, peer_source: &[PeerShouldHave], + cx: &SyncNetworkContext, ) { - let lookup = self.search_block_with(block_root, parent_components, peer_source); + let lookup = self.new_current_lookup(block_root, parent_components, peer_source, cx); if let Some(lookup) = lookup { self.add_single_lookup(lookup) } @@ -211,8 +183,8 @@ impl BlockLookups { /// Attempts to trigger the request matching the given `block_root`. pub fn trigger_single_lookup( &mut self, - mut single_block_lookup: SingleBlockLookup, - cx: &mut SyncNetworkContext, + mut single_block_lookup: SingleBlockLookup, + cx: &SyncNetworkContext, ) { if !single_block_lookup.triggered && single_block_lookup.request_block_and_blobs(cx).is_ok() { @@ -221,11 +193,9 @@ impl BlockLookups { } } - pub fn add_single_lookup( - &mut self, - single_block_lookup: SingleBlockLookup, - ) { - self.single_block_lookups.push(single_block_lookup); + pub fn add_single_lookup(&mut self, single_block_lookup: SingleBlockLookup) { + self.single_block_lookups + .insert(single_block_lookup.id, single_block_lookup); metrics::set_gauge( &metrics::SYNC_SINGLE_BLOCK_LOOKUPS, @@ -236,12 +206,13 @@ impl BlockLookups { pub fn trigger_lookup_by_root( &mut self, block_root: Hash256, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, ) -> Result<(), ()> { - for lookup in self.single_block_lookups.iter_mut() { + for (_, lookup) in self.single_block_lookups.iter_mut() { if lookup.block_request_state.requested_block_root == block_root && !lookup.triggered { - lookup.request_block_and_blobs(cx)?; - lookup.triggered = true; + if lookup.request_block_and_blobs(cx).is_ok() { + lookup.triggered = true; + } } } Ok(()) @@ -249,22 +220,23 @@ impl BlockLookups { pub fn remove_lookup_by_root(&mut self, block_root: Hash256) { self.single_block_lookups - .retain(|lookup| lookup.block_request_state.requested_block_root != block_root); + .retain(|_id, lookup| lookup.block_request_state.requested_block_root != block_root); } /// Searches for a single block hash. If the blocks parent is unknown, a chain of blocks is /// constructed. - pub fn search_block_with( + pub fn new_current_lookup( &mut self, block_root: Hash256, parent_components: Option>, peers: &[PeerShouldHave], - ) -> Option> { + cx: &SyncNetworkContext, + ) -> Option> { // Do not re-request a block that is already being requested - if let Some(lookup) = self + if let Some((_, lookup)) = self .single_block_lookups .iter_mut() - .find(|lookup| lookup.is_for_block(block_root)) + .find(|(id, lookup)| lookup.is_for_block(block_root)) { lookup.add_peers(peers); if let Some(components) = parent_components { @@ -304,6 +276,7 @@ impl BlockLookups { parent_components, peers, self.da_checker.clone(), + cx, )) } @@ -315,7 +288,7 @@ impl BlockLookups { block_root: Hash256, parent_root: Hash256, peer_id: PeerId, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, ) { // Gossip blocks or blobs shouldn't be propagated if parents are unavailable. let peer_source = PeerShouldHave::BlockAndBlobs(peer_id); @@ -345,210 +318,131 @@ impl BlockLookups { // we are already processing this block, ignore it. return; } - - let parent_lookup = ParentLookup::new( + let mut parent_lookup = ParentLookup::new( block_root, parent_root, peer_source, self.da_checker.clone(), + cx, ); - self.request_parent_block_and_blobs(parent_lookup, cx); + if let Ok(()) = parent_lookup + .current_parent_request + .request_block_and_blobs(cx) + { + self.parent_lookups.push(parent_lookup); + } } /* Lookup responses */ - pub fn single_block_lookup_response( + pub fn single_lookup_response>( &mut self, id: Id, peer_id: PeerId, - block: Option>>, + response: Option, seen_timestamp: Duration, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, ) { - let stream_terminator = block.is_none().into(); + let is_stream_terminator = response.is_none(); + let response_type = R::response_type(); let log = self.log.clone(); - let Some((has_pending_parent_request, request_ref)) = self.find_single_lookup_request(id, stream_terminator, ResponseType::Block) else { - return; - }; - - let should_remove = match request_ref.verify_block(block) { - Ok(Some((block_root, block))) => { - if let Some(parent_components) = request_ref.unknown_parent_components.as_mut() { - parent_components.add_unknown_parent_block(block.clone()); - }; - - if !has_pending_parent_request { - let rpc_block = request_ref - .get_downloaded_block() - .unwrap_or(RpcBlock::new_without_blobs(block)); - // This is the correct block, send it for processing - match self.send_block_for_processing( - block_root, - rpc_block, - seen_timestamp, - BlockProcessType::SingleBlock { id }, - cx, - ) { - Ok(()) => ShouldRemoveLookup::False, - Err(()) => ShouldRemoveLookup::True, - } - } else { - ShouldRemoveLookup::False - } + let Some(lookup) = self.single_block_lookups.get_mut(&id) else { + if !is_stream_terminator { + warn!( + self.log, + "Block returned for single block lookup not present"; + "response_type" => ?response_type, + ); } - Ok(None) => ShouldRemoveLookup::False, - Err(e) => handle_block_lookup_verify_error( - request_ref, - ResponseType::Block, - peer_id, - e, - cx, - &log, - ), + return; }; - if matches!(should_remove, ShouldRemoveLookup::True) { - self.single_block_lookups - .retain(|req| req.id.block_request_id != Some(id)); - } - - metrics::set_gauge( - &metrics::SYNC_SINGLE_BLOCK_LOOKUPS, - self.single_block_lookups.len() as i64, - ); - } + let expected_block_root = lookup.block_request_state.requested_block_root; - pub fn single_blob_lookup_response( - &mut self, - id: Id, - peer_id: PeerId, - blob: Option>>, - seen_timestamp: Duration, - cx: &mut SyncNetworkContext, - ) { - let stream_terminator = blob.is_none().into(); + let has_pending_parent_request = self + .parent_lookups + .iter() + .any(|parent_lookup| parent_lookup.chain_hash() == expected_block_root); - let log = self.log.clone(); + let request_state = R::request_state_mut(lookup); - let Some((has_pending_parent_requests, request_ref)) = - self.find_single_lookup_request(id, stream_terminator, ResponseType::Blob) else { - return; - }; + match request_state.verify_response(expected_block_root, response) { + Ok(Some((root, verified_response))) => { + if let Some(parent_components) = lookup.unknown_parent_components.as_mut() { + R::add_to_parent_components(verified_response, parent_components); - let should_remove = match request_ref.verify_blob(blob) { - Ok(Some((block_root, blobs))) => { - if let Some(parent_components) = request_ref.unknown_parent_components.as_mut() { - parent_components.add_unknown_parent_blobs(blobs); - - if !has_pending_parent_requests { - request_ref - .get_downloaded_block() - .map(|block| { - match self.send_block_for_processing( - block_root, - block, - seen_timestamp, - BlockProcessType::SingleBlock { id }, - cx, - ) { - Ok(()) => ShouldRemoveLookup::False, - Err(()) => ShouldRemoveLookup::True, - } - }) - .unwrap_or(ShouldRemoveLookup::False) - } else { - ShouldRemoveLookup::False + if !has_pending_parent_request { + if let Some(rpc_block) = lookup.get_downloaded_block() { + if let Err(()) = self.send_block_for_processing( + expected_block_root, + rpc_block, + seen_timestamp, + BlockProcessType::SingleBlock { id }, + cx, + ) { + self.single_block_lookups.remove(&id); + } + } } } else { - // These are the correct blobs, send them for processing - match self.send_blobs_for_processing( - block_root, - blobs, + if let Err(()) = R::send_for_processing( + id, + self, + root, + R::verified_to_reconstructed(verified_response), seen_timestamp, - BlockProcessType::SingleBlock { id }, - cx, + &cx, ) { - Ok(()) => ShouldRemoveLookup::False, - Err(()) => ShouldRemoveLookup::True, + self.single_block_lookups.remove(&id); } } } - Ok(None) => ShouldRemoveLookup::False, - Err(e) => handle_block_lookup_verify_error( - request_ref, - ResponseType::Blob, - peer_id, - e, - cx, - &log, - ), - }; + Ok(None) => {} + Err(e) => { + let msg = if matches!(e, LookupVerifyError::BenignFailure) { + request_state + .get_state_mut() + .remove_peer_if_useless(&peer_id); + "peer could not response to request" + } else { + let msg = e.into(); + cx.report_peer(peer_id, PeerAction::LowToleranceError, msg); + msg + }; - if matches!(should_remove, ShouldRemoveLookup::True) { - self.single_block_lookups - .retain(|req| req.id.blob_request_id != Some(id)); + debug!(log, "Single block lookup failed"; + "peer_id" => %peer_id, + "error" => msg, + "block_root" => ?expected_block_root, + "response_type" => ?response_type + ); + if let Err(()) = request_state.retry_request_after_failure(id, cx, &log) { + self.single_block_lookups.remove(&id); + } + } } + //TODO(sean) move metric to trait to differentiate block and blob metrics::set_gauge( &metrics::SYNC_SINGLE_BLOCK_LOOKUPS, self.single_block_lookups.len() as i64, ); } - /// Returns the lookup along with a `bool` representing whether the lookup has an outstanding - /// parent lookup that has yet to be resolved. This determines whether we send the - /// block or blob for processing because we would fail block processing and trigger a new lookup - /// via `UnknownParentBlock` or `UnknownParentBlob` until we process the parent. - fn find_single_lookup_request( - &mut self, - target_id: Id, - stream_terminator: StreamTerminator, - response_type: ResponseType, - ) -> Option<( - bool, - &mut SingleBlockLookup, - )> { - let lookup = self.single_block_lookups.iter_mut().find_map(|req| { - let id_opt = match response_type { - ResponseType::Block => req.id.block_request_id, - ResponseType::Blob => req.id.blob_request_id, - }; - if let Some(lookup_id) = id_opt { - if lookup_id == target_id { - let has_pending_parent_request = self.parent_lookups.iter().any(|lookup| { - lookup.chain_hash() == req.block_request_state.requested_block_root - }); - - return Some((has_pending_parent_request, req)); - } - } - None - }); - - if lookup.is_none() && matches!(stream_terminator, StreamTerminator::False) { - warn!( - self.log, - "Block returned for single block lookup not present"; - "response_type" => ?response_type, - ); - } - lookup - } - /// Process a response received from a parent lookup request. - pub fn parent_lookup_response( + pub fn parent_lookup_response>( &mut self, id: Id, peer_id: PeerId, - block: Option>>, + block: Option, seen_timestamp: Duration, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, ) { let mut parent_lookup = if let Some(pos) = self .parent_lookups .iter() - .position(|request| request.pending_block_response(id)) + .position(|request| request.current_parent_request.id == id) { self.parent_lookups.remove(pos) } else { @@ -558,91 +452,18 @@ impl BlockLookups { return; }; - match parent_lookup.verify_block(block, &mut self.failed_chains) { - Ok(Some((block_root, block))) => { - parent_lookup.add_current_request_block(block); - if let Some(rpc_block) = parent_lookup.current_parent_request.get_downloaded_block() + match parent_lookup.verify_block::(block, &mut self.failed_chains) { + Ok(Some((block_root, verified_response))) => { + if let Some(parent_components) = parent_lookup + .current_parent_request + .unknown_parent_components + .as_mut() { - let chain_hash = parent_lookup.chain_hash(); - if self - .send_block_for_processing( - block_root, - rpc_block, - seen_timestamp, - BlockProcessType::ParentLookup { chain_hash }, - cx, - ) - .is_ok() - { - self.parent_lookups.push(parent_lookup) - } - } else { - let outstanding_blobs_req = parent_lookup - .current_parent_request - .id - .blob_request_id - .is_some(); - if !outstanding_blobs_req { - if let Ok(peer_id) = parent_lookup - .current_parent_request - .downloading_peer(ResponseType::Blob) - { - cx.report_peer( - peer_id.to_peer_id(), - PeerAction::MidToleranceError, - "bbroot_failed_chains", - ); - } - - self.request_parent_blobs(parent_lookup, cx); - } else { - self.parent_lookups.push(parent_lookup) - } + R::add_to_parent_components(verified_response, parent_components); } - } - Ok(None) => { - // Request finished successfully, nothing else to do. It will be removed after the - // processing result arrives. - self.parent_lookups.push(parent_lookup); - } - Err(e) => { - self.handle_parent_verify_error(peer_id, parent_lookup, ResponseType::Block, e, cx) - } - }; - - metrics::set_gauge( - &metrics::SYNC_PARENT_BLOCK_LOOKUPS, - self.parent_lookups.len() as i64, - ); - } - - pub fn parent_lookup_blob_response( - &mut self, - id: Id, - peer_id: PeerId, - blob: Option>>, - seen_timestamp: Duration, - cx: &mut SyncNetworkContext, - ) { - let mut parent_lookup = if let Some(pos) = self - .parent_lookups - .iter() - .position(|request| request.pending_blob_response(id)) - { - self.parent_lookups.remove(pos) - } else { - if blob.is_some() { - debug!(self.log, "Response for a parent lookup blob request that was not found"; "peer_id" => %peer_id); - } - return; - }; - - match parent_lookup.verify_blob(blob, &mut self.failed_chains) { - Ok(Some((block_root, blobs))) => { - parent_lookup.add_current_request_blobs(blobs); - let chain_hash = parent_lookup.chain_hash(); if let Some(rpc_block) = parent_lookup.current_parent_request.get_downloaded_block() { + let chain_hash = parent_lookup.chain_hash(); if self .send_block_for_processing( block_root, @@ -656,31 +477,32 @@ impl BlockLookups { self.parent_lookups.push(parent_lookup) } } else { + //TODO(sean) here, we could penalize a peer who previously sent us a blob list + // that was incomplete, and trigger a re-request immediately self.parent_lookups.push(parent_lookup) } } Ok(None) => { - // Waiting for more blobs to arrive + // Request finished successfully, nothing else to do. It will be removed after the + // processing result arrives. self.parent_lookups.push(parent_lookup); } - Err(e) => { - self.handle_parent_verify_error(peer_id, parent_lookup, ResponseType::Blob, e, cx) - } + Err(e) => self.handle_parent_verify_error::(peer_id, parent_lookup, e, cx), }; + //TODO(sean) move metric to trait to differentiate block and blob metrics::set_gauge( &metrics::SYNC_PARENT_BLOCK_LOOKUPS, self.parent_lookups.len() as i64, ); } - fn handle_parent_verify_error( + fn handle_parent_verify_error>( &mut self, peer_id: PeerId, mut parent_lookup: ParentLookup, - response_type: ResponseType, e: ParentVerifyError, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, ) { match e { ParentVerifyError::RootMismatch @@ -699,10 +521,7 @@ impl BlockLookups { cx.report_peer(peer_id, PeerAction::LowToleranceError, e); // We try again if possible. - match response_type { - ResponseType::Block => self.request_parent_block(parent_lookup, cx), - ResponseType::Blob => self.request_parent_blobs(parent_lookup, cx), - }; + self.request_parent(parent_lookup, cx) } ParentVerifyError::PreviousFailure { parent_root } => { debug!( @@ -724,13 +543,9 @@ impl BlockLookups { self.log, "Requested peer could not respond to block request, requesting a new peer"; ); - parent_lookup - .current_parent_request - .remove_peer_if_useless(&peer_id, response_type); - match response_type { - ResponseType::Block => self.request_parent_block(parent_lookup, cx), - ResponseType::Blob => self.request_parent_blobs(parent_lookup, cx), - }; + let request_state = R::request_state_mut(&mut parent_lookup.current_parent_request); + request_state.remove_if_useless(&peer_id); + self.request_parent(parent_lookup, cx) } } } @@ -738,14 +553,12 @@ impl BlockLookups { /* Error responses */ pub fn peer_disconnected(&mut self, peer_id: &PeerId, cx: &mut SyncNetworkContext) { - self.single_block_lookups.retain_mut(|req| { - let should_remove_block = - should_remove_disconnected_peer(ResponseType::Block, peer_id, cx, req, &self.log); - let should_remove_blob = - should_remove_disconnected_peer(ResponseType::Blob, peer_id, cx, req, &self.log); - - matches!(should_remove_block, ShouldRemoveLookup::False) - && matches!(should_remove_blob, ShouldRemoveLookup::False) + /* Check disconnection for single lookups */ + self.single_block_lookups.retain(|id, req| { + let should_remove_lookup = + req.should_remove_disconnected_peer(*id, peer_id, cx, &self.log); + + !should_remove_lookup }); /* Check disconnection for parent lookups */ @@ -755,83 +568,60 @@ impl BlockLookups { }) { let parent_lookup = self.parent_lookups.remove(pos); trace!(self.log, "Parent lookup's peer disconnected"; &parent_lookup); - self.request_parent_block_and_blobs(parent_lookup, cx); + self.request_parent(parent_lookup, cx); } } /// An RPC error has occurred during a parent lookup. This function handles this case. - pub fn parent_lookup_failed( + pub fn parent_lookup_failed>( &mut self, id: Id, peer_id: PeerId, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, error: RPCError, ) { let msg = error.as_static_str(); if let Some(pos) = self .parent_lookups .iter() - .position(|request| request.pending_block_response(id)) + .position(|request| request.current_parent_request.id == id) { let mut parent_lookup = self.parent_lookups.remove(pos); - parent_lookup.block_download_failed(); + R::request_state_mut(&mut parent_lookup.current_parent_request) + .register_failure_downloading(); trace!(self.log, "Parent lookup block request failed"; &parent_lookup, "error" => msg); - self.request_parent_block(parent_lookup, cx); + self.request_parent(parent_lookup, cx); } else { return debug!(self.log, "RPC failure for a block parent lookup request that was not found"; "peer_id" => %peer_id, "error" => msg); }; - if let Some(pos) = self - .parent_lookups - .iter() - .position(|request| request.pending_blob_response(id)) - { - let mut parent_lookup = self.parent_lookups.remove(pos); - parent_lookup.blob_download_failed(); - trace!(self.log, "Parent lookup blobs request failed"; &parent_lookup, "error" => msg); - - self.request_parent_blobs(parent_lookup, cx); - } else { - return debug!(self.log, "RPC failure for a blobs parent lookup request that was not found"; "peer_id" => %peer_id, "error" => msg); - }; metrics::set_gauge( &metrics::SYNC_PARENT_BLOCK_LOOKUPS, self.parent_lookups.len() as i64, ); } - pub fn single_block_lookup_failed( + pub fn single_block_lookup_failed>( &mut self, id: Id, peer_id: &PeerId, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, error: RPCError, ) { let msg = error.as_static_str(); - self.single_block_lookups.retain_mut(|req| { - let should_remove_block = should_remove_failed_lookup( - id, - ResponseType::Block, - msg, - peer_id, - cx, - req, - &self.log, - ); - let should_remove_blob = should_remove_failed_lookup( - id, - ResponseType::Blob, - msg, - peer_id, - cx, - req, - &self.log, - ); - - matches!(should_remove_block, ShouldRemoveLookup::False) - && matches!(should_remove_blob, ShouldRemoveLookup::False) - }); + let Some(lookup) = self.single_block_lookups.get_mut(&id) else { + debug!(self.log, "Error response to dropped lookup"; "error" => ?error); + return; + }; + let root = lookup.block_request_state.requested_block_root; + let request_state = R::request_state_mut(lookup); + request_state.register_failure_downloading(); + let response_type = R::response_type(); + trace!(self.log, "Single lookup failed"; "block_root" => ?root, "error" => msg, "response_type" => ?response_type); + if let Err(()) = request_state.retry_request_after_failure(id, cx, &self.log) { + self.single_block_lookups.remove(&id); + }; metrics::set_gauge( &metrics::SYNC_SINGLE_BLOCK_LOOKUPS, @@ -841,48 +631,71 @@ impl BlockLookups { /* Processing responses */ - pub fn single_block_component_processed( + pub fn single_block_component_processed>( &mut self, target_id: Id, result: BlockProcessingResult, - response_type: ResponseType, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, ) { - let lookup_components_opt = - self.single_block_lookups - .iter_mut() - .enumerate() - .find_map(|(index, req)| { - let block_match = req.id.block_request_id.as_ref() == Some(&target_id); - let blob_match = req.id.blob_request_id.as_ref() == Some(&target_id); - (block_match || blob_match).then_some((index, req)) - }); - let (index, request_ref) = match lookup_components_opt { - Some(req) => req, - None => { - return debug!( - self.log, - "Block component processed for single block lookup not present" - ); - } - }; + let Some(request_ref) = self.single_block_lookups.get_mut(&target_id) else { + debug!(self.log, "Block component processed for single block lookup not present" ); + return; + }; let root = request_ref.block_request_state.requested_block_root; - let peer_id = request_ref.processing_peer(response_type); + let request_state = R::request_state_mut(request_ref); + let peer_id = request_state.get_state().processing_peer(); + request_state.get_state_mut().component_processed = true; let peer_id = match peer_id { Ok(peer) => peer, Err(_) => return, }; - let should_remove_lookup = match result { + match result { BlockProcessingResult::Ok(status) => match status { AvailabilityProcessingStatus::Imported(root) => { trace!(self.log, "Single block processing succeeded"; "block" => %root); - ShouldRemoveLookup::True + self.single_block_lookups.remove(&target_id); } AvailabilityProcessingStatus::MissingComponents(_, _block_root) => { - should_remove_missing_components(request_ref, response_type, cx, &self.log) + // if this was the result of a blocks request, the block peer did nothing wrong. + // if we already had a blobs resposne, we should penalize the blobs peer because + // they did not provide all blobs. + if request_ref.both_components_processed() { + if let Ok(blob_peer) = + request_ref.blob_request_state.state.processing_peer() + { + if let PeerShouldHave::BlockAndBlobs(blob_peer) = blob_peer { + cx.report_peer( + blob_peer, + PeerAction::MidToleranceError, + "single_block_failure", + ); + } + request_ref + .blob_request_state + .state + .remove_peer_if_useless(blob_peer.as_peer_id()); + if !::EthSpec, + > as RequestState>::downloading( + &request_ref.blob_request_state, + ) { + // Try it again if possible. + if let Err(()) = request_ref + .blob_request_state + .retry_request_after_failure(target_id, cx, &self.log) + { + self.single_block_lookups.remove(&target_id); + }; + } + } else { + trace!(self.log, "Dropped blob peer prior to penalizing"; "root" => ?root); + self.single_block_lookups.remove(&target_id); + }; + } } }, BlockProcessingResult::Ignored => { @@ -893,26 +706,25 @@ impl BlockLookups { "Single block processing was ignored, cpu might be overloaded"; "action" => "dropping single block request" ); - ShouldRemoveLookup::True + self.single_block_lookups.remove(&target_id); } BlockProcessingResult::Err(e) => { trace!(self.log, "Single block processing failed"; "block" => %root, "error" => %e); match e { BlockError::BlockIsAlreadyKnown => { // No error here - ShouldRemoveLookup::True + self.single_block_lookups.remove(&target_id); } BlockError::BeaconChainError(e) => { // Internal error error!(self.log, "Beacon chain error processing single block"; "block_root" => %root, "error" => ?e); - ShouldRemoveLookup::True + self.single_block_lookups.remove(&target_id); } BlockError::ParentUnknown(block) => { let slot = block.slot(); let parent_root = block.parent_root(); request_ref.add_unknown_parent_components(block.into()); self.search_parent(slot, root, parent_root, peer_id.to_peer_id(), cx); - ShouldRemoveLookup::False } ref e @ BlockError::ExecutionPayloadError(ref epe) if !epe.penalize_peer() => { // These errors indicate that the execution layer is offline @@ -923,7 +735,7 @@ impl BlockLookups { "root" => %root, "error" => ?e ); - ShouldRemoveLookup::True + self.single_block_lookups.remove(&target_id); } BlockError::AvailabilityCheck( AvailabilityCheckError::KzgVerificationFailed, @@ -931,27 +743,28 @@ impl BlockLookups { | BlockError::AvailabilityCheck(AvailabilityCheckError::Kzg(_)) | BlockError::BlobValidation(_) => { warn!(self.log, "Blob validation failure"; "root" => %root, "peer_id" => %peer_id); - if let Ok(blob_peer) = request_ref.processing_peer(ResponseType::Blob) { + if let Ok(blob_peer) = + request_ref.blob_request_state.state.processing_peer() + { cx.report_peer( blob_peer.to_peer_id(), PeerAction::MidToleranceError, "single_blob_failure", ); // Try it again if possible. - retry_request_after_failure( - request_ref, - ResponseType::Blob, - peer_id.as_peer_id(), - cx, - &self.log, - ) - } else { - ShouldRemoveLookup::False + if let Err(()) = request_ref + .blob_request_state + .retry_request_after_failure(target_id, cx, &self.log) + { + self.single_block_lookups.remove(&target_id); + }; } } other => { warn!(self.log, "Peer sent invalid block in single block lookup"; "root" => %root, "error" => ?other, "peer_id" => %peer_id); - if let Ok(block_peer) = request_ref.processing_peer(ResponseType::Block) { + if let Ok(block_peer) = + request_ref.block_request_state.state.processing_peer() + { cx.report_peer( block_peer.to_peer_id(), PeerAction::MidToleranceError, @@ -959,25 +772,19 @@ impl BlockLookups { ); // Try it again if possible. - retry_request_after_failure( - request_ref, - ResponseType::Block, - block_peer.as_peer_id(), - cx, - &self.log, - ) - } else { - ShouldRemoveLookup::False + if let Err(()) = request_ref + .blob_request_state + .retry_request_after_failure(target_id, cx, &self.log) + { + self.single_block_lookups.remove(&target_id); + }; } } } } }; - if matches!(should_remove_lookup, ShouldRemoveLookup::True) { - self.single_block_lookups.remove(index); - } - + //TODO(sean) move metrics to lookup response trait metrics::set_gauge( &metrics::SYNC_SINGLE_BLOCK_LOOKUPS, self.single_block_lookups.len() as i64, @@ -989,7 +796,7 @@ impl BlockLookups { chain_hash: Hash256, result: BlockProcessingResult, response_type: ResponseType, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, ) { let index = self .parent_lookups @@ -1002,13 +809,9 @@ impl BlockLookups { return debug!(self.log, "Process response for a parent lookup request that was not found"; "chain_hash" => %chain_hash); }; - let peer_id = parent_lookup - .current_parent_request - .processing_peer(response_type); - - let peer_id = match peer_id { - Ok(peer) => peer, - Err(_) => return, + let Ok(peer_id) = + parent_lookup.processing_peer() else { + return }; match &result { @@ -1042,7 +845,7 @@ impl BlockLookups { } BlockProcessingResult::Err(BlockError::ParentUnknown(block)) => { parent_lookup.add_unknown_parent_block(block); - self.request_parent_block_and_blobs(parent_lookup, cx); + self.request_parent(parent_lookup, cx); } BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(_)) | BlockProcessingResult::Err(BlockError::BlockIsAlreadyKnown { .. }) => { @@ -1059,15 +862,27 @@ impl BlockLookups { }; let (chain_hash, mut blocks, hashes, block_request) = parent_lookup.parts_for_processing(); - if let Some(child_block) = self.single_block_lookups.iter_mut().find_map(|req| { - if req.block_request_state.requested_block_root == chain_hash { - req.get_downloaded_block() + + // Find the child block that spawned the parent lookup request and add it to the chain + // to send for processing. + if let Some(child_lookup_id) = + self.single_block_lookups.iter().find_map(|(id, lookup)| { + (lookup.block_request_state.requested_block_root == chain_hash).then(|| *id) + }) + { + let Some(child_lookup) = self.single_block_lookups.get_mut(&child_lookup_id) else { + debug!(self.log, "Missing child for parent lookup request"; "child_root" => ?chain_hash); + return; + }; + if let Some(rpc_block) = child_lookup.get_downloaded_block() { + blocks.push(rpc_block); } else { - None + trace!(self.log, "Parent lookup chain complete, awaiting child response"; "chain_hash" => ?chain_hash); } - }) { - blocks.push(child_block); + } else { + debug!(self.log, "Missing child for parent lookup request"; "child_root" => ?chain_hash); }; + let process_id = ChainSegmentProcessId::ParentLookup(chain_hash); match beacon_processor.send_chain_segment(process_id, blocks) { @@ -1120,7 +935,7 @@ impl BlockLookups { &mut self, outcome: BlockError<::EthSpec>, peer_id: PeerId, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, mut parent_lookup: ParentLookup, ) { // all else we consider the chain a failure and downvote the peer that sent @@ -1135,16 +950,15 @@ impl BlockLookups { // ambiguity. cx.report_peer(peer_id, PeerAction::MidToleranceError, "parent_request_err"); // Try again if possible - parent_lookup.block_processing_failed(); - parent_lookup.blob_processing_failed(); - self.request_parent_block_and_blobs(parent_lookup, cx); + parent_lookup.processing_failed(); + self.request_parent(parent_lookup, cx); } pub fn parent_chain_processed( &mut self, chain_hash: Hash256, result: BatchProcessResult, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, ) { let request = match self.processing_parent_lookups.remove(&chain_hash) { Some((_hashes, request)) => request, @@ -1156,42 +970,36 @@ impl BlockLookups { debug!(self.log, "Parent chain processed"; "chain_hash" => %chain_hash, "result" => ?result); match result { BatchProcessResult::Success { .. } => { - if let Some((index, _)) = self + let Some(id) = self .single_block_lookups .iter() - .enumerate() - .find(|(_, req)| req.block_request_state.requested_block_root == chain_hash) - { - if let Some((lookup_id, rpc_block)) = - self.single_block_lookups.get_mut(index).and_then(|lookup| { - lookup - .get_downloaded_block() - .map(|block| (lookup.id.clone(), block)) - }) - { - let LookupId { - block_request_id, - blob_request_id, - } = lookup_id; - let Some(id) = block_request_id.or(blob_request_id) else { - warn!(self.log, "No id found for single block lookup"; "chain_hash" => %chain_hash); - return; - }; + .find_map(|(id, req)| + (req.block_request_state.requested_block_root == chain_hash).then(|| *id)) else { + warn!(self.log, "No id found for single block lookup"; "chain_hash" => %chain_hash); + return; + }; - // This is the correct block, send it for processing - if self - .send_block_for_processing( - chain_hash, - rpc_block, - Duration::from_secs(0), //TODO(sean) pipe this through - BlockProcessType::SingleBlock { id }, - cx, - ) - .is_err() - { - // Remove to avoid inconsistencies - self.single_block_lookups.remove(index); - } + let Some(lookup) = self + .single_block_lookups + .get_mut(&id) else { + warn!(self.log, "No id found for single block lookup"; "chain_hash" => %chain_hash); + return; + }; + + if let Some(rpc_block) = lookup.get_downloaded_block() { + // This is the correct block, send it for processing + if self + .send_block_for_processing( + chain_hash, + rpc_block, + Duration::from_secs(0), //TODO(sean) pipe this through + BlockProcessType::SingleBlock { id: id }, + cx, + ) + .is_err() + { + // Remove to avoid inconsistencies + self.single_block_lookups.remove(&id); } } } @@ -1225,7 +1033,7 @@ impl BlockLookups { block: RpcBlock, duration: Duration, process_type: BlockProcessType, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, ) -> Result<(), ()> { match cx.beacon_processor_if_enabled() { Some(beacon_processor) => { @@ -1259,7 +1067,7 @@ impl BlockLookups { blobs: FixedBlobSidecarList, duration: Duration, process_type: BlockProcessType, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, ) -> Result<(), ()> { let blob_count = blobs.iter().filter(|b| b.is_some()).count(); if blob_count == 0 { @@ -1288,49 +1096,10 @@ impl BlockLookups { } } - fn request_parent_block( - &mut self, - mut parent_lookup: ParentLookup, - cx: &mut SyncNetworkContext, - ) { - let response = parent_lookup.request_parent_block(cx); - self.handle_response(parent_lookup, cx, response, ResponseType::Block); - } - - fn request_parent_blobs( - &mut self, - mut parent_lookup: ParentLookup, - cx: &mut SyncNetworkContext, - ) { - let response = parent_lookup.request_parent_blobs(cx); - self.handle_response(parent_lookup, cx, response, ResponseType::Blob); - } - - fn request_parent_block_and_blobs( - &mut self, - mut parent_lookup: ParentLookup, - cx: &mut SyncNetworkContext, - ) { - let block_res = parent_lookup.request_parent_block(cx); - match block_res { - Ok(()) => { - let blob_res = parent_lookup.request_parent_blobs(cx); - self.handle_response(parent_lookup, cx, blob_res, ResponseType::Blob) - } - Err(e) => { - self.handle_response(parent_lookup, cx, Err(e), ResponseType::Block); - } - } - } + fn request_parent(&mut self, mut parent_lookup: ParentLookup, cx: &SyncNetworkContext) { + let response = parent_lookup.request_parent(cx); - fn handle_response( - &mut self, - parent_lookup: ParentLookup, - cx: &mut SyncNetworkContext, - result: Result<(), parent_lookup::RequestError>, - response_type: ResponseType, - ) { - match result { + match response { Err(e) => { debug!(self.log, "Failed to request parent"; &parent_lookup, "error" => e.as_static()); match e { @@ -1340,7 +1109,7 @@ impl BlockLookups { parent_lookup::RequestError::ChainTooLong => { self.failed_chains.insert(parent_lookup.chain_hash()); // This indicates faulty peers. - for &peer_id in parent_lookup.used_peers(response_type) { + for &peer_id in parent_lookup.used_peers() { cx.report_peer(peer_id, PeerAction::LowToleranceError, e.as_static()) } } @@ -1353,7 +1122,7 @@ impl BlockLookups { self.failed_chains.insert(parent_lookup.chain_hash()); } // This indicates faulty peers. - for &peer_id in parent_lookup.used_peers(response_type) { + for &peer_id in parent_lookup.used_peers() { cx.report_peer(peer_id, PeerAction::LowToleranceError, e.as_static()) } } @@ -1361,6 +1130,7 @@ impl BlockLookups { // This happens if the peer disconnects while the block is being // processed. Drop the request without extra penalty } + parent_lookup::RequestError::AlreadyDownloaded => {} } } Ok(_) => { @@ -1389,175 +1159,8 @@ impl BlockLookups { } } -fn handle_block_lookup_verify_error( - request_ref: &mut SingleBlockLookup, - response_type: ResponseType, - peer_id: PeerId, - e: LookupVerifyError, - cx: &mut SyncNetworkContext, - log: &Logger, -) -> ShouldRemoveLookup { - let msg = if matches!(e, LookupVerifyError::BenignFailure) { - request_ref.remove_peer_if_useless(&peer_id, response_type); - "peer could not response to request" - } else { - let msg = e.into(); - cx.report_peer(peer_id, PeerAction::LowToleranceError, msg); - msg - }; - - debug!(log, "Single block lookup failed"; - "peer_id" => %peer_id, - "error" => msg, - "block_root" => ?request_ref.block_request_state.requested_block_root, - "response_type" => ?response_type - ); - retry_request_after_failure(request_ref, response_type, &peer_id, cx, log) -} - -fn retry_request_after_failure( - request_ref: &mut SingleBlockLookup, - response_type: ResponseType, - initial_peer_id: &PeerId, - cx: &mut SyncNetworkContext, - log: &Logger, -) -> ShouldRemoveLookup { - let requested_block_root = request_ref.block_request_state.requested_block_root; - - // try the request again if possible - match response_type { - ResponseType::Block => { - let id = request_ref.request_block().map(|request_opt| { - request_opt - .map(|(peer_id, request)| cx.single_block_lookup_request(peer_id, request)) - }); - match id { - Ok(Some(Ok(id))) => { - request_ref.id.block_request_id = Some(id); - } - Ok(Some(Err(e))) => { - debug!(log, "Single block lookup failed"; - "peer_id" => %initial_peer_id, - "error" => ?e, - "block_root" => ?requested_block_root, - "response_type" => ?response_type); - return ShouldRemoveLookup::True; - } - Ok(None) => { - request_ref.id.block_request_id = None; - // The lookup failed but the block or blob was found via other means. - } - Err(e) => { - debug!(log, "Single block lookup failed"; - "peer_id" => %initial_peer_id, - "error" => ?e, - "block_root" => ?requested_block_root, - "response_type" => ?response_type); - return ShouldRemoveLookup::True; - } - } - } - ResponseType::Blob => { - let id = request_ref.request_blobs().map(|request_opt| { - request_opt - .map(|(peer_id, request)| cx.single_blobs_lookup_request(peer_id, request)) - }); - - match id { - Ok(Some(Ok(id))) => { - request_ref.id.blob_request_id = Some(id); - } - Ok(Some(Err(e))) => { - debug!(log, "Single block lookup failed"; - "peer_id" => %initial_peer_id, - "error" => ?e, - "block_root" => ?requested_block_root, - "response_type" => ?response_type); - return ShouldRemoveLookup::True; - } - Ok(None) => { - request_ref.id.blob_request_id = None; - // The lookup failed but the block or blob was found via other means. - } - Err(e) => { - debug!(log, "Single block lookup failed"; - "peer_id" => %initial_peer_id, - "error" => ?e, - "block_root" => ?requested_block_root, - "response_type" => ?response_type); - return ShouldRemoveLookup::True; - } - } - } - }; - ShouldRemoveLookup::False -} - -fn should_remove_disconnected_peer( - response_type: ResponseType, - peer_id: &PeerId, - cx: &mut SyncNetworkContext, - req: &mut SingleBlockLookup, - log: &Logger, -) -> ShouldRemoveLookup { - if req.check_peer_disconnected(peer_id, response_type).is_err() { - trace!(log, "Single lookup failed on peer disconnection"; "block_root" => ?req.block_request_state.requested_block_root, "response_type" => ?response_type); - retry_request_after_failure(req, response_type, peer_id, cx, log) - } else { - ShouldRemoveLookup::False - } -} - -fn should_remove_failed_lookup( - id: Id, - response_type: ResponseType, - msg: &'static str, - peer_id: &PeerId, - cx: &mut SyncNetworkContext, - req: &mut SingleBlockLookup, - log: &Logger, -) -> ShouldRemoveLookup { - if req.id.block_request_id == Some(id) || req.id.blob_request_id == Some(id) { - req.register_failure_downloading(response_type); - trace!(log, "Single lookup failed"; "block" => %req.block_request_state.requested_block_root, "error" => msg, "response_type" => ?response_type); - retry_request_after_failure(req, response_type, peer_id, cx, log) - } else { - ShouldRemoveLookup::False - } -} - -fn should_remove_missing_components( - request_ref: &mut SingleBlockLookup, - response_type: ResponseType, - cx: &mut SyncNetworkContext, - log: &Logger, -) -> ShouldRemoveLookup { - request_ref.set_component_processed(response_type); - - // If we get a missing component response after processing both a blob and a block response, the - // blobs must be what are missing. - if request_ref.both_components_processed() { - let Ok(blob_peer) = request_ref.processing_peer(ResponseType::Blob) else { - return ShouldRemoveLookup::False; - }; - if let PeerShouldHave::BlockAndBlobs(blob_peer) = blob_peer { - cx.report_peer( - blob_peer, - PeerAction::MidToleranceError, - "single_block_failure", - ); - } - request_ref.remove_peer_if_useless(blob_peer.as_peer_id(), ResponseType::Blob); - if !request_ref.downloading(ResponseType::Blob) { - // Try it again if possible. - return retry_request_after_failure( - request_ref, - ResponseType::Blob, - blob_peer.as_peer_id(), - cx, - log, - ); - } - } - ShouldRemoveLookup::False +#[derive(Debug, Copy, Clone)] +pub enum LookupType { + Current, + Parent, } diff --git a/beacon_node/network/src/sync/block_lookups/parent_lookup.rs b/beacon_node/network/src/sync/block_lookups/parent_lookup.rs index 6d870b5aba3..1df51bf7f06 100644 --- a/beacon_node/network/src/sync/block_lookups/parent_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/parent_lookup.rs @@ -1,18 +1,21 @@ use super::single_block_lookup::{LookupRequestError, LookupVerifyError, SingleBlockLookup}; -use super::{BlobRequestId, BlockRequestId, DownloadedBlocks, PeerShouldHave, ResponseType}; -use crate::sync::block_lookups::single_block_lookup::{State, UnknownParentComponents}; -use crate::sync::block_lookups::{RootBlobsTuple, RootBlockTuple}; +use super::{DownloadedBlocks, PeerShouldHave}; +use crate::sync::block_lookups::single_block_lookup::{ + Parent, RequestState, State, UnknownParentComponents, +}; +use crate::sync::block_lookups::Lookup; use crate::sync::{manager::SLOT_IMPORT_TOLERANCE, network_context::SyncNetworkContext}; use beacon_chain::block_verification_types::AsBlock; use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::data_availability_checker::DataAvailabilityChecker; use beacon_chain::BeaconChainTypes; +use itertools::Itertools; use lighthouse_network::PeerId; use std::sync::Arc; use store::Hash256; use strum::IntoStaticStr; use types::blob_sidecar::FixedBlobSidecarList; -use types::{BlobSidecar, SignedBeaconBlock}; +use types::SignedBeaconBlock; /// How many attempts we try to find a parent of a block before we give up trying. pub(crate) const PARENT_FAIL_TOLERANCE: u8 = 5; @@ -28,7 +31,7 @@ pub(crate) struct ParentLookup { /// The blocks that have currently been downloaded. downloaded_blocks: Vec>, /// Request of the last parent. - pub current_parent_request: SingleBlockLookup, + pub current_parent_request: SingleBlockLookup, } #[derive(Debug, PartialEq, Eq, IntoStaticStr)] @@ -55,6 +58,7 @@ pub enum RequestError { cannot_process: bool, }, NoPeers, + AlreadyDownloaded, } impl ParentLookup { @@ -63,9 +67,15 @@ impl ParentLookup { parent_root: Hash256, peer_id: PeerShouldHave, da_checker: Arc>, + cx: &SyncNetworkContext, ) -> Self { - let current_parent_request = - SingleBlockLookup::new(parent_root, Some(<_>::default()), &[peer_id], da_checker); + let current_parent_request = SingleBlockLookup::new( + parent_root, + Some(<_>::default()), + &[peer_id], + da_checker, + cx, + ); Self { chain_hash: block_root, @@ -85,52 +95,15 @@ impl ParentLookup { } /// Attempts to request the next unknown parent. If the request fails, it should be removed. - pub fn request_parent_block( - &mut self, - cx: &mut SyncNetworkContext, - ) -> Result<(), RequestError> { - // check to make sure this request hasn't failed - if self.downloaded_blocks.len() + 1 >= PARENT_DEPTH_TOLERANCE { - return Err(RequestError::ChainTooLong); - } - - if let Some((peer_id, request)) = self.current_parent_request.request_block()? { - match cx.parent_lookup_block_request(peer_id, request) { - Ok(request_id) => { - self.current_parent_request.id.block_request_id = Some(request_id); - return Ok(()); - } - Err(reason) => { - self.current_parent_request.id.block_request_id = None; - return Err(RequestError::SendFailed(reason)); - } - } - } - Ok(()) - } - - pub fn request_parent_blobs( - &mut self, - cx: &mut SyncNetworkContext, - ) -> Result<(), RequestError> { + pub fn request_parent(&mut self, cx: &SyncNetworkContext) -> Result<(), RequestError> { // check to make sure this request hasn't failed if self.downloaded_blocks.len() + 1 >= PARENT_DEPTH_TOLERANCE { return Err(RequestError::ChainTooLong); } - if let Some((peer_id, request)) = self.current_parent_request.request_blobs()? { - match cx.parent_lookup_blobs_request(peer_id, request) { - Ok(request_id) => { - self.current_parent_request.id.blob_request_id = Some(request_id); - return Ok(()); - } - Err(reason) => { - self.current_parent_request.id.blob_request_id = None; - return Err(RequestError::SendFailed(reason)); - } - } - } - Ok(()) + self.current_parent_request + .request_block_and_blobs(cx) + .map_err(Into::into) } pub fn check_block_peer_disconnected(&mut self, peer_id: &PeerId) -> Result<(), ()> { @@ -162,11 +135,9 @@ impl ParentLookup { .block_request_state .requested_block_root = next_parent; self.current_parent_request.block_request_state.state.state = State::AwaitingDownload; - self.current_parent_request.id.block_request_id = None; // Update the blobs request. self.current_parent_request.blob_request_state.state.state = State::AwaitingDownload; - self.current_parent_request.id.blob_request_id = None; // Reset the unknown parent components. self.current_parent_request.unknown_parent_components = @@ -176,25 +147,24 @@ impl ParentLookup { pub fn add_current_request_block(&mut self, block: Arc>) { // Cache the block. self.current_parent_request.add_unknown_parent_block(block); - - // Update the request. - self.current_parent_request.id.block_request_id = None; } pub fn add_current_request_blobs(&mut self, blobs: FixedBlobSidecarList) { // Cache the blobs. self.current_parent_request.add_unknown_parent_blobs(blobs); - - // Update the request. - self.current_parent_request.id.blob_request_id = None; - } - - pub fn pending_block_response(&self, req_id: BlockRequestId) -> bool { - self.current_parent_request.id.block_request_id == Some(req_id) } - pub fn pending_blob_response(&self, req_id: BlobRequestId) -> bool { - self.current_parent_request.id.blob_request_id == Some(req_id) + pub fn processing_peer(&self) -> Result { + self.current_parent_request + .block_request_state + .state + .processing_peer() + .or_else(|()| { + self.current_parent_request + .blob_request_state + .state + .processing_peer() + }) } /// Consumes the parent request and destructures it into it's parts. @@ -205,7 +175,7 @@ impl ParentLookup { Hash256, Vec>, Vec, - SingleBlockLookup, + SingleBlockLookup, ) { let ParentLookup { chain_hash, @@ -227,38 +197,11 @@ impl ParentLookup { self.chain_hash } - pub fn block_download_failed(&mut self) { - self.current_parent_request - .block_request_state - .state - .register_failure_downloading(); - self.current_parent_request.id.block_request_id = None; - } - - pub fn blob_download_failed(&mut self) { - self.current_parent_request - .blob_request_state - .state - .register_failure_downloading(); - self.current_parent_request.id.blob_request_id = None; - } - - pub fn block_processing_failed(&mut self) { + pub fn processing_failed(&mut self) { self.current_parent_request .block_request_state .state .register_failure_processing(); - if let Some(components) = self - .current_parent_request - .unknown_parent_components - .as_mut() - { - components.downloaded_block = None; - } - self.current_parent_request.id.block_request_id = None; - } - - pub fn blob_processing_failed(&mut self) { self.current_parent_request .blob_request_state .state @@ -268,32 +211,34 @@ impl ParentLookup { .unknown_parent_components .as_mut() { + components.downloaded_block = None; components.downloaded_blobs = <_>::default(); } - self.current_parent_request.id.blob_request_id = None; } /// Verifies that the received block is what we requested. If so, parent lookup now waits for /// the processing result of the block. - pub fn verify_block( + pub fn verify_block>( &mut self, - block: Option>>, + block: Option, failed_chains: &mut lru_cache::LRUTimeCache, - ) -> Result>, ParentVerifyError> { - let root_and_block = self.current_parent_request.verify_block(block)?; + ) -> Result, ParentVerifyError> { + let expected_block_root = self + .current_parent_request + .block_request_state + .requested_block_root; + let request_state = R::request_state_mut(&mut self.current_parent_request); + let root_and_block = request_state.verify_response(expected_block_root, block)?; // check if the parent of this block isn't in the failed cache. If it is, this chain should // be dropped and the peer downscored. if let Some(parent_root) = root_and_block .as_ref() - .map(|(_, block)| block.parent_root()) + .map(|(_, block)| R::get_parent_root(block)) + .flatten() { if failed_chains.contains(&parent_root) { - self.current_parent_request - .block_request_state - .state - .register_failure_downloading(); - self.current_parent_request.id.block_request_id = None; + request_state.register_failure_downloading(); return Err(ParentVerifyError::PreviousFailure { parent_root }); } } @@ -301,49 +246,24 @@ impl ParentLookup { Ok(root_and_block) } - pub fn verify_blob( - &mut self, - blob: Option>>, - failed_chains: &mut lru_cache::LRUTimeCache, - ) -> Result>, ParentVerifyError> { - let parent_root_opt = blob.as_ref().map(|b| b.block_parent_root); - let blobs = self.current_parent_request.verify_blob(blob)?; - - // check if the parent of this block isn't in the failed cache. If it is, this chain should - // be dropped and the peer downscored. - if let Some(parent_root) = parent_root_opt { - if failed_chains.contains(&parent_root) { - self.current_parent_request - .blob_request_state - .state - .register_failure_downloading(); - self.current_parent_request.id.blob_request_id = None; - return Err(ParentVerifyError::PreviousFailure { parent_root }); - } - } - - Ok(blobs) - } - pub fn add_peers(&mut self, peer_source: &[PeerShouldHave]) { self.current_parent_request.add_peers(peer_source) } - pub fn used_peers(&self, response_type: ResponseType) -> impl Iterator + '_ { - match response_type { - ResponseType::Block => self - .current_parent_request - .block_request_state - .state - .used_peers - .iter(), - ResponseType::Blob => self - .current_parent_request - .blob_request_state - .state - .used_peers - .iter(), - } + pub fn used_peers(&self) -> impl Iterator + '_ { + self.current_parent_request + .block_request_state + .state + .used_peers + .iter() + .chain( + self.current_parent_request + .blob_request_state + .state + .used_peers + .iter(), + ) + .unique() } } @@ -371,6 +291,8 @@ impl From for RequestError { RequestError::TooManyAttempts { cannot_process } } E::NoPeers => RequestError::NoPeers, + E::AlreadyDownloaded => RequestError::AlreadyDownloaded, + E::SendFailed(msg) => RequestError::SendFailed(msg), } } } @@ -398,6 +320,7 @@ impl RequestError { } RequestError::TooManyAttempts { cannot_process: _ } => "too_many_downloading_attempts", RequestError::NoPeers => "no_peers", + RequestError::AlreadyDownloaded => "already_downloaded", } } } diff --git a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs index 90829905b8a..e3143f75f99 100644 --- a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs @@ -1,26 +1,59 @@ -use crate::sync::block_lookups::{BlobRequestId, BlockRequestId, RootBlobsTuple, RootBlockTuple}; +use super::{PeerShouldHave, ResponseType}; +use crate::sync::block_lookups::parent_lookup::RequestError::SendFailed; +use crate::sync::block_lookups::parent_lookup::PARENT_FAIL_TOLERANCE; +use crate::sync::block_lookups::{ + BlockLookups, Id, LookupType, RootBlobsTuple, RootBlockTuple, SINGLE_BLOCK_LOOKUP_MAX_ATTEMPTS, +}; +use crate::sync::manager::BlockProcessType; use crate::sync::network_context::SyncNetworkContext; -use beacon_chain::block_verification_types::RpcBlock; +use beacon_chain::block_verification_types::{AsBlock, RpcBlock}; use beacon_chain::data_availability_checker::DataAvailabilityChecker; use beacon_chain::{get_block_root, BeaconChainTypes}; +use itertools::Itertools; use lighthouse_network::rpc::methods::BlobsByRootRequest; use lighthouse_network::{rpc::BlocksByRootRequest, PeerId}; use rand::seq::IteratorRandom; +use slog::{debug, trace, Logger}; use ssz_types::VariableList; use std::collections::HashSet; +use std::marker::PhantomData; use std::ops::IndexMut; use std::sync::Arc; +use std::time::Duration; use store::Hash256; use strum::IntoStaticStr; use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; use types::{BlobSidecar, EthSpec, SignedBeaconBlock}; -use super::{PeerShouldHave, ResponseType}; +pub trait Lookup { + const MAX_ATTEMPTS: u8; + fn lookup_type() -> LookupType; + fn max_attempts() -> u8 { + Self::MAX_ATTEMPTS + } +} + +pub struct Parent; +pub struct Current; + +impl Lookup for Parent { + const MAX_ATTEMPTS: u8 = PARENT_FAIL_TOLERANCE; + fn lookup_type() -> LookupType { + LookupType::Parent + } +} -pub struct SingleBlockLookup { - pub id: LookupId, - pub block_request_state: BlockRequestState, - pub blob_request_state: BlobRequestState, +impl Lookup for Current { + const MAX_ATTEMPTS: u8 = SINGLE_BLOCK_LOOKUP_MAX_ATTEMPTS; + fn lookup_type() -> LookupType { + LookupType::Current + } +} + +pub struct SingleBlockLookup { + pub id: Id, + pub block_request_state: BlockRequestState, + pub blob_request_state: BlobRequestState, pub da_checker: Arc>, /// Only necessary for requests triggered by an `UnknownBlockParent` or `UnknownBlockParent` because any /// blocks or blobs without parents won't hit the data availability cache. @@ -30,101 +63,418 @@ pub struct SingleBlockLookup { pub triggered: bool, } -#[derive(Default, Clone)] -pub struct LookupId { - pub block_request_id: Option, - pub blob_request_id: Option, +// generic across block + blob +pub trait RequestState { + type RequestType; + type ResponseType; + type ReconstructedResponseType; + type VerifiedResponseType; + + // response verify + fn response_type() -> ResponseType; + fn get_parent_root(verified_response: &Self::VerifiedResponseType) -> Option; + fn request_state_mut(request: &mut SingleBlockLookup) -> &mut Self; + fn add_to_parent_components( + verified_response: Self::VerifiedResponseType, + components: &mut UnknownParentComponents, + ); + fn verified_to_reconstructed( + verified: Self::VerifiedResponseType, + ) -> Self::ReconstructedResponseType; + fn send_for_processing( + id: Id, + bl: &mut BlockLookups, + block_root: Hash256, + verified: Self::ReconstructedResponseType, + duration: Duration, + cx: &SyncNetworkContext, + ) -> Result<(), ()>; + + fn get_state(&self) -> &SingleLookupRequestState; + fn get_state_mut(&mut self) -> &mut SingleLookupRequestState; + fn processing_peer(&self) -> Result { + self.get_state().processing_peer() + } + fn downloading_peer(&self) -> Result { + self.get_state().peer() + } + fn set_component_processed(&mut self) { + self.get_state_mut().component_processed = true; + } + fn new_request(&self) -> Self::RequestType; + fn max_attempts() -> u8; + fn retry_request( + id: Id, + cx: &SyncNetworkContext, + peer_id: PeerId, + request: Self::RequestType, + ) -> Result<(), &'static str>; + fn verify_response( + &mut self, + expected_block_root: Hash256, + response: Option, + ) -> Result, LookupVerifyError> { + let request_state = self.get_state_mut(); + match request_state.state { + State::AwaitingDownload => { + request_state.register_failure_downloading(); + Err(LookupVerifyError::ExtraBlocksReturned) + } + State::Downloading { peer_id } => { + self.verify_response_inner(expected_block_root, response, peer_id) + } + State::Processing { peer_id: _ } => match response { + Some(_) => { + // We sent the block for processing and received an extra block. + request_state.register_failure_downloading(); + Err(LookupVerifyError::ExtraBlocksReturned) + } + None => { + // This is simply the stream termination and we are already processing the + // block + Ok(None) + } + }, + } + } + fn verify_response_inner( + &mut self, + expected_block_root: Hash256, + response: Option, + peer_id: PeerShouldHave, + ) -> Result, LookupVerifyError>; + + fn retry_request_after_failure( + &mut self, + id: Id, + cx: &SyncNetworkContext, + log: &Logger, + ) -> Result<(), ()> { + if let Err(e) = self + .build_request() + .map_err(Into::into) + .and_then(|(peer_id, request)| Self::retry_request(id, cx, peer_id, request)) + { + //TODO(sean) pass this error up? check downloaded contents prior to retry-ing? + debug!(log, "Single block lookup failed"; + "error" => ?e, + ); + return Err(()); + } + Ok(()) + } + fn build_request(&mut self) -> Result<(PeerId, Self::RequestType), LookupRequestError> { + debug_assert!(matches!(self.get_state().state, State::AwaitingDownload)); + self.too_many_attempts()?; + let peer = self.get_peer()?; + let request = self.new_request(); + Ok((peer, request)) + } + fn too_many_attempts(&self) -> Result<(), LookupRequestError> { + let max_attempts = Self::max_attempts(); + if self.get_state().failed_attempts() >= max_attempts { + Err(LookupRequestError::TooManyAttempts { + cannot_process: self.cannot_process(), + }) + } else { + Ok(()) + } + } + fn cannot_process(&self) -> bool { + let request_state = self.get_state(); + request_state.failed_processing >= request_state.failed_downloading + } + fn get_peer(&mut self) -> Result { + let mut request_state = self.get_state_mut(); + let Some(peer_id) = request_state + .available_peers + .iter() + .choose(&mut rand::thread_rng()) + .copied() + .map(PeerShouldHave::BlockAndBlobs).or(request_state + .potential_peers + .iter() + .choose(&mut rand::thread_rng()) + .copied() + .map(PeerShouldHave::Neither)) else { + return Err(LookupRequestError::NoPeers); + }; + request_state.used_peers.insert(peer_id.to_peer_id()); + request_state.state = State::Downloading { peer_id }; + Ok(peer_id.to_peer_id()) + } + fn check_peer_disconnected(&mut self, peer: &PeerId) -> Result<(), ()> { + self.get_state_mut().check_peer_disconnected(peer) + } + fn remove_if_useless(&mut self, peer: &PeerId) { + self.get_state_mut().remove_peer_if_useless(peer) + } + fn downloading(&self) -> bool { + matches!(self.get_state().state, State::Downloading { .. }) + } + fn register_failure_downloading(&mut self) { + self.get_state_mut().register_failure_downloading() + } } -pub struct BlobRequestState { +impl RequestState for BlockRequestState { + type RequestType = BlocksByRootRequest; + type ResponseType = Arc>; + type ReconstructedResponseType = RpcBlock; + type VerifiedResponseType = Arc>; + + // response verify + fn response_type() -> ResponseType { + ResponseType::Block + } + + fn get_parent_root(verified_response: &Arc>) -> Option { + Some(verified_response.parent_root()) + } + fn request_state_mut(request: &mut SingleBlockLookup) -> &mut Self { + &mut request.block_request_state + } + fn add_to_parent_components( + verified_response: Arc>, + components: &mut UnknownParentComponents, + ) { + components.add_unknown_parent_block(verified_response); + } + + fn verified_to_reconstructed( + block: Arc>, + ) -> RpcBlock { + RpcBlock::new_without_blobs(block) + } + + fn send_for_processing( + id: Id, + bl: &mut BlockLookups, + block_root: Hash256, + constructed: RpcBlock, + duration: Duration, + cx: &SyncNetworkContext, + ) -> Result<(), ()> { + bl.send_block_for_processing( + block_root, + constructed, + duration, + BlockProcessType::SingleBlock { id }, + cx, + ) + } + + fn get_state(&self) -> &SingleLookupRequestState { + &self.state + } + fn get_state_mut(&mut self) -> &mut SingleLookupRequestState { + &mut self.state + } + fn new_request(&self) -> BlocksByRootRequest { + BlocksByRootRequest::new(VariableList::from(vec![self.requested_block_root])) + } + fn max_attempts() -> u8 { + L::MAX_ATTEMPTS + } + fn retry_request( + id: Id, + cx: &SyncNetworkContext, + peer_id: PeerId, + request: Self::RequestType, + ) -> Result<(), &'static str> { + cx.single_block_lookup_request_retry(id, peer_id, request, L::lookup_type()) + } + + fn verify_response_inner( + &mut self, + expected_block_root: Hash256, + response: Option, + peer_id: PeerShouldHave, + ) -> Result>, LookupVerifyError> { + match response { + Some(block) => { + // Compute the block root using this specific function so that we can get timing + // metrics. + let block_root = get_block_root(&block); + if block_root != expected_block_root { + // return an error and drop the block + // NOTE: we take this is as a download failure to prevent counting the + // attempt as a chain failure, but simply a peer failure. + self.state.register_failure_downloading(); + Err(LookupVerifyError::RootMismatch) + } else { + // Return the block for processing. + self.state.state = State::Processing { peer_id }; + Ok(Some((block_root, block))) + } + } + None => { + if peer_id.should_have_block() { + self.state.register_failure_downloading(); + Err(LookupVerifyError::NoBlockReturned) + } else { + self.state.state = State::AwaitingDownload; + Err(LookupVerifyError::BenignFailure) + } + } + } + } +} + +impl RequestState for BlobRequestState { + type RequestType = BlobsByRootRequest; + type ResponseType = Arc>; + type ReconstructedResponseType = FixedBlobSidecarList; + type VerifiedResponseType = FixedBlobSidecarList; + + // response verify + fn response_type() -> ResponseType { + ResponseType::Blob + } + + fn get_parent_root(verified_response: &FixedBlobSidecarList) -> Option { + verified_response + .into_iter() + .filter_map(|blob| blob.as_ref()) + .map(|blob| blob.block_parent_root) + .next() + } + fn request_state_mut(request: &mut SingleBlockLookup) -> &mut Self { + &mut request.blob_request_state + } + fn add_to_parent_components( + verified_response: FixedBlobSidecarList, + components: &mut UnknownParentComponents, + ) { + components.add_unknown_parent_blobs(verified_response); + } + fn verified_to_reconstructed( + blobs: FixedBlobSidecarList, + ) -> FixedBlobSidecarList { + blobs + } + + fn send_for_processing( + id: Id, + bl: &mut BlockLookups, + block_root: Hash256, + verified: FixedBlobSidecarList, + duration: Duration, + cx: &SyncNetworkContext, + ) -> Result<(), ()> { + bl.send_blobs_for_processing( + block_root, + verified, + duration, + BlockProcessType::SingleBlob { id }, + cx, + ) + } + + fn get_state(&self) -> &SingleLookupRequestState { + &self.state + } + fn get_state_mut(&mut self) -> &mut SingleLookupRequestState { + &mut self.state + } + fn new_request(&self) -> BlobsByRootRequest { + BlobsByRootRequest { + blob_ids: VariableList::from(self.requested_ids.clone()), + } + } + fn max_attempts() -> u8 { + L::MAX_ATTEMPTS + } + fn retry_request( + id: Id, + cx: &SyncNetworkContext, + peer_id: PeerId, + request: Self::RequestType, + ) -> Result<(), &'static str> { + cx.single_blob_lookup_request_retry(id, peer_id, request, L::lookup_type()) + } + + fn verify_response_inner( + &mut self, + expected_block_root: Hash256, + blob: Option, + peer_id: PeerShouldHave, + ) -> Result>, LookupVerifyError> { + match blob { + Some(blob) => { + let received_id = blob.id(); + if !self.requested_ids.contains(&received_id) { + self.state.register_failure_downloading(); + Err(LookupVerifyError::UnrequestedBlobId) + } else { + // State should remain downloading until we receive the stream terminator. + self.requested_ids.retain(|id| *id != received_id); + let blob_index = blob.index; + + if blob_index >= T::EthSpec::max_blobs_per_block() as u64 { + return Err(LookupVerifyError::InvalidIndex(blob.index)); + } + *self.blob_download_queue.index_mut(blob_index as usize) = Some(blob); + Ok(None) + } + } + None => { + self.state.state = State::Processing { peer_id }; + let blobs = std::mem::take(&mut self.blob_download_queue); + Ok(Some((expected_block_root, blobs))) + } + } + } +} + +pub struct BlobRequestState { pub requested_ids: Vec, /// Where we store blobs until we receive the stream terminator. pub blob_download_queue: FixedBlobSidecarList, - pub state: SingleLookupRequestState, + pub state: SingleLookupRequestState, + _phantom: PhantomData, } -impl BlobRequestState { +impl BlobRequestState { pub fn new(peer_source: &[PeerShouldHave]) -> Self { Self { requested_ids: <_>::default(), blob_download_queue: <_>::default(), state: SingleLookupRequestState::new(peer_source), + _phantom: PhantomData::default(), } } } -pub struct BlockRequestState { +pub struct BlockRequestState { pub requested_block_root: Hash256, - pub state: SingleLookupRequestState, + pub state: SingleLookupRequestState, + _phantom: PhantomData, } -impl BlockRequestState { +impl BlockRequestState { pub fn new(block_root: Hash256, peers: &[PeerShouldHave]) -> Self { Self { requested_block_root: block_root, state: SingleLookupRequestState::new(peers), + _phantom: PhantomData::default(), } } } -impl SingleBlockLookup { - pub(crate) fn register_failure_downloading(&mut self, response_type: ResponseType) { - match response_type { - ResponseType::Block => self - .block_request_state - .state - .register_failure_downloading(), - ResponseType::Blob => self.blob_request_state.state.register_failure_downloading(), - } - } -} - -impl SingleBlockLookup { - pub(crate) fn downloading(&mut self, response_type: ResponseType) -> bool { - match response_type { - ResponseType::Block => { - matches!( - self.block_request_state.state.state, - State::Downloading { .. } - ) - } - ResponseType::Blob => { - matches!( - self.blob_request_state.state.state, - State::Downloading { .. } - ) - } - } - } - - pub(crate) fn remove_peer_if_useless(&mut self, peer_id: &PeerId, response_type: ResponseType) { - match response_type { - ResponseType::Block => self - .block_request_state - .state - .remove_peer_if_useless(peer_id), - ResponseType::Blob => self - .blob_request_state - .state - .remove_peer_if_useless(peer_id), +impl SingleBlockLookup { + pub(crate) fn block_already_downloaded(&self) -> bool { + if let Some(components) = self.unknown_parent_components.as_ref() { + components.downloaded_block.is_some() + } else { + self.da_checker + .has_block(&self.block_request_state.requested_block_root) } } - pub(crate) fn check_peer_disconnected( - &mut self, - peer_id: &PeerId, - response_type: ResponseType, - ) -> Result<(), ()> { - match response_type { - ResponseType::Block => self - .block_request_state - .state - .check_peer_disconnected(peer_id), - ResponseType::Blob => self - .blob_request_state - .state - .check_peer_disconnected(peer_id), - } + pub(crate) fn blobs_already_downloaded(&mut self) -> bool { + self.update_blobs_request(); + self.blob_request_state.requested_ids.is_empty() } } @@ -179,7 +529,7 @@ impl UnknownParentComponents { /// Object representing the state of a single block or blob lookup request. #[derive(PartialEq, Eq, Debug)] -pub struct SingleLookupRequestState { +pub struct SingleLookupRequestState { /// State of this request. pub state: State, /// Peers that should have this block or blob. @@ -225,17 +575,20 @@ pub enum LookupRequestError { cannot_process: bool, }, NoPeers, + SendFailed(&'static str), + AlreadyDownloaded, } -impl SingleBlockLookup { +impl SingleBlockLookup { pub fn new( requested_block_root: Hash256, unknown_parent_components: Option>, peers: &[PeerShouldHave], da_checker: Arc>, + cx: &SyncNetworkContext, ) -> Self { Self { - id: <_>::default(), + id: cx.next_id(), block_request_state: BlockRequestState::new(requested_block_root, peers), blob_request_state: BlobRequestState::new(peers), da_checker, @@ -251,27 +604,60 @@ impl SingleBlockLookup) -> Result<(), ()> { - let block_request_id = if let Ok(Some((peer_id, block_request))) = self.request_block() { - cx.single_block_lookup_request(peer_id, block_request).ok() - } else { - None - }; - - let blob_request_id = if let Ok(Some((peer_id, blob_request))) = self.request_blobs() { - cx.single_blobs_lookup_request(peer_id, blob_request).ok() - } else { - None - }; - - if block_request_id.is_none() && blob_request_id.is_none() { - return Err(()); + pub fn request_block_and_blobs( + &mut self, + cx: &SyncNetworkContext, + ) -> Result<(), LookupRequestError> { + let block_root = self.block_request_state.requested_block_root; + if self.block_already_downloaded() && self.blobs_already_downloaded() { + // drop lookup + trace!(cx.log, "Lookup request already completed"; "block_root"=> ?block_root); + return Err(LookupRequestError::AlreadyDownloaded); } - self.id = LookupId { - block_request_id, - blob_request_id, + let (block_peer_id, block_request) = + match as RequestState>::build_request( + &mut self.block_request_state, + ) { + Ok(opt) => opt, + Err(e) => { + // drop lookup + debug!(cx.log, + "Lookup request block error, dropping lookup"; + "block_root"=> ?block_root, + "error"=> ?e + ); + return Err(e); + } + }; + + let (blob_peer_id, blob_request) = match ::EthSpec, + > as RequestState>::build_request( + &mut self.blob_request_state + ) { + Ok(opt) => opt, + Err(e) => { + // drop lookup + debug!(cx.log, + "Lookup request blob error, dropping lookup"; + "block_root"=> ?block_root, + "error"=> ?e + ); + return Err(e); + } }; + + cx.single_lookup_request( + self.id, + block_peer_id, + block_request, + blob_peer_id, + blob_request, + L::lookup_type(), + ) + .map_err(LookupRequestError::SendFailed)?; Ok(()) } @@ -368,268 +754,6 @@ impl SingleBlockLookup>>, - ) -> Result>, LookupVerifyError> { - match self.block_request_state.state.state { - State::AwaitingDownload => { - self.block_request_state - .state - .register_failure_downloading(); - Err(LookupVerifyError::ExtraBlocksReturned) - } - State::Downloading { peer_id } => { - match block { - Some(block) => { - // Compute the block root using this specific function so that we can get timing - // metrics. - let block_root = get_block_root(&block); - if block_root != self.block_request_state.requested_block_root { - // return an error and drop the block - // NOTE: we take this is as a download failure to prevent counting the - // attempt as a chain failure, but simply a peer failure. - self.block_request_state - .state - .register_failure_downloading(); - Err(LookupVerifyError::RootMismatch) - } else { - // Return the block for processing. - self.block_request_state.state.state = State::Processing { peer_id }; - Ok(Some((block_root, block))) - } - } - None => { - if peer_id.should_have_block() { - self.block_request_state - .state - .register_failure_downloading(); - Err(LookupVerifyError::NoBlockReturned) - } else { - self.block_request_state.state.state = State::AwaitingDownload; - Err(LookupVerifyError::BenignFailure) - } - } - } - } - State::Processing { peer_id: _ } => match block { - Some(_) => { - // We sent the block for processing and received an extra block. - self.block_request_state - .state - .register_failure_downloading(); - Err(LookupVerifyError::ExtraBlocksReturned) - } - None => { - // This is simply the stream termination and we are already processing the - // block - Ok(None) - } - }, - } - } - - pub fn verify_blob( - &mut self, - blob: Option>>, - ) -> Result>, LookupVerifyError> { - match self.blob_request_state.state.state { - State::AwaitingDownload => { - self.blob_request_state.state.register_failure_downloading(); - Err(LookupVerifyError::ExtraBlobsReturned) - } - State::Downloading { - peer_id: peer_source, - } => match blob { - Some(blob) => { - let received_id = blob.id(); - if !self.blob_request_state.requested_ids.contains(&received_id) { - self.blob_request_state.state.register_failure_downloading(); - Err(LookupVerifyError::UnrequestedBlobId) - } else { - // State should remain downloading until we receive the stream terminator. - self.blob_request_state - .requested_ids - .retain(|id| *id != received_id); - let blob_index = blob.index; - - if blob_index >= T::EthSpec::max_blobs_per_block() as u64 { - return Err(LookupVerifyError::InvalidIndex(blob.index)); - } - *self - .blob_request_state - .blob_download_queue - .index_mut(blob_index as usize) = Some(blob); - Ok(None) - } - } - None => { - self.blob_request_state.state.state = State::Processing { - peer_id: peer_source, - }; - Ok(Some(( - self.block_request_state.requested_block_root, - std::mem::take(&mut self.blob_request_state.blob_download_queue), - ))) - } - }, - State::Processing { peer_id: _ } => match blob { - Some(_) => { - // We sent the blob for processing and received an extra blob. - self.blob_request_state.state.register_failure_downloading(); - Err(LookupVerifyError::ExtraBlobsReturned) - } - None => { - // This is simply the stream termination and we are already processing the - // block - Ok(None) - } - }, - } - } - - pub fn request_block( - &mut self, - ) -> Result, LookupRequestError> { - let block_already_downloaded = - if let Some(components) = self.unknown_parent_components.as_ref() { - components.downloaded_block.is_some() - } else { - self.da_checker - .has_block(&self.block_request_state.requested_block_root) - }; - - if block_already_downloaded { - return Ok(None); - } - - debug_assert!(matches!( - self.block_request_state.state.state, - State::AwaitingDownload - )); - let request = BlocksByRootRequest::new(VariableList::from(vec![ - self.block_request_state.requested_block_root, - ])); - let response_type = ResponseType::Block; - if self.too_many_attempts(response_type) { - Err(LookupRequestError::TooManyAttempts { - cannot_process: self.cannot_process(response_type), - }) - } else if let Some(peer_id) = self.get_peer(response_type) { - self.add_used_peer(peer_id, response_type); - Ok(Some((peer_id.to_peer_id(), request))) - } else { - Err(LookupRequestError::NoPeers) - } - } - - pub fn request_blobs( - &mut self, - ) -> Result, LookupRequestError> { - self.update_blobs_request(); - - if self.blob_request_state.requested_ids.is_empty() { - return Ok(None); - } - - debug_assert!(matches!( - self.blob_request_state.state.state, - State::AwaitingDownload - )); - let request = BlobsByRootRequest { - blob_ids: VariableList::from(self.blob_request_state.requested_ids.clone()), - }; - let response_type = ResponseType::Blob; - if self.too_many_attempts(response_type) { - Err(LookupRequestError::TooManyAttempts { - cannot_process: self.cannot_process(response_type), - }) - } else if let Some(peer_id) = self.get_peer(response_type) { - self.add_used_peer(peer_id, response_type); - Ok(Some((peer_id.to_peer_id(), request))) - } else { - Err(LookupRequestError::NoPeers) - } - } - - fn too_many_attempts(&self, response_type: ResponseType) -> bool { - match response_type { - ResponseType::Block => self.block_request_state.state.failed_attempts() >= MAX_ATTEMPTS, - ResponseType::Blob => self.blob_request_state.state.failed_attempts() >= MAX_ATTEMPTS, - } - } - - fn cannot_process(&self, response_type: ResponseType) -> bool { - match response_type { - ResponseType::Block => { - self.block_request_state.state.failed_processing - >= self.block_request_state.state.failed_downloading - } - ResponseType::Blob => { - self.blob_request_state.state.failed_processing - >= self.blob_request_state.state.failed_downloading - } - } - } - - fn get_peer(&self, response_type: ResponseType) -> Option { - match response_type { - ResponseType::Block => self - .block_request_state - .state - .available_peers - .iter() - .choose(&mut rand::thread_rng()) - .copied() - .map(PeerShouldHave::BlockAndBlobs) - .or(self - .block_request_state - .state - .potential_peers - .iter() - .choose(&mut rand::thread_rng()) - .copied() - .map(PeerShouldHave::Neither)), - ResponseType::Blob => self - .blob_request_state - .state - .available_peers - .iter() - .choose(&mut rand::thread_rng()) - .copied() - .map(PeerShouldHave::BlockAndBlobs) - .or(self - .blob_request_state - .state - .potential_peers - .iter() - .choose(&mut rand::thread_rng()) - .copied() - .map(PeerShouldHave::Neither)), - } - } - - fn add_used_peer(&mut self, peer_id: PeerShouldHave, response_type: ResponseType) { - match response_type { - ResponseType::Block => { - self.block_request_state - .state - .used_peers - .insert(peer_id.to_peer_id()); - self.block_request_state.state.state = State::Downloading { peer_id }; - } - ResponseType::Blob => { - self.blob_request_state - .state - .used_peers - .insert(peer_id.to_peer_id()); - self.blob_request_state.state.state = State::Downloading { peer_id }; - } - } - } - pub fn add_peers(&mut self, peers: &[PeerShouldHave]) { for peer in peers { match peer { @@ -645,34 +769,48 @@ impl SingleBlockLookup Result { - match response_type { - ResponseType::Block => self.block_request_state.state.processing_peer(), - ResponseType::Blob => self.blob_request_state.state.processing_peer(), - } - } - - pub fn downloading_peer(&self, response_type: ResponseType) -> Result { - match response_type { - ResponseType::Block => self.block_request_state.state.peer(), - ResponseType::Blob => self.blob_request_state.state.peer(), - } - } - pub fn both_components_processed(&self) -> bool { self.block_request_state.state.component_processed && self.blob_request_state.state.component_processed } - pub fn set_component_processed(&mut self, response_type: ResponseType) { - match response_type { - ResponseType::Block => self.block_request_state.state.component_processed = true, - ResponseType::Blob => self.blob_request_state.state.component_processed = true, - } + pub fn should_remove_disconnected_peer( + &mut self, + id: Id, + peer_id: &PeerId, + cx: &SyncNetworkContext, + log: &Logger, + ) -> bool { + let useless_block_peer = + if as RequestState>::check_peer_disconnected( + &mut self.block_request_state, + peer_id, + ) + .is_err() + { + trace!(log, "Single lookup failed on peer disconnection"; "block_root" => ?self.block_request_state.requested_block_root, "response_type" => ?ResponseType::Block); + self.block_request_state + .retry_request_after_failure(id, cx, log) + .is_err() + } else { + false + }; + let useless_blob_peer = if ::EthSpec> as RequestState>::check_peer_disconnected(&mut self + .blob_request_state, peer_id) + .is_err() + { + trace!(log, "Single lookup failed on peer disconnection"; "block_root" => ?self.block_request_state.requested_block_root, "response_type" => ?ResponseType::Blob); + self.blob_request_state + .retry_request_after_failure(id, cx, log) + .is_err() + } else { + false + }; + useless_block_peer && useless_blob_peer } } -impl SingleLookupRequestState { +impl SingleLookupRequestState { pub fn new(peers: &[PeerShouldHave]) -> Self { let mut available_peers = HashSet::default(); let mut potential_peers = HashSet::default(); @@ -763,9 +901,7 @@ impl SingleLookupRequestState { } } -impl slog::Value - for SingleBlockLookup -{ +impl slog::Value for SingleBlockLookup { fn serialize( &self, _record: &slog::Record, @@ -773,6 +909,7 @@ impl slog::Value serializer: &mut dyn slog::Serializer, ) -> slog::Result { serializer.emit_str("request", key)?; + serializer.emit_arguments("lookup_type", &format_args!("{:?}", L::lookup_type()))?; serializer.emit_arguments( "hash", &format_args!("{}", self.block_request_state.requested_block_root), @@ -793,7 +930,7 @@ impl slog::Value } } -impl slog::Value for SingleLookupRequestState { +impl slog::Value for SingleLookupRequestState { fn serialize( &self, record: &slog::Record, diff --git a/beacon_node/network/src/sync/block_lookups/tests.rs b/beacon_node/network/src/sync/block_lookups/tests.rs index 561593e6d14..12f345ac7e1 100644 --- a/beacon_node/network/src/sync/block_lookups/tests.rs +++ b/beacon_node/network/src/sync/block_lookups/tests.rs @@ -304,7 +304,7 @@ fn test_single_block_lookup_happy_path() { // The peer provides the correct block, should not be penalized. Now the block should be sent // for processing. - bl.single_block_lookup_response(id, peer_id, Some(block.into()), D, &mut cx); + bl.single_lookup_response(id, peer_id, Some(block.into()), D, &mut cx); rig.expect_empty_network(); rig.expect_block_process(response_type); @@ -313,7 +313,7 @@ fn test_single_block_lookup_happy_path() { // Send the stream termination. Peer should have not been penalized, and the request removed // after processing. - bl.single_block_lookup_response(id, peer_id, None, D, &mut cx); + bl.single_lookup_response(id, peer_id, None, D, &mut cx); bl.single_block_component_processed( id, BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(block_root)), @@ -346,7 +346,7 @@ fn test_single_block_lookup_empty_response() { } // The peer does not have the block. It should be penalized. - bl.single_block_lookup_response(id, peer_id, None, D, &mut cx); + bl.single_lookup_response(id, peer_id, None, D, &mut cx); rig.expect_penalty(); rig.expect_block_request(response_type); // it should be retried @@ -375,12 +375,12 @@ fn test_single_block_lookup_wrong_response() { // Peer sends something else. It should be penalized. let bad_block = rig.rand_block(fork_name); - bl.single_block_lookup_response(id, peer_id, Some(bad_block.into()), D, &mut cx); + bl.single_lookup_response(id, peer_id, Some(bad_block.into()), D, &mut cx); rig.expect_penalty(); rig.expect_block_request(response_type); // should be retried // Send the stream termination. This should not produce an additional penalty. - bl.single_block_lookup_response(id, peer_id, None, D, &mut cx); + bl.single_lookup_response(id, peer_id, None, D, &mut cx); rig.expect_empty_network(); } @@ -438,7 +438,7 @@ fn test_single_block_lookup_becomes_parent_request() { // The peer provides the correct block, should not be penalized. Now the block should be sent // for processing. - bl.single_block_lookup_response(id, peer_id, Some(block.clone()), D, &mut cx); + bl.single_lookup_response(id, peer_id, Some(block.clone()), D, &mut cx); rig.expect_empty_network(); rig.expect_block_process(response_type); @@ -971,7 +971,7 @@ fn test_single_block_lookup_ignored_response() { // The peer provides the correct block, should not be penalized. Now the block should be sent // for processing. - bl.single_block_lookup_response(id, peer_id, Some(block.into()), D, &mut cx); + bl.single_lookup_response(id, peer_id, Some(block.into()), D, &mut cx); rig.expect_empty_network(); rig.expect_block_process(response_type); @@ -980,7 +980,7 @@ fn test_single_block_lookup_ignored_response() { // Send the stream termination. Peer should have not been penalized, and the request removed // after processing. - bl.single_block_lookup_response(id, peer_id, None, D, &mut cx); + bl.single_lookup_response(id, peer_id, None, D, &mut cx); // Send an Ignored response, the request should be dropped bl.single_block_component_processed(id, BlockProcessingResult::Ignored, response_type, &mut cx); rig.expect_empty_network(); @@ -1353,7 +1353,7 @@ mod deneb_only { fn block_response(mut self) -> Self { // The peer provides the correct block, should not be penalized. Now the block should be sent // for processing. - self.bl.single_block_lookup_response( + self.bl.single_lookup_response( self.block_req_id.expect("block request id"), self.peer_id, self.block.clone(), @@ -1402,7 +1402,7 @@ mod deneb_only { } fn empty_block_response(mut self) -> Self { - self.bl.single_block_lookup_response( + self.bl.single_lookup_response( self.block_req_id.expect("block request id"), self.peer_id, None, diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 5e8fc4a4e9e..31062336580 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -43,8 +43,9 @@ use crate::service::NetworkMessage; use crate::status::ToStatusMessage; use crate::sync::block_lookups::delayed_lookup; use crate::sync::block_lookups::delayed_lookup::DelayedLookupMessage; -pub use crate::sync::block_lookups::ResponseType; -use crate::sync::block_lookups::UnknownParentComponents; +use crate::sync::block_lookups::{ + BlobRequestState, BlockRequestState, Current, Parent, RequestState, UnknownParentComponents, +}; use crate::sync::range_sync::ByRangeRequestType; use beacon_chain::block_verification_types::AsBlock; use beacon_chain::block_verification_types::RpcBlock; @@ -83,27 +84,47 @@ pub const DELAY_QUEUE_CHANNEL_SIZE: usize = 128; pub type Id = u32; +#[derive(Debug)] +pub enum ResponseType { + Block, + Blob, +} + /// Id of rpc requests sent by sync to the network. #[derive(Debug, Hash, PartialEq, Eq, Clone, Copy)] pub enum RequestId { /// Request searching for a block given a hash. - SingleBlock { id: Id }, + SingleBlock { + id: Id, + }, + SingleBlob { + id: Id, + }, /// Request searching for a block's parent. The id is the chain - ParentLookup { id: Id }, + ParentLookup { + id: Id, + }, + ParentLookupBlob { + id: Id, + }, /// Request was from the backfill sync algorithm. - BackFillBlocks { id: Id }, + BackFillBlocks { + id: Id, + }, /// Backfill request that is composed by both a block range request and a blob range request. - BackFillBlockAndBlobs { id: Id }, + BackFillBlockAndBlobs { + id: Id, + }, /// The request was from a chain in the range sync algorithm. - RangeBlocks { id: Id }, + RangeBlocks { + id: Id, + }, /// Range request that is composed by both a block range request and a blob range request. - RangeBlockAndBlobs { id: Id }, + RangeBlockAndBlobs { + id: Id, + }, } -// TODO(diva) I'm updating functions what at a time, but this should be revisited because I think -// some code paths that are split for blobs and blocks can be made just one after sync as a whole -// is updated. - #[derive(Debug)] /// A message that can be sent to the sync manager thread. pub enum SyncMessage { @@ -174,6 +195,7 @@ pub enum SyncMessage { #[derive(Debug, Clone)] pub enum BlockProcessType { SingleBlock { id: Id }, + SingleBlob { id: Id }, ParentLookup { chain_hash: Hash256 }, } @@ -324,16 +346,40 @@ impl SyncManager { trace!(self.log, "Sync manager received a failed RPC"); match request_id { RequestId::SingleBlock { id } => { - self.block_lookups.single_block_lookup_failed( - id, - &peer_id, - &mut self.network, - error, - ); + self.block_lookups + .single_block_lookup_failed::>( + id, + &peer_id, + &mut self.network, + error, + ); + } + RequestId::SingleBlob { id } => { + self.block_lookups + .single_block_lookup_failed::>( + id, + &peer_id, + &mut self.network, + error, + ); } RequestId::ParentLookup { id } => { self.block_lookups - .parent_lookup_failed(id, peer_id, &mut self.network, error); + .parent_lookup_failed::>( + id, + peer_id, + &mut self.network, + error, + ); + } + RequestId::ParentLookupBlob { id } => { + self.block_lookups + .parent_lookup_failed::>( + id, + peer_id, + &mut self.network, + error, + ); } RequestId::BackFillBlocks { id } => { if let Some(batch_id) = self @@ -652,8 +698,11 @@ impl SyncManager { // If we are not synced, ignore this block. if self.synced_and_connected(&peer_id) { if self.should_delay_lookup(slot) { - self.block_lookups - .search_block_delayed(block_root, PeerShouldHave::Neither(peer_id)); + self.block_lookups.search_block_delayed( + block_root, + PeerShouldHave::Neither(peer_id), + &self.network, + ); if let Err(e) = self .delayed_lookups .try_send(DelayedLookupMessage::MissingComponents(block_root)) @@ -695,7 +744,18 @@ impl SyncManager { } => match process_type { BlockProcessType::SingleBlock { id } => self .block_lookups - .single_block_component_processed(id, result, response_type, &mut self.network), + .single_block_component_processed::>( + id, + result, + &self.network, + ), + BlockProcessType::SingleBlob { id } => self + .block_lookups + .single_block_component_processed::>( + id, + result, + &self.network, + ), BlockProcessType::ParentLookup { chain_hash } => self .block_lookups .parent_block_processed(chain_hash, result, response_type, &mut self.network), @@ -753,6 +813,7 @@ impl SyncManager { block_root, parent_components, &[PeerShouldHave::Neither(peer_id)], + &self.network, ); if let Err(e) = self .delayed_lookups @@ -883,20 +944,30 @@ impl SyncManager { seen_timestamp: Duration, ) { match request_id { - RequestId::SingleBlock { id } => self.block_lookups.single_block_lookup_response( - id, - peer_id, - block, - seen_timestamp, - &mut self.network, - ), - RequestId::ParentLookup { id } => self.block_lookups.parent_lookup_response( - id, - peer_id, - block, - seen_timestamp, - &mut self.network, - ), + RequestId::SingleBlock { id } => self + .block_lookups + .single_lookup_response::>( + id, + peer_id, + block, + seen_timestamp, + &self.network, + ), + RequestId::SingleBlob { id } => { + crit!(self.log, "Blob received during block request"; "peer_id" => %peer_id ); + } + RequestId::ParentLookup { id } => self + .block_lookups + .parent_lookup_response::>( + id, + peer_id, + block, + seen_timestamp, + &self.network, + ), + RequestId::ParentLookupBlob { id } => { + crit!(self.log, "Blob received during parent block request"; "peer_id" => %peer_id ); + } RequestId::BackFillBlocks { id } => { let is_stream_terminator = block.is_none(); if let Some(batch_id) = self @@ -954,20 +1025,31 @@ impl SyncManager { seen_timestamp: Duration, ) { match request_id { - RequestId::SingleBlock { id } => self.block_lookups.single_blob_lookup_response( - id, - peer_id, - blob, - seen_timestamp, - &mut self.network, - ), - RequestId::ParentLookup { id } => self.block_lookups.parent_lookup_blob_response( - id, - peer_id, - blob, - seen_timestamp, - &mut self.network, - ), + RequestId::SingleBlock { id } => { + crit!(self.log, "Single blob received during block request"; "peer_id" => %peer_id ); + } + RequestId::SingleBlob { id } => self + .block_lookups + .single_lookup_response::>( + id, + peer_id, + blob, + seen_timestamp, + &mut self.network, + ), + + RequestId::ParentLookup { id } => { + crit!(self.log, "Single blob received during parent block request"; "peer_id" => %peer_id ); + } + RequestId::ParentLookupBlob { id } => self + .block_lookups + .parent_lookup_response::>( + id, + peer_id, + blob, + seen_timestamp, + &mut self.network, + ), RequestId::BackFillBlocks { id: _ } => { crit!(self.log, "Blob received during backfill block request"; "peer_id" => %peer_id ); } diff --git a/beacon_node/network/src/sync/network_context.rs b/beacon_node/network/src/sync/network_context.rs index d635dd2ea18..3524da78594 100644 --- a/beacon_node/network/src/sync/network_context.rs +++ b/beacon_node/network/src/sync/network_context.rs @@ -7,7 +7,7 @@ use super::range_sync::{BatchId, ByRangeRequestType, ChainId}; use crate::network_beacon_processor::NetworkBeaconProcessor; use crate::service::{NetworkMessage, RequestId}; use crate::status::ToStatusMessage; -use crate::sync::block_lookups::{BlobRequestId, BlockRequestId}; +use crate::sync::block_lookups::LookupType; use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::{BeaconChain, BeaconChainTypes, EngineState}; use fnv::FnvHashMap; @@ -37,7 +37,7 @@ pub struct SyncNetworkContext { network_send: mpsc::UnboundedSender>, /// A sequential ID for all RPC requests. - request_id: Id, + request_id: std::cell::Cell, /// BlocksByRange requests made by the range syncing algorithm. range_requests: FnvHashMap, @@ -62,7 +62,7 @@ pub struct SyncNetworkContext { pub chain: Arc>, /// Logger for the `SyncNetworkContext`. - log: slog::Logger, + pub log: slog::Logger, } /// Small enumeration to make dealing with block and blob requests easier. @@ -93,7 +93,7 @@ impl SyncNetworkContext { SyncNetworkContext { network_send, execution_engine_state: EngineState::Online, // always assume `Online` at the start - request_id: 1, + request_id: std::cell::Cell::new(1), range_requests: FnvHashMap::default(), backfill_requests: FnvHashMap::default(), range_blocks_and_blobs_requests: FnvHashMap::default(), @@ -118,11 +118,7 @@ impl SyncNetworkContext { .unwrap_or_default() } - pub fn status_peers( - &mut self, - chain: &C, - peers: impl Iterator, - ) { + pub fn status_peers(&self, chain: &C, peers: impl Iterator) { let status_message = chain.status_message(); for peer_id in peers { debug!( @@ -409,70 +405,39 @@ impl SyncNetworkContext { } /// Sends a blocks by root request for a parent request. - pub fn single_block_lookup_request( - &mut self, + pub fn single_lookup_request( + &self, + id: Id, peer_id: PeerId, request: BlocksByRootRequest, - ) -> Result { - let id = self.next_id(); - let request_id = RequestId::Sync(SyncRequestId::SingleBlock { id }); - - trace!( - self.log, - "Sending BlocksByRoot Request"; - "method" => "BlocksByRoot", - "count" => request.block_roots().len(), - "peer" => %peer_id - ); - - self.send_network_msg(NetworkMessage::SendRequest { - peer_id, - request: Request::BlocksByRoot(request), - request_id, - })?; - Ok(id) + blob_peer_id: PeerId, + blob_request: BlobsByRootRequest, + lookup_type: LookupType, + ) -> Result<(), &'static str> { + self.single_block_lookup_request_retry(id, peer_id, request, lookup_type)?; + self.single_blob_lookup_request_retry(id, blob_peer_id, blob_request, lookup_type)?; + Ok(()) } - - /// Sends a blobs by root request for a parent request. - pub fn single_blobs_lookup_request( - &mut self, - peer_id: PeerId, - request: BlobsByRootRequest, - ) -> Result { - let id = self.next_id(); - let request_id = RequestId::Sync(SyncRequestId::SingleBlock { id }); - - trace!( - self.log, - "Sending BlobsByRoot Request"; - "method" => "BlobsByRoot", - "count" => request.blob_ids.len(), - "peer" => %peer_id - ); - - self.send_network_msg(NetworkMessage::SendRequest { - peer_id, - request: Request::BlobsByRoot(request), - request_id, - })?; - Ok(id) - } - - /// Sends a blocks by root request for a parent request. - pub fn parent_lookup_block_request( - &mut self, + pub fn single_block_lookup_request_retry( + &self, + id: Id, peer_id: PeerId, request: BlocksByRootRequest, - ) -> Result { - let id = self.next_id(); - let request_id = RequestId::Sync(SyncRequestId::ParentLookup { id }); + lookup_type: LookupType, + ) -> Result<(), &'static str> { + let sync_id = match lookup_type { + LookupType::Current => SyncRequestId::SingleBlock { id }, + LookupType::Parent => SyncRequestId::ParentLookup { id }, + }; + let request_id = RequestId::Sync(sync_id); trace!( self.log, - "Sending parent BlocksByRoot Request"; + "Sending BlocksByRoot Request"; "method" => "BlocksByRoot", "count" => request.block_roots().len(), - "peer" => %peer_id + "peer" => %peer_id, + "lookup_type" => ?lookup_type ); self.send_network_msg(NetworkMessage::SendRequest { @@ -480,32 +445,34 @@ impl SyncNetworkContext { request: Request::BlocksByRoot(request), request_id, })?; - Ok(id) + Ok(()) } - /// Sends a blocks by root request for a parent request. - pub fn parent_lookup_blobs_request( - &mut self, - peer_id: PeerId, - request: BlobsByRootRequest, - ) -> Result { - let id = self.next_id(); - let request_id = RequestId::Sync(SyncRequestId::ParentLookup { id }); - - trace!( - self.log, - "Sending parent BlobsByRoot Request"; - "method" => "BlobsByRoot", - "count" => request.blob_ids.len(), - "peer" => %peer_id - ); + pub fn single_blob_lookup_request_retry( + &self, + id: Id, + blob_peer_id: PeerId, + blob_request: BlobsByRootRequest, + lookup_type: LookupType, + ) -> Result<(), &'static str> { + let request_id = RequestId::Sync(SyncRequestId::SingleBlob { id }); + + if !blob_request.blob_ids.is_empty() { + trace!( + self.log, + "Sending BlobsByRoot Request"; + "method" => "BlobsByRoot", + "count" => blob_request.blob_ids.len(), + "peer" => %blob_peer_id + ); - self.send_network_msg(NetworkMessage::SendRequest { - peer_id, - request: Request::BlobsByRoot(request), - request_id, - })?; - Ok(id) + self.send_network_msg(NetworkMessage::SendRequest { + peer_id: blob_peer_id, + request: Request::BlobsByRoot(blob_request), + request_id, + })?; + } + Ok(()) } pub fn is_execution_engine_online(&self) -> bool { @@ -532,7 +499,7 @@ impl SyncNetworkContext { } /// Reports to the scoring algorithm the behaviour of a peer. - pub fn report_peer(&mut self, peer_id: PeerId, action: PeerAction, msg: &'static str) { + pub fn report_peer(&self, peer_id: PeerId, action: PeerAction, msg: &'static str) { debug!(self.log, "Sync reporting peer"; "peer_id" => %peer_id, "action" => %action); self.network_send .send(NetworkMessage::ReportPeer { @@ -547,7 +514,7 @@ impl SyncNetworkContext { } /// Subscribes to core topics. - pub fn subscribe_core_topics(&mut self) { + pub fn subscribe_core_topics(&self) { self.network_send .send(NetworkMessage::SubscribeCoreTopics) .unwrap_or_else(|e| { @@ -556,7 +523,7 @@ impl SyncNetworkContext { } /// Sends an arbitrary network message. - fn send_network_msg(&mut self, msg: NetworkMessage) -> Result<(), &'static str> { + fn send_network_msg(&self, msg: NetworkMessage) -> Result<(), &'static str> { self.network_send.send(msg).map_err(|_| { debug!(self.log, "Could not send message to the network service"); "Network channel send Failed" @@ -572,10 +539,10 @@ impl SyncNetworkContext { &self.network_beacon_processor } - fn next_id(&mut self) -> Id { - let id = self.request_id; - self.request_id += 1; - id + pub fn next_id(&self) -> Id { + let current_value = self.request_id.get(); + self.request_id.set(current_value + 1); + current_value } /// Check whether a batch for this epoch (and only this epoch) should request just blocks or @@ -587,25 +554,17 @@ impl SyncNetworkContext { const _: () = assert!( super::backfill_sync::BACKFILL_EPOCHS_PER_BATCH == 1 && super::range_sync::EPOCHS_PER_BATCH == 1, - "To deal with alignment with 4844 boundaries, batches need to be of just one epoch" + "To deal with alignment with deneb boundaries, batches need to be of just one epoch" ); - #[cfg(test)] - { - // Keep tests only for blocks. - ByRangeRequestType::Blocks - } - #[cfg(not(test))] - { - if let Some(data_availability_boundary) = self.chain.data_availability_boundary() { - if epoch >= data_availability_boundary { - ByRangeRequestType::BlocksAndBlobs - } else { - ByRangeRequestType::Blocks - } + if let Some(data_availability_boundary) = self.chain.data_availability_boundary() { + if epoch >= data_availability_boundary { + ByRangeRequestType::BlocksAndBlobs } else { ByRangeRequestType::Blocks } + } else { + ByRangeRequestType::Blocks } } } From e27161ed2171e34527d17cb8067c0a16a684e132 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Tue, 25 Jul 2023 16:18:59 -0400 Subject: [PATCH 05/18] get tests compiling --- .../network_beacon_processor/sync_methods.rs | 4 - .../network/src/sync/block_lookups/mod.rs | 71 +-- .../src/sync/block_lookups/parent_lookup.rs | 20 +- .../sync/block_lookups/single_block_lookup.rs | 114 ++--- .../network/src/sync/block_lookups/tests.rs | 422 +++++++++++------- beacon_node/network/src/sync/manager.rs | 40 +- .../network/src/sync/network_context.rs | 17 +- 7 files changed, 385 insertions(+), 303 deletions(-) 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 b21bc6abde8..a1176bebbca 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -1,6 +1,5 @@ use crate::metrics; use crate::network_beacon_processor::{NetworkBeaconProcessor, FUTURE_SLOT_TOLERANCE}; -use crate::sync::manager::ResponseType; use crate::sync::BatchProcessResult; use crate::sync::{ manager::{BlockProcessType, SyncMessage}, @@ -96,7 +95,6 @@ impl NetworkBeaconProcessor { self.send_sync_message(SyncMessage::BlockComponentProcessed { process_type, result: crate::sync::manager::BlockProcessingResult::Ignored, - response_type: crate::sync::manager::ResponseType::Block, }); }; (process_fn, Box::new(ignore_fn)) @@ -251,7 +249,6 @@ impl NetworkBeaconProcessor { self.send_sync_message(SyncMessage::BlockComponentProcessed { process_type, result: result.into(), - response_type: ResponseType::Block, }); // Drop the handle to remove the entry from the cache @@ -303,7 +300,6 @@ impl NetworkBeaconProcessor { self.send_sync_message(SyncMessage::BlockComponentProcessed { process_type, result: result.into(), - response_type: ResponseType::Blob, }); } diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index d9184cee594..79ef277468f 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -5,7 +5,9 @@ use super::BatchProcessResult; use super::{manager::BlockProcessType, network_context::SyncNetworkContext}; use crate::metrics; use crate::network_beacon_processor::ChainSegmentProcessId; -use crate::sync::block_lookups::single_block_lookup::LookupId; +use crate::sync::block_lookups::parent_lookup::ParentLookup; +use crate::sync::block_lookups::single_block_lookup::LookupVerifyError; +use crate::sync::manager::{Id, ResponseType}; use beacon_chain::block_verification_types::{AsBlock, RpcBlock}; use beacon_chain::data_availability_checker::{AvailabilityCheckError, DataAvailabilityChecker}; use beacon_chain::{AvailabilityProcessingStatus, BeaconChainTypes, BlockError}; @@ -26,7 +28,7 @@ use std::time::Duration; use store::{Hash256, SignedBeaconBlock}; use strum::Display; use types::blob_sidecar::FixedBlobSidecarList; -use types::{BlobSidecar, EthSpec, Slot}; +use types::Slot; pub(crate) mod delayed_lookup; mod parent_lookup; @@ -116,7 +118,7 @@ impl BlockLookups { &mut self, block_root: Hash256, peer_source: PeerShouldHave, - cx: &SyncNetworkContext, + cx: &mut SyncNetworkContext, ) { let lookup = self.new_current_lookup(block_root, None, &[peer_source], cx); if let Some(lookup) = lookup { @@ -131,7 +133,7 @@ impl BlockLookups { &mut self, block_root: Hash256, peer_source: PeerShouldHave, - cx: &SyncNetworkContext, + cx: &mut SyncNetworkContext, ) { let lookup = self.new_current_lookup(block_root, None, &[peer_source], cx); if let Some(lookup) = lookup { @@ -150,7 +152,7 @@ impl BlockLookups { block_root: Hash256, parent_components: Option>, peer_source: &[PeerShouldHave], - cx: &SyncNetworkContext, + cx: &mut SyncNetworkContext, ) { let lookup = self.new_current_lookup(block_root, parent_components, peer_source, cx); if let Some(lookup) = lookup { @@ -170,7 +172,7 @@ impl BlockLookups { block_root: Hash256, parent_components: Option>, peer_source: &[PeerShouldHave], - cx: &SyncNetworkContext, + cx: &mut SyncNetworkContext, ) { let lookup = self.new_current_lookup(block_root, parent_components, peer_source, cx); if let Some(lookup) = lookup { @@ -207,10 +209,11 @@ impl BlockLookups { cx: &SyncNetworkContext, ) -> Result<(), ()> { for (_, lookup) in self.single_block_lookups.iter_mut() { - if lookup.block_request_state.requested_block_root == block_root && !lookup.triggered { - if lookup.request_block_and_blobs(cx).is_ok() { - lookup.triggered = true; - } + if lookup.block_request_state.requested_block_root == block_root + && !lookup.triggered + && lookup.request_block_and_blobs(cx).is_ok() + { + lookup.triggered = true; } } Ok(()) @@ -228,13 +231,13 @@ impl BlockLookups { block_root: Hash256, parent_components: Option>, peers: &[PeerShouldHave], - cx: &SyncNetworkContext, + cx: &mut SyncNetworkContext, ) -> Option> { // Do not re-request a block that is already being requested if let Some((_, lookup)) = self .single_block_lookups .iter_mut() - .find(|(id, lookup)| lookup.is_for_block(block_root)) + .find(|(_id, lookup)| lookup.is_for_block(block_root)) { lookup.add_peers(peers); if let Some(components) = parent_components { @@ -274,7 +277,7 @@ impl BlockLookups { parent_components, peers, self.da_checker.clone(), - cx, + cx.next_id(), )) } @@ -286,7 +289,7 @@ impl BlockLookups { block_root: Hash256, parent_root: Hash256, peer_id: PeerId, - cx: &SyncNetworkContext, + cx: &mut SyncNetworkContext, ) { // Gossip blocks or blobs shouldn't be propagated if parents are unavailable. let peer_source = PeerShouldHave::BlockAndBlobs(peer_id); @@ -383,17 +386,15 @@ impl BlockLookups { } } } - } else { - if let Err(()) = R::send_for_processing( - id, - self, - root, - R::verified_to_reconstructed(verified_response), - seen_timestamp, - &cx, - ) { - self.single_block_lookups.remove(&id); - } + } else if let Err(()) = R::send_for_processing( + id, + self, + root, + R::verified_to_reconstructed(verified_response), + seen_timestamp, + cx, + ) { + self.single_block_lookups.remove(&id); } } Ok(None) => {} @@ -616,7 +617,13 @@ impl BlockLookups { let request_state = R::request_state_mut(lookup); request_state.register_failure_downloading(); let response_type = R::response_type(); - trace!(self.log, "Single lookup failed"; "block_root" => ?root, "error" => msg, "response_type" => ?response_type); + trace!(self.log, + "Single lookup failed"; + "block_root" => ?root, + "error" => msg, + "peer_id" => %peer_id, + "response_type" => ?response_type + ); if let Err(()) = request_state.retry_request_after_failure(id, cx, &self.log) { self.single_block_lookups.remove(&id); }; @@ -633,7 +640,7 @@ impl BlockLookups { &mut self, target_id: Id, result: BlockProcessingResult, - cx: &SyncNetworkContext, + cx: &mut SyncNetworkContext, ) { let Some(request_ref) = self.single_block_lookups.get_mut(&target_id) else { debug!(self.log, "Block component processed for single block lookup not present" ); @@ -793,8 +800,7 @@ impl BlockLookups { &mut self, chain_hash: Hash256, result: BlockProcessingResult, - response_type: ResponseType, - cx: &SyncNetworkContext, + cx: &mut SyncNetworkContext, ) { let index = self .parent_lookups @@ -865,7 +871,8 @@ impl BlockLookups { // to send for processing. if let Some(child_lookup_id) = self.single_block_lookups.iter().find_map(|(id, lookup)| { - (lookup.block_request_state.requested_block_root == chain_hash).then(|| *id) + (lookup.block_request_state.requested_block_root == chain_hash) + .then_some(*id) }) { let Some(child_lookup) = self.single_block_lookups.get_mut(&child_lookup_id) else { @@ -972,7 +979,7 @@ impl BlockLookups { .single_block_lookups .iter() .find_map(|(id, req)| - (req.block_request_state.requested_block_root == chain_hash).then(|| *id)) else { + (req.block_request_state.requested_block_root == chain_hash).then_some(*id)) else { warn!(self.log, "No id found for single block lookup"; "chain_hash" => %chain_hash); return; }; @@ -991,7 +998,7 @@ impl BlockLookups { chain_hash, rpc_block, Duration::from_secs(0), //TODO(sean) pipe this through - BlockProcessType::SingleBlock { id: id }, + BlockProcessType::SingleBlock { id }, cx, ) .is_err() diff --git a/beacon_node/network/src/sync/block_lookups/parent_lookup.rs b/beacon_node/network/src/sync/block_lookups/parent_lookup.rs index 1df51bf7f06..6afeaccc7b7 100644 --- a/beacon_node/network/src/sync/block_lookups/parent_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/parent_lookup.rs @@ -3,7 +3,6 @@ use super::{DownloadedBlocks, PeerShouldHave}; use crate::sync::block_lookups::single_block_lookup::{ Parent, RequestState, State, UnknownParentComponents, }; -use crate::sync::block_lookups::Lookup; use crate::sync::{manager::SLOT_IMPORT_TOLERANCE, network_context::SyncNetworkContext}; use beacon_chain::block_verification_types::AsBlock; use beacon_chain::block_verification_types::RpcBlock; @@ -14,8 +13,6 @@ use lighthouse_network::PeerId; use std::sync::Arc; use store::Hash256; use strum::IntoStaticStr; -use types::blob_sidecar::FixedBlobSidecarList; -use types::SignedBeaconBlock; /// How many attempts we try to find a parent of a block before we give up trying. pub(crate) const PARENT_FAIL_TOLERANCE: u8 = 5; @@ -67,14 +64,14 @@ impl ParentLookup { parent_root: Hash256, peer_id: PeerShouldHave, da_checker: Arc>, - cx: &SyncNetworkContext, + cx: &mut SyncNetworkContext, ) -> Self { let current_parent_request = SingleBlockLookup::new( parent_root, Some(<_>::default()), &[peer_id], da_checker, - cx, + cx.next_id(), ); Self { @@ -144,16 +141,6 @@ impl ParentLookup { Some(UnknownParentComponents::default()); } - pub fn add_current_request_block(&mut self, block: Arc>) { - // Cache the block. - self.current_parent_request.add_unknown_parent_block(block); - } - - pub fn add_current_request_blobs(&mut self, blobs: FixedBlobSidecarList) { - // Cache the blobs. - self.current_parent_request.add_unknown_parent_blobs(blobs); - } - pub fn processing_peer(&self) -> Result { self.current_parent_request .block_request_state @@ -234,8 +221,7 @@ impl ParentLookup { // be dropped and the peer downscored. if let Some(parent_root) = root_and_block .as_ref() - .map(|(_, block)| R::get_parent_root(block)) - .flatten() + .and_then(|(_, block)| R::get_parent_root(block)) { if failed_chains.contains(&parent_root) { request_state.register_failure_downloading(); diff --git a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs index e6e1e72e72f..5499c669864 100644 --- a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs @@ -1,5 +1,4 @@ use super::{PeerShouldHave, ResponseType}; -use crate::sync::block_lookups::parent_lookup::RequestError::SendFailed; use crate::sync::block_lookups::parent_lookup::PARENT_FAIL_TOLERANCE; use crate::sync::block_lookups::{ BlockLookups, Id, LookupType, RootBlobsTuple, RootBlockTuple, SINGLE_BLOCK_LOOKUP_MAX_ATTEMPTS, @@ -9,7 +8,6 @@ use crate::sync::network_context::SyncNetworkContext; use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::data_availability_checker::DataAvailabilityChecker; use beacon_chain::{get_block_root, BeaconChainTypes}; -use itertools::Itertools; use lighthouse_network::rpc::methods::BlobsByRootRequest; use lighthouse_network::{rpc::BlocksByRootRequest, PeerId}; use rand::seq::IteratorRandom; @@ -95,9 +93,6 @@ pub trait RequestState { fn processing_peer(&self) -> Result { self.get_state().processing_peer() } - fn downloading_peer(&self) -> Result { - self.get_state().peer() - } fn set_component_processed(&mut self) { self.get_state_mut().component_processed = true; } @@ -185,7 +180,7 @@ pub trait RequestState { request_state.failed_processing >= request_state.failed_downloading } fn get_peer(&mut self) -> Result { - let mut request_state = self.get_state_mut(); + let request_state = self.get_state_mut(); let Some(peer_id) = request_state .available_peers .iter() @@ -441,7 +436,7 @@ impl BlobRequestState { requested_ids: <_>::default(), blob_download_queue: <_>::default(), state: SingleLookupRequestState::new(peer_source), - _phantom: PhantomData::default(), + _phantom: PhantomData, } } } @@ -457,7 +452,7 @@ impl BlockRequestState { Self { requested_block_root: block_root, state: SingleLookupRequestState::new(peers), - _phantom: PhantomData::default(), + _phantom: PhantomData, } } } @@ -585,10 +580,10 @@ impl SingleBlockLookup { unknown_parent_components: Option>, peers: &[PeerShouldHave], da_checker: Arc>, - cx: &SyncNetworkContext, + id: Id, ) -> Self { Self { - id: cx.next_id(), + id, block_request_state: BlockRequestState::new(requested_block_root, peers), blob_request_state: BlobRequestState::new(peers), da_checker, @@ -732,27 +727,6 @@ impl SingleBlockLookup { self.unknown_parent_components = Some(components); } } - pub fn add_unknown_parent_block(&mut self, block: Arc>) { - if let Some(ref mut components) = self.unknown_parent_components { - components.add_unknown_parent_block(block) - } else { - self.unknown_parent_components = Some(UnknownParentComponents { - downloaded_block: Some(block), - downloaded_blobs: FixedBlobSidecarList::default(), - }) - } - } - - pub fn add_unknown_parent_blobs(&mut self, blobs: FixedBlobSidecarList) { - if let Some(ref mut components) = self.unknown_parent_components { - components.add_unknown_parent_blobs(blobs) - } else { - self.unknown_parent_components = Some(UnknownParentComponents { - downloaded_block: None, - downloaded_blobs: blobs, - }) - } - } pub fn add_peers(&mut self, peers: &[PeerShouldHave]) { for peer in peers { @@ -886,14 +860,6 @@ impl SingleLookupRequestState { } } - pub fn peer(&self) -> Result { - match &self.state { - State::Processing { peer_id } => Ok(*peer_id), - State::Downloading { peer_id } => Ok(*peer_id), - _ => Err(()), - } - } - pub fn remove_peer_if_useless(&mut self, peer_id: &PeerId) { if !self.available_peers.is_empty() || self.potential_peers.len() > 1 { self.potential_peers.remove(peer_id); @@ -981,6 +947,26 @@ mod tests { } type T = Witness, E, MemoryStore, MemoryStore>; + struct TestLookup1; + + impl Lookup for TestLookup1 { + const MAX_ATTEMPTS: u8 = 3; + + fn lookup_type() -> LookupType { + panic!() + } + } + + struct TestLookup2; + + impl Lookup for TestLookup2 { + const MAX_ATTEMPTS: u8 = 4; + + fn lookup_type() -> LookupType { + panic!() + } + } + #[test] fn test_happy_path() { let peer_id = PeerShouldHave::BlockAndBlobs(PeerId::random()); @@ -998,15 +984,28 @@ mod tests { DataAvailabilityChecker::new(slot_clock, None, store.into(), spec) .expect("data availability checker"), ); - let mut sl = - SingleBlockLookup::<4, T>::new(block.canonical_root(), None, &[peer_id], da_checker); - sl.request_block().unwrap(); - sl.verify_block(Some(block.into())).unwrap().unwrap(); + let mut sl = SingleBlockLookup::::new( + block.canonical_root(), + None, + &[peer_id], + da_checker, + 1, + ); + as RequestState>::build_request( + &mut sl.block_request_state, + ) + .unwrap(); + as RequestState>::verify_response( + &mut sl.block_request_state, + block.canonical_root(), + Some(block.into()), + ) + .unwrap() + .unwrap(); } #[test] fn test_block_lookup_failures() { - const FAILURES: u8 = 3; let peer_id = PeerShouldHave::BlockAndBlobs(PeerId::random()); let block = rand_block(); let spec = E::default_spec(); @@ -1024,25 +1023,40 @@ mod tests { .expect("data availability checker"), ); - let mut sl = SingleBlockLookup::::new( + let mut sl = SingleBlockLookup::::new( block.canonical_root(), None, &[peer_id], da_checker, + 1, ); - for _ in 1..FAILURES { - sl.request_block().unwrap(); + for _ in 1..TestLookup2::MAX_ATTEMPTS { + as RequestState>::build_request( + &mut sl.block_request_state, + ) + .unwrap(); sl.block_request_state.state.register_failure_downloading(); } // Now we receive the block and send it for processing - sl.request_block().unwrap(); - sl.verify_block(Some(block.into())).unwrap().unwrap(); + as RequestState>::build_request( + &mut sl.block_request_state, + ) + .unwrap(); + as RequestState>::verify_response( + &mut sl.block_request_state, + block.canonical_root(), + Some(block.into()), + ) + .unwrap() + .unwrap(); // One processing failure maxes the available attempts sl.block_request_state.state.register_failure_processing(); assert_eq!( - sl.request_block(), + as RequestState>::build_request( + &mut sl.block_request_state + ), Err(LookupRequestError::TooManyAttempts { cannot_process: false }) diff --git a/beacon_node/network/src/sync/block_lookups/tests.rs b/beacon_node/network/src/sync/block_lookups/tests.rs index c12f101a3ec..d7c9688e502 100644 --- a/beacon_node/network/src/sync/block_lookups/tests.rs +++ b/beacon_node/network/src/sync/block_lookups/tests.rs @@ -20,7 +20,8 @@ use tokio::sync::mpsc; use types::{ map_fork_name, map_fork_name_with, test_utils::{SeedableRng, TestRandom, XorShiftRng}, - BeaconBlock, EthSpec, ForkName, FullPayloadDeneb, MinimalEthSpec as E, SignedBeaconBlock, + BeaconBlock, BlobSidecar, EthSpec, ForkName, FullPayloadDeneb, MinimalEthSpec as E, + SignedBeaconBlock, }; type T = Witness, E, MemoryStore, MemoryStore>; @@ -304,7 +305,13 @@ fn test_single_block_lookup_happy_path() { // The peer provides the correct block, should not be penalized. Now the block should be sent // for processing. - bl.single_lookup_response(id, peer_id, Some(block.into()), D, &mut cx); + bl.single_lookup_response::>( + id, + peer_id, + Some(block.into()), + D, + &cx, + ); rig.expect_empty_network(); rig.expect_block_process(response_type); @@ -313,11 +320,10 @@ fn test_single_block_lookup_happy_path() { // Send the stream termination. Peer should have not been penalized, and the request removed // after processing. - bl.single_lookup_response(id, peer_id, None, D, &mut cx); - bl.single_block_component_processed( + bl.single_lookup_response::>(id, peer_id, None, D, &cx); + bl.single_block_component_processed::>( id, BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(block_root)), - response_type, &mut cx, ); rig.expect_empty_network(); @@ -346,7 +352,7 @@ fn test_single_block_lookup_empty_response() { } // The peer does not have the block. It should be penalized. - bl.single_lookup_response(id, peer_id, None, D, &mut cx); + bl.single_lookup_response::>(id, peer_id, None, D, &cx); rig.expect_penalty(); rig.expect_block_request(response_type); // it should be retried @@ -375,12 +381,18 @@ fn test_single_block_lookup_wrong_response() { // Peer sends something else. It should be penalized. let bad_block = rig.rand_block(fork_name); - bl.single_lookup_response(id, peer_id, Some(bad_block.into()), D, &mut cx); + bl.single_lookup_response::>( + id, + peer_id, + Some(bad_block.into()), + D, + &cx, + ); rig.expect_penalty(); rig.expect_block_request(response_type); // should be retried // Send the stream termination. This should not produce an additional penalty. - bl.single_lookup_response(id, peer_id, None, D, &mut cx); + bl.single_lookup_response::>(id, peer_id, None, D, &cx); rig.expect_empty_network(); } @@ -406,7 +418,12 @@ fn test_single_block_lookup_failure() { } // The request fails. RPC failures are handled elsewhere so we should not penalize the peer. - bl.single_block_lookup_failed(id, &peer_id, &mut cx, RPCError::UnsupportedProtocol); + bl.single_block_lookup_failed::>( + id, + &peer_id, + &cx, + RPCError::UnsupportedProtocol, + ); rig.expect_block_request(response_type); rig.expect_empty_network(); } @@ -438,7 +455,13 @@ fn test_single_block_lookup_becomes_parent_request() { // The peer provides the correct block, should not be penalized. Now the block should be sent // for processing. - bl.single_lookup_response(id, peer_id, Some(block.clone()), D, &mut cx); + bl.single_lookup_response::>( + id, + peer_id, + Some(block.clone()), + D, + &cx, + ); rig.expect_empty_network(); rig.expect_block_process(response_type); @@ -447,10 +470,9 @@ fn test_single_block_lookup_becomes_parent_request() { // Send the stream termination. Peer should have not been penalized, and the request moved to a // parent request after processing. - bl.single_block_component_processed( + bl.single_block_component_processed::>( id, BlockError::ParentUnknown(block.into()).into(), - response_type, &mut cx, ); assert_eq!(bl.single_block_lookups.len(), 1); @@ -491,22 +513,23 @@ fn test_parent_lookup_happy_path() { } // Peer sends the right block, it should be sent for processing. Peer should not be penalized. - bl.parent_lookup_response(id, peer_id, Some(parent.into()), D, &mut cx); + bl.parent_lookup_response::>( + id, + peer_id, + Some(parent.into()), + D, + &cx, + ); rig.expect_block_process(response_type); rig.expect_empty_network(); // Processing succeeds, now the rest of the chain should be sent for processing. - bl.parent_block_processed( - chain_hash, - BlockError::BlockIsAlreadyKnown.into(), - response_type, - &mut cx, - ); + bl.parent_block_processed(chain_hash, BlockError::BlockIsAlreadyKnown.into(), &mut cx); rig.expect_parent_chain_process(); let process_result = BatchProcessResult::Success { was_non_empty: true, }; - bl.parent_chain_processed(chain_hash, process_result, &mut cx); + bl.parent_chain_processed(chain_hash, process_result, &cx); assert_eq!(bl.parent_lookups.len(), 0); } @@ -538,30 +561,41 @@ fn test_parent_lookup_wrong_response() { // Peer sends the wrong block, peer should be penalized and the block re-requested. let bad_block = rig.rand_block(fork_name); - bl.parent_lookup_response(id1, peer_id, Some(bad_block.into()), D, &mut cx); + bl.parent_lookup_response::>( + id1, + peer_id, + Some(bad_block.into()), + D, + &cx, + ); rig.expect_penalty(); let id2 = rig.expect_parent_request(response_type); // Send the stream termination for the first request. This should not produce extra penalties. - bl.parent_lookup_response(id1, peer_id, None, D, &mut cx); + bl.parent_lookup_response::>(id1, peer_id, None, D, &cx); rig.expect_empty_network(); // Send the right block this time. - bl.parent_lookup_response(id2, peer_id, Some(parent.into()), D, &mut cx); + bl.parent_lookup_response::>( + id2, + peer_id, + Some(parent.into()), + D, + &cx, + ); rig.expect_block_process(response_type); // Processing succeeds, now the rest of the chain should be sent for processing. bl.parent_block_processed( chain_hash, BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(block_root)), - response_type, &mut cx, ); rig.expect_parent_chain_process(); let process_result = BatchProcessResult::Success { was_non_empty: true, }; - bl.parent_chain_processed(chain_hash, process_result, &mut cx); + bl.parent_chain_processed(chain_hash, process_result, &cx); assert_eq!(bl.parent_lookups.len(), 0); } @@ -592,26 +626,31 @@ fn test_parent_lookup_empty_response() { } // Peer sends an empty response, peer should be penalized and the block re-requested. - bl.parent_lookup_response(id1, peer_id, None, D, &mut cx); + bl.parent_lookup_response::>(id1, peer_id, None, D, &cx); rig.expect_penalty(); let id2 = rig.expect_parent_request(response_type); // Send the right block this time. - bl.parent_lookup_response(id2, peer_id, Some(parent.into()), D, &mut cx); + bl.parent_lookup_response::>( + id2, + peer_id, + Some(parent.into()), + D, + &cx, + ); rig.expect_block_process(response_type); // Processing succeeds, now the rest of the chain should be sent for processing. bl.parent_block_processed( chain_hash, BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(block_root)), - response_type, &mut cx, ); rig.expect_parent_chain_process(); let process_result = BatchProcessResult::Success { was_non_empty: true, }; - bl.parent_chain_processed(chain_hash, process_result, &mut cx); + bl.parent_chain_processed(chain_hash, process_result, &cx); assert_eq!(bl.parent_lookups.len(), 0); } @@ -642,10 +681,10 @@ fn test_parent_lookup_rpc_failure() { } // The request fails. It should be tried again. - bl.parent_lookup_failed( + bl.parent_lookup_failed::>( id1, peer_id, - &mut cx, + &cx, RPCError::ErrorResponse( RPCResponseErrorCode::ResourceUnavailable, "older than deneb".into(), @@ -654,21 +693,26 @@ fn test_parent_lookup_rpc_failure() { let id2 = rig.expect_parent_request(response_type); // Send the right block this time. - bl.parent_lookup_response(id2, peer_id, Some(parent.into()), D, &mut cx); + bl.parent_lookup_response::>( + id2, + peer_id, + Some(parent.into()), + D, + &cx, + ); rig.expect_block_process(response_type); // Processing succeeds, now the rest of the chain should be sent for processing. bl.parent_block_processed( chain_hash, BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(block_root)), - response_type, &mut cx, ); rig.expect_parent_chain_process(); let process_result = BatchProcessResult::Success { was_non_empty: true, }; - bl.parent_chain_processed(chain_hash, process_result, &mut cx); + bl.parent_chain_processed(chain_hash, process_result, &cx); assert_eq!(bl.parent_lookups.len(), 0); } @@ -701,10 +745,10 @@ fn test_parent_lookup_too_many_attempts() { // make sure every error is accounted for 0 => { // The request fails. It should be tried again. - bl.parent_lookup_failed( + bl.parent_lookup_failed::>( id, peer_id, - &mut cx, + &cx, RPCError::ErrorResponse( RPCResponseErrorCode::ResourceUnavailable, "older than deneb".into(), @@ -714,9 +758,15 @@ fn test_parent_lookup_too_many_attempts() { _ => { // Send a bad block this time. It should be tried again. let bad_block = rig.rand_block(fork_name); - bl.parent_lookup_response(id, peer_id, Some(bad_block.into()), D, &mut cx); + bl.parent_lookup_response::>( + id, + peer_id, + Some(bad_block.into()), + D, + &cx, + ); // Send the stream termination - bl.parent_lookup_response(id, peer_id, None, D, &mut cx); + bl.parent_lookup_response::>(id, peer_id, None, D, &cx); rig.expect_penalty(); } } @@ -764,10 +814,10 @@ fn test_parent_lookup_too_many_download_attempts_no_blacklist() { } if i % 2 != 0 { // The request fails. It should be tried again. - bl.parent_lookup_failed( + bl.parent_lookup_failed::>( id, peer_id, - &mut cx, + &cx, RPCError::ErrorResponse( RPCResponseErrorCode::ResourceUnavailable, "older than deneb".into(), @@ -776,7 +826,13 @@ fn test_parent_lookup_too_many_download_attempts_no_blacklist() { } else { // Send a bad block this time. It should be tried again. let bad_block = rig.rand_block(fork_name); - bl.parent_lookup_response(id, peer_id, Some(bad_block.into()), D, &mut cx); + bl.parent_lookup_response::>( + id, + peer_id, + Some(bad_block.into()), + D, + &cx, + ); rig.expect_penalty(); } if i < parent_lookup::PARENT_FAIL_TOLERANCE { @@ -825,10 +881,10 @@ fn test_parent_lookup_too_many_processing_attempts_must_blacklist() { let _ = rig.expect_parent_request(ResponseType::Blob); } // The request fails. It should be tried again. - bl.parent_lookup_failed( + bl.parent_lookup_failed::>( id, peer_id, - &mut cx, + &cx, RPCError::ErrorResponse( RPCResponseErrorCode::ResourceUnavailable, "older than deneb".into(), @@ -846,14 +902,15 @@ fn test_parent_lookup_too_many_processing_attempts_must_blacklist() { // we don't require a response because we're generateing 0-blob blocks in this test. assert!(!bl.failed_chains.contains(&block_root)); // send the right parent but fail processing - bl.parent_lookup_response(id, peer_id, Some(parent.clone()), D, &mut cx); - bl.parent_block_processed( - block_root, - BlockError::InvalidSignature.into(), - response_type, - &mut cx, + bl.parent_lookup_response::>( + id, + peer_id, + Some(parent.clone()), + D, + &cx, ); - bl.parent_lookup_response(id, peer_id, None, D, &mut cx); + bl.parent_block_processed(block_root, BlockError::InvalidSignature.into(), &mut cx); + bl.parent_lookup_response::>(id, peer_id, None, D, &cx); rig.expect_penalty(); } @@ -902,16 +959,21 @@ fn test_parent_lookup_too_deep() { let _ = rig.expect_parent_request(ResponseType::Blob); } // the block - bl.parent_lookup_response(id, peer_id, Some(block.clone()), D, &mut cx); + bl.parent_lookup_response::>( + id, + peer_id, + Some(block.clone()), + D, + &cx, + ); // the stream termination - bl.parent_lookup_response(id, peer_id, None, D, &mut cx); + bl.parent_lookup_response::>(id, peer_id, None, D, &cx); // the processing request rig.expect_block_process(response_type); // the processing result bl.parent_block_processed( chain_hash, BlockError::ParentUnknown(block.into()).into(), - response_type, &mut cx, ) } @@ -971,7 +1033,13 @@ fn test_single_block_lookup_ignored_response() { // The peer provides the correct block, should not be penalized. Now the block should be sent // for processing. - bl.single_lookup_response(id, peer_id, Some(block.into()), D, &mut cx); + bl.single_lookup_response::>( + id, + peer_id, + Some(block.into()), + D, + &cx, + ); rig.expect_empty_network(); rig.expect_block_process(response_type); @@ -980,9 +1048,13 @@ fn test_single_block_lookup_ignored_response() { // Send the stream termination. Peer should have not been penalized, and the request removed // after processing. - bl.single_lookup_response(id, peer_id, None, D, &mut cx); + bl.single_lookup_response::>(id, peer_id, None, D, &cx); // Send an Ignored response, the request should be dropped - bl.single_block_component_processed(id, BlockProcessingResult::Ignored, response_type, &mut cx); + bl.single_block_component_processed::>( + id, + BlockProcessingResult::Ignored, + &mut cx, + ); rig.expect_empty_network(); assert_eq!(bl.single_block_lookups.len(), 0); } @@ -1015,17 +1087,18 @@ fn test_parent_lookup_ignored_response() { } // Peer sends the right block, it should be sent for processing. Peer should not be penalized. - bl.parent_lookup_response(id, peer_id, Some(parent.into()), D, &mut cx); + bl.parent_lookup_response::>( + id, + peer_id, + Some(parent.into()), + D, + &cx, + ); rig.expect_block_process(response_type); rig.expect_empty_network(); // Return an Ignored result. The request should be dropped - bl.parent_block_processed( - chain_hash, - BlockProcessingResult::Ignored, - response_type, - &mut cx, - ); + bl.parent_block_processed(chain_hash, BlockProcessingResult::Ignored, &mut cx); rig.expect_empty_network(); assert_eq!(bl.parent_lookups.len(), 0); } @@ -1092,25 +1165,25 @@ fn test_same_chain_race_condition() { let _ = rig.expect_parent_request(ResponseType::Blob); } // the block - bl.parent_lookup_response(id, peer_id, Some(block.clone()), D, &mut cx); + bl.parent_lookup_response::>( + id, + peer_id, + Some(block.clone()), + D, + &cx, + ); // the stream termination - bl.parent_lookup_response(id, peer_id, None, D, &mut cx); + bl.parent_lookup_response::>(id, peer_id, None, D, &cx); // the processing request rig.expect_block_process(response_type); // the processing result if i + 2 == depth { // one block was removed - bl.parent_block_processed( - chain_hash, - BlockError::BlockIsAlreadyKnown.into(), - response_type, - &mut cx, - ) + bl.parent_block_processed(chain_hash, BlockError::BlockIsAlreadyKnown.into(), &mut cx) } else { bl.parent_block_processed( chain_hash, BlockError::ParentUnknown(block.into()).into(), - response_type, &mut cx, ) } @@ -1137,7 +1210,7 @@ fn test_same_chain_race_condition() { let process_result = BatchProcessResult::Success { was_non_empty: true, }; - bl.parent_chain_processed(chain_hash, process_result, &mut cx); + bl.parent_chain_processed(chain_hash, process_result, &cx); assert_eq!(bl.parent_lookups.len(), 0); } @@ -1307,12 +1380,12 @@ mod deneb_only { fn parent_block_response(mut self) -> Self { self.rig.expect_empty_network(); - self.bl.parent_lookup_response( + self.bl.parent_lookup_response::>( self.parent_block_req_id.expect("parent request id"), self.peer_id, self.parent_block.clone(), D, - &mut self.cx, + &self.cx, ); assert_eq!(self.bl.parent_lookups.len(), 1); @@ -1321,22 +1394,24 @@ mod deneb_only { fn parent_blob_response(mut self) -> Self { for blob in &self.parent_blobs { - self.bl.parent_lookup_blob_response( - self.parent_blob_req_id.expect("parent blob request id"), + self.bl + .parent_lookup_response::>( + self.parent_blob_req_id.expect("parent blob request id"), + self.peer_id, + Some(blob.clone()), + D, + &self.cx, + ); + assert_eq!(self.bl.parent_lookups.len(), 1); + } + self.bl + .parent_lookup_response::>( + self.parent_blob_req_id.expect("blob request id"), self.peer_id, - Some(blob.clone()), + None, D, - &mut self.cx, + &self.cx, ); - assert_eq!(self.bl.parent_lookups.len(), 1); - } - self.bl.parent_lookup_blob_response( - self.parent_blob_req_id.expect("blob request id"), - self.peer_id, - None, - D, - &mut self.cx, - ); self } @@ -1353,13 +1428,14 @@ mod deneb_only { fn block_response(mut self) -> Self { // The peer provides the correct block, should not be penalized. Now the block should be sent // for processing. - self.bl.single_lookup_response( - self.block_req_id.expect("block request id"), - self.peer_id, - self.block.clone(), - D, - &mut self.cx, - ); + self.bl + .single_lookup_response::>( + self.block_req_id.expect("block request id"), + self.peer_id, + self.block.clone(), + D, + &self.cx, + ); self.rig.expect_empty_network(); // The request should still be active. @@ -1369,22 +1445,24 @@ mod deneb_only { fn blobs_response(mut self) -> Self { for blob in &self.blobs { - self.bl.single_blob_lookup_response( + self.bl + .single_lookup_response::>( + self.blob_req_id.expect("blob request id"), + self.peer_id, + Some(blob.clone()), + D, + &self.cx, + ); + assert_eq!(self.bl.single_block_lookups.len(), 1); + } + self.bl + .single_lookup_response::>( self.blob_req_id.expect("blob request id"), self.peer_id, - Some(blob.clone()), + None, D, - &mut self.cx, + &self.cx, ); - assert_eq!(self.bl.single_block_lookups.len(), 1); - } - self.bl.single_blob_lookup_response( - self.blob_req_id.expect("blob request id"), - self.peer_id, - None, - D, - &mut self.cx, - ); self } @@ -1402,58 +1480,63 @@ mod deneb_only { } fn empty_block_response(mut self) -> Self { - self.bl.single_lookup_response( - self.block_req_id.expect("block request id"), - self.peer_id, - None, - D, - &mut self.cx, - ); + self.bl + .single_lookup_response::>( + self.block_req_id.expect("block request id"), + self.peer_id, + None, + D, + &self.cx, + ); self } fn empty_blobs_response(mut self) -> Self { - self.bl.single_blob_lookup_response( - self.blob_req_id.expect("blob request id"), - self.peer_id, - None, - D, - &mut self.cx, - ); + self.bl + .single_lookup_response::>( + self.blob_req_id.expect("blob request id"), + self.peer_id, + None, + D, + &self.cx, + ); self } fn empty_parent_block_response(mut self) -> Self { - self.bl.parent_lookup_response( + self.bl.parent_lookup_response::>( self.parent_block_req_id.expect("block request id"), self.peer_id, None, D, - &mut self.cx, + &self.cx, ); self } fn empty_parent_blobs_response(mut self) -> Self { - self.bl.parent_lookup_blob_response( - self.parent_blob_req_id.expect("blob request id"), - self.peer_id, - None, - D, - &mut self.cx, - ); + self.bl + .parent_lookup_response::>( + self.parent_blob_req_id.expect("blob request id"), + self.peer_id, + None, + D, + &self.cx, + ); self } fn block_imported(mut self) -> Self { // Missing blobs should be the request is not removed, the outstanding blobs request should // mean we do not send a new request. - self.bl.single_block_component_processed( - self.block_req_id.expect("block request id"), - BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(self.block_root)), - ResponseType::Block, - &mut self.cx, - ); + self.bl + .single_block_component_processed::>( + self.block_req_id.expect("block request id"), + BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported( + self.block_root, + )), + &mut self.cx, + ); self.rig.expect_empty_network(); assert_eq!(self.bl.single_block_lookups.len(), 0); self @@ -1463,7 +1546,6 @@ mod deneb_only { self.bl.parent_block_processed( self.block_root, BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(self.block_root)), - ResponseType::Block, &mut self.cx, ); self.rig.expect_empty_network(); @@ -1477,7 +1559,6 @@ mod deneb_only { BlockProcessingResult::Err(BlockError::ParentUnknown(RpcBlock::new_without_blobs( self.parent_block.clone().expect("parent block"), ))), - ResponseType::Block, &mut self.cx, ); assert_eq!(self.bl.parent_lookups.len(), 1); @@ -1488,7 +1569,6 @@ mod deneb_only { self.bl.parent_block_processed( self.block_root, BlockProcessingResult::Err(BlockError::ProposalSignatureInvalid), - ResponseType::Block, &mut self.cx, ); assert_eq!(self.bl.parent_lookups.len(), 1); @@ -1496,53 +1576,53 @@ mod deneb_only { } fn invalid_block_processed(mut self) -> Self { - self.bl.single_block_component_processed( - self.block_req_id.expect("block request id"), - BlockProcessingResult::Err(BlockError::ProposalSignatureInvalid), - ResponseType::Block, - &mut self.cx, - ); + self.bl + .single_block_component_processed::>( + self.block_req_id.expect("block request id"), + BlockProcessingResult::Err(BlockError::ProposalSignatureInvalid), + &mut self.cx, + ); assert_eq!(self.bl.single_block_lookups.len(), 1); self } fn invalid_blob_processed(mut self) -> Self { - self.bl.single_block_component_processed( - self.blob_req_id.expect("blob request id"), - BlockProcessingResult::Err(BlockError::BlobValidation( - BlobError::ProposerSignatureInvalid, - )), - ResponseType::Blob, - &mut self.cx, - ); + self.bl + .single_block_component_processed::>( + self.blob_req_id.expect("blob request id"), + BlockProcessingResult::Err(BlockError::BlobValidation( + BlobError::ProposerSignatureInvalid, + )), + &mut self.cx, + ); assert_eq!(self.bl.single_block_lookups.len(), 1); self } fn missing_components_from_block_request(mut self) -> Self { - self.bl.single_block_component_processed( - self.block_req_id.expect("block request id"), - BlockProcessingResult::Ok(AvailabilityProcessingStatus::MissingComponents( - self.slot, - self.block_root, - )), - ResponseType::Block, - &mut self.cx, - ); + self.bl + .single_block_component_processed::>( + self.block_req_id.expect("block request id"), + BlockProcessingResult::Ok(AvailabilityProcessingStatus::MissingComponents( + self.slot, + self.block_root, + )), + &mut self.cx, + ); assert_eq!(self.bl.single_block_lookups.len(), 1); self } fn missing_components_from_blob_request(mut self) -> Self { - self.bl.single_block_component_processed( - self.blob_req_id.expect("blob request id"), - BlockProcessingResult::Ok(AvailabilityProcessingStatus::MissingComponents( - self.slot, - self.block_root, - )), - ResponseType::Blob, - &mut self.cx, - ); + self.bl + .single_block_component_processed::>( + self.blob_req_id.expect("blob request id"), + BlockProcessingResult::Ok(AvailabilityProcessingStatus::MissingComponents( + self.slot, + self.block_root, + )), + &mut self.cx, + ); assert_eq!(self.bl.single_block_lookups.len(), 1); self } diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 31062336580..cedad8890f0 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -44,7 +44,7 @@ use crate::status::ToStatusMessage; use crate::sync::block_lookups::delayed_lookup; use crate::sync::block_lookups::delayed_lookup::DelayedLookupMessage; use crate::sync::block_lookups::{ - BlobRequestState, BlockRequestState, Current, Parent, RequestState, UnknownParentComponents, + BlobRequestState, BlockRequestState, Current, Parent, UnknownParentComponents, }; use crate::sync::range_sync::ByRangeRequestType; use beacon_chain::block_verification_types::AsBlock; @@ -84,7 +84,7 @@ pub const DELAY_QUEUE_CHANNEL_SIZE: usize = 128; pub type Id = u32; -#[derive(Debug)] +#[derive(Debug, Copy, Clone)] pub enum ResponseType { Block, Blob, @@ -187,7 +187,6 @@ pub enum SyncMessage { BlockComponentProcessed { process_type: BlockProcessType, result: BlockProcessingResult, - response_type: ResponseType, }, } @@ -350,7 +349,7 @@ impl SyncManager { .single_block_lookup_failed::>( id, &peer_id, - &mut self.network, + &self.network, error, ); } @@ -359,7 +358,7 @@ impl SyncManager { .single_block_lookup_failed::>( id, &peer_id, - &mut self.network, + &self.network, error, ); } @@ -368,7 +367,7 @@ impl SyncManager { .parent_lookup_failed::>( id, peer_id, - &mut self.network, + &self.network, error, ); } @@ -377,7 +376,7 @@ impl SyncManager { .parent_lookup_failed::>( id, peer_id, - &mut self.network, + &self.network, error, ); } @@ -701,7 +700,7 @@ impl SyncManager { self.block_lookups.search_block_delayed( block_root, PeerShouldHave::Neither(peer_id), - &self.network, + &mut self.network, ); if let Err(e) = self .delayed_lookups @@ -722,7 +721,7 @@ impl SyncManager { SyncMessage::MissingGossipBlockComponentsDelayed(block_root) => { if self .block_lookups - .trigger_lookup_by_root(block_root, &mut self.network) + .trigger_lookup_by_root(block_root, &self.network) .is_err() { // No request was made for block or blob so the lookup is dropped. @@ -740,25 +739,24 @@ impl SyncManager { SyncMessage::BlockComponentProcessed { process_type, result, - response_type, } => match process_type { BlockProcessType::SingleBlock { id } => self .block_lookups .single_block_component_processed::>( id, result, - &self.network, + &mut self.network, ), BlockProcessType::SingleBlob { id } => self .block_lookups .single_block_component_processed::>( id, result, - &self.network, + &mut self.network, ), BlockProcessType::ParentLookup { chain_hash } => self .block_lookups - .parent_block_processed(chain_hash, result, response_type, &mut self.network), + .parent_block_processed(chain_hash, result, &mut self.network), }, SyncMessage::BatchProcessed { sync_type, result } => match sync_type { ChainSegmentProcessId::RangeBatchId(chain_id, epoch) => { @@ -787,7 +785,7 @@ impl SyncManager { } ChainSegmentProcessId::ParentLookup(chain_hash) => self .block_lookups - .parent_chain_processed(chain_hash, result, &mut self.network), + .parent_chain_processed(chain_hash, result, &self.network), }, } } @@ -813,7 +811,7 @@ impl SyncManager { block_root, parent_components, &[PeerShouldHave::Neither(peer_id)], - &self.network, + &mut self.network, ); if let Err(e) = self .delayed_lookups @@ -953,7 +951,7 @@ impl SyncManager { seen_timestamp, &self.network, ), - RequestId::SingleBlob { id } => { + RequestId::SingleBlob { id: _ } => { crit!(self.log, "Blob received during block request"; "peer_id" => %peer_id ); } RequestId::ParentLookup { id } => self @@ -965,7 +963,7 @@ impl SyncManager { seen_timestamp, &self.network, ), - RequestId::ParentLookupBlob { id } => { + RequestId::ParentLookupBlob { id: _ } => { crit!(self.log, "Blob received during parent block request"; "peer_id" => %peer_id ); } RequestId::BackFillBlocks { id } => { @@ -1025,7 +1023,7 @@ impl SyncManager { seen_timestamp: Duration, ) { match request_id { - RequestId::SingleBlock { id } => { + RequestId::SingleBlock { id: _ } => { crit!(self.log, "Single blob received during block request"; "peer_id" => %peer_id ); } RequestId::SingleBlob { id } => self @@ -1035,10 +1033,10 @@ impl SyncManager { peer_id, blob, seen_timestamp, - &mut self.network, + &self.network, ), - RequestId::ParentLookup { id } => { + RequestId::ParentLookup { id: _ } => { crit!(self.log, "Single blob received during parent block request"; "peer_id" => %peer_id ); } RequestId::ParentLookupBlob { id } => self @@ -1048,7 +1046,7 @@ impl SyncManager { peer_id, blob, seen_timestamp, - &mut self.network, + &self.network, ), RequestId::BackFillBlocks { id: _ } => { crit!(self.log, "Blob received during backfill block request"; "peer_id" => %peer_id ); diff --git a/beacon_node/network/src/sync/network_context.rs b/beacon_node/network/src/sync/network_context.rs index 21c25a1e5df..b7c6de2fc63 100644 --- a/beacon_node/network/src/sync/network_context.rs +++ b/beacon_node/network/src/sync/network_context.rs @@ -7,7 +7,7 @@ use super::range_sync::{BatchId, ByRangeRequestType, ChainId}; use crate::network_beacon_processor::NetworkBeaconProcessor; use crate::service::{NetworkMessage, RequestId}; use crate::status::ToStatusMessage; -use crate::sync::block_lookups::{BlobRequestId, BlockRequestId}; +use crate::sync::block_lookups::LookupType; use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::{BeaconChain, BeaconChainTypes, EngineState}; use fnv::FnvHashMap; @@ -37,7 +37,7 @@ pub struct SyncNetworkContext { network_send: mpsc::UnboundedSender>, /// A sequential ID for all RPC requests. - request_id: std::cell::Cell, + request_id: Id, /// BlocksByRange requests made by the range syncing algorithm. range_requests: FnvHashMap, @@ -93,7 +93,7 @@ impl SyncNetworkContext { SyncNetworkContext { network_send, execution_engine_state: EngineState::Online, // always assume `Online` at the start - request_id: std::cell::Cell::new(1), + request_id: 1, range_requests: FnvHashMap::default(), backfill_requests: FnvHashMap::default(), range_blocks_and_blobs_requests: FnvHashMap::default(), @@ -463,7 +463,8 @@ impl SyncNetworkContext { "Sending BlobsByRoot Request"; "method" => "BlobsByRoot", "count" => blob_request.blob_ids.len(), - "peer" => %blob_peer_id + "peer" => %blob_peer_id, + "lookup_type" => ?lookup_type ); self.send_network_msg(NetworkMessage::SendRequest { @@ -539,10 +540,10 @@ impl SyncNetworkContext { &self.network_beacon_processor } - pub fn next_id(&self) -> Id { - let current_value = self.request_id.get(); - self.request_id.set(current_value + 1); - current_value + pub(crate) fn next_id(&mut self) -> Id { + let id = self.request_id; + self.request_id += 1; + id } /// Check whether a batch for this epoch (and only this epoch) should request just blocks or From 6bcfaf4fdea843302d4333d639a504a87e9690e8 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Tue, 1 Aug 2023 17:14:45 -0400 Subject: [PATCH 06/18] clean up everything add child component, fix peer scoring and retry logic --- .../src/block_verification_types.rs | 18 + .../src/network_beacon_processor/mod.rs | 4 + .../src/network_beacon_processor/tests.rs | 2 + .../network/src/sync/block_lookups/common.rs | 448 +++++++ .../network/src/sync/block_lookups/mod.rs | 1109 +++++++++++------ .../src/sync/block_lookups/parent_lookup.rs | 72 +- .../sync/block_lookups/single_block_lookup.rs | 955 +++++--------- .../network/src/sync/block_lookups/tests.rs | 44 +- beacon_node/network/src/sync/manager.rs | 78 +- beacon_node/network/src/sync/mod.rs | 2 +- .../network/src/sync/network_context.rs | 37 +- 11 files changed, 1596 insertions(+), 1173 deletions(-) create mode 100644 beacon_node/network/src/sync/block_lookups/common.rs diff --git a/beacon_node/beacon_chain/src/block_verification_types.rs b/beacon_node/beacon_chain/src/block_verification_types.rs index c41090c421f..ab110c2104e 100644 --- a/beacon_node/beacon_chain/src/block_verification_types.rs +++ b/beacon_node/beacon_chain/src/block_verification_types.rs @@ -5,8 +5,10 @@ use crate::eth1_finalization_cache::Eth1FinalizationData; use crate::{data_availability_checker, GossipVerifiedBlock, PayloadVerificationOutcome}; use derivative::Derivative; use ssz_derive::{Decode, Encode}; +use ssz_types::VariableList; use state_processing::ConsensusContext; use std::sync::Arc; +use types::blob_sidecar::FixedBlobSidecarList; use types::{ blob_sidecar::BlobIdentifier, ssz_tagged_beacon_state, ssz_tagged_signed_beacon_block, ssz_tagged_signed_beacon_block_arc, @@ -72,6 +74,22 @@ impl RpcBlock { Ok(Self { block: inner }) } + pub fn new_from_fixed( + block: Arc>, + blobs: FixedBlobSidecarList, + ) -> Result { + let filtered = blobs + .into_iter() + .filter_map(|b| b.clone()) + .collect::>(); + let blobs = if filtered.is_empty() { + None + } else { + Some(VariableList::from(filtered)) + }; + Self::new(block, blobs) + } + pub fn deconstruct(self) -> (Arc>, Option>) { match self.block { RpcBlockInner::Block(block) => (block, None), diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 4a214c3637a..3906dcaaf6f 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -434,6 +434,10 @@ impl NetworkBeaconProcessor { seen_timestamp: Duration, process_type: BlockProcessType, ) -> Result<(), Error> { + let blob_count = blobs.iter().filter(|b| b.is_some()).count(); + if blob_count == 0 { + return Ok(()); + } let process_fn = self.clone().generate_rpc_blobs_process_fn( block_root, blobs, diff --git a/beacon_node/network/src/network_beacon_processor/tests.rs b/beacon_node/network/src/network_beacon_processor/tests.rs index 2c37d177aab..1bdc3a88166 100644 --- a/beacon_node/network/src/network_beacon_processor/tests.rs +++ b/beacon_node/network/src/network_beacon_processor/tests.rs @@ -318,6 +318,7 @@ impl TestRig { } pub fn enqueue_single_lookup_rpc_blobs(&self) { if let Some(blobs) = self.next_blobs.clone() { + dbg!(blobs.len()); let blobs = FixedBlobSidecarList::from( blobs .into_iter() @@ -1003,6 +1004,7 @@ async fn test_rpc_block_reprocessing() { rig.enqueue_single_lookup_rpc_blobs(); if rig.next_blobs.as_ref().map(|b| b.len()).unwrap_or(0) > 0 { + dbg!("here"); rig.assert_event_journal(&[RPC_BLOBS, WORKER_FREED, NOTHING_TO_DO]) .await; } diff --git a/beacon_node/network/src/sync/block_lookups/common.rs b/beacon_node/network/src/sync/block_lookups/common.rs new file mode 100644 index 00000000000..b4711e26be5 --- /dev/null +++ b/beacon_node/network/src/sync/block_lookups/common.rs @@ -0,0 +1,448 @@ +use crate::sync::block_lookups::parent_lookup::PARENT_FAIL_TOLERANCE; +use crate::sync::block_lookups::single_block_lookup::{ + LookupRequestError, LookupVerifyError, SingleBlockLookup, SingleLookupRequestState, State, +}; +use crate::sync::block_lookups::{ + BlobRequestState, BlockLookups, BlockRequestState, PeerShouldHave, + SINGLE_BLOCK_LOOKUP_MAX_ATTEMPTS, +}; +use crate::sync::manager::{BlockProcessType, Id, SingleLookupReqId}; +use crate::sync::network_context::SyncNetworkContext; +use crate::sync::ChildComponents; +use beacon_chain::block_verification_types::RpcBlock; +use beacon_chain::{get_block_root, BeaconChainTypes}; +use lighthouse_network::rpc::methods::BlobsByRootRequest; +use lighthouse_network::rpc::BlocksByRootRequest; +use lighthouse_network::PeerId; +use rand::prelude::IteratorRandom; +use ssz_types::VariableList; +use std::ops::IndexMut; +use std::sync::Arc; +use std::time::Duration; +use types::blob_sidecar::FixedBlobSidecarList; +use types::{BlobSidecar, EthSpec, Hash256, SignedBeaconBlock}; + +#[derive(Debug, Copy, Clone)] +pub enum ResponseType { + Block, + Blob, +} + +#[derive(Debug, Copy, Clone)] +pub enum LookupType { + Current, + Parent, +} + +/// This trait helps differentiate `SingleBlockLookup`s from `ParentLookup`s .This is useful in +/// ensuring requests and responses are handled separately and enables us to use different failure +/// tolerances for each, while re-using the same basic request and retry logic. +pub trait Lookup { + const MAX_ATTEMPTS: u8; + fn lookup_type() -> LookupType; + fn max_attempts() -> u8 { + Self::MAX_ATTEMPTS + } +} + +/// A `Lookup` that is a part of a `ParentLookup`. +pub struct Parent; + +impl Lookup for Parent { + const MAX_ATTEMPTS: u8 = PARENT_FAIL_TOLERANCE; + fn lookup_type() -> LookupType { + LookupType::Parent + } +} + +/// A `Lookup` that part of a single block lookup. +pub struct Current; + +impl Lookup for Current { + const MAX_ATTEMPTS: u8 = SINGLE_BLOCK_LOOKUP_MAX_ATTEMPTS; + fn lookup_type() -> LookupType { + LookupType::Current + } +} + +/// This trait unifies common single block lookup functionality across blocks and blobs. This +/// includes making requests, verifying responses, and handling processing results. A +/// `SingleBlockLookup` includes both a `BlockRequestState` and a `BlobRequestState`, this trait is +/// implemented for each. +/// +/// The use of the `ResponseType` associated type gives us a degree of type +/// safety when handling a block/blob response ensuring we only mutate the correct corresponding +/// state. +pub trait RequestState { + /// The type of the request . + type RequestType; + + /// A block or blob response. + type ResponseType; + + /// The type created after validation. + type VerifiedResponseType: Clone; + + /// We convert a `VerifiedResponseType` to this type prior to sending it to the beacon processor. + type ReconstructedResponseType; + + /* Request building methods */ + + /// Construct a new request. + fn build_request(&mut self) -> Result<(PeerId, Self::RequestType), LookupRequestError> { + debug_assert!(matches!(self.get_state().state, State::AwaitingDownload)); + self.too_many_attempts()?; + let peer = self.get_peer()?; + let request = self.new_request(); + self.get_state_mut().req_counter += 1; + Ok((peer, request)) + } + + /// Verify the current request has not exceeded the maximum number of attempts. + fn too_many_attempts(&self) -> Result<(), LookupRequestError> { + let max_attempts = L::max_attempts(); + let request_state = self.get_state(); + + if request_state.failed_attempts() >= max_attempts { + let cannot_process = + request_state.failed_processing >= request_state.failed_downloading; + Err(LookupRequestError::TooManyAttempts { cannot_process }) + } else { + Ok(()) + } + } + + /// Get the next peer to request. Draws from the set of peers we think should have both the + /// block and blob first. If that fails, we draw from the set of peers that may have either. + fn get_peer(&mut self) -> Result { + let request_state = self.get_state_mut(); + let available_peer_opt = request_state + .available_peers + .iter() + .choose(&mut rand::thread_rng()) + .copied() + .map(PeerShouldHave::BlockAndBlobs); + + let Some(peer_id) = available_peer_opt.or_else(||request_state + .potential_peers + .iter() + .choose(&mut rand::thread_rng()) + .copied() + .map(PeerShouldHave::Neither)) else { + return Err(LookupRequestError::NoPeers); + }; + request_state.used_peers.insert(peer_id.to_peer_id()); + request_state.state = State::Downloading { peer_id }; + Ok(peer_id.to_peer_id()) + } + + /// Initialize `Self::RequestType`. + fn new_request(&self) -> Self::RequestType; + + /// Send the request to the network service. + fn make_request( + id: SingleLookupReqId, + peer_id: PeerId, + request: Self::RequestType, + cx: &SyncNetworkContext, + ) -> Result<(), LookupRequestError>; + + /* Response handling methods */ + + /// Verify the response is valid based on what we requested. + fn verify_response( + &mut self, + expected_block_root: Hash256, + response: Option, + ) -> Result, LookupVerifyError> { + let request_state = self.get_state_mut(); + match request_state.state { + State::AwaitingDownload => { + request_state.register_failure_downloading(); + Err(LookupVerifyError::ExtraBlocksReturned) + } + State::Downloading { peer_id } => { + self.verify_response_inner(expected_block_root, response, peer_id) + } + State::Processing { peer_id: _ } => match response { + Some(_) => { + // We sent the block for processing and received an extra block. + request_state.register_failure_downloading(); + Err(LookupVerifyError::ExtraBlocksReturned) + } + None => { + // This is simply the stream termination and we are already processing the + // block + Ok(None) + } + }, + } + } + + /// The response verification unique to block or blobs. + fn verify_response_inner( + &mut self, + expected_block_root: Hash256, + response: Option, + peer_id: PeerShouldHave, + ) -> Result, LookupVerifyError>; + + /// A getter for the parent root of the response. Returns an `Option` because we won't know + /// the blob parent if we don't end up getting any blobs in the response. + fn get_parent_root(verified_response: &Self::VerifiedResponseType) -> Option; + + /// Caches the verified response in the lookup if necessary. This is only necessary for lookups + /// triggered by `UnknownParent` errors. + fn add_to_child_components( + verified_response: Self::VerifiedResponseType, + components: &mut ChildComponents, + ); + + /// Convert a verified response to the type we send to the beacon processor. + fn verified_to_reconstructed( + verified: Self::VerifiedResponseType, + ) -> Self::ReconstructedResponseType; + + /// Send the response to the beacon processor. + fn send_for_processing( + id: Id, + bl: &BlockLookups, + block_root: Hash256, + verified: Self::ReconstructedResponseType, + duration: Duration, + cx: &SyncNetworkContext, + ) -> Result<(), LookupRequestError>; + + /// Remove the peer from the lookup if it is useless. + fn remove_if_useless(&mut self, peer: &PeerId) { + self.get_state_mut().remove_peer_if_useless(peer) + } + + /// Register a failure to process the block or blob. + fn register_failure_downloading(&mut self) { + self.get_state_mut().register_failure_downloading() + } + + /* Utility methods */ + + /// Returns the `ResponseType` associated with this trait implementation. Useful in logging. + fn response_type() -> ResponseType; + + /// A getter for the `BlockRequestState` or `BlobRequestState` associated with this trait. + fn request_state_mut(request: &mut SingleBlockLookup) -> &mut Self; + + /// A getter for a reference to the `SingleLookupRequestState` associated with this trait. + fn get_state(&self) -> &SingleLookupRequestState; + + /// A getter for a mutable reference to the SingleLookupRequestState associated with this trait. + fn get_state_mut(&mut self) -> &mut SingleLookupRequestState; +} + +impl RequestState for BlockRequestState { + type RequestType = BlocksByRootRequest; + type ResponseType = Arc>; + type VerifiedResponseType = Arc>; + type ReconstructedResponseType = RpcBlock; + + fn new_request(&self) -> BlocksByRootRequest { + BlocksByRootRequest::new(VariableList::from(vec![self.requested_block_root])) + } + + fn make_request( + id: SingleLookupReqId, + peer_id: PeerId, + request: Self::RequestType, + cx: &SyncNetworkContext, + ) -> Result<(), LookupRequestError> { + cx.block_lookup_request(id, peer_id, request, L::lookup_type()) + .map_err(LookupRequestError::SendFailed) + } + + fn verify_response_inner( + &mut self, + expected_block_root: Hash256, + response: Option, + peer_id: PeerShouldHave, + ) -> Result>>, LookupVerifyError> { + match response { + Some(block) => { + // Compute the block root using this specific function so that we can get timing + // metrics. + let block_root = get_block_root(&block); + if block_root != expected_block_root { + // return an error and drop the block + // NOTE: we take this is as a download failure to prevent counting the + // attempt as a chain failure, but simply a peer failure. + self.state.register_failure_downloading(); + Err(LookupVerifyError::RootMismatch) + } else { + // Return the block for processing. + self.state.state = State::Processing { peer_id }; + Ok(Some(block)) + } + } + None => { + if peer_id.should_have_block() { + self.state.register_failure_downloading(); + Err(LookupVerifyError::NoBlockReturned) + } else { + self.state.state = State::AwaitingDownload; + Err(LookupVerifyError::BenignFailure) + } + } + } + } + + fn get_parent_root(verified_response: &Arc>) -> Option { + Some(verified_response.parent_root()) + } + + fn add_to_child_components( + verified_response: Arc>, + components: &mut ChildComponents, + ) { + components.add_unknown_parent_block(verified_response); + } + + fn verified_to_reconstructed( + block: Arc>, + ) -> RpcBlock { + RpcBlock::new_without_blobs(block) + } + + fn send_for_processing( + id: Id, + bl: &BlockLookups, + block_root: Hash256, + constructed: RpcBlock, + duration: Duration, + cx: &SyncNetworkContext, + ) -> Result<(), LookupRequestError> { + bl.send_block_for_processing( + block_root, + constructed, + duration, + BlockProcessType::SingleBlock { id }, + cx, + ) + } + + fn response_type() -> ResponseType { + ResponseType::Block + } + fn request_state_mut(request: &mut SingleBlockLookup) -> &mut Self { + &mut request.block_request_state + } + fn get_state(&self) -> &SingleLookupRequestState { + &self.state + } + fn get_state_mut(&mut self) -> &mut SingleLookupRequestState { + &mut self.state + } +} + +impl RequestState for BlobRequestState { + type RequestType = BlobsByRootRequest; + type ResponseType = Arc>; + type VerifiedResponseType = FixedBlobSidecarList; + type ReconstructedResponseType = FixedBlobSidecarList; + + fn new_request(&self) -> BlobsByRootRequest { + BlobsByRootRequest { + blob_ids: VariableList::from(self.requested_ids.clone()), + } + } + + fn make_request( + id: SingleLookupReqId, + peer_id: PeerId, + request: Self::RequestType, + cx: &SyncNetworkContext, + ) -> Result<(), LookupRequestError> { + cx.blob_lookup_request(id, peer_id, request, L::lookup_type()) + .map_err(LookupRequestError::SendFailed) + } + + fn verify_response_inner( + &mut self, + _expected_block_root: Hash256, + blob: Option, + peer_id: PeerShouldHave, + ) -> Result>, LookupVerifyError> { + match blob { + Some(blob) => { + let received_id = blob.id(); + if !self.requested_ids.contains(&received_id) { + self.state.register_failure_downloading(); + Err(LookupVerifyError::UnrequestedBlobId) + } else { + // State should remain downloading until we receive the stream terminator. + self.requested_ids.retain(|id| *id != received_id); + let blob_index = blob.index; + + if blob_index >= T::EthSpec::max_blobs_per_block() as u64 { + return Err(LookupVerifyError::InvalidIndex(blob.index)); + } + *self.blob_download_queue.index_mut(blob_index as usize) = Some(blob); + Ok(None) + } + } + None => { + self.state.state = State::Processing { peer_id }; + let blobs = std::mem::take(&mut self.blob_download_queue); + Ok(Some(blobs)) + } + } + } + + fn get_parent_root(verified_response: &FixedBlobSidecarList) -> Option { + verified_response + .into_iter() + .filter_map(|blob| blob.as_ref()) + .map(|blob| blob.block_parent_root) + .next() + } + + fn add_to_child_components( + verified_response: FixedBlobSidecarList, + components: &mut ChildComponents, + ) { + components.add_unknown_parent_blobs(verified_response); + } + + fn verified_to_reconstructed( + blobs: FixedBlobSidecarList, + ) -> FixedBlobSidecarList { + blobs + } + + fn send_for_processing( + id: Id, + bl: &BlockLookups, + block_root: Hash256, + verified: FixedBlobSidecarList, + duration: Duration, + cx: &SyncNetworkContext, + ) -> Result<(), LookupRequestError> { + bl.send_blobs_for_processing( + block_root, + verified, + duration, + BlockProcessType::SingleBlob { id }, + cx, + ) + } + + fn response_type() -> ResponseType { + ResponseType::Blob + } + fn request_state_mut(request: &mut SingleBlockLookup) -> &mut Self { + &mut request.blob_request_state + } + fn get_state(&self) -> &SingleLookupRequestState { + &self.state + } + fn get_state_mut(&mut self) -> &mut SingleLookupRequestState { + &mut self.state + } +} diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index 79ef277468f..43fb5330181 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -5,31 +5,38 @@ use super::BatchProcessResult; use super::{manager::BlockProcessType, network_context::SyncNetworkContext}; use crate::metrics; use crate::network_beacon_processor::ChainSegmentProcessId; -use crate::sync::block_lookups::parent_lookup::ParentLookup; -use crate::sync::block_lookups::single_block_lookup::LookupVerifyError; -use crate::sync::manager::{Id, ResponseType}; +use crate::sync::block_lookups::parent_lookup::{ParentLookup, RequestError}; +use crate::sync::block_lookups::single_block_lookup::{ + CachedChild, LookupRequestError, LookupVerifyError, +}; +use crate::sync::manager::{Id, SingleLookupReqId}; use beacon_chain::block_verification_types::{AsBlock, RpcBlock}; -use beacon_chain::data_availability_checker::{AvailabilityCheckError, DataAvailabilityChecker}; +use beacon_chain::data_availability_checker::DataAvailabilityChecker; +use beacon_chain::validator_monitor::timestamp_now; use beacon_chain::{AvailabilityProcessingStatus, BeaconChainTypes, BlockError}; +pub use common::Current; +pub use common::Lookup; +use common::LookupType; +pub use common::Parent; +pub use common::RequestState; use fnv::FnvHashMap; use lighthouse_network::rpc::RPCError; use lighthouse_network::{PeerAction, PeerId}; use lru_cache::LRUTimeCache; -pub use single_block_lookup::UnknownParentComponents; -pub use single_block_lookup::{ - BlobRequestState, BlockRequestState, Current, Lookup, Parent, RequestState, -}; +pub use single_block_lookup::ChildComponents; +pub use single_block_lookup::{BlobRequestState, BlockRequestState}; use slog::{debug, error, trace, warn, Logger}; use smallvec::SmallVec; use std::collections::HashMap; use std::fmt::Debug; use std::sync::Arc; use std::time::Duration; -use store::{Hash256, SignedBeaconBlock}; +use store::Hash256; use strum::Display; use types::blob_sidecar::FixedBlobSidecarList; use types::Slot; +pub mod common; pub(crate) mod delayed_lookup; mod parent_lookup; mod single_block_lookup; @@ -37,29 +44,10 @@ mod single_block_lookup; mod tests; pub type DownloadedBlocks = (Hash256, RpcBlock); -pub type RootBlockTuple = (Hash256, Arc>); -pub type RootBlobsTuple = (Hash256, FixedBlobSidecarList); const FAILED_CHAINS_CACHE_EXPIRY_SECONDS: u64 = 60; pub const SINGLE_BLOCK_LOOKUP_MAX_ATTEMPTS: u8 = 3; -pub struct BlockLookups { - /// Parent chain lookups being downloaded. - parent_lookups: SmallVec<[ParentLookup; 3]>, - - processing_parent_lookups: HashMap, SingleBlockLookup)>, - - /// A cache of failed chain lookups to prevent duplicate searches. - failed_chains: LRUTimeCache, - - single_block_lookups: FnvHashMap>, - - da_checker: Arc>, - - /// The logger for the import manager. - log: Logger, -} - /// This enum is used to track what a peer *should* be able to respond with respond based on /// other messages we've seen from this peer on the network. This is useful for peer scoring. /// We expect a peer tracked by the `BlockAndBlobs` variant to be able to respond to all @@ -97,6 +85,23 @@ impl PeerShouldHave { } } +pub struct BlockLookups { + /// Parent chain lookups being downloaded. + parent_lookups: SmallVec<[ParentLookup; 3]>, + + processing_parent_lookups: HashMap, SingleBlockLookup)>, + + /// A cache of failed chain lookups to prevent duplicate searches. + failed_chains: LRUTimeCache, + + single_block_lookups: FnvHashMap>, + + da_checker: Arc>, + + /// The logger for the import manager. + log: Logger, +} + impl BlockLookups { pub fn new(da_checker: Arc>, log: Logger) -> Self { Self { @@ -143,14 +148,14 @@ impl BlockLookups { /// Creates a lookup for the block with the given `block_root`, while caching other block /// components we've already received. The block components are cached here because we haven't - /// imported it's parent and therefore can't fully validate it and store it in the data + /// imported its parent and therefore can't fully validate it and store it in the data /// availability cache. /// /// The request is immediately triggered. pub fn search_child_block( &mut self, block_root: Hash256, - parent_components: Option>, + parent_components: Option>, peer_source: &[PeerShouldHave], cx: &mut SyncNetworkContext, ) { @@ -170,7 +175,7 @@ impl BlockLookups { pub fn search_child_delayed( &mut self, block_root: Hash256, - parent_components: Option>, + parent_components: Option>, peer_source: &[PeerShouldHave], cx: &mut SyncNetworkContext, ) { @@ -186,13 +191,19 @@ impl BlockLookups { mut single_block_lookup: SingleBlockLookup, cx: &SyncNetworkContext, ) { - if !single_block_lookup.triggered && single_block_lookup.request_block_and_blobs(cx).is_ok() - { - single_block_lookup.triggered = true; - self.add_single_lookup(single_block_lookup) + let block_root = single_block_lookup.block_root(); + match single_block_lookup.request_block_and_blobs(cx) { + Ok(()) => self.add_single_lookup(single_block_lookup), + Err(e) => { + debug!(self.log, "Single block lookup failed"; + "error" => ?e, + "block_root" => ?block_root, + ); + } } } + /// Adds a lookup to the `single_block_lookups` map. pub fn add_single_lookup(&mut self, single_block_lookup: SingleBlockLookup) { self.single_block_lookups .insert(single_block_lookup.id, single_block_lookup); @@ -203,25 +214,20 @@ impl BlockLookups { ); } - pub fn trigger_lookup_by_root( - &mut self, - block_root: Hash256, - cx: &SyncNetworkContext, - ) -> Result<(), ()> { - for (_, lookup) in self.single_block_lookups.iter_mut() { - if lookup.block_request_state.requested_block_root == block_root - && !lookup.triggered - && lookup.request_block_and_blobs(cx).is_ok() - { - lookup.triggered = true; + /// Trigger any lookups that are waiting for the given `block_root`. + pub fn trigger_lookup_by_root(&mut self, block_root: Hash256, cx: &SyncNetworkContext) { + self.single_block_lookups.retain(|_id, lookup| { + if lookup.block_root() == block_root { + if let Err(e) = lookup.request_block_and_blobs(cx) { + debug!(self.log, "Delayed single block lookup failed"; + "error" => ?e, + "block_root" => ?block_root, + ); + return false; + } } - } - Ok(()) - } - - pub fn remove_lookup_by_root(&mut self, block_root: Hash256) { - self.single_block_lookups - .retain(|_id, lookup| lookup.block_request_state.requested_block_root != block_root); + true + }); } /// Searches for a single block hash. If the blocks parent is unknown, a chain of blocks is @@ -229,7 +235,7 @@ impl BlockLookups { pub fn new_current_lookup( &mut self, block_root: Hash256, - parent_components: Option>, + parent_components: Option>, peers: &[PeerShouldHave], cx: &mut SyncNetworkContext, ) -> Option> { @@ -241,7 +247,7 @@ impl BlockLookups { { lookup.add_peers(peers); if let Some(components) = parent_components { - lookup.add_unknown_parent_components(components); + lookup.add_child_components(components); } return None; } @@ -319,37 +325,68 @@ impl BlockLookups { // we are already processing this block, ignore it. return; } - let mut parent_lookup = ParentLookup::new( + let parent_lookup = ParentLookup::new( block_root, parent_root, peer_source, self.da_checker.clone(), cx, ); - if let Ok(()) = parent_lookup - .current_parent_request - .request_block_and_blobs(cx) - { - self.parent_lookups.push(parent_lookup); - } + self.request_parent(parent_lookup, cx); } /* Lookup responses */ + /// Get a single block lookup by its ID. This method additionally ensures the `req_counter` + /// matches the current `req_counter` for the lookup. This any stale responses from requests + /// that have been retried are ignored. + fn get_single_lookup>( + &mut self, + id: SingleLookupReqId, + ) -> Option> { + let mut lookup = self.single_block_lookups.remove(&id.id)?; + + let request_state = R::request_state_mut(&mut lookup); + if id.req_counter != request_state.get_state().req_counter { + // We don't want to drop the lookup, just ignore the old response. + self.single_block_lookups.insert(id.id, lookup); + return None; + } + Some(lookup) + } + + /// Checks whether a single block lookup is waiting for a parent lookup to complete. This is + /// necessary because we want to make sure all parents are processed before sending a child + /// for processing, otherwise the block will fail validation and will be returned to the network + /// layer with an `UnknownParent` error. + pub fn has_pending_parent_request(&self, target_id: Id) -> bool { + self.single_block_lookups.iter().any(|(id, lookup)| { + if *id == target_id { + self.parent_lookups + .iter() + .any(|parent_lookup| parent_lookup.chain_hash() == lookup.block_root()) + } else { + false + } + }) + } + + /// Process a block or blob response received from a single lookup request. pub fn single_lookup_response>( &mut self, - id: Id, + lookup_id: SingleLookupReqId, peer_id: PeerId, response: Option, seen_timestamp: Duration, cx: &SyncNetworkContext, ) { - let is_stream_terminator = response.is_none(); + let id = lookup_id.id; let response_type = R::response_type(); - let log = self.log.clone(); - let Some(lookup) = self.single_block_lookups.get_mut(&id) else { - if !is_stream_terminator { + let chain_hash_opt = self.has_pending_parent_request(id); + + let Some(lookup) = self.get_single_lookup::(lookup_id) else { + if response.is_some() { warn!( self.log, "Block returned for single block lookup not present"; @@ -359,150 +396,258 @@ impl BlockLookups { return; }; - let expected_block_root = lookup.block_request_state.requested_block_root; + let expected_block_root = lookup.block_root(); - let has_pending_parent_request = self - .parent_lookups - .iter() - .any(|parent_lookup| parent_lookup.chain_hash() == expected_block_root); + match self.single_lookup_response_inner::( + peer_id, + response, + seen_timestamp, + cx, + chain_hash_opt, + lookup, + ) { + Ok(lookup) => { + self.single_block_lookups.insert(id, lookup); + } + Err(e) => { + debug!(self.log, + "Single lookup retry failed"; + "error" => ?e, + "block_root" => ?expected_block_root, + ); + } + } - let request_state = R::request_state_mut(lookup); + metrics::set_gauge( + &metrics::SYNC_SINGLE_BLOCK_LOOKUPS, + self.single_block_lookups.len() as i64, + ); + } + + /// Consolidates error handling for `single_lookup_response`. An `Err` here should always mean + /// the lookup is dropped. + fn single_lookup_response_inner>( + &self, + peer_id: PeerId, + response: Option, + seen_timestamp: Duration, + cx: &SyncNetworkContext, + delay_send: bool, + mut lookup: SingleBlockLookup, + ) -> Result, LookupRequestError> { + let response_type = R::response_type(); + let log = self.log.clone(); + let expected_block_root = lookup.block_root(); + let request_state = R::request_state_mut(&mut lookup); match request_state.verify_response(expected_block_root, response) { - Ok(Some((root, verified_response))) => { - if let Some(parent_components) = lookup.unknown_parent_components.as_mut() { - R::add_to_parent_components(verified_response, parent_components); - - if !has_pending_parent_request { - if let Some(rpc_block) = lookup.get_downloaded_block() { - if let Err(()) = self.send_block_for_processing( - expected_block_root, - rpc_block, - seen_timestamp, - BlockProcessType::SingleBlock { id }, - cx, - ) { - self.single_block_lookups.remove(&id); - } - } - } - } else if let Err(()) = R::send_for_processing( - id, - self, - root, - R::verified_to_reconstructed(verified_response), + Ok(Some(verified_response)) => { + self.handle_verified_response::( seen_timestamp, cx, - ) { - self.single_block_lookups.remove(&id); - } + delay_send, + None, + verified_response, + &mut lookup, + )?; } Ok(None) => {} Err(e) => { - let msg = if matches!(e, LookupVerifyError::BenignFailure) { + debug!( + log, + "Single lookup response verification failed, retrying"; + "block_root" => ?expected_block_root, + "peer_id" => %peer_id, + "response_type" => ?response_type, + "error" => ?e + ); + if matches!(e, LookupVerifyError::BenignFailure) { request_state .get_state_mut() .remove_peer_if_useless(&peer_id); - "peer could not response to request" } else { let msg = e.into(); cx.report_peer(peer_id, PeerAction::LowToleranceError, msg); - msg }; - debug!(log, "Single block lookup failed"; - "peer_id" => %peer_id, - "error" => msg, - "block_root" => ?expected_block_root, - "response_type" => ?response_type - ); - if let Err(()) = request_state.retry_request_after_failure(id, cx, &log) { - self.single_block_lookups.remove(&id); + request_state.register_failure_downloading(); + lookup.request_block_and_blobs(cx)?; + } + } + Ok(lookup) + } + + fn handle_verified_response>( + &self, + seen_timestamp: Duration, + cx: &SyncNetworkContext, + delay_send: bool, + chain_hash: Option, + verified_response: R::VerifiedResponseType, + lookup: &mut SingleBlockLookup, + ) -> Result<(), LookupRequestError> { + let id = lookup.id; + let block_root = lookup.block_root(); + + R::request_state_mut(lookup) + .get_state_mut() + .component_downloaded = true; + + let cached_child = lookup.add_response::(verified_response.clone()); + + match cached_child { + CachedChild::Ok(block) => { + if let Some(chain_hash) = chain_hash { + if !delay_send { + let process_type = match L::lookup_type() { + LookupType::Parent => BlockProcessType::ParentLookup { chain_hash }, + LookupType::Current => BlockProcessType::SingleBlock { id }, + }; + self.send_block_for_processing( + block_root, + block, + seen_timestamp, + process_type, + cx, + )? + } + } + } + CachedChild::DownloadIncomplete => { + // If this was the result of a block request, we can't determined if the block peer + // did anything wrong. If we already had both a block and blobs response processed, + // we should penalize the blobs peer because they did not provide all blobs on the + // initial request. + if lookup.both_components_downloaded() { + lookup.penalize_lazy_blob_peer(cx); + lookup + .blob_request_state + .state + .register_failure_downloading(); } + lookup.request_block_and_blobs(cx)?; + } + CachedChild::NotRequired => R::send_for_processing( + id, + self, + block_root, + R::verified_to_reconstructed(verified_response), + seen_timestamp, + cx, + )?, + CachedChild::Err(e) => { + warn!(self.log, "Consistency error in cached block"; + "error" => ?e, + "block_root" => ?block_root + ); + lookup.handle_consistency_failure(cx); + lookup.request_block_and_blobs(cx)?; } } + Ok(()) + } - //TODO(sean) move metric to trait to differentiate block and blob - metrics::set_gauge( - &metrics::SYNC_SINGLE_BLOCK_LOOKUPS, - self.single_block_lookups.len() as i64, - ); + /// Get a parent block lookup by its ID. This method additionally ensures the `req_counter` + /// matches the current `req_counter` for the lookup. This any stale responses from requests + /// that have been retried are ignored. + fn get_parent_lookup>( + &mut self, + id: SingleLookupReqId, + ) -> Option> { + let mut parent_lookup = if let Some(pos) = self + .parent_lookups + .iter() + .position(|request| request.current_parent_request.id == id.id) + { + self.parent_lookups.remove(pos) + } else { + return None; + }; + + if R::request_state_mut(&mut parent_lookup.current_parent_request) + .get_state() + .req_counter + != id.req_counter + { + self.parent_lookups.push(parent_lookup); + return None; + } + Some(parent_lookup) } /// Process a response received from a parent lookup request. pub fn parent_lookup_response>( &mut self, - id: Id, + id: SingleLookupReqId, peer_id: PeerId, - block: Option, + response: Option, seen_timestamp: Duration, cx: &SyncNetworkContext, ) { - let mut parent_lookup = if let Some(pos) = self - .parent_lookups - .iter() - .position(|request| request.current_parent_request.id == id) - { - self.parent_lookups.remove(pos) - } else { - if block.is_some() { + let Some(mut parent_lookup) = self.get_parent_lookup::(id) else { + if response.is_some() { debug!(self.log, "Response for a parent lookup request that was not found"; "peer_id" => %peer_id); } - return; + return }; - match parent_lookup.verify_block::(block, &mut self.failed_chains) { - Ok(Some((block_root, verified_response))) => { - if let Some(parent_components) = parent_lookup - .current_parent_request - .unknown_parent_components - .as_mut() - { - R::add_to_parent_components(verified_response, parent_components); - } - if let Some(rpc_block) = parent_lookup.current_parent_request.get_downloaded_block() - { - let chain_hash = parent_lookup.chain_hash(); - if self - .send_block_for_processing( - block_root, - rpc_block, - seen_timestamp, - BlockProcessType::ParentLookup { chain_hash }, - cx, - ) - .is_ok() - { - self.parent_lookups.push(parent_lookup) - } - } else { - //TODO(sean) here, we could penalize a peer who previously sent us a blob list - // that was incomplete, and trigger a re-request immediately - self.parent_lookups.push(parent_lookup) - } - } - Ok(None) => { - // Request finished successfully, nothing else to do. It will be removed after the - // processing result arrives. + match self.parent_lookup_response_inner::( + peer_id, + response, + seen_timestamp, + cx, + &mut parent_lookup, + ) { + Ok(()) => { + debug!(self.log, "Requesting parent"; &parent_lookup); self.parent_lookups.push(parent_lookup); } - Err(e) => self.handle_parent_verify_error::(peer_id, parent_lookup, e, cx), - }; + Err(e) => { + self.handle_parent_request_error(&mut parent_lookup, cx, e); + } + } - //TODO(sean) move metric to trait to differentiate block and blob metrics::set_gauge( &metrics::SYNC_PARENT_BLOCK_LOOKUPS, self.parent_lookups.len() as i64, ); } + /// Consolidates error handling for `parent_lookup_response`. An `Err` here should always mean + /// the lookup is dropped. + fn parent_lookup_response_inner>( + &mut self, + peer_id: PeerId, + block: Option, + seen_timestamp: Duration, + cx: &SyncNetworkContext, + parent_lookup: &mut ParentLookup, + ) -> Result<(), RequestError> { + match parent_lookup.verify_block::(block, &mut self.failed_chains) { + Ok(Some(verified_response)) => { + self.handle_verified_response::( + seen_timestamp, + cx, + false, + Some(parent_lookup.chain_hash()), + verified_response, + &mut parent_lookup.current_parent_request, + )?; + } + Ok(None) => {} + Err(e) => self.handle_parent_verify_error::(peer_id, parent_lookup, e, cx)?, + }; + Ok(()) + } + + /// Handle logging and peer scoring for `ParentVerifyError`s during parent lookup requests. fn handle_parent_verify_error>( &mut self, peer_id: PeerId, - mut parent_lookup: ParentLookup, + parent_lookup: &mut ParentLookup, e: ParentVerifyError, cx: &SyncNetworkContext, - ) { + ) -> Result<(), RequestError> { match e { ParentVerifyError::RootMismatch | ParentVerifyError::NoBlockReturned @@ -520,7 +665,7 @@ impl BlockLookups { cx.report_peer(peer_id, PeerAction::LowToleranceError, e); // We try again if possible. - self.request_parent(parent_lookup, cx) + parent_lookup.request_parent(cx)?; } ParentVerifyError::PreviousFailure { parent_root } => { debug!( @@ -544,7 +689,47 @@ impl BlockLookups { ); let request_state = R::request_state_mut(&mut parent_lookup.current_parent_request); request_state.remove_if_useless(&peer_id); - self.request_parent(parent_lookup, cx) + parent_lookup.request_parent(cx)?; + } + } + Ok(()) + } + + /// Handle logging and peer scoring for `RequestError`s during parent lookup requests. + fn handle_parent_request_error( + &mut self, + parent_lookup: &mut ParentLookup, + cx: &SyncNetworkContext, + e: RequestError, + ) { + debug!(self.log, "Failed to request parent"; "error" => e.as_static()); + match e { + RequestError::SendFailed(_) => { + // Probably shutting down, nothing to do here. Drop the request + } + RequestError::ChainTooLong => { + self.failed_chains.insert(parent_lookup.chain_hash()); + // This indicates faulty peers. + for &peer_id in parent_lookup.used_peers() { + cx.report_peer(peer_id, PeerAction::LowToleranceError, e.as_static()) + } + } + RequestError::TooManyAttempts { cannot_process } => { + // We only consider the chain failed if we were unable to process it. + // We could have failed because one peer continually failed to send us + // bad blocks. We still allow other peers to send us this chain. Note + // that peers that do this, still get penalised. + if cannot_process { + self.failed_chains.insert(parent_lookup.chain_hash()); + } + // This indicates faulty peers. + for &peer_id in parent_lookup.used_peers() { + cx.report_peer(peer_id, PeerAction::LowToleranceError, e.as_static()) + } + } + RequestError::NoPeers => { + // This happens if the peer disconnects while the block is being + // processed. Drop the request without extra penalty } } } @@ -553,18 +738,19 @@ impl BlockLookups { pub fn peer_disconnected(&mut self, peer_id: &PeerId, cx: &mut SyncNetworkContext) { /* Check disconnection for single lookups */ - self.single_block_lookups.retain(|id, req| { - let should_remove_lookup = - req.should_remove_disconnected_peer(*id, peer_id, cx, &self.log); + self.single_block_lookups.retain(|_, req| { + let should_drop_lookup = + req.should_drop_lookup_on_disconnected_peer(peer_id, cx, &self.log); - !should_remove_lookup + !should_drop_lookup }); /* Check disconnection for parent lookups */ - while let Some(pos) = self.parent_lookups.iter_mut().position(|req| { - req.check_block_peer_disconnected(peer_id).is_err() - || req.check_blob_peer_disconnected(peer_id).is_err() - }) { + while let Some(pos) = self + .parent_lookups + .iter_mut() + .position(|req| req.check_peer_disconnected(peer_id).is_err()) + { let parent_lookup = self.parent_lookups.remove(pos); trace!(self.log, "Parent lookup's peer disconnected"; &parent_lookup); self.request_parent(parent_lookup, cx); @@ -574,26 +760,25 @@ impl BlockLookups { /// An RPC error has occurred during a parent lookup. This function handles this case. pub fn parent_lookup_failed>( &mut self, - id: Id, + id: SingleLookupReqId, peer_id: PeerId, cx: &SyncNetworkContext, error: RPCError, ) { let msg = error.as_static_str(); - if let Some(pos) = self - .parent_lookups - .iter() - .position(|request| request.current_parent_request.id == id) - { - let mut parent_lookup = self.parent_lookups.remove(pos); - R::request_state_mut(&mut parent_lookup.current_parent_request) - .register_failure_downloading(); - trace!(self.log, "Parent lookup block request failed"; &parent_lookup, "error" => msg); - - self.request_parent(parent_lookup, cx); - } else { - return debug!(self.log, "RPC failure for a block parent lookup request that was not found"; "peer_id" => %peer_id, "error" => msg); + let Some(mut parent_lookup) = self.get_parent_lookup::(id) else { + debug!(self.log, + "RPC failure for a block parent lookup request that was not found"; + "peer_id" => %peer_id, + "error" => msg + ); + return }; + R::request_state_mut(&mut parent_lookup.current_parent_request) + .register_failure_downloading(); + trace!(self.log, "Parent lookup block request failed"; &parent_lookup, "error" => msg); + + self.request_parent(parent_lookup, cx); metrics::set_gauge( &metrics::SYNC_PARENT_BLOCK_LOOKUPS, @@ -601,32 +786,40 @@ impl BlockLookups { ); } + /// An RPC error has occurred during a single lookup. This function handles this case.\ pub fn single_block_lookup_failed>( &mut self, - id: Id, + id: SingleLookupReqId, peer_id: &PeerId, cx: &SyncNetworkContext, error: RPCError, ) { let msg = error.as_static_str(); - let Some(lookup) = self.single_block_lookups.get_mut(&id) else { - debug!(self.log, "Error response to dropped lookup"; "error" => ?error); + let log = self.log.clone(); + let Some(mut lookup) = self.get_single_lookup::(id) else { + debug!(log, "Error response to dropped lookup"; "error" => ?error); return; }; - let root = lookup.block_request_state.requested_block_root; - let request_state = R::request_state_mut(lookup); - request_state.register_failure_downloading(); + let block_root = lookup.block_root(); + let request_state = R::request_state_mut(&mut lookup); let response_type = R::response_type(); - trace!(self.log, + trace!(log, "Single lookup failed"; - "block_root" => ?root, + "block_root" => ?block_root, "error" => msg, "peer_id" => %peer_id, "response_type" => ?response_type ); - if let Err(()) = request_state.retry_request_after_failure(id, cx, &self.log) { + let id = id.id; + request_state.register_failure_downloading(); + if let Err(e) = lookup.request_block_and_blobs(cx) { + debug!(self.log, + "Single lookup retry failed"; + "error" => ?e, + "block_root" => ?block_root, + ); self.single_block_lookups.remove(&id); - }; + } metrics::set_gauge( &metrics::SYNC_SINGLE_BLOCK_LOOKUPS, @@ -642,64 +835,35 @@ impl BlockLookups { result: BlockProcessingResult, cx: &mut SyncNetworkContext, ) { - let Some(request_ref) = self.single_block_lookups.get_mut(&target_id) else { - debug!(self.log, "Block component processed for single block lookup not present" ); + let Some(mut lookup) = self.single_block_lookups.remove(&target_id) else { return; }; - let root = request_ref.block_request_state.requested_block_root; - let request_state = R::request_state_mut(request_ref); - let peer_id = request_state.get_state().processing_peer(); - request_state.get_state_mut().component_processed = true; + let root = lookup.block_root(); + let request_state = R::request_state_mut(&mut lookup); - let peer_id = match peer_id { - Ok(peer) => peer, - Err(_) => return, + let Ok(peer_id) = request_state.get_state().processing_peer() else { + return }; + debug!( + self.log, + "Block component processed for single block lookup not present" + ); match result { BlockProcessingResult::Ok(status) => match status { AvailabilityProcessingStatus::Imported(root) => { trace!(self.log, "Single block processing succeeded"; "block" => %root); - self.single_block_lookups.remove(&target_id); } AvailabilityProcessingStatus::MissingComponents(_, _block_root) => { - // if this was the result of a blocks request, the block peer did nothing wrong. - // if we already had a blobs resposne, we should penalize the blobs peer because - // they did not provide all blobs. - if request_ref.both_components_processed() { - if let Ok(blob_peer) = - request_ref.blob_request_state.state.processing_peer() - { - if let PeerShouldHave::BlockAndBlobs(blob_peer) = blob_peer { - cx.report_peer( - blob_peer, - PeerAction::MidToleranceError, - "single_block_failure", - ); - } - request_ref - .blob_request_state - .state - .remove_peer_if_useless(blob_peer.as_peer_id()); - if !::EthSpec, - > as RequestState>::downloading( - &request_ref.blob_request_state, - ) { - // Try it again if possible. - if let Err(()) = request_ref - .blob_request_state - .retry_request_after_failure(target_id, cx, &self.log) - { - self.single_block_lookups.remove(&target_id); - }; - } - } else { - trace!(self.log, "Dropped blob peer prior to penalizing"; "root" => ?root); - self.single_block_lookups.remove(&target_id); - }; + match self.handle_missing_components::(cx, &mut lookup) { + Ok(()) => { + self.single_block_lookups.insert(target_id, lookup); + } + Err(e) => { + // Drop with an additional error. + warn!(self.log, "Single block lookup failed"; "block" => %root, "error" => ?e); + } } } }, @@ -711,89 +875,132 @@ impl BlockLookups { "Single block processing was ignored, cpu might be overloaded"; "action" => "dropping single block request" ); - self.single_block_lookups.remove(&target_id); } BlockProcessingResult::Err(e) => { - trace!(self.log, "Single block processing failed"; "block" => %root, "error" => %e); - match e { - BlockError::BlockIsAlreadyKnown => { - // No error here - self.single_block_lookups.remove(&target_id); - } - BlockError::BeaconChainError(e) => { - // Internal error - error!(self.log, "Beacon chain error processing single block"; "block_root" => %root, "error" => ?e); - self.single_block_lookups.remove(&target_id); + match self.handle_single_lookup_block_error(cx, lookup, peer_id, e) { + Ok(Some(lookup)) => { + self.single_block_lookups.insert(target_id, lookup); } - BlockError::ParentUnknown(block) => { - let slot = block.slot(); - let parent_root = block.parent_root(); - request_ref.add_unknown_parent_components(block.into()); - self.search_parent(slot, root, parent_root, peer_id.to_peer_id(), cx); + Ok(None) => { + // Drop without an additional error. } - ref e @ BlockError::ExecutionPayloadError(ref epe) if !epe.penalize_peer() => { - // These errors indicate that the execution layer is offline - // and failed to validate the execution payload. Do not downscore peer. - debug!( - self.log, - "Single block lookup failed. Execution layer is offline / unsynced / misconfigured"; - "root" => %root, - "error" => ?e - ); - self.single_block_lookups.remove(&target_id); - } - BlockError::AvailabilityCheck( - AvailabilityCheckError::KzgVerificationFailed, - ) - | BlockError::AvailabilityCheck(AvailabilityCheckError::Kzg(_)) - | BlockError::BlobValidation(_) => { - warn!(self.log, "Blob validation failure"; "root" => %root, "peer_id" => %peer_id); - if let Ok(blob_peer) = - request_ref.blob_request_state.state.processing_peer() - { - cx.report_peer( - blob_peer.to_peer_id(), - PeerAction::MidToleranceError, - "single_blob_failure", - ); - // Try it again if possible. - if let Err(()) = request_ref - .blob_request_state - .retry_request_after_failure(target_id, cx, &self.log) - { - self.single_block_lookups.remove(&target_id); - }; - } - } - other => { - warn!(self.log, "Peer sent invalid block in single block lookup"; "root" => %root, "error" => ?other, "peer_id" => %peer_id); - if let Ok(block_peer) = - request_ref.block_request_state.state.processing_peer() - { - cx.report_peer( - block_peer.to_peer_id(), - PeerAction::MidToleranceError, - "single_block_failure", - ); - - // Try it again if possible. - if let Err(()) = request_ref - .blob_request_state - .retry_request_after_failure(target_id, cx, &self.log) - { - self.single_block_lookups.remove(&target_id); - }; - } + Err(e) => { + // Drop with an additional error. + warn!(self.log, "Single block lookup failed"; "block" => %root, "error" => ?e); } } } }; + } - //TODO(sean) move metrics to lookup response trait - metrics::set_gauge( - &metrics::SYNC_SINGLE_BLOCK_LOOKUPS, - self.single_block_lookups.len() as i64, - ); + /// Handles a `MissingComponents` block processing error. Handles peer scoring and retries. + /// + /// If this was the result of a block request, we can't determined if the block peer did anything + /// wrong. If we already had both a block and blobs response processed, we should penalize the + /// blobs peer because they did not provide all blobs on the initial request. + fn handle_missing_components>( + &self, + cx: &mut SyncNetworkContext, + lookup: &mut SingleBlockLookup, + ) -> Result<(), LookupRequestError> { + let request_state = R::request_state_mut(lookup); + + request_state.get_state_mut().component_processed = true; + if lookup.both_components_processed() { + lookup.penalize_lazy_blob_peer(cx); + + // Try it again if possible. + lookup + .blob_request_state + .state + .register_failure_processing(); + lookup.request_block_and_blobs(cx)?; + } + Ok(()) + } + + /// Handles peer scoring and retries related to a `BlockError` in response to a single block + /// or blob lookup processing result. + fn handle_single_lookup_block_error( + &mut self, + cx: &mut SyncNetworkContext, + mut lookup: SingleBlockLookup, + peer_id: PeerShouldHave, + e: BlockError, + ) -> Result>, LookupRequestError> { + let root = lookup.block_root(); + trace!(self.log, "Single block processing failed"; "block" => %root, "error" => %e); + match e { + BlockError::BlockIsAlreadyKnown => { + // No error here + return Ok(None); + } + BlockError::BeaconChainError(e) => { + // Internal error + error!(self.log, "Beacon chain error processing single block"; "block_root" => %root, "error" => ?e); + return Ok(None); + } + BlockError::ParentUnknown(block) => { + let slot = block.slot(); + let parent_root = block.parent_root(); + lookup.add_child_components(block.into()); + lookup.request_block_and_blobs(cx)?; + self.search_parent(slot, root, parent_root, peer_id.to_peer_id(), cx); + } + ref e @ BlockError::ExecutionPayloadError(ref epe) if !epe.penalize_peer() => { + // These errors indicate that the execution layer is offline + // and failed to validate the execution payload. Do not downscore peer. + debug!( + self.log, + "Single block lookup failed. Execution layer is offline / unsynced / misconfigured"; + "root" => %root, + "error" => ?e + ); + return Ok(None); + } + BlockError::AvailabilityCheck(e) => { + warn!(self.log, "Availability check failure"; "root" => %root, "peer_id" => %peer_id, "error" => ?e); + lookup.handle_availability_check_failure(cx); + lookup.request_block_and_blobs(cx)? + } + BlockError::BlobValidation(_) => { + warn!(self.log, "Blob validation failure"; "root" => %root, "peer_id" => %peer_id); + if let Ok(blob_peer) = lookup.blob_request_state.state.processing_peer() { + cx.report_peer( + blob_peer.to_peer_id(), + PeerAction::MidToleranceError, + "single_block_failure", + ); + lookup + .blob_request_state + .state + .remove_peer_if_useless(blob_peer.as_peer_id()); + } + lookup + .blob_request_state + .state + .register_failure_processing(); + lookup.request_block_and_blobs(cx)? + } + other => { + warn!(self.log, "Peer sent invalid block in single block lookup"; "root" => %root, "error" => ?other, "peer_id" => %peer_id); + if let Ok(block_peer) = lookup.block_request_state.state.processing_peer() { + cx.report_peer( + block_peer.to_peer_id(), + PeerAction::MidToleranceError, + "single_block_failure", + ); + + // Try it again if possible. + lookup + .block_request_state + .state + .register_failure_processing(); + lookup.request_block_and_blobs(cx)? + } + } + } + Ok(Some(lookup)) } pub fn parent_block_processed( @@ -813,11 +1020,6 @@ impl BlockLookups { return debug!(self.log, "Process response for a parent lookup request that was not found"; "chain_hash" => %chain_hash); }; - let Ok(peer_id) = - parent_lookup.processing_peer() else { - return - }; - match &result { BlockProcessingResult::Ok(status) => match status { AvailabilityProcessingStatus::Imported(block_root) => { @@ -845,7 +1047,32 @@ impl BlockLookups { _, block_root, )) => { - self.search_block(block_root, peer_id, cx); + let expected_block_root = parent_lookup.current_parent_request.block_root(); + if block_root != expected_block_root { + warn!( + self.log, + "Parent block processing result/request root mismatch"; + "request" =>?expected_block_root, + "result" => ?block_root + ); + return; + } + + // We only send parent blocks + blobs for processing together. This means a + // `MissingComponents` response here indicates missing blobs. Therefore we always + // register a blob processing failure here. + parent_lookup + .current_parent_request + .blob_request_state + .state + .register_failure_processing(); + match parent_lookup + .current_parent_request + .request_block_and_blobs(cx) + { + Ok(()) => self.parent_lookups.push(parent_lookup), + Err(e) => self.handle_parent_request_error(&mut parent_lookup, cx, e.into()), + } } BlockProcessingResult::Err(BlockError::ParentUnknown(block)) => { parent_lookup.add_unknown_parent_block(block); @@ -864,29 +1091,10 @@ impl BlockLookups { ); } }; - let (chain_hash, mut blocks, hashes, block_request) = + let (chain_hash, blocks, hashes, block_request) = parent_lookup.parts_for_processing(); - // Find the child block that spawned the parent lookup request and add it to the chain - // to send for processing. - if let Some(child_lookup_id) = - self.single_block_lookups.iter().find_map(|(id, lookup)| { - (lookup.block_request_state.requested_block_root == chain_hash) - .then_some(*id) - }) - { - let Some(child_lookup) = self.single_block_lookups.get_mut(&child_lookup_id) else { - debug!(self.log, "Missing child for parent lookup request"; "child_root" => ?chain_hash); - return; - }; - if let Some(rpc_block) = child_lookup.get_downloaded_block() { - blocks.push(rpc_block); - } else { - trace!(self.log, "Parent lookup chain complete, awaiting child response"; "chain_hash" => ?chain_hash); - } - } else { - debug!(self.log, "Missing child for parent lookup request"; "child_root" => ?chain_hash); - }; + let blocks = self.add_child_block_to_chain(chain_hash, blocks, cx); let process_id = ChainSegmentProcessId::ParentLookup(chain_hash); @@ -917,7 +1125,7 @@ impl BlockLookups { ); } BlockProcessingResult::Err(outcome) => { - self.handle_invalid_block(outcome, peer_id.to_peer_id(), cx, parent_lookup); + self.handle_parent_block_error(outcome, cx, parent_lookup); } BlockProcessingResult::Ignored => { // Beacon processor signalled to ignore the block processing result. @@ -936,24 +1144,107 @@ impl BlockLookups { ); } - fn handle_invalid_block( + /// Find the child block that spawned the parent lookup request and add it to the chain + /// to send for processing. + fn add_child_block_to_chain( + &mut self, + chain_hash: Hash256, + mut blocks: Vec>, + cx: &SyncNetworkContext, + ) -> Vec> { + // Find the child block that spawned the parent lookup request and add it to the chain + // to send for processing. + if let Some(child_lookup_id) = self + .single_block_lookups + .iter() + .find_map(|(id, lookup)| (lookup.block_root() == chain_hash).then_some(*id)) + { + let Some(child_lookup) = self.single_block_lookups.get_mut(&child_lookup_id) else { + debug!(self.log, "Missing child for parent lookup request"; "child_root" => ?chain_hash); + return blocks; + }; + match child_lookup.get_cached_child_block() { + CachedChild::Ok(rpc_block) => { + blocks.push(rpc_block); + } + CachedChild::DownloadIncomplete => { + trace!(self.log, "Parent lookup chain complete, awaiting child response"; "chain_hash" => ?chain_hash); + } + CachedChild::NotRequired => { + warn!(self.log, "Child not cached for parent lookup"; "chain_hash" => %chain_hash); + } + CachedChild::Err(e) => { + warn!( + self.log, + "Consistency error in child block triggering chain or parent lookups"; + "error" => ?e, + "chain_hash" => ?chain_hash + ); + child_lookup.handle_consistency_failure(cx); + if child_lookup.request_block_and_blobs(cx).is_err() { + self.single_block_lookups.remove(&child_lookup_id); + } + } + } + } else { + debug!(self.log, "Missing child for parent lookup request"; "child_root" => ?chain_hash); + }; + blocks + } + + /// Handle the peer scoring, retries, and logging related to a `BlockError` returned from + /// processing a block + blobs for a parent lookup. + fn handle_parent_block_error( &mut self, outcome: BlockError<::EthSpec>, - peer_id: PeerId, cx: &SyncNetworkContext, mut parent_lookup: ParentLookup, ) { + // We should always have a block peer. + let Ok(block_peer_id) = + parent_lookup.block_processing_peer() else { + return + }; + let block_peer_id = block_peer_id.to_peer_id(); + + // We may not have a blob peer, if there were no blobs required for this block. + let blob_peer_id = parent_lookup + .blob_processing_peer() + .ok() + .map(PeerShouldHave::to_peer_id); + // all else we consider the chain a failure and downvote the peer that sent // us the last block warn!( self.log, "Invalid parent chain"; "score_adjustment" => %PeerAction::MidToleranceError, "outcome" => ?outcome, - "last_peer" => %peer_id, + "block_peer_id" => %block_peer_id, ); // This currently can be a host of errors. We permit this due to the partial // ambiguity. - cx.report_peer(peer_id, PeerAction::MidToleranceError, "parent_request_err"); + cx.report_peer( + block_peer_id, + PeerAction::MidToleranceError, + "parent_request_err", + ); + // Don't downscore the same peer twice + if let Some(blob_peer_id) = blob_peer_id { + if block_peer_id != blob_peer_id { + debug!( + self.log, "Additionally down-scoring blob peer"; + "score_adjustment" => %PeerAction::MidToleranceError, + "outcome" => ?outcome, + "blob_peer_id" => %blob_peer_id, + ); + cx.report_peer( + blob_peer_id, + PeerAction::MidToleranceError, + "parent_request_err", + ); + } + } + // Try again if possible parent_lookup.processing_failed(); self.request_parent(parent_lookup, cx); @@ -979,7 +1270,7 @@ impl BlockLookups { .single_block_lookups .iter() .find_map(|(id, req)| - (req.block_request_state.requested_block_root == chain_hash).then_some(*id)) else { + (req.block_root() == chain_hash).then_some(*id)) else { warn!(self.log, "No id found for single block lookup"; "chain_hash" => %chain_hash); return; }; @@ -991,20 +1282,40 @@ impl BlockLookups { return; }; - if let Some(rpc_block) = lookup.get_downloaded_block() { - // This is the correct block, send it for processing - if self - .send_block_for_processing( - chain_hash, - rpc_block, - Duration::from_secs(0), //TODO(sean) pipe this through - BlockProcessType::SingleBlock { id }, - cx, - ) - .is_err() - { - // Remove to avoid inconsistencies - self.single_block_lookups.remove(&id); + match lookup.get_cached_child_block() { + CachedChild::Ok(rpc_block) => { + // This is the correct block, send it for processing + if self + .send_block_for_processing( + chain_hash, + rpc_block, + timestamp_now(), + BlockProcessType::SingleBlock { id }, + cx, + ) + .is_err() + { + // Remove to avoid inconsistencies + self.single_block_lookups.remove(&id); + } + } + CachedChild::DownloadIncomplete => { + trace!(self.log, "Parent chain complete, awaiting child response"; "chain_hash" => %chain_hash); + } + CachedChild::NotRequired => { + warn!(self.log, "Child not cached for parent lookup"; "chain_hash" => %chain_hash); + } + CachedChild::Err(e) => { + warn!( + self.log, + "Consistency error in child block triggering parent lookup"; + "chain_hash" => %chain_hash, + "error" => ?e + ); + lookup.handle_consistency_failure(cx); + if lookup.request_block_and_blobs(cx).is_err() { + self.single_block_lookups.remove(&id); + } } } } @@ -1013,9 +1324,7 @@ impl BlockLookups { penalty, } => { self.failed_chains.insert(chain_hash); - let mut all_peers = request.block_request_state.state.used_peers.clone(); - all_peers.extend(request.blob_request_state.state.used_peers); - for peer_source in all_peers { + for peer_source in request.all_peers() { cx.report_peer(peer_source, penalty, "parent_chain_failure") } } @@ -1033,13 +1342,13 @@ impl BlockLookups { /* Helper functions */ fn send_block_for_processing( - &mut self, + &self, block_root: Hash256, block: RpcBlock, duration: Duration, process_type: BlockProcessType, cx: &SyncNetworkContext, - ) -> Result<(), ()> { + ) -> Result<(), LookupRequestError> { match cx.beacon_processor_if_enabled() { Some(beacon_processor) => { trace!(self.log, "Sending block for processing"; "block" => ?block_root, "process" => ?process_type); @@ -1054,14 +1363,18 @@ impl BlockLookups { "Failed to send sync block to processor"; "error" => ?e ); - Err(()) + Err(LookupRequestError::SendFailed( + "beacon processor send failure", + )) } else { Ok(()) } } None => { trace!(self.log, "Dropping block ready for processing. Beacon processor not available"; "block" => %block_root); - Err(()) + Err(LookupRequestError::SendFailed( + "beacon processor unavailable", + )) } } } @@ -1073,11 +1386,7 @@ impl BlockLookups { duration: Duration, process_type: BlockProcessType, cx: &SyncNetworkContext, - ) -> Result<(), ()> { - let blob_count = blobs.iter().filter(|b| b.is_some()).count(); - if blob_count == 0 { - return Ok(()); - } + ) -> Result<(), LookupRequestError> { match cx.beacon_processor_if_enabled() { Some(beacon_processor) => { trace!(self.log, "Sending blobs for processing"; "block" => ?block_root, "process_type" => ?process_type); @@ -1089,54 +1398,30 @@ impl BlockLookups { "Failed to send sync blobs to processor"; "error" => ?e ); - Err(()) + Err(LookupRequestError::SendFailed( + "beacon processor send failure", + )) } else { Ok(()) } } None => { trace!(self.log, "Dropping blobs ready for processing. Beacon processor not available"; "block_root" => %block_root); - Err(()) + Err(LookupRequestError::SendFailed( + "beacon processor unavailable", + )) } } } + /// Attempts to request the next unknown parent. This method handles peer scoring and dropping + /// the lookup in the event of failure. fn request_parent(&mut self, mut parent_lookup: ParentLookup, cx: &SyncNetworkContext) { let response = parent_lookup.request_parent(cx); match response { Err(e) => { - debug!(self.log, "Failed to request parent"; &parent_lookup, "error" => e.as_static()); - match e { - parent_lookup::RequestError::SendFailed(_) => { - // Probably shutting down, nothing to do here. Drop the request - } - parent_lookup::RequestError::ChainTooLong => { - self.failed_chains.insert(parent_lookup.chain_hash()); - // This indicates faulty peers. - for &peer_id in parent_lookup.used_peers() { - cx.report_peer(peer_id, PeerAction::LowToleranceError, e.as_static()) - } - } - parent_lookup::RequestError::TooManyAttempts { cannot_process } => { - // We only consider the chain failed if we were unable to process it. - // We could have failed because one peer continually failed to send us - // bad blocks. We still allow other peers to send us this chain. Note - // that peers that do this, still get penalised. - if cannot_process { - self.failed_chains.insert(parent_lookup.chain_hash()); - } - // This indicates faulty peers. - for &peer_id in parent_lookup.used_peers() { - cx.report_peer(peer_id, PeerAction::LowToleranceError, e.as_static()) - } - } - parent_lookup::RequestError::NoPeers => { - // This happens if the peer disconnects while the block is being - // processed. Drop the request without extra penalty - } - parent_lookup::RequestError::AlreadyDownloaded => {} - } + self.handle_parent_request_error(&mut parent_lookup, cx, e); } Ok(_) => { debug!(self.log, "Requesting parent"; &parent_lookup); @@ -1163,9 +1448,3 @@ impl BlockLookups { self.parent_lookups.drain(..).len() } } - -#[derive(Debug, Copy, Clone)] -pub enum LookupType { - Current, - Parent, -} diff --git a/beacon_node/network/src/sync/block_lookups/parent_lookup.rs b/beacon_node/network/src/sync/block_lookups/parent_lookup.rs index 6afeaccc7b7..b94a3be71ea 100644 --- a/beacon_node/network/src/sync/block_lookups/parent_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/parent_lookup.rs @@ -1,8 +1,7 @@ use super::single_block_lookup::{LookupRequestError, LookupVerifyError, SingleBlockLookup}; use super::{DownloadedBlocks, PeerShouldHave}; -use crate::sync::block_lookups::single_block_lookup::{ - Parent, RequestState, State, UnknownParentComponents, -}; +use crate::sync::block_lookups::common::Parent; +use crate::sync::block_lookups::common::RequestState; use crate::sync::{manager::SLOT_IMPORT_TOLERANCE, network_context::SyncNetworkContext}; use beacon_chain::block_verification_types::AsBlock; use beacon_chain::block_verification_types::RpcBlock; @@ -55,7 +54,6 @@ pub enum RequestError { cannot_process: bool, }, NoPeers, - AlreadyDownloaded, } impl ParentLookup { @@ -103,55 +101,42 @@ impl ParentLookup { .map_err(Into::into) } - pub fn check_block_peer_disconnected(&mut self, peer_id: &PeerId) -> Result<(), ()> { + pub fn check_peer_disconnected(&mut self, peer_id: &PeerId) -> Result<(), ()> { self.current_parent_request .block_request_state .state .check_peer_disconnected(peer_id) - } - - pub fn check_blob_peer_disconnected(&mut self, peer_id: &PeerId) -> Result<(), ()> { - self.current_parent_request - .blob_request_state - .state - .check_peer_disconnected(peer_id) + .and_then(|()| { + self.current_parent_request + .blob_request_state + .state + .check_peer_disconnected(peer_id) + }) } pub fn add_unknown_parent_block(&mut self, block: RpcBlock) { let next_parent = block.parent_root(); - // Cache the block. - let current_root = self - .current_parent_request - .block_request_state - .requested_block_root; + let current_root = self.current_parent_request.block_root(); self.downloaded_blocks.push((current_root, block)); - // Update the block request. + // Update the parent request. self.current_parent_request - .block_request_state - .requested_block_root = next_parent; - self.current_parent_request.block_request_state.state.state = State::AwaitingDownload; - - // Update the blobs request. - self.current_parent_request.blob_request_state.state.state = State::AwaitingDownload; - - // Reset the unknown parent components. - self.current_parent_request.unknown_parent_components = - Some(UnknownParentComponents::default()); + .update_requested_parent_block(next_parent) } - pub fn processing_peer(&self) -> Result { + pub fn block_processing_peer(&self) -> Result { self.current_parent_request .block_request_state .state .processing_peer() - .or_else(|()| { - self.current_parent_request - .blob_request_state - .state - .processing_peer() - }) + } + + pub fn blob_processing_peer(&self) -> Result { + self.current_parent_request + .blob_request_state + .state + .processing_peer() } /// Consumes the parent request and destructures it into it's parts. @@ -193,11 +178,7 @@ impl ParentLookup { .blob_request_state .state .register_failure_processing(); - if let Some(components) = self - .current_parent_request - .unknown_parent_components - .as_mut() - { + if let Some(components) = self.current_parent_request.cached_child_components.as_mut() { components.downloaded_block = None; components.downloaded_blobs = <_>::default(); } @@ -209,11 +190,8 @@ impl ParentLookup { &mut self, block: Option, failed_chains: &mut lru_cache::LRUTimeCache, - ) -> Result, ParentVerifyError> { - let expected_block_root = self - .current_parent_request - .block_request_state - .requested_block_root; + ) -> Result, ParentVerifyError> { + let expected_block_root = self.current_parent_request.block_root(); let request_state = R::request_state_mut(&mut self.current_parent_request); let root_and_block = request_state.verify_response(expected_block_root, block)?; @@ -221,7 +199,7 @@ impl ParentLookup { // be dropped and the peer downscored. if let Some(parent_root) = root_and_block .as_ref() - .and_then(|(_, block)| R::get_parent_root(block)) + .and_then(|block| R::get_parent_root(block)) { if failed_chains.contains(&parent_root) { request_state.register_failure_downloading(); @@ -277,7 +255,6 @@ impl From for RequestError { RequestError::TooManyAttempts { cannot_process } } E::NoPeers => RequestError::NoPeers, - E::AlreadyDownloaded => RequestError::AlreadyDownloaded, E::SendFailed(msg) => RequestError::SendFailed(msg), } } @@ -306,7 +283,6 @@ impl RequestError { } RequestError::TooManyAttempts { cannot_process: _ } => "too_many_downloading_attempts", RequestError::NoPeers => "no_peers", - RequestError::AlreadyDownloaded => "already_downloaded", } } } diff --git a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs index 5499c669864..3cb084d9be9 100644 --- a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs @@ -1,51 +1,52 @@ -use super::{PeerShouldHave, ResponseType}; -use crate::sync::block_lookups::parent_lookup::PARENT_FAIL_TOLERANCE; -use crate::sync::block_lookups::{ - BlockLookups, Id, LookupType, RootBlobsTuple, RootBlockTuple, SINGLE_BLOCK_LOOKUP_MAX_ATTEMPTS, -}; -use crate::sync::manager::BlockProcessType; +use super::PeerShouldHave; +use crate::sync::block_lookups::common::{Lookup, RequestState}; +use crate::sync::block_lookups::Id; +use crate::sync::manager::SingleLookupReqId; use crate::sync::network_context::SyncNetworkContext; use beacon_chain::block_verification_types::RpcBlock; -use beacon_chain::data_availability_checker::DataAvailabilityChecker; -use beacon_chain::{get_block_root, BeaconChainTypes}; -use lighthouse_network::rpc::methods::BlobsByRootRequest; -use lighthouse_network::{rpc::BlocksByRootRequest, PeerId}; -use rand::seq::IteratorRandom; -use slog::{debug, trace, Logger}; -use ssz_types::VariableList; +use beacon_chain::data_availability_checker::{AvailabilityCheckError, DataAvailabilityChecker}; +use beacon_chain::BeaconChainTypes; +use lighthouse_network::{PeerAction, PeerId}; +use slog::{trace, Logger}; use std::collections::HashSet; use std::marker::PhantomData; -use std::ops::IndexMut; use std::sync::Arc; -use std::time::Duration; use store::Hash256; use strum::IntoStaticStr; use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; -use types::{BlobSidecar, EthSpec, SignedBeaconBlock}; +use types::{EthSpec, SignedBeaconBlock}; -pub trait Lookup { - const MAX_ATTEMPTS: u8; - fn lookup_type() -> LookupType; - fn max_attempts() -> u8 { - Self::MAX_ATTEMPTS - } +#[derive(Debug, PartialEq, Eq)] +pub enum State { + AwaitingDownload, + Downloading { peer_id: PeerShouldHave }, + Processing { peer_id: PeerShouldHave }, } -pub struct Parent; -pub struct Current; - -impl Lookup for Parent { - const MAX_ATTEMPTS: u8 = PARENT_FAIL_TOLERANCE; - fn lookup_type() -> LookupType { - LookupType::Parent - } +#[derive(Debug, PartialEq, Eq, IntoStaticStr)] +pub enum LookupVerifyError { + RootMismatch, + NoBlockReturned, + ExtraBlocksReturned, + UnrequestedBlobId, + ExtraBlobsReturned, + NotEnoughBlobsReturned, + InvalidIndex(u64), + /// We don't have enough information to know + /// whether the peer is at fault or simply missed + /// what was requested on gossip. + BenignFailure, } -impl Lookup for Current { - const MAX_ATTEMPTS: u8 = SINGLE_BLOCK_LOOKUP_MAX_ATTEMPTS; - fn lookup_type() -> LookupType { - LookupType::Current - } +#[derive(Debug, PartialEq, Eq, IntoStaticStr)] +pub enum LookupRequestError { + /// Too many failed attempts + TooManyAttempts { + /// The failed attempts were primarily due to processing failures. + cannot_process: bool, + }, + NoPeers, + SendFailed(&'static str), } pub struct SingleBlockLookup { @@ -53,376 +54,297 @@ pub struct SingleBlockLookup { pub block_request_state: BlockRequestState, pub blob_request_state: BlobRequestState, pub da_checker: Arc>, - /// Only necessary for requests triggered by an `UnknownBlockParent` or `UnknownBlockParent` because any - /// blocks or blobs without parents won't hit the data availability cache. - pub unknown_parent_components: Option>, - /// We may want to delay the actual request trigger to give us a chance to receive all block - /// components over gossip. - pub triggered: bool, + /// Only necessary for requests triggered by an `UnknownBlockParent` or `UnknownBlockParent` + /// because any blocks or blobs without parents won't hit the data availability cache. + pub cached_child_components: Option>, } -// generic across block + blob -pub trait RequestState { - type RequestType; - type ResponseType; - type ReconstructedResponseType; - type VerifiedResponseType; - - // response verify - fn response_type() -> ResponseType; - fn get_parent_root(verified_response: &Self::VerifiedResponseType) -> Option; - fn request_state_mut(request: &mut SingleBlockLookup) -> &mut Self; - fn add_to_parent_components( - verified_response: Self::VerifiedResponseType, - components: &mut UnknownParentComponents, - ); - fn verified_to_reconstructed( - verified: Self::VerifiedResponseType, - ) -> Self::ReconstructedResponseType; - fn send_for_processing( +impl SingleBlockLookup { + pub fn new( + requested_block_root: Hash256, + unknown_parent_components: Option>, + peers: &[PeerShouldHave], + da_checker: Arc>, id: Id, - bl: &mut BlockLookups, - block_root: Hash256, - verified: Self::ReconstructedResponseType, - duration: Duration, - cx: &SyncNetworkContext, - ) -> Result<(), ()>; + ) -> Self { + Self { + id, + block_request_state: BlockRequestState::new(requested_block_root, peers), + blob_request_state: BlobRequestState::new(peers), + da_checker, + cached_child_components: unknown_parent_components, + } + } - fn get_state(&self) -> &SingleLookupRequestState; - fn get_state_mut(&mut self) -> &mut SingleLookupRequestState; - fn processing_peer(&self) -> Result { - self.get_state().processing_peer() + /// Get the block root that is being requested. + pub fn block_root(&self) -> Hash256 { + self.block_request_state.requested_block_root } - fn set_component_processed(&mut self) { - self.get_state_mut().component_processed = true; + + /// Check the block root matches the requested block root. + pub fn is_for_block(&self, block_root: Hash256) -> bool { + self.block_root() == block_root } - fn new_request(&self) -> Self::RequestType; - fn max_attempts() -> u8; - fn retry_request( - id: Id, - cx: &SyncNetworkContext, - peer_id: PeerId, - request: Self::RequestType, - ) -> Result<(), &'static str>; - fn verify_response( + + /// Update the requested block, this should only be used in a chain of parent lookups to request + /// the next parent. + pub fn update_requested_parent_block(&mut self, block_root: Hash256) { + self.block_request_state.requested_block_root = block_root; + self.block_request_state.state.state = State::AwaitingDownload; + self.blob_request_state.state.state = State::AwaitingDownload; + self.cached_child_components = Some(ChildComponents::default()); + } + + /// Get all unique peers across block and blob requests. + pub fn all_peers(&self) -> HashSet { + let mut all_peers = self.block_request_state.state.used_peers.clone(); + all_peers.extend(self.blob_request_state.state.used_peers.clone()); + all_peers + } + + /// Send the necessary requests for blocks and/or blobs. This will check whether we have + /// downloaded the block and/or blobs already and will not send requests if so. It will also + /// inspect the request state or blocks and blobs to ensure we are not already processing or + /// downloading the block and/or blobs. + pub fn request_block_and_blobs( &mut self, - expected_block_root: Hash256, - response: Option, - ) -> Result, LookupVerifyError> { - let request_state = self.get_state_mut(); - match request_state.state { - State::AwaitingDownload => { - request_state.register_failure_downloading(); - Err(LookupVerifyError::ExtraBlocksReturned) - } - State::Downloading { peer_id } => { - self.verify_response_inner(expected_block_root, response, peer_id) - } - State::Processing { peer_id: _ } => match response { - Some(_) => { - // We sent the block for processing and received an extra block. - request_state.register_failure_downloading(); - Err(LookupVerifyError::ExtraBlocksReturned) - } - None => { - // This is simply the stream termination and we are already processing the - // block - Ok(None) - } - }, + cx: &SyncNetworkContext, + ) -> Result<(), LookupRequestError> { + let block_root = self.block_root(); + let block_already_downloaded = self.block_already_downloaded(); + let blobs_already_downloaded = self.blobs_already_downloaded(); + + if block_already_downloaded && blobs_already_downloaded { + trace!(cx.log, "Lookup request already completed"; "block_root"=> ?block_root); + return Ok(()); } + + self.request_generic::>(block_already_downloaded, cx)?; + self.request_generic::>(blobs_already_downloaded, cx) } - fn verify_response_inner( - &mut self, - expected_block_root: Hash256, - response: Option, - peer_id: PeerShouldHave, - ) -> Result, LookupVerifyError>; - fn retry_request_after_failure( + /// Common checks and request logic for blocks and blobs. + fn request_generic>( &mut self, - id: Id, + already_downloaded: bool, cx: &SyncNetworkContext, - log: &Logger, - ) -> Result<(), ()> { - if let Err(e) = self - .build_request() - .map_err(Into::into) - .and_then(|(peer_id, request)| Self::retry_request(id, cx, peer_id, request)) - { - //TODO(sean) pass this error up? check downloaded contents prior to retry-ing? - debug!(log, "Single block lookup failed"; - "error" => ?e, - ); - return Err(()); + ) -> Result<(), LookupRequestError> { + let id = self.id; + let request_state = R::request_state_mut(self); + + let should_request = !already_downloaded + && matches!(request_state.get_state().state, State::AwaitingDownload); + if should_request { + let (peer_id, request) = request_state.build_request()?; + let id = SingleLookupReqId { + id, + req_counter: request_state.get_state().req_counter, + }; + R::make_request(id, peer_id, request, cx)?; } Ok(()) } - fn build_request(&mut self) -> Result<(PeerId, Self::RequestType), LookupRequestError> { - debug_assert!(matches!(self.get_state().state, State::AwaitingDownload)); - self.too_many_attempts()?; - let peer = self.get_peer()?; - let request = self.new_request(); - Ok((peer, request)) - } - fn too_many_attempts(&self) -> Result<(), LookupRequestError> { - let max_attempts = Self::max_attempts(); - if self.get_state().failed_attempts() >= max_attempts { - Err(LookupRequestError::TooManyAttempts { - cannot_process: self.cannot_process(), - }) + + /// Returns a `CachedChild`, which is a wrapper around a `RpcBlock` that is either: + /// + /// 1. `NotRequired`: there is no child caching required for this lookup. + /// 2. `DownloadIncomplete`: Child caching is required, but all components are not yet downloaded. + /// 3. `Ok`: The child is required and we have downloaded it. + /// 4. `Err`: The child is required, but has failed consistency checks. + pub fn get_cached_child_block(&self) -> CachedChild { + if let Some(components) = self.cached_child_components.as_ref() { + let Some(block) = components.downloaded_block.as_ref()else { + return CachedChild::DownloadIncomplete + }; + + if !self.missing_blob_ids().is_empty() { + return CachedChild::DownloadIncomplete; + } + + match RpcBlock::new_from_fixed(block.clone(), components.downloaded_blobs.clone()) { + Ok(rpc_block) => CachedChild::Ok(rpc_block), + Err(e) => CachedChild::Err(e), + } } else { - Ok(()) + CachedChild::NotRequired } } - fn cannot_process(&self) -> bool { - let request_state = self.get_state(); - request_state.failed_processing >= request_state.failed_downloading - } - fn get_peer(&mut self) -> Result { - let request_state = self.get_state_mut(); - let Some(peer_id) = request_state - .available_peers - .iter() - .choose(&mut rand::thread_rng()) - .copied() - .map(PeerShouldHave::BlockAndBlobs).or(request_state - .potential_peers - .iter() - .choose(&mut rand::thread_rng()) - .copied() - .map(PeerShouldHave::Neither)) else { - return Err(LookupRequestError::NoPeers); - }; - request_state.used_peers.insert(peer_id.to_peer_id()); - request_state.state = State::Downloading { peer_id }; - Ok(peer_id.to_peer_id()) - } - fn check_peer_disconnected(&mut self, peer: &PeerId) -> Result<(), ()> { - self.get_state_mut().check_peer_disconnected(peer) - } - fn remove_if_useless(&mut self, peer: &PeerId) { - self.get_state_mut().remove_peer_if_useless(peer) - } - fn downloading(&self) -> bool { - matches!(self.get_state().state, State::Downloading { .. }) - } - fn register_failure_downloading(&mut self) { - self.get_state_mut().register_failure_downloading() - } -} - -impl RequestState for BlockRequestState { - type RequestType = BlocksByRootRequest; - type ResponseType = Arc>; - type ReconstructedResponseType = RpcBlock; - type VerifiedResponseType = Arc>; - // response verify - fn response_type() -> ResponseType { - ResponseType::Block + /// Accepts a verified response, and adds it to the child components if required. This method + /// returns a `CachedChild` which provides a completed block + blob response if all components have been + /// received, or information about whether the child is required and if it has been downloaded. + pub fn add_response>( + &mut self, + verified_response: R::VerifiedResponseType, + ) -> CachedChild { + if let Some(cached_child_components) = self.cached_child_components.as_mut() { + R::add_to_child_components(verified_response, cached_child_components); + self.get_cached_child_block() + } else { + CachedChild::NotRequired + } } - fn get_parent_root(verified_response: &Arc>) -> Option { - Some(verified_response.parent_root()) - } - fn request_state_mut(request: &mut SingleBlockLookup) -> &mut Self { - &mut request.block_request_state - } - fn add_to_parent_components( - verified_response: Arc>, - components: &mut UnknownParentComponents, - ) { - components.add_unknown_parent_block(verified_response); + /// Add a child component to the lookup request. Merges with any existing child components. + pub fn add_child_components(&mut self, components: ChildComponents) { + if let Some(ref mut existing_components) = self.cached_child_components { + let ChildComponents { + downloaded_block, + downloaded_blobs, + } = components; + if let Some(block) = downloaded_block { + existing_components.add_unknown_parent_block(block); + } + existing_components.add_unknown_parent_blobs(downloaded_blobs); + } else { + self.cached_child_components = Some(components); + } } - fn verified_to_reconstructed( - block: Arc>, - ) -> RpcBlock { - RpcBlock::new_without_blobs(block) + /// Add all given peers to both block and blob request states. + pub fn add_peers(&mut self, peers: &[PeerShouldHave]) { + for peer in peers { + match peer { + PeerShouldHave::BlockAndBlobs(peer_id) => { + self.block_request_state.state.add_peer(peer_id); + self.blob_request_state.state.add_peer(peer_id); + } + PeerShouldHave::Neither(peer_id) => { + self.block_request_state.state.add_potential_peer(peer_id); + self.blob_request_state.state.add_potential_peer(peer_id); + } + } + } } - fn send_for_processing( - id: Id, - bl: &mut BlockLookups, - block_root: Hash256, - constructed: RpcBlock, - duration: Duration, - cx: &SyncNetworkContext, - ) -> Result<(), ()> { - bl.send_block_for_processing( - block_root, - constructed, - duration, - BlockProcessType::SingleBlock { id }, - cx, - ) + /// Returns true if the block has already been downloaded. + pub fn both_components_downloaded(&self) -> bool { + self.block_request_state.state.component_downloaded + && self.blob_request_state.state.component_downloaded } - fn get_state(&self) -> &SingleLookupRequestState { - &self.state - } - fn get_state_mut(&mut self) -> &mut SingleLookupRequestState { - &mut self.state - } - fn new_request(&self) -> BlocksByRootRequest { - BlocksByRootRequest::new(VariableList::from(vec![self.requested_block_root])) - } - fn max_attempts() -> u8 { - L::MAX_ATTEMPTS - } - fn retry_request( - id: Id, - cx: &SyncNetworkContext, - peer_id: PeerId, - request: Self::RequestType, - ) -> Result<(), &'static str> { - cx.single_block_lookup_request_retry(id, peer_id, request, L::lookup_type()) + /// Returns true if the block has already been downloaded. + pub fn both_components_processed(&self) -> bool { + self.block_request_state.state.component_processed + && self.blob_request_state.state.component_processed } - fn verify_response_inner( + /// Checks both the block and blob request states to see if the peer is disconnected. + /// + /// Returns true if the lookup should be dropped. + pub fn should_drop_lookup_on_disconnected_peer( &mut self, - expected_block_root: Hash256, - response: Option, - peer_id: PeerShouldHave, - ) -> Result>, LookupVerifyError> { - match response { - Some(block) => { - // Compute the block root using this specific function so that we can get timing - // metrics. - let block_root = get_block_root(&block); - if block_root != expected_block_root { - // return an error and drop the block - // NOTE: we take this is as a download failure to prevent counting the - // attempt as a chain failure, but simply a peer failure. - self.state.register_failure_downloading(); - Err(LookupVerifyError::RootMismatch) - } else { - // Return the block for processing. - self.state.state = State::Processing { peer_id }; - Ok(Some((block_root, block))) - } - } - None => { - if peer_id.should_have_block() { - self.state.register_failure_downloading(); - Err(LookupVerifyError::NoBlockReturned) - } else { - self.state.state = State::AwaitingDownload; - Err(LookupVerifyError::BenignFailure) - } + peer_id: &PeerId, + cx: &SyncNetworkContext, + log: &Logger, + ) -> bool { + let block_root = self.block_root(); + let block_peer_disconnected = self + .block_request_state + .state + .check_peer_disconnected(peer_id) + .is_err(); + let blob_peer_disconnected = self + .blob_request_state + .state + .check_peer_disconnected(peer_id) + .is_err(); + + if block_peer_disconnected || blob_peer_disconnected { + if let Err(e) = self.request_block_and_blobs(cx) { + trace!(log, "Single lookup failed on peer disconnection"; "block_root" => ?block_root, "error" => ?e); + return true; } } + false } -} - -impl RequestState for BlobRequestState { - type RequestType = BlobsByRootRequest; - type ResponseType = Arc>; - type ReconstructedResponseType = FixedBlobSidecarList; - type VerifiedResponseType = FixedBlobSidecarList; - // response verify - fn response_type() -> ResponseType { - ResponseType::Blob + /// Returns `true` if the block has already been downloaded. + pub(crate) fn block_already_downloaded(&self) -> bool { + if let Some(components) = self.cached_child_components.as_ref() { + components.downloaded_block.is_some() + } else { + self.da_checker.has_block(&self.block_root()) + } } - fn get_parent_root(verified_response: &FixedBlobSidecarList) -> Option { - verified_response - .into_iter() - .filter_map(|blob| blob.as_ref()) - .map(|blob| blob.block_parent_root) - .next() - } - fn request_state_mut(request: &mut SingleBlockLookup) -> &mut Self { - &mut request.blob_request_state - } - fn add_to_parent_components( - verified_response: FixedBlobSidecarList, - components: &mut UnknownParentComponents, - ) { - components.add_unknown_parent_blobs(verified_response); - } - fn verified_to_reconstructed( - blobs: FixedBlobSidecarList, - ) -> FixedBlobSidecarList { - blobs + /// Updates the `requested_ids` field of the `BlockRequestState` with the most recent picture + /// of which blobs still need to be requested. Returns `true` if there are no more blobs to + /// request. + pub(crate) fn blobs_already_downloaded(&mut self) -> bool { + self.update_blobs_request(); + self.blob_request_state.requested_ids.is_empty() } - fn send_for_processing( - id: Id, - bl: &mut BlockLookups, - block_root: Hash256, - verified: FixedBlobSidecarList, - duration: Duration, - cx: &SyncNetworkContext, - ) -> Result<(), ()> { - bl.send_blobs_for_processing( - block_root, - verified, - duration, - BlockProcessType::SingleBlob { id }, - cx, - ) + /// Updates this request with the most recent picture of which blobs still need to be requested. + pub fn update_blobs_request(&mut self) { + self.blob_request_state.requested_ids = self.missing_blob_ids() } - fn get_state(&self) -> &SingleLookupRequestState { - &self.state - } - fn get_state_mut(&mut self) -> &mut SingleLookupRequestState { - &mut self.state - } - fn new_request(&self) -> BlobsByRootRequest { - BlobsByRootRequest { - blob_ids: VariableList::from(self.requested_ids.clone()), + /// If `unknown_parent_components` is `Some`, we know block components won't hit the data + /// availability cache, so we don't check it. In either case we use the data availability + /// checker to get a picture of outstanding blob requirements for the block root. + pub(crate) fn missing_blob_ids(&self) -> Vec { + if let Some(components) = self.cached_child_components.as_ref() { + let blobs = components.downloaded_indices(); + self.da_checker + .get_missing_blob_ids( + self.block_root(), + components.downloaded_block.as_ref(), + Some(blobs), + ) + .unwrap_or_default() + } else { + self.da_checker + .get_missing_blob_ids_checking_cache(self.block_root()) + .unwrap_or_default() } } - fn max_attempts() -> u8 { - L::MAX_ATTEMPTS + + /// Penalizes a blob peer if it should have blobs but didn't return them to us. Does not penalize + /// a peer who we request blobs from based on seeing a block or blobs over gossip. This may + /// have been a benign failure. + pub fn penalize_lazy_blob_peer(&mut self, cx: &SyncNetworkContext) { + if let Ok(blob_peer) = self.blob_request_state.state.processing_peer() { + if let PeerShouldHave::BlockAndBlobs(blob_peer) = blob_peer { + cx.report_peer( + blob_peer, + PeerAction::MidToleranceError, + "single_block_failure", + ); + } + self.blob_request_state + .state + .remove_peer_if_useless(blob_peer.as_peer_id()); + } } - fn retry_request( - id: Id, - cx: &SyncNetworkContext, - peer_id: PeerId, - request: Self::RequestType, - ) -> Result<(), &'static str> { - cx.single_blob_lookup_request_retry(id, peer_id, request, L::lookup_type()) + + /// This failure occurs on download, so register a failure downloading, penalize the peer if + /// necessary and clear the blob cache. + pub fn handle_consistency_failure(&mut self, cx: &SyncNetworkContext) { + self.penalize_lazy_blob_peer(cx); + if let Some(cached_child) = self.cached_child_components.as_mut() { + cached_child.clear_blobs(); + } + self.blob_request_state.state.register_failure_downloading() } - fn verify_response_inner( - &mut self, - expected_block_root: Hash256, - blob: Option, - peer_id: PeerShouldHave, - ) -> Result>, LookupVerifyError> { - match blob { - Some(blob) => { - let received_id = blob.id(); - if !self.requested_ids.contains(&received_id) { - self.state.register_failure_downloading(); - Err(LookupVerifyError::UnrequestedBlobId) - } else { - // State should remain downloading until we receive the stream terminator. - self.requested_ids.retain(|id| *id != received_id); - let blob_index = blob.index; - - if blob_index >= T::EthSpec::max_blobs_per_block() as u64 { - return Err(LookupVerifyError::InvalidIndex(blob.index)); - } - *self.blob_download_queue.index_mut(blob_index as usize) = Some(blob); - Ok(None) - } - } - None => { - self.state.state = State::Processing { peer_id }; - let blobs = std::mem::take(&mut self.blob_download_queue); - Ok(Some((expected_block_root, blobs))) - } + /// This failure occurs after processing, so register a failure processing, penalize the peer if + /// necessary and clear the blob cache. + pub fn handle_availability_check_failure(&mut self, cx: &SyncNetworkContext) { + self.penalize_lazy_blob_peer(cx); + if let Some(cached_child) = self.cached_child_components.as_mut() { + cached_child.clear_blobs(); } + self.blob_request_state.state.register_failure_processing() } } +/// The state of the blob request component of a `SingleBlockLookup`. pub struct BlobRequestState { + /// The latest picture of which blobs still need to be requested. This includes information + /// from both block/blobs downloaded in the network layer and any blocks/blobs that exist in + /// the data availability checker. pub requested_ids: Vec, /// Where we store blobs until we receive the stream terminator. pub blob_download_queue: FixedBlobSidecarList, @@ -430,7 +352,7 @@ pub struct BlobRequestState { _phantom: PhantomData, } -impl BlobRequestState { +impl BlobRequestState { pub fn new(peer_source: &[PeerShouldHave]) -> Self { Self { requested_ids: <_>::default(), @@ -441,6 +363,7 @@ impl BlobRequestState { } } +/// The state of the block request component of a `SingleBlockLookup`. pub struct BlockRequestState { pub requested_block_root: Hash256, pub state: SingleLookupRequestState, @@ -457,33 +380,34 @@ impl BlockRequestState { } } -impl SingleBlockLookup { - pub(crate) fn block_already_downloaded(&self) -> bool { - if let Some(components) = self.unknown_parent_components.as_ref() { - components.downloaded_block.is_some() - } else { - self.da_checker - .has_block(&self.block_request_state.requested_block_root) - } - } - - pub(crate) fn blobs_already_downloaded(&mut self) -> bool { - self.update_blobs_request(); - self.blob_request_state.requested_ids.is_empty() - } +/// This is the status of cached components for a lookup if they are required. It provides information +/// about whether we should send a responses immediately for processing, whether we require more +/// responses, or whether all cached components have been received and the reconstructed block +/// should be sent for processing. +pub enum CachedChild { + /// All child components have been received, this is the reconstructed block, including all. + /// It has been checked for consistency between blobs and block, but no consensus checks have + /// been performed and no kzg verification has been performed. + Ok(RpcBlock), + /// All child components have not yet been received. + DownloadIncomplete, + /// Child components should not be cached, send this directly for processing. + NotRequired, + /// There was an error during consistency checks between block and blobs. + Err(AvailabilityCheckError), } -/// For requests triggered by an `UnknownBlockParent` or `UnknownBlockParent`, this struct -/// is used to cache components as they are sent to the networking layer. We can't use the -/// data availability cache currently because any blocks or blobs without parents won't hit +/// For requests triggered by an `UnknownBlockParent` or `UnknownBlobParent`, this struct +/// is used to cache components as they are sent to the network service. We can't use the +/// data availability cache currently because any blocks or blobs without parents /// won't pass validation and therefore won't make it into the cache. #[derive(Default)] -pub struct UnknownParentComponents { +pub struct ChildComponents { pub downloaded_block: Option>>, pub downloaded_blobs: FixedBlobSidecarList, } -impl From> for UnknownParentComponents { +impl From> for ChildComponents { fn from(value: RpcBlock) -> Self { let (block, blobs) = value.deconstruct(); let fixed_blobs = blobs.map(|blobs| { @@ -493,7 +417,7 @@ impl From> for UnknownParentComponents { } } -impl UnknownParentComponents { +impl ChildComponents { pub fn new( block: Option>>, blobs: Option>, @@ -503,9 +427,15 @@ impl UnknownParentComponents { downloaded_blobs: blobs.unwrap_or_default(), } } + + pub fn clear_blobs(&mut self) { + self.downloaded_blobs = FixedBlobSidecarList::default(); + } + pub fn add_unknown_parent_block(&mut self, block: Arc>) { self.downloaded_block = Some(block); } + pub fn add_unknown_parent_blobs(&mut self, blobs: FixedBlobSidecarList) { for (index, blob_opt) in self.downloaded_blobs.iter_mut().enumerate() { if let Some(Some(downloaded_blob)) = blobs.get(index) { @@ -513,6 +443,7 @@ impl UnknownParentComponents { } } } + pub fn downloaded_indices(&self) -> HashSet { self.downloaded_blobs .iter() @@ -534,254 +465,19 @@ pub struct SingleLookupRequestState { /// Peers from which we have requested this block. pub used_peers: HashSet, /// How many times have we attempted to process this block or blob. - failed_processing: u8, + pub failed_processing: u8, /// How many times have we attempted to download this block or blob. - failed_downloading: u8, + pub failed_downloading: u8, + /// Whether or not we have downloaded this block or blob. + pub component_downloaded: bool, + /// Whether or not we have processed this block or blob. pub component_processed: bool, -} - -#[derive(Debug, PartialEq, Eq)] -pub enum State { - AwaitingDownload, - Downloading { peer_id: PeerShouldHave }, - Processing { peer_id: PeerShouldHave }, -} - -#[derive(Debug, PartialEq, Eq, IntoStaticStr)] -pub enum LookupVerifyError { - RootMismatch, - NoBlockReturned, - ExtraBlocksReturned, - UnrequestedBlobId, - ExtraBlobsReturned, - NotEnoughBlobsReturned, - InvalidIndex(u64), - /// We don't have enough information to know - /// whether the peer is at fault or simply missed - /// what was requested on gossip. - BenignFailure, -} - -#[derive(Debug, PartialEq, Eq, IntoStaticStr)] -pub enum LookupRequestError { - /// Too many failed attempts - TooManyAttempts { - /// The failed attempts were primarily due to processing failures. - cannot_process: bool, - }, - NoPeers, - SendFailed(&'static str), - AlreadyDownloaded, -} - -impl SingleBlockLookup { - pub fn new( - requested_block_root: Hash256, - unknown_parent_components: Option>, - peers: &[PeerShouldHave], - da_checker: Arc>, - id: Id, - ) -> Self { - Self { - id, - block_request_state: BlockRequestState::new(requested_block_root, peers), - blob_request_state: BlobRequestState::new(peers), - da_checker, - unknown_parent_components, - triggered: false, - } - } - - pub fn is_for_block(&self, block_root: Hash256) -> bool { - self.block_request_state.requested_block_root == block_root - } - - /// Send the necessary request for blobs and blocks and update `self.id` with the latest - /// request `Id`s. This will return `Err(())` if neither the block nor blob request could be made - /// or are no longer required. - pub fn request_block_and_blobs( - &mut self, - cx: &SyncNetworkContext, - ) -> Result<(), LookupRequestError> { - let block_root = self.block_request_state.requested_block_root; - if self.block_already_downloaded() && self.blobs_already_downloaded() { - // drop lookup - trace!(cx.log, "Lookup request already completed"; "block_root"=> ?block_root); - return Err(LookupRequestError::AlreadyDownloaded); - } - - let (block_peer_id, block_request) = - match as RequestState>::build_request( - &mut self.block_request_state, - ) { - Ok(opt) => opt, - Err(e) => { - // drop lookup - debug!(cx.log, - "Lookup request block error, dropping lookup"; - "block_root"=> ?block_root, - "error"=> ?e - ); - return Err(e); - } - }; - - let (blob_peer_id, blob_request) = match ::EthSpec, - > as RequestState>::build_request( - &mut self.blob_request_state - ) { - Ok(opt) => opt, - Err(e) => { - // drop lookup - debug!(cx.log, - "Lookup request blob error, dropping lookup"; - "block_root"=> ?block_root, - "error"=> ?e - ); - return Err(e); - } - }; - - cx.single_lookup_request( - self.id, - block_peer_id, - block_request, - blob_peer_id, - blob_request, - L::lookup_type(), - ) - .map_err(LookupRequestError::SendFailed)?; - Ok(()) - } - - pub fn update_blobs_request(&mut self) { - self.blob_request_state.requested_ids = if let Some(components) = - self.unknown_parent_components.as_ref() - { - let blobs = components.downloaded_indices(); - self.da_checker - .get_missing_blob_ids( - self.block_request_state.requested_block_root, - components.downloaded_block.as_ref(), - Some(blobs), - ) - .unwrap_or_default() - } else { - self.da_checker - .get_missing_blob_ids_checking_cache(self.block_request_state.requested_block_root) - .unwrap_or_default() - }; - } - - pub fn get_downloaded_block(&mut self) -> Option> { - self.unknown_parent_components - .as_mut() - .and_then(|components| { - let downloaded_block = components.downloaded_block.as_ref(); - let downloaded_indices = components.downloaded_indices(); - let missing_ids = self.da_checker.get_missing_blob_ids( - self.block_request_state.requested_block_root, - downloaded_block, - Some(downloaded_indices), - ); - let download_complete = - missing_ids.map_or(true, |missing_ids| missing_ids.is_empty()); - if download_complete { - let UnknownParentComponents { - downloaded_block, - downloaded_blobs, - } = components; - downloaded_block.as_ref().and_then(|block| { - //TODO(sean) figure out how to properly deal with a consistency error here, - // should we downscore the peer sending blobs? - let blobs = std::mem::take(downloaded_blobs); - let filtered = blobs - .into_iter() - .filter_map(|b| b.clone()) - .collect::>(); - let blobs = VariableList::from(filtered); - RpcBlock::new(block.clone(), Some(blobs)).ok() - }) - } else { - None - } - }) - } - - pub fn add_unknown_parent_components( - &mut self, - components: UnknownParentComponents, - ) { - if let Some(ref mut existing_components) = self.unknown_parent_components { - let UnknownParentComponents { - downloaded_block, - downloaded_blobs, - } = components; - if let Some(block) = downloaded_block { - existing_components.add_unknown_parent_block(block); - } - existing_components.add_unknown_parent_blobs(downloaded_blobs); - } else { - self.unknown_parent_components = Some(components); - } - } - - pub fn add_peers(&mut self, peers: &[PeerShouldHave]) { - for peer in peers { - match peer { - PeerShouldHave::BlockAndBlobs(peer_id) => { - self.block_request_state.state.add_peer(peer_id); - self.blob_request_state.state.add_peer(peer_id); - } - PeerShouldHave::Neither(peer_id) => { - self.block_request_state.state.add_potential_peer(peer_id); - self.blob_request_state.state.add_potential_peer(peer_id); - } - } - } - } - - pub fn both_components_processed(&self) -> bool { - self.block_request_state.state.component_processed - && self.blob_request_state.state.component_processed - } - - pub fn should_remove_disconnected_peer( - &mut self, - id: Id, - peer_id: &PeerId, - cx: &SyncNetworkContext, - log: &Logger, - ) -> bool { - let useless_block_peer = - if as RequestState>::check_peer_disconnected( - &mut self.block_request_state, - peer_id, - ) - .is_err() - { - trace!(log, "Single lookup failed on peer disconnection"; "block_root" => ?self.block_request_state.requested_block_root, "response_type" => ?ResponseType::Block); - self.block_request_state - .retry_request_after_failure(id, cx, log) - .is_err() - } else { - false - }; - let useless_blob_peer = if ::EthSpec> as RequestState>::check_peer_disconnected(&mut self - .blob_request_state, peer_id) - .is_err() - { - trace!(log, "Single lookup failed on peer disconnection"; "block_root" => ?self.block_request_state.requested_block_root, "response_type" => ?ResponseType::Blob); - self.blob_request_state - .retry_request_after_failure(id, cx, log) - .is_err() - } else { - false - }; - useless_block_peer && useless_blob_peer - } + /// Should be incremented everytime this request is retried. The purpose of this is to + /// differentiate retries of the same block/blob request within a lookup. We currently penalize + /// peers and retry requests prior to receiving the stream terminator. This means responses + /// from a prior request may arrive after a new request has been sent, this counter allows + /// us to differentiate these two responses. + pub req_counter: u32, } impl SingleLookupRequestState { @@ -805,7 +501,9 @@ impl SingleLookupRequestState { used_peers: HashSet::default(), failed_processing: 0, failed_downloading: 0, + component_downloaded: false, component_processed: false, + req_counter: 0, } } @@ -827,11 +525,13 @@ impl SingleLookupRequestState { self.failed_processing + self.failed_downloading } + /// This method should be used for peers wrapped in `PeerShouldHave::BlockAndBlobs`. pub fn add_peer(&mut self, peer_id: &PeerId) { self.potential_peers.remove(peer_id); self.available_peers.insert(*peer_id); } + /// This method should be used for peers wrapped in `PeerShouldHave::Neither`. pub fn add_potential_peer(&mut self, peer_id: &PeerId) { if !self.available_peers.contains(peer_id) { self.potential_peers.insert(*peer_id); @@ -852,6 +552,8 @@ impl SingleLookupRequestState { Ok(()) } + /// Returns the id peer we downloaded from if we have downloaded a verified block, otherwise + /// returns an error. pub fn processing_peer(&self) -> Result { if let State::Processing { peer_id } = &self.state { Ok(*peer_id) @@ -860,6 +562,8 @@ impl SingleLookupRequestState { } } + /// Remove the given peer from the set of potential peers, so long as there is at least one + /// other potential peer or we have any available peers. pub fn remove_peer_if_useless(&mut self, peer_id: &PeerId) { if !self.available_peers.is_empty() || self.potential_peers.len() > 1 { self.potential_peers.remove(peer_id); @@ -876,10 +580,7 @@ impl slog::Value for SingleBlockLookup { ) -> slog::Result { serializer.emit_str("request", key)?; serializer.emit_arguments("lookup_type", &format_args!("{:?}", L::lookup_type()))?; - serializer.emit_arguments( - "hash", - &format_args!("{}", self.block_request_state.requested_block_root), - )?; + serializer.emit_arguments("hash", &format_args!("{}", self.block_root()))?; serializer.emit_arguments( "blob_ids", &format_args!("{:?}", self.blob_request_state.requested_ids), @@ -924,6 +625,8 @@ impl slog::Value for SingleLookupRequestState { #[cfg(test)] mod tests { use super::*; + use crate::sync::block_lookups::common::LookupType; + use crate::sync::block_lookups::common::{Lookup, RequestState}; use beacon_chain::builder::Witness; use beacon_chain::eth1_chain::CachingEth1Backend; use sloggers::null::NullLoggerBuilder; diff --git a/beacon_node/network/src/sync/block_lookups/tests.rs b/beacon_node/network/src/sync/block_lookups/tests.rs index d7c9688e502..305c0279cb1 100644 --- a/beacon_node/network/src/sync/block_lookups/tests.rs +++ b/beacon_node/network/src/sync/block_lookups/tests.rs @@ -1,12 +1,13 @@ use crate::network_beacon_processor::NetworkBeaconProcessor; use crate::service::RequestId; -use crate::sync::manager::RequestId as SyncId; +use crate::sync::manager::{RequestId as SyncId, SingleLookupReqId}; use crate::NetworkMessage; use std::sync::Arc; use super::*; +use crate::sync::block_lookups::common::ResponseType; use beacon_chain::builder::Witness; use beacon_chain::eth1_chain::CachingEth1Backend; use beacon_chain::test_utils::{build_log, BeaconChainHarness, EphemeralHarnessType}; @@ -156,7 +157,7 @@ impl TestRig { } #[track_caller] - fn expect_block_request(&mut self, response_type: ResponseType) -> Id { + fn expect_block_request(&mut self, response_type: ResponseType) -> SingleLookupReqId { match response_type { ResponseType::Block => match self.network_rx.try_recv() { Ok(NetworkMessage::SendRequest { @@ -182,7 +183,7 @@ impl TestRig { } #[track_caller] - fn expect_parent_request(&mut self, response_type: ResponseType) -> Id { + fn expect_parent_request(&mut self, response_type: ResponseType) -> SingleLookupReqId { match response_type { ResponseType::Block => match self.network_rx.try_recv() { Ok(NetworkMessage::SendRequest { @@ -322,7 +323,7 @@ fn test_single_block_lookup_happy_path() { // after processing. bl.single_lookup_response::>(id, peer_id, None, D, &cx); bl.single_block_component_processed::>( - id, + id.id, BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(block_root)), &mut cx, ); @@ -471,7 +472,7 @@ fn test_single_block_lookup_becomes_parent_request() { // Send the stream termination. Peer should have not been penalized, and the request moved to a // parent request after processing. bl.single_block_component_processed::>( - id, + id.id, BlockError::ParentUnknown(block.into()).into(), &mut cx, ); @@ -766,6 +767,14 @@ fn test_parent_lookup_too_many_attempts() { &cx, ); // Send the stream termination + + // Note, previously we would send the same lookup id with a stream terminator, + // we'd ignore it because we'd intrepret it as an unrequested response, since + // we already got one response for the block. I'm not sure what the intent is + // for having this stream terminator line in this test at all. Receiving an invalid + // block and a stream terminator with the same Id now results in two failed attempts, + // I'm unsure if this is how it should behave? + // bl.parent_lookup_response::>(id, peer_id, None, D, &cx); rig.expect_penalty(); } @@ -1051,7 +1060,7 @@ fn test_single_block_lookup_ignored_response() { bl.single_lookup_response::>(id, peer_id, None, D, &cx); // Send an Ignored response, the request should be dropped bl.single_block_component_processed::>( - id, + id.id, BlockProcessingResult::Ignored, &mut cx, ); @@ -1216,6 +1225,7 @@ fn test_same_chain_race_condition() { mod deneb_only { use super::*; + use crate::sync::block_lookups::common::ResponseType; use beacon_chain::blob_verification::BlobError; use std::ops::IndexMut; use std::str::FromStr; @@ -1229,10 +1239,10 @@ mod deneb_only { parent_block: Option>>, parent_blobs: Vec>>, peer_id: PeerId, - block_req_id: Option, - parent_block_req_id: Option, - blob_req_id: Option, - parent_blob_req_id: Option, + block_req_id: Option, + parent_block_req_id: Option, + blob_req_id: Option, + parent_blob_req_id: Option, slot: Slot, block_root: Hash256, } @@ -1296,7 +1306,7 @@ mod deneb_only { block_root = child_root; bl.search_child_block( child_root, - Some(UnknownParentComponents::new(Some(child_block), None)), + Some(ChildComponents::new(Some(child_block), None)), &[PeerShouldHave::Neither(peer_id)], &mut cx, ); @@ -1334,7 +1344,7 @@ mod deneb_only { *blobs.index_mut(0) = Some(child_blob); bl.search_child_block( child_root, - Some(UnknownParentComponents::new(None, Some(blobs))), + Some(ChildComponents::new(None, Some(blobs))), &[PeerShouldHave::Neither(peer_id)], &mut cx, ); @@ -1531,7 +1541,7 @@ mod deneb_only { // mean we do not send a new request. self.bl .single_block_component_processed::>( - self.block_req_id.expect("block request id"), + self.block_req_id.expect("block request id").id, BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported( self.block_root, )), @@ -1578,7 +1588,7 @@ mod deneb_only { fn invalid_block_processed(mut self) -> Self { self.bl .single_block_component_processed::>( - self.block_req_id.expect("block request id"), + self.block_req_id.expect("block request id").id, BlockProcessingResult::Err(BlockError::ProposalSignatureInvalid), &mut self.cx, ); @@ -1589,7 +1599,7 @@ mod deneb_only { fn invalid_blob_processed(mut self) -> Self { self.bl .single_block_component_processed::>( - self.blob_req_id.expect("blob request id"), + self.blob_req_id.expect("blob request id").id, BlockProcessingResult::Err(BlockError::BlobValidation( BlobError::ProposerSignatureInvalid, )), @@ -1602,7 +1612,7 @@ mod deneb_only { fn missing_components_from_block_request(mut self) -> Self { self.bl .single_block_component_processed::>( - self.block_req_id.expect("block request id"), + self.block_req_id.expect("block request id").id, BlockProcessingResult::Ok(AvailabilityProcessingStatus::MissingComponents( self.slot, self.block_root, @@ -1616,7 +1626,7 @@ mod deneb_only { fn missing_components_from_blob_request(mut self) -> Self { self.bl .single_block_component_processed::>( - self.blob_req_id.expect("blob request id"), + self.blob_req_id.expect("blob request id").id, BlockProcessingResult::Ok(AvailabilityProcessingStatus::MissingComponents( self.slot, self.block_root, diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index cedad8890f0..696a2949d5b 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -41,11 +41,10 @@ use super::range_sync::{RangeSync, RangeSyncType, EPOCHS_PER_BATCH}; use crate::network_beacon_processor::{ChainSegmentProcessId, NetworkBeaconProcessor}; use crate::service::NetworkMessage; use crate::status::ToStatusMessage; +use crate::sync::block_lookups::common::{Current, Parent}; use crate::sync::block_lookups::delayed_lookup; use crate::sync::block_lookups::delayed_lookup::DelayedLookupMessage; -use crate::sync::block_lookups::{ - BlobRequestState, BlockRequestState, Current, Parent, UnknownParentComponents, -}; +use crate::sync::block_lookups::{BlobRequestState, BlockRequestState, ChildComponents}; use crate::sync::range_sync::ByRangeRequestType; use beacon_chain::block_verification_types::AsBlock; use beacon_chain::block_verification_types::RpcBlock; @@ -84,45 +83,39 @@ pub const DELAY_QUEUE_CHANNEL_SIZE: usize = 128; pub type Id = u32; -#[derive(Debug, Copy, Clone)] -pub enum ResponseType { - Block, - Blob, +#[derive(Debug, Hash, PartialEq, Eq, Clone, Copy)] +pub struct SingleLookupReqId { + pub id: Id, + pub req_counter: Id, +} + +#[derive(Debug, Hash, PartialEq, Eq, Clone, Copy)] +pub struct ParentLookupReqId { + pub id: Id, + pub req_counter: Id, } /// Id of rpc requests sent by sync to the network. #[derive(Debug, Hash, PartialEq, Eq, Clone, Copy)] pub enum RequestId { /// Request searching for a block given a hash. - SingleBlock { - id: Id, - }, - SingleBlob { - id: Id, - }, - /// Request searching for a block's parent. The id is the chain - ParentLookup { - id: Id, - }, - ParentLookupBlob { - id: Id, - }, + SingleBlock { id: SingleLookupReqId }, + /// Request searching for a set of blobs given a hash. + SingleBlob { id: SingleLookupReqId }, + /// Request searching for a block's parent. The id is the chain, share with the corresponding + /// blob id. + ParentLookup { id: SingleLookupReqId }, + /// Request searching for a block's parent blobs. The id is the chain, shared with the corresponding + /// block id. + ParentLookupBlob { id: SingleLookupReqId }, /// Request was from the backfill sync algorithm. - BackFillBlocks { - id: Id, - }, + BackFillBlocks { id: Id }, /// Backfill request that is composed by both a block range request and a blob range request. - BackFillBlockAndBlobs { - id: Id, - }, + BackFillBlockAndBlobs { id: Id }, /// The request was from a chain in the range sync algorithm. - RangeBlocks { - id: Id, - }, + RangeBlocks { id: Id }, /// Range request that is composed by both a block range request and a blob range request. - RangeBlockAndBlobs { - id: Id, - }, + RangeBlockAndBlobs { id: Id }, } #[derive(Debug)] @@ -680,7 +673,7 @@ impl SyncManager { block_root, parent_root, blob_slot, - Some(UnknownParentComponents::new(None, Some(blobs))), + Some(ChildComponents::new(None, Some(blobs))), ); } SyncMessage::UnknownBlockHashFromAttestation(peer_id, block_hash) => { @@ -718,16 +711,9 @@ impl SyncManager { } } } - SyncMessage::MissingGossipBlockComponentsDelayed(block_root) => { - if self - .block_lookups - .trigger_lookup_by_root(block_root, &self.network) - .is_err() - { - // No request was made for block or blob so the lookup is dropped. - self.block_lookups.remove_lookup_by_root(block_root); - } - } + SyncMessage::MissingGossipBlockComponentsDelayed(block_root) => self + .block_lookups + .trigger_lookup_by_root(block_root, &self.network), SyncMessage::Disconnect(peer_id) => { self.peer_disconnect(&peer_id); } @@ -796,7 +782,7 @@ impl SyncManager { block_root: Hash256, parent_root: Hash256, slot: Slot, - parent_components: Option>, + parent_components: Option>, ) { if self.should_search_for_block(slot, &peer_id) { self.block_lookups.search_parent( @@ -951,7 +937,7 @@ impl SyncManager { seen_timestamp, &self.network, ), - RequestId::SingleBlob { id: _ } => { + RequestId::SingleBlob { .. } => { crit!(self.log, "Blob received during block request"; "peer_id" => %peer_id ); } RequestId::ParentLookup { id } => self @@ -1023,7 +1009,7 @@ impl SyncManager { seen_timestamp: Duration, ) { match request_id { - RequestId::SingleBlock { id: _ } => { + RequestId::SingleBlock { .. } => { crit!(self.log, "Single blob received during block request"; "peer_id" => %peer_id ); } RequestId::SingleBlob { id } => self diff --git a/beacon_node/network/src/sync/mod.rs b/beacon_node/network/src/sync/mod.rs index 1dd33bd31c8..23e055f60af 100644 --- a/beacon_node/network/src/sync/mod.rs +++ b/beacon_node/network/src/sync/mod.rs @@ -9,6 +9,6 @@ mod network_context; mod peer_sync_info; mod range_sync; -pub use block_lookups::UnknownParentComponents; +pub use block_lookups::ChildComponents; pub use manager::{BatchProcessResult, SyncMessage}; pub use range_sync::{BatchOperationOutcome, ChainId}; diff --git a/beacon_node/network/src/sync/network_context.rs b/beacon_node/network/src/sync/network_context.rs index b7c6de2fc63..adc9469d71f 100644 --- a/beacon_node/network/src/sync/network_context.rs +++ b/beacon_node/network/src/sync/network_context.rs @@ -7,7 +7,8 @@ use super::range_sync::{BatchId, ByRangeRequestType, ChainId}; use crate::network_beacon_processor::NetworkBeaconProcessor; use crate::service::{NetworkMessage, RequestId}; use crate::status::ToStatusMessage; -use crate::sync::block_lookups::LookupType; +use crate::sync::block_lookups::common::LookupType; +use crate::sync::manager::SingleLookupReqId; use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::{BeaconChain, BeaconChainTypes, EngineState}; use fnv::FnvHashMap; @@ -404,23 +405,9 @@ impl SyncNetworkContext { } } - /// Sends a blocks by root request for a parent request. - pub fn single_lookup_request( + pub fn block_lookup_request( &self, - id: Id, - peer_id: PeerId, - request: BlocksByRootRequest, - blob_peer_id: PeerId, - blob_request: BlobsByRootRequest, - lookup_type: LookupType, - ) -> Result<(), &'static str> { - self.single_block_lookup_request_retry(id, peer_id, request, lookup_type)?; - self.single_blob_lookup_request_retry(id, blob_peer_id, blob_request, lookup_type)?; - Ok(()) - } - pub fn single_block_lookup_request_retry( - &self, - id: Id, + id: SingleLookupReqId, peer_id: PeerId, request: BlocksByRootRequest, lookup_type: LookupType, @@ -448,14 +435,18 @@ impl SyncNetworkContext { Ok(()) } - pub fn single_blob_lookup_request_retry( + pub fn blob_lookup_request( &self, - id: Id, + id: SingleLookupReqId, blob_peer_id: PeerId, blob_request: BlobsByRootRequest, lookup_type: LookupType, ) -> Result<(), &'static str> { - let request_id = RequestId::Sync(SyncRequestId::SingleBlob { id }); + let sync_id = match lookup_type { + LookupType::Current => SyncRequestId::SingleBlock { id }, + LookupType::Parent => SyncRequestId::ParentLookup { id }, + }; + let request_id = RequestId::Sync(sync_id); if !blob_request.blob_ids.is_empty() { trace!( @@ -558,6 +549,12 @@ impl SyncNetworkContext { "To deal with alignment with deneb boundaries, batches need to be of just one epoch" ); + #[cfg(test)] + { + // Keep tests only for blocks. + ByRangeRequestType::Blocks + } + #[cfg(not(test))] if let Some(data_availability_boundary) = self.chain.data_availability_boundary() { if epoch >= data_availability_boundary { ByRangeRequestType::BlocksAndBlobs From 4215160bc19c00163336afe0783856e7898af376 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Tue, 1 Aug 2023 18:31:01 -0400 Subject: [PATCH 07/18] smol cleanup and a bugfix --- .../src/network_beacon_processor/tests.rs | 2 - .../network/src/sync/block_lookups/mod.rs | 83 +++++++++---------- 2 files changed, 38 insertions(+), 47 deletions(-) diff --git a/beacon_node/network/src/network_beacon_processor/tests.rs b/beacon_node/network/src/network_beacon_processor/tests.rs index 1bdc3a88166..2c37d177aab 100644 --- a/beacon_node/network/src/network_beacon_processor/tests.rs +++ b/beacon_node/network/src/network_beacon_processor/tests.rs @@ -318,7 +318,6 @@ impl TestRig { } pub fn enqueue_single_lookup_rpc_blobs(&self) { if let Some(blobs) = self.next_blobs.clone() { - dbg!(blobs.len()); let blobs = FixedBlobSidecarList::from( blobs .into_iter() @@ -1004,7 +1003,6 @@ async fn test_rpc_block_reprocessing() { rig.enqueue_single_lookup_rpc_blobs(); if rig.next_blobs.as_ref().map(|b| b.len()).unwrap_or(0) > 0 { - dbg!("here"); rig.assert_event_journal(&[RPC_BLOBS, WORKER_FREED, NOTHING_TO_DO]) .await; } diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index 43fb5330181..4e6e5f987f5 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -5,6 +5,7 @@ use super::BatchProcessResult; use super::{manager::BlockProcessType, network_context::SyncNetworkContext}; use crate::metrics; use crate::network_beacon_processor::ChainSegmentProcessId; +use crate::sync::block_lookups::common::LookupType; use crate::sync::block_lookups::parent_lookup::{ParentLookup, RequestError}; use crate::sync::block_lookups::single_block_lookup::{ CachedChild, LookupRequestError, LookupVerifyError, @@ -16,7 +17,6 @@ use beacon_chain::validator_monitor::timestamp_now; use beacon_chain::{AvailabilityProcessingStatus, BeaconChainTypes, BlockError}; pub use common::Current; pub use common::Lookup; -use common::LookupType; pub use common::Parent; pub use common::RequestState; use fnv::FnvHashMap; @@ -359,16 +359,10 @@ impl BlockLookups { /// necessary because we want to make sure all parents are processed before sending a child /// for processing, otherwise the block will fail validation and will be returned to the network /// layer with an `UnknownParent` error. - pub fn has_pending_parent_request(&self, target_id: Id) -> bool { - self.single_block_lookups.iter().any(|(id, lookup)| { - if *id == target_id { - self.parent_lookups - .iter() - .any(|parent_lookup| parent_lookup.chain_hash() == lookup.block_root()) - } else { - false - } - }) + pub fn has_pending_parent_request(&self, block_root: Hash256) -> bool { + self.parent_lookups + .iter() + .any(|parent_lookup| parent_lookup.chain_hash() == block_root) } /// Process a block or blob response received from a single lookup request. @@ -383,8 +377,6 @@ impl BlockLookups { let id = lookup_id.id; let response_type = R::response_type(); - let chain_hash_opt = self.has_pending_parent_request(id); - let Some(lookup) = self.get_single_lookup::(lookup_id) else { if response.is_some() { warn!( @@ -398,14 +390,8 @@ impl BlockLookups { let expected_block_root = lookup.block_root(); - match self.single_lookup_response_inner::( - peer_id, - response, - seen_timestamp, - cx, - chain_hash_opt, - lookup, - ) { + match self.single_lookup_response_inner::(peer_id, response, seen_timestamp, cx, lookup) + { Ok(lookup) => { self.single_block_lookups.insert(id, lookup); } @@ -432,7 +418,6 @@ impl BlockLookups { response: Option, seen_timestamp: Duration, cx: &SyncNetworkContext, - delay_send: bool, mut lookup: SingleBlockLookup, ) -> Result, LookupRequestError> { let response_type = R::response_type(); @@ -445,8 +430,7 @@ impl BlockLookups { self.handle_verified_response::( seen_timestamp, cx, - delay_send, - None, + BlockProcessType::SingleBlock { id: lookup.id }, verified_response, &mut lookup, )?; @@ -481,8 +465,7 @@ impl BlockLookups { &self, seen_timestamp: Duration, cx: &SyncNetworkContext, - delay_send: bool, - chain_hash: Option, + process_type: BlockProcessType, verified_response: R::VerifiedResponseType, lookup: &mut SingleBlockLookup, ) -> Result<(), LookupRequestError> { @@ -494,23 +477,24 @@ impl BlockLookups { .component_downloaded = true; let cached_child = lookup.add_response::(verified_response.clone()); - match cached_child { CachedChild::Ok(block) => { - if let Some(chain_hash) = chain_hash { - if !delay_send { - let process_type = match L::lookup_type() { - LookupType::Parent => BlockProcessType::ParentLookup { chain_hash }, - LookupType::Current => BlockProcessType::SingleBlock { id }, - }; - self.send_block_for_processing( - block_root, - block, - seen_timestamp, - process_type, - cx, - )? - } + // If we have an outstanding parent request for this block, delay sending the response until + // all parent blocks have been processed, otherwise we will fail validation with an + // `UnknownParent`. + let delay_send = match L::lookup_type() { + LookupType::Parent => false, + LookupType::Current => self.has_pending_parent_request(lookup.block_root()), + }; + + if !delay_send { + self.send_block_for_processing( + block_root, + block, + seen_timestamp, + process_type, + cx, + )? } } CachedChild::DownloadIncomplete => { @@ -628,8 +612,9 @@ impl BlockLookups { self.handle_verified_response::( seen_timestamp, cx, - false, - Some(parent_lookup.chain_hash()), + BlockProcessType::ParentLookup { + chain_hash: parent_lookup.chain_hash(), + }, verified_response, &mut parent_lookup.current_parent_request, )?; @@ -1181,7 +1166,11 @@ impl BlockLookups { "chain_hash" => ?chain_hash ); child_lookup.handle_consistency_failure(cx); - if child_lookup.request_block_and_blobs(cx).is_err() { + if let Err(e) = child_lookup.request_block_and_blobs(cx) { + debug!(self.log, + "Failed to request block and blobs, dropping lookup"; + "error" => ?e + ); self.single_block_lookups.remove(&child_lookup_id); } } @@ -1313,7 +1302,11 @@ impl BlockLookups { "error" => ?e ); lookup.handle_consistency_failure(cx); - if lookup.request_block_and_blobs(cx).is_err() { + if let Err(e) = lookup.request_block_and_blobs(cx) { + debug!(self.log, + "Failed to request block and blobs, droppingn lookup"; + "error" => ?e + ); self.single_block_lookups.remove(&id); } } From c71e011a1cc4482c2d31b0d37481e5b4b6dc2ac2 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Tue, 1 Aug 2023 19:08:41 -0400 Subject: [PATCH 08/18] remove ParentLookupReqId --- beacon_node/network/src/sync/manager.rs | 6 ------ 1 file changed, 6 deletions(-) diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 696a2949d5b..03d7fdc2a93 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -89,12 +89,6 @@ pub struct SingleLookupReqId { pub req_counter: Id, } -#[derive(Debug, Hash, PartialEq, Eq, Clone, Copy)] -pub struct ParentLookupReqId { - pub id: Id, - pub req_counter: Id, -} - /// Id of rpc requests sent by sync to the network. #[derive(Debug, Hash, PartialEq, Eq, Clone, Copy)] pub enum RequestId { From 122def5ef4bf7c7a9d0c7b4d934388317acb1610 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 3 Aug 2023 09:43:50 -0400 Subject: [PATCH 09/18] Update beacon_node/network/src/sync/manager.rs Co-authored-by: Jimmy Chen --- beacon_node/network/src/sync/manager.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 03d7fdc2a93..9f6b14d4d57 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -944,7 +944,7 @@ impl SyncManager { &self.network, ), RequestId::ParentLookupBlob { id: _ } => { - crit!(self.log, "Blob received during parent block request"; "peer_id" => %peer_id ); + crit!(self.log, "Block received during parent blob request"; "peer_id" => %peer_id ); } RequestId::BackFillBlocks { id } => { let is_stream_terminator = block.is_none(); From 5736891b60405d44c7493d752d892d70432fd766 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 3 Aug 2023 09:44:07 -0400 Subject: [PATCH 10/18] Update beacon_node/network/src/sync/manager.rs Co-authored-by: Jimmy Chen --- beacon_node/network/src/sync/manager.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 9f6b14d4d57..d61e34c21ca 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -932,7 +932,7 @@ impl SyncManager { &self.network, ), RequestId::SingleBlob { .. } => { - crit!(self.log, "Blob received during block request"; "peer_id" => %peer_id ); + crit!(self.log, "Block received during blob request"; "peer_id" => %peer_id ); } RequestId::ParentLookup { id } => self .block_lookups From 064bf64dff86b3229316aeed0431c3f4251571a5 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 3 Aug 2023 09:50:52 -0400 Subject: [PATCH 11/18] update unreachables to crits --- beacon_node/network/src/router.rs | 24 +++++++++++++++--------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/beacon_node/network/src/router.rs b/beacon_node/network/src/router.rs index 927229e7b7e..3b7431822e7 100644 --- a/beacon_node/network/src/router.rs +++ b/beacon_node/network/src/router.rs @@ -21,7 +21,7 @@ use lighthouse_network::{ MessageId, NetworkGlobals, PeerId, PeerRequestId, PubsubMessage, Request, Response, }; use logging::TimeLatch; -use slog::{debug, o, trace}; +use slog::{crit, debug, o, trace}; use slog::{error, warn}; use std::sync::Arc; use std::time::{Duration, SystemTime, UNIX_EPOCH}; @@ -486,14 +486,16 @@ impl Router { | SyncId::SingleBlob { .. } | SyncId::ParentLookup { .. } | SyncId::ParentLookupBlob { .. } => { - unreachable!("Block lookups do not request BBRange requests") + crit!(self.log, "Block lookups do not request BBRange requests"; "peer_id" => %peer_id); } id @ (SyncId::BackFillBlocks { .. } | SyncId::RangeBlocks { .. } | SyncId::BackFillBlockAndBlobs { .. } | SyncId::RangeBlockAndBlobs { .. }) => id, }, - RequestId::Router => unreachable!("All BBRange requests belong to sync"), + RequestId::Router => { + crit!(self.log, "All BBRange requests belong to sync"; "peer_id" => %peer_id) + } }; trace!( @@ -551,13 +553,15 @@ impl Router { | SyncId::RangeBlocks { .. } | SyncId::RangeBlockAndBlobs { .. } | SyncId::BackFillBlockAndBlobs { .. } => { - unreachable!("Batch syncing do not request BBRoot requests") + crit!(self.log, "Batch syncing do not request BBRoot requests"; "peer_id" => %peer_id) } SyncId::SingleBlob { .. } | SyncId::ParentLookupBlob { .. } => { - unreachable!("Blob response to block by roots request") + crit!(self.log, "Blob response to block by roots request"; "peer_id" => %peer_id) } }, - RequestId::Router => unreachable!("All BBRoot requests belong to sync"), + RequestId::Router => { + crit!(self.log, "All BBRoot requests belong to sync"; "peer_id" => %peer_id) + } }; trace!( @@ -584,16 +588,18 @@ impl Router { RequestId::Sync(sync_id) => match sync_id { id @ (SyncId::SingleBlob { .. } | SyncId::ParentLookupBlob { .. }) => id, SyncId::SingleBlock { .. } | SyncId::ParentLookup { .. } => { - unreachable!("Block response to blobs by roots request") + crit!(self.log, "Block response to blobs by roots request"; "peer_id" => %peer_id) } SyncId::BackFillBlocks { .. } | SyncId::RangeBlocks { .. } | SyncId::RangeBlockAndBlobs { .. } | SyncId::BackFillBlockAndBlobs { .. } => { - unreachable!("Batch syncing does not request BBRoot requests") + crit!(self.log, "Batch syncing does not request BBRoot requests"; "peer_id" => %peer_id) } }, - RequestId::Router => unreachable!("All BlobsByRoot requests belong to sync"), + RequestId::Router => { + crit!(self.log, "All BlobsByRoot requests belong to sync"; "peer_id" => %peer_id) + } }; trace!( From 460f7121cad09bb85371fe0dfc9672050a83a378 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 3 Aug 2023 10:05:22 -0400 Subject: [PATCH 12/18] Revert "update unreachables to crits" This reverts commit 064bf64dff86b3229316aeed0431c3f4251571a5. --- beacon_node/network/src/router.rs | 24 +++++++++--------------- 1 file changed, 9 insertions(+), 15 deletions(-) diff --git a/beacon_node/network/src/router.rs b/beacon_node/network/src/router.rs index 3b7431822e7..927229e7b7e 100644 --- a/beacon_node/network/src/router.rs +++ b/beacon_node/network/src/router.rs @@ -21,7 +21,7 @@ use lighthouse_network::{ MessageId, NetworkGlobals, PeerId, PeerRequestId, PubsubMessage, Request, Response, }; use logging::TimeLatch; -use slog::{crit, debug, o, trace}; +use slog::{debug, o, trace}; use slog::{error, warn}; use std::sync::Arc; use std::time::{Duration, SystemTime, UNIX_EPOCH}; @@ -486,16 +486,14 @@ impl Router { | SyncId::SingleBlob { .. } | SyncId::ParentLookup { .. } | SyncId::ParentLookupBlob { .. } => { - crit!(self.log, "Block lookups do not request BBRange requests"; "peer_id" => %peer_id); + unreachable!("Block lookups do not request BBRange requests") } id @ (SyncId::BackFillBlocks { .. } | SyncId::RangeBlocks { .. } | SyncId::BackFillBlockAndBlobs { .. } | SyncId::RangeBlockAndBlobs { .. }) => id, }, - RequestId::Router => { - crit!(self.log, "All BBRange requests belong to sync"; "peer_id" => %peer_id) - } + RequestId::Router => unreachable!("All BBRange requests belong to sync"), }; trace!( @@ -553,15 +551,13 @@ impl Router { | SyncId::RangeBlocks { .. } | SyncId::RangeBlockAndBlobs { .. } | SyncId::BackFillBlockAndBlobs { .. } => { - crit!(self.log, "Batch syncing do not request BBRoot requests"; "peer_id" => %peer_id) + unreachable!("Batch syncing do not request BBRoot requests") } SyncId::SingleBlob { .. } | SyncId::ParentLookupBlob { .. } => { - crit!(self.log, "Blob response to block by roots request"; "peer_id" => %peer_id) + unreachable!("Blob response to block by roots request") } }, - RequestId::Router => { - crit!(self.log, "All BBRoot requests belong to sync"; "peer_id" => %peer_id) - } + RequestId::Router => unreachable!("All BBRoot requests belong to sync"), }; trace!( @@ -588,18 +584,16 @@ impl Router { RequestId::Sync(sync_id) => match sync_id { id @ (SyncId::SingleBlob { .. } | SyncId::ParentLookupBlob { .. }) => id, SyncId::SingleBlock { .. } | SyncId::ParentLookup { .. } => { - crit!(self.log, "Block response to blobs by roots request"; "peer_id" => %peer_id) + unreachable!("Block response to blobs by roots request") } SyncId::BackFillBlocks { .. } | SyncId::RangeBlocks { .. } | SyncId::RangeBlockAndBlobs { .. } | SyncId::BackFillBlockAndBlobs { .. } => { - crit!(self.log, "Batch syncing does not request BBRoot requests"; "peer_id" => %peer_id) + unreachable!("Batch syncing does not request BBRoot requests") } }, - RequestId::Router => { - crit!(self.log, "All BlobsByRoot requests belong to sync"; "peer_id" => %peer_id) - } + RequestId::Router => unreachable!("All BlobsByRoot requests belong to sync"), }; trace!( From 74373a9bd7a460911fcc3e7e8b23018c2506f821 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 3 Aug 2023 10:32:03 -0400 Subject: [PATCH 13/18] update make request/build request to make more sense --- .../network/src/sync/block_lookups/common.rs | 39 ++++++++++++++++--- .../network/src/sync/block_lookups/mod.rs | 2 +- .../sync/block_lookups/single_block_lookup.rs | 28 ++----------- 3 files changed, 38 insertions(+), 31 deletions(-) diff --git a/beacon_node/network/src/sync/block_lookups/common.rs b/beacon_node/network/src/sync/block_lookups/common.rs index b4711e26be5..a5d822776d6 100644 --- a/beacon_node/network/src/sync/block_lookups/common.rs +++ b/beacon_node/network/src/sync/block_lookups/common.rs @@ -89,15 +89,43 @@ pub trait RequestState { /* Request building methods */ /// Construct a new request. - fn build_request(&mut self) -> Result<(PeerId, Self::RequestType), LookupRequestError> { - debug_assert!(matches!(self.get_state().state, State::AwaitingDownload)); + fn build_request( + &mut self, + ) -> Result<(PeerShouldHave, Self::RequestType), LookupRequestError> { + // Verify and construct request. self.too_many_attempts()?; let peer = self.get_peer()?; let request = self.new_request(); - self.get_state_mut().req_counter += 1; Ok((peer, request)) } + /// Construct a new request and send it. + fn build_request_and_send( + &mut self, + id: Id, + already_downloaded: bool, + cx: &SyncNetworkContext, + ) -> Result<(), LookupRequestError> { + // Check if request is necessary. + if already_downloaded || !matches!(self.get_state().state, State::AwaitingDownload) { + return Ok(()); + } + + // Construct request. + let (peer_id, request) = self.build_request()?; + + // Update request state. + self.get_state_mut().state = State::Downloading { peer_id }; + self.get_state_mut().req_counter += 1; + + // Make request + let id = SingleLookupReqId { + id, + req_counter: self.get_state().req_counter, + }; + Self::make_request(id, peer_id.to_peer_id(), request, cx) + } + /// Verify the current request has not exceeded the maximum number of attempts. fn too_many_attempts(&self) -> Result<(), LookupRequestError> { let max_attempts = L::max_attempts(); @@ -114,7 +142,7 @@ pub trait RequestState { /// Get the next peer to request. Draws from the set of peers we think should have both the /// block and blob first. If that fails, we draw from the set of peers that may have either. - fn get_peer(&mut self) -> Result { + fn get_peer(&mut self) -> Result { let request_state = self.get_state_mut(); let available_peer_opt = request_state .available_peers @@ -132,8 +160,7 @@ pub trait RequestState { return Err(LookupRequestError::NoPeers); }; request_state.used_peers.insert(peer_id.to_peer_id()); - request_state.state = State::Downloading { peer_id }; - Ok(peer_id.to_peer_id()) + Ok(peer_id) } /// Initialize `Self::RequestType`. diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index 4e6e5f987f5..4374c4ac5b3 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -885,7 +885,7 @@ impl BlockLookups { /// blobs peer because they did not provide all blobs on the initial request. fn handle_missing_components>( &self, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, lookup: &mut SingleBlockLookup, ) -> Result<(), LookupRequestError> { let request_state = R::request_state_mut(lookup); diff --git a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs index 3cb084d9be9..eeea6b1d989 100644 --- a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs @@ -118,31 +118,11 @@ impl SingleBlockLookup { trace!(cx.log, "Lookup request already completed"; "block_root"=> ?block_root); return Ok(()); } - - self.request_generic::>(block_already_downloaded, cx)?; - self.request_generic::>(blobs_already_downloaded, cx) - } - - /// Common checks and request logic for blocks and blobs. - fn request_generic>( - &mut self, - already_downloaded: bool, - cx: &SyncNetworkContext, - ) -> Result<(), LookupRequestError> { let id = self.id; - let request_state = R::request_state_mut(self); - - let should_request = !already_downloaded - && matches!(request_state.get_state().state, State::AwaitingDownload); - if should_request { - let (peer_id, request) = request_state.build_request()?; - let id = SingleLookupReqId { - id, - req_counter: request_state.get_state().req_counter, - }; - R::make_request(id, peer_id, request, cx)?; - } - Ok(()) + self.block_request_state + .build_request_and_send(id, block_already_downloaded, cx)?; + self.blob_request_state + .build_request_and_send(id, blobs_already_downloaded, cx) } /// Returns a `CachedChild`, which is a wrapper around a `RpcBlock` that is either: From af5e0d455c3699de979729bb6746254b8e06894b Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 3 Aug 2023 11:04:32 -0400 Subject: [PATCH 14/18] pr feedback --- beacon_node/network/src/router.rs | 32 +++++++--- .../network/src/sync/block_lookups/common.rs | 18 +++--- .../network/src/sync/block_lookups/mod.rs | 10 ++-- .../sync/block_lookups/single_block_lookup.rs | 21 ++++--- .../network/src/sync/block_lookups/tests.rs | 58 +++++++++---------- beacon_node/network/src/sync/manager.rs | 6 +- beacon_node/network/src/sync/mod.rs | 2 +- .../network/src/sync/network_context.rs | 4 +- 8 files changed, 83 insertions(+), 68 deletions(-) diff --git a/beacon_node/network/src/router.rs b/beacon_node/network/src/router.rs index 927229e7b7e..86181c347d3 100644 --- a/beacon_node/network/src/router.rs +++ b/beacon_node/network/src/router.rs @@ -21,7 +21,7 @@ use lighthouse_network::{ MessageId, NetworkGlobals, PeerId, PeerRequestId, PubsubMessage, Request, Response, }; use logging::TimeLatch; -use slog::{debug, o, trace}; +use slog::{crit, debug, o, trace}; use slog::{error, warn}; use std::sync::Arc; use std::time::{Duration, SystemTime, UNIX_EPOCH}; @@ -486,14 +486,18 @@ impl Router { | SyncId::SingleBlob { .. } | SyncId::ParentLookup { .. } | SyncId::ParentLookupBlob { .. } => { - unreachable!("Block lookups do not request BBRange requests") + crit!(self.log, "Block lookups do not request BBRange requests"; "peer_id" => %peer_id); + return; } id @ (SyncId::BackFillBlocks { .. } | SyncId::RangeBlocks { .. } | SyncId::BackFillBlockAndBlobs { .. } | SyncId::RangeBlockAndBlobs { .. }) => id, }, - RequestId::Router => unreachable!("All BBRange requests belong to sync"), + RequestId::Router => { + crit!(self.log, "All BBRange requests belong to sync"; "peer_id" => %peer_id); + return; + } }; trace!( @@ -551,13 +555,18 @@ impl Router { | SyncId::RangeBlocks { .. } | SyncId::RangeBlockAndBlobs { .. } | SyncId::BackFillBlockAndBlobs { .. } => { - unreachable!("Batch syncing do not request BBRoot requests") + crit!(self.log, "Batch syncing do not request BBRoot requests"; "peer_id" => %peer_id); + return; } SyncId::SingleBlob { .. } | SyncId::ParentLookupBlob { .. } => { - unreachable!("Blob response to block by roots request") + crit!(self.log, "Blob response to block by roots request"; "peer_id" => %peer_id); + return; } }, - RequestId::Router => unreachable!("All BBRoot requests belong to sync"), + RequestId::Router => { + crit!(self.log, "All BBRoot requests belong to sync"; "peer_id" => %peer_id); + return; + } }; trace!( @@ -584,16 +593,21 @@ impl Router { RequestId::Sync(sync_id) => match sync_id { id @ (SyncId::SingleBlob { .. } | SyncId::ParentLookupBlob { .. }) => id, SyncId::SingleBlock { .. } | SyncId::ParentLookup { .. } => { - unreachable!("Block response to blobs by roots request") + crit!(self.log, "Block response to blobs by roots request"; "peer_id" => %peer_id); + return; } SyncId::BackFillBlocks { .. } | SyncId::RangeBlocks { .. } | SyncId::RangeBlockAndBlobs { .. } | SyncId::BackFillBlockAndBlobs { .. } => { - unreachable!("Batch syncing does not request BBRoot requests") + crit!(self.log, "Batch syncing does not request BBRoot requests"; "peer_id" => %peer_id); + return; } }, - RequestId::Router => unreachable!("All BlobsByRoot requests belong to sync"), + RequestId::Router => { + crit!(self.log, "All BlobsByRoot requests belong to sync"; "peer_id" => %peer_id); + return; + } }; trace!( diff --git a/beacon_node/network/src/sync/block_lookups/common.rs b/beacon_node/network/src/sync/block_lookups/common.rs index a5d822776d6..aa12acc12c7 100644 --- a/beacon_node/network/src/sync/block_lookups/common.rs +++ b/beacon_node/network/src/sync/block_lookups/common.rs @@ -8,7 +8,7 @@ use crate::sync::block_lookups::{ }; use crate::sync::manager::{BlockProcessType, Id, SingleLookupReqId}; use crate::sync::network_context::SyncNetworkContext; -use crate::sync::ChildComponents; +use crate::sync::CachedChildComponents; use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::{get_block_root, BeaconChainTypes}; use lighthouse_network::rpc::methods::BlobsByRootRequest; @@ -89,9 +89,7 @@ pub trait RequestState { /* Request building methods */ /// Construct a new request. - fn build_request( - &mut self, - ) -> Result<(PeerShouldHave, Self::RequestType), LookupRequestError> { + fn build_request(&mut self) -> Result<(PeerShouldHave, Self::RequestType), LookupRequestError> { // Verify and construct request. self.too_many_attempts()?; let peer = self.get_peer()?; @@ -222,7 +220,7 @@ pub trait RequestState { /// triggered by `UnknownParent` errors. fn add_to_child_components( verified_response: Self::VerifiedResponseType, - components: &mut ChildComponents, + components: &mut CachedChildComponents, ); /// Convert a verified response to the type we send to the beacon processor. @@ -231,7 +229,7 @@ pub trait RequestState { ) -> Self::ReconstructedResponseType; /// Send the response to the beacon processor. - fn send_for_processing( + fn send_reconstructed_for_processing( id: Id, bl: &BlockLookups, block_root: Hash256, @@ -326,7 +324,7 @@ impl RequestState for BlockRequestState fn add_to_child_components( verified_response: Arc>, - components: &mut ChildComponents, + components: &mut CachedChildComponents, ) { components.add_unknown_parent_block(verified_response); } @@ -337,7 +335,7 @@ impl RequestState for BlockRequestState RpcBlock::new_without_blobs(block) } - fn send_for_processing( + fn send_reconstructed_for_processing( id: Id, bl: &BlockLookups, block_root: Hash256, @@ -432,7 +430,7 @@ impl RequestState for BlobRequestState, - components: &mut ChildComponents, + components: &mut CachedChildComponents, ) { components.add_unknown_parent_blobs(verified_response); } @@ -443,7 +441,7 @@ impl RequestState for BlobRequestState, block_root: Hash256, diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index 4374c4ac5b3..0c1bc4e15bd 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -23,7 +23,7 @@ use fnv::FnvHashMap; use lighthouse_network::rpc::RPCError; use lighthouse_network::{PeerAction, PeerId}; use lru_cache::LRUTimeCache; -pub use single_block_lookup::ChildComponents; +pub use single_block_lookup::CachedChildComponents; pub use single_block_lookup::{BlobRequestState, BlockRequestState}; use slog::{debug, error, trace, warn, Logger}; use smallvec::SmallVec; @@ -155,7 +155,7 @@ impl BlockLookups { pub fn search_child_block( &mut self, block_root: Hash256, - parent_components: Option>, + parent_components: Option>, peer_source: &[PeerShouldHave], cx: &mut SyncNetworkContext, ) { @@ -175,7 +175,7 @@ impl BlockLookups { pub fn search_child_delayed( &mut self, block_root: Hash256, - parent_components: Option>, + parent_components: Option>, peer_source: &[PeerShouldHave], cx: &mut SyncNetworkContext, ) { @@ -235,7 +235,7 @@ impl BlockLookups { pub fn new_current_lookup( &mut self, block_root: Hash256, - parent_components: Option>, + parent_components: Option>, peers: &[PeerShouldHave], cx: &mut SyncNetworkContext, ) -> Option> { @@ -511,7 +511,7 @@ impl BlockLookups { } lookup.request_block_and_blobs(cx)?; } - CachedChild::NotRequired => R::send_for_processing( + CachedChild::NotRequired => R::send_reconstructed_for_processing( id, self, block_root, diff --git a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs index eeea6b1d989..fef676c8719 100644 --- a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs @@ -1,7 +1,6 @@ use super::PeerShouldHave; use crate::sync::block_lookups::common::{Lookup, RequestState}; use crate::sync::block_lookups::Id; -use crate::sync::manager::SingleLookupReqId; use crate::sync::network_context::SyncNetworkContext; use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::data_availability_checker::{AvailabilityCheckError, DataAvailabilityChecker}; @@ -56,13 +55,13 @@ pub struct SingleBlockLookup { pub da_checker: Arc>, /// Only necessary for requests triggered by an `UnknownBlockParent` or `UnknownBlockParent` /// because any blocks or blobs without parents won't hit the data availability cache. - pub cached_child_components: Option>, + pub cached_child_components: Option>, } impl SingleBlockLookup { pub fn new( requested_block_root: Hash256, - unknown_parent_components: Option>, + unknown_parent_components: Option>, peers: &[PeerShouldHave], da_checker: Arc>, id: Id, @@ -92,7 +91,7 @@ impl SingleBlockLookup { self.block_request_state.requested_block_root = block_root; self.block_request_state.state.state = State::AwaitingDownload; self.blob_request_state.state.state = State::AwaitingDownload; - self.cached_child_components = Some(ChildComponents::default()); + self.cached_child_components = Some(CachedChildComponents::default()); } /// Get all unique peers across block and blob requests. @@ -166,9 +165,9 @@ impl SingleBlockLookup { } /// Add a child component to the lookup request. Merges with any existing child components. - pub fn add_child_components(&mut self, components: ChildComponents) { + pub fn add_child_components(&mut self, components: CachedChildComponents) { if let Some(ref mut existing_components) = self.cached_child_components { - let ChildComponents { + let CachedChildComponents { downloaded_block, downloaded_blobs, } = components; @@ -382,12 +381,12 @@ pub enum CachedChild { /// data availability cache currently because any blocks or blobs without parents /// won't pass validation and therefore won't make it into the cache. #[derive(Default)] -pub struct ChildComponents { +pub struct CachedChildComponents { pub downloaded_block: Option>>, pub downloaded_blobs: FixedBlobSidecarList, } -impl From> for ChildComponents { +impl From> for CachedChildComponents { fn from(value: RpcBlock) -> Self { let (block, blobs) = value.deconstruct(); let fixed_blobs = blobs.map(|blobs| { @@ -397,7 +396,7 @@ impl From> for ChildComponents { } } -impl ChildComponents { +impl CachedChildComponents { pub fn new( block: Option>>, blobs: Option>, @@ -678,6 +677,8 @@ mod tests { &mut sl.block_request_state, ) .unwrap(); + sl.block_request_state.state.state = State::Downloading { peer_id }; + as RequestState>::verify_response( &mut sl.block_request_state, block.canonical_root(), @@ -726,6 +727,8 @@ mod tests { &mut sl.block_request_state, ) .unwrap(); + sl.block_request_state.state.state = State::Downloading { peer_id }; + as RequestState>::verify_response( &mut sl.block_request_state, block.canonical_root(), diff --git a/beacon_node/network/src/sync/block_lookups/tests.rs b/beacon_node/network/src/sync/block_lookups/tests.rs index 305c0279cb1..5799a3762e6 100644 --- a/beacon_node/network/src/sync/block_lookups/tests.rs +++ b/beacon_node/network/src/sync/block_lookups/tests.rs @@ -157,7 +157,7 @@ impl TestRig { } #[track_caller] - fn expect_block_request(&mut self, response_type: ResponseType) -> SingleLookupReqId { + fn expect_lookup_request(&mut self, response_type: ResponseType) -> SingleLookupReqId { match response_type { ResponseType::Block => match self.network_rx.try_recv() { Ok(NetworkMessage::SendRequest { @@ -173,7 +173,7 @@ impl TestRig { Ok(NetworkMessage::SendRequest { peer_id: _, request: Request::BlobsByRoot(_request), - request_id: RequestId::Sync(SyncId::SingleBlock { id }), + request_id: RequestId::Sync(SyncId::SingleBlob { id }), }) => id, other => { panic!("Expected blob request, found {:?}", other); @@ -197,7 +197,7 @@ impl TestRig { Ok(NetworkMessage::SendRequest { peer_id: _, request: Request::BlobsByRoot(_request), - request_id: RequestId::Sync(SyncId::ParentLookup { id }), + request_id: RequestId::Sync(SyncId::ParentLookupBlob { id }), }) => id, other => panic!("Expected parent blobs request, found {:?}", other), }, @@ -297,11 +297,11 @@ fn test_single_block_lookup_happy_path() { let block_root = block.canonical_root(); // Trigger the request bl.search_block(block_root, PeerShouldHave::BlockAndBlobs(peer_id), &mut cx); - let id = rig.expect_block_request(response_type); + let id = rig.expect_lookup_request(response_type); // If we're in deneb, a blob request should have been triggered as well, // we don't require a response because we're generateing 0-blob blocks in this test. if matches!(fork_name, ForkName::Deneb) { - let _ = rig.expect_block_request(ResponseType::Blob); + let _ = rig.expect_lookup_request(ResponseType::Blob); } // The peer provides the correct block, should not be penalized. Now the block should be sent @@ -345,18 +345,18 @@ fn test_single_block_lookup_empty_response() { // Trigger the request bl.search_block(block_hash, PeerShouldHave::BlockAndBlobs(peer_id), &mut cx); - let id = rig.expect_block_request(response_type); + let id = rig.expect_lookup_request(response_type); // If we're in deneb, a blob request should have been triggered as well, // we don't require a response because we're generateing 0-blob blocks in this test. if matches!(fork_name, ForkName::Deneb) { - let _ = rig.expect_block_request(ResponseType::Blob); + let _ = rig.expect_lookup_request(ResponseType::Blob); } // The peer does not have the block. It should be penalized. bl.single_lookup_response::>(id, peer_id, None, D, &cx); rig.expect_penalty(); - rig.expect_block_request(response_type); // it should be retried + rig.expect_lookup_request(response_type); // it should be retried } #[test] @@ -373,11 +373,11 @@ fn test_single_block_lookup_wrong_response() { // Trigger the request bl.search_block(block_hash, PeerShouldHave::BlockAndBlobs(peer_id), &mut cx); - let id = rig.expect_block_request(response_type); + let id = rig.expect_lookup_request(response_type); // If we're in deneb, a blob request should have been triggered as well, // we don't require a response because we're generateing 0-blob blocks in this test. if matches!(fork_name, ForkName::Deneb) { - let _ = rig.expect_block_request(ResponseType::Blob); + let _ = rig.expect_lookup_request(ResponseType::Blob); } // Peer sends something else. It should be penalized. @@ -390,7 +390,7 @@ fn test_single_block_lookup_wrong_response() { &cx, ); rig.expect_penalty(); - rig.expect_block_request(response_type); // should be retried + rig.expect_lookup_request(response_type); // should be retried // Send the stream termination. This should not produce an additional penalty. bl.single_lookup_response::>(id, peer_id, None, D, &cx); @@ -411,11 +411,11 @@ fn test_single_block_lookup_failure() { // Trigger the request bl.search_block(block_hash, PeerShouldHave::BlockAndBlobs(peer_id), &mut cx); - let id = rig.expect_block_request(response_type); + let id = rig.expect_lookup_request(response_type); // If we're in deneb, a blob request should have been triggered as well, // we don't require a response because we're generateing 0-blob blocks in this test. if matches!(fork_name, ForkName::Deneb) { - let _ = rig.expect_block_request(ResponseType::Blob); + let _ = rig.expect_lookup_request(ResponseType::Blob); } // The request fails. RPC failures are handled elsewhere so we should not penalize the peer. @@ -425,7 +425,7 @@ fn test_single_block_lookup_failure() { &cx, RPCError::UnsupportedProtocol, ); - rig.expect_block_request(response_type); + rig.expect_lookup_request(response_type); rig.expect_empty_network(); } @@ -447,11 +447,11 @@ fn test_single_block_lookup_becomes_parent_request() { PeerShouldHave::BlockAndBlobs(peer_id), &mut cx, ); - let id = rig.expect_block_request(response_type); + let id = rig.expect_lookup_request(response_type); // If we're in deneb, a blob request should have been triggered as well, // we don't require a response because we're generateing 0-blob blocks in this test. if matches!(fork_name, ForkName::Deneb) { - let _ = rig.expect_block_request(ResponseType::Blob); + let _ = rig.expect_lookup_request(ResponseType::Blob); } // The peer provides the correct block, should not be penalized. Now the block should be sent @@ -1033,11 +1033,11 @@ fn test_single_block_lookup_ignored_response() { PeerShouldHave::BlockAndBlobs(peer_id), &mut cx, ); - let id = rig.expect_block_request(response_type); + let id = rig.expect_lookup_request(response_type); // If we're in deneb, a blob request should have been triggered as well, // we don't require a response because we're generateing 0-blob blocks in this test. if matches!(fork_name, ForkName::Deneb) { - let _ = rig.expect_block_request(ResponseType::Blob); + let _ = rig.expect_lookup_request(ResponseType::Blob); } // The peer provides the correct block, should not be penalized. Now the block should be sent @@ -1285,8 +1285,8 @@ mod deneb_only { PeerShouldHave::BlockAndBlobs(peer_id), &mut cx, ); - let block_req_id = rig.expect_block_request(ResponseType::Block); - let blob_req_id = rig.expect_block_request(ResponseType::Blob); + let block_req_id = rig.expect_lookup_request(ResponseType::Block); + let blob_req_id = rig.expect_lookup_request(ResponseType::Blob); (Some(block_req_id), Some(blob_req_id), None, None) } RequestTrigger::GossipUnknownParentBlock => { @@ -1306,12 +1306,12 @@ mod deneb_only { block_root = child_root; bl.search_child_block( child_root, - Some(ChildComponents::new(Some(child_block), None)), + Some(CachedChildComponents::new(Some(child_block), None)), &[PeerShouldHave::Neither(peer_id)], &mut cx, ); - let blob_req_id = rig.expect_block_request(ResponseType::Blob); + let blob_req_id = rig.expect_lookup_request(ResponseType::Blob); rig.expect_empty_network(); // expect no block request bl.search_parent(slot, child_root, parent_root, peer_id, &mut cx); let parent_block_req_id = rig.expect_parent_request(ResponseType::Block); @@ -1344,13 +1344,13 @@ mod deneb_only { *blobs.index_mut(0) = Some(child_blob); bl.search_child_block( child_root, - Some(ChildComponents::new(None, Some(blobs))), + Some(CachedChildComponents::new(None, Some(blobs))), &[PeerShouldHave::Neither(peer_id)], &mut cx, ); - let block_req_id = rig.expect_block_request(ResponseType::Block); - let blobs_req_id = rig.expect_block_request(ResponseType::Blob); + let block_req_id = rig.expect_lookup_request(ResponseType::Block); + let blobs_req_id = rig.expect_lookup_request(ResponseType::Blob); rig.expect_empty_network(); // expect no block request bl.search_parent(slot, child_root, parent_root, peer_id, &mut cx); let parent_block_req_id = rig.expect_parent_request(ResponseType::Block); @@ -1364,8 +1364,8 @@ mod deneb_only { } RequestTrigger::GossipUnknownBlockOrBlob => { bl.search_block(block_root, PeerShouldHave::Neither(peer_id), &mut cx); - let block_req_id = rig.expect_block_request(ResponseType::Block); - let blob_req_id = rig.expect_block_request(ResponseType::Blob); + let block_req_id = rig.expect_lookup_request(ResponseType::Block); + let blob_req_id = rig.expect_lookup_request(ResponseType::Blob); (Some(block_req_id), Some(blob_req_id), None, None) } }; @@ -1646,12 +1646,12 @@ mod deneb_only { self } fn expect_block_request(mut self) -> Self { - let id = self.rig.expect_block_request(ResponseType::Block); + let id = self.rig.expect_lookup_request(ResponseType::Block); self.block_req_id = Some(id); self } fn expect_blobs_request(mut self) -> Self { - let id = self.rig.expect_block_request(ResponseType::Blob); + let id = self.rig.expect_lookup_request(ResponseType::Blob); self.blob_req_id = Some(id); self } diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index d61e34c21ca..f823e127092 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -44,7 +44,7 @@ use crate::status::ToStatusMessage; use crate::sync::block_lookups::common::{Current, Parent}; use crate::sync::block_lookups::delayed_lookup; use crate::sync::block_lookups::delayed_lookup::DelayedLookupMessage; -use crate::sync::block_lookups::{BlobRequestState, BlockRequestState, ChildComponents}; +use crate::sync::block_lookups::{BlobRequestState, BlockRequestState, CachedChildComponents}; use crate::sync::range_sync::ByRangeRequestType; use beacon_chain::block_verification_types::AsBlock; use beacon_chain::block_verification_types::RpcBlock; @@ -667,7 +667,7 @@ impl SyncManager { block_root, parent_root, blob_slot, - Some(ChildComponents::new(None, Some(blobs))), + Some(CachedChildComponents::new(None, Some(blobs))), ); } SyncMessage::UnknownBlockHashFromAttestation(peer_id, block_hash) => { @@ -776,7 +776,7 @@ impl SyncManager { block_root: Hash256, parent_root: Hash256, slot: Slot, - parent_components: Option>, + parent_components: Option>, ) { if self.should_search_for_block(slot, &peer_id) { self.block_lookups.search_parent( diff --git a/beacon_node/network/src/sync/mod.rs b/beacon_node/network/src/sync/mod.rs index 23e055f60af..b6ed1b3c3d5 100644 --- a/beacon_node/network/src/sync/mod.rs +++ b/beacon_node/network/src/sync/mod.rs @@ -9,6 +9,6 @@ mod network_context; mod peer_sync_info; mod range_sync; -pub use block_lookups::ChildComponents; +pub use block_lookups::CachedChildComponents; pub use manager::{BatchProcessResult, SyncMessage}; pub use range_sync::{BatchOperationOutcome, ChainId}; diff --git a/beacon_node/network/src/sync/network_context.rs b/beacon_node/network/src/sync/network_context.rs index adc9469d71f..df48005e473 100644 --- a/beacon_node/network/src/sync/network_context.rs +++ b/beacon_node/network/src/sync/network_context.rs @@ -443,8 +443,8 @@ impl SyncNetworkContext { lookup_type: LookupType, ) -> Result<(), &'static str> { let sync_id = match lookup_type { - LookupType::Current => SyncRequestId::SingleBlock { id }, - LookupType::Parent => SyncRequestId::ParentLookup { id }, + LookupType::Current => SyncRequestId::SingleBlob { id }, + LookupType::Parent => SyncRequestId::ParentLookupBlob { id }, }; let request_id = RequestId::Sync(sync_id); From 335e57d0f8ad49a2443d614e2ca294da7995840a Mon Sep 17 00:00:00 2001 From: realbigsean Date: Fri, 4 Aug 2023 09:43:02 -0400 Subject: [PATCH 15/18] Update beacon_node/network/src/sync/block_lookups/mod.rs Co-authored-by: Jimmy Chen --- beacon_node/network/src/sync/block_lookups/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index 0c1bc4e15bd..f04d8cbf920 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -338,7 +338,7 @@ impl BlockLookups { /* Lookup responses */ /// Get a single block lookup by its ID. This method additionally ensures the `req_counter` - /// matches the current `req_counter` for the lookup. This any stale responses from requests + /// matches the current `req_counter` for the lookup. This ensures any stale responses from requests /// that have been retried are ignored. fn get_single_lookup>( &mut self, From 8404cc8ef6877f87918ffe496a07e8367c4a6df3 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Fri, 4 Aug 2023 09:43:15 -0400 Subject: [PATCH 16/18] Update beacon_node/network/src/sync/block_lookups/mod.rs Co-authored-by: Jimmy Chen --- beacon_node/network/src/sync/block_lookups/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index f04d8cbf920..c4d9e8287e5 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -1304,7 +1304,7 @@ impl BlockLookups { lookup.handle_consistency_failure(cx); if let Err(e) = lookup.request_block_and_blobs(cx) { debug!(self.log, - "Failed to request block and blobs, droppingn lookup"; + "Failed to request block and blobs, dropping lookup"; "error" => ?e ); self.single_block_lookups.remove(&id); From 4db34e9d7cd90db826eed0332e41045f0d7761ec Mon Sep 17 00:00:00 2001 From: realbigsean Date: Fri, 4 Aug 2023 10:24:41 -0400 Subject: [PATCH 17/18] more pr feedback, fix availability check error handling --- .../src/data_availability_checker.rs | 1 - .../gossip_methods.rs | 1 - .../network/src/sync/block_lookups/common.rs | 4 +- .../network/src/sync/block_lookups/mod.rs | 87 +++++++++++-------- .../src/sync/block_lookups/parent_lookup.rs | 6 +- .../sync/block_lookups/single_block_lookup.rs | 20 ++--- beacon_node/network/src/sync/manager.rs | 10 ++- 7 files changed, 74 insertions(+), 55 deletions(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index 3e7685efd81..f6130d26ec2 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -41,7 +41,6 @@ pub enum AvailabilityCheckError { num_blobs: usize, }, MissingBlobs, - TxKzgCommitmentMismatch(String), KzgCommitmentMismatch { blob_index: u64, }, 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 97cd82e7190..5b3a3ba42d0 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -1169,7 +1169,6 @@ impl NetworkBeaconProcessor { AvailabilityCheckError::Kzg(_) | AvailabilityCheckError::KzgVerificationFailed | AvailabilityCheckError::NumBlobsMismatch { .. } - | AvailabilityCheckError::TxKzgCommitmentMismatch(_) | AvailabilityCheckError::BlobIndexInvalid(_) | AvailabilityCheckError::UnorderedBlobs { .. } | AvailabilityCheckError::BlockBlobRootMismatch { .. } diff --git a/beacon_node/network/src/sync/block_lookups/common.rs b/beacon_node/network/src/sync/block_lookups/common.rs index aa12acc12c7..4f071a04358 100644 --- a/beacon_node/network/src/sync/block_lookups/common.rs +++ b/beacon_node/network/src/sync/block_lookups/common.rs @@ -326,7 +326,7 @@ impl RequestState for BlockRequestState verified_response: Arc>, components: &mut CachedChildComponents, ) { - components.add_unknown_parent_block(verified_response); + components.add_cached_child_block(verified_response); } fn verified_to_reconstructed( @@ -432,7 +432,7 @@ impl RequestState for BlobRequestState, components: &mut CachedChildComponents, ) { - components.add_unknown_parent_blobs(verified_response); + components.add_cached_child_blobs(verified_response); } fn verified_to_reconstructed( diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index 0c1bc4e15bd..36389223eab 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -12,7 +12,7 @@ use crate::sync::block_lookups::single_block_lookup::{ }; use crate::sync::manager::{Id, SingleLookupReqId}; use beacon_chain::block_verification_types::{AsBlock, RpcBlock}; -use beacon_chain::data_availability_checker::DataAvailabilityChecker; +use beacon_chain::data_availability_checker::{AvailabilityCheckError, DataAvailabilityChecker}; use beacon_chain::validator_monitor::timestamp_now; use beacon_chain::{AvailabilityProcessingStatus, BeaconChainTypes, BlockError}; pub use common::Current; @@ -43,7 +43,7 @@ mod single_block_lookup; #[cfg(test)] mod tests; -pub type DownloadedBlocks = (Hash256, RpcBlock); +pub type DownloadedBlock = (Hash256, RpcBlock); const FAILED_CHAINS_CACHE_EXPIRY_SECONDS: u64 = 60; pub const SINGLE_BLOCK_LOOKUP_MAX_ATTEMPTS: u8 = 3; @@ -155,11 +155,11 @@ impl BlockLookups { pub fn search_child_block( &mut self, block_root: Hash256, - parent_components: Option>, + child_components: Option>, peer_source: &[PeerShouldHave], cx: &mut SyncNetworkContext, ) { - let lookup = self.new_current_lookup(block_root, parent_components, peer_source, cx); + let lookup = self.new_current_lookup(block_root, child_components, peer_source, cx); if let Some(lookup) = lookup { self.trigger_single_lookup(lookup, cx); } @@ -175,11 +175,11 @@ impl BlockLookups { pub fn search_child_delayed( &mut self, block_root: Hash256, - parent_components: Option>, + child_components: Option>, peer_source: &[PeerShouldHave], cx: &mut SyncNetworkContext, ) { - let lookup = self.new_current_lookup(block_root, parent_components, peer_source, cx); + let lookup = self.new_current_lookup(block_root, child_components, peer_source, cx); if let Some(lookup) = lookup { self.add_single_lookup(lookup) } @@ -235,7 +235,7 @@ impl BlockLookups { pub fn new_current_lookup( &mut self, block_root: Hash256, - parent_components: Option>, + child_components: Option>, peers: &[PeerShouldHave], cx: &mut SyncNetworkContext, ) -> Option> { @@ -246,7 +246,7 @@ impl BlockLookups { .find(|(_id, lookup)| lookup.is_for_block(block_root)) { lookup.add_peers(peers); - if let Some(components) = parent_components { + if let Some(components) = child_components { lookup.add_child_components(components); } return None; @@ -280,7 +280,7 @@ impl BlockLookups { Some(SingleBlockLookup::new( block_root, - parent_components, + child_components, peers, self.da_checker.clone(), cx.next_id(), @@ -397,7 +397,7 @@ impl BlockLookups { } Err(e) => { debug!(self.log, - "Single lookup retry failed"; + "Single lookup request failed"; "error" => ?e, "block_root" => ?expected_block_root, ); @@ -503,7 +503,7 @@ impl BlockLookups { // we should penalize the blobs peer because they did not provide all blobs on the // initial request. if lookup.both_components_downloaded() { - lookup.penalize_lazy_blob_peer(cx); + lookup.penalize_blob_peer(false, cx); lookup .blob_request_state .state @@ -602,12 +602,12 @@ impl BlockLookups { fn parent_lookup_response_inner>( &mut self, peer_id: PeerId, - block: Option, + response: Option, seen_timestamp: Duration, cx: &SyncNetworkContext, parent_lookup: &mut ParentLookup, ) -> Result<(), RequestError> { - match parent_lookup.verify_block::(block, &mut self.failed_chains) { + match parent_lookup.verify_response::(response, &mut self.failed_chains) { Ok(Some(verified_response)) => { self.handle_verified_response::( seen_timestamp, @@ -892,7 +892,7 @@ impl BlockLookups { request_state.get_state_mut().component_processed = true; if lookup.both_components_processed() { - lookup.penalize_lazy_blob_peer(cx); + lookup.penalize_blob_peer(false, cx); // Try it again if possible. lookup @@ -944,28 +944,45 @@ impl BlockLookups { return Ok(None); } BlockError::AvailabilityCheck(e) => { - warn!(self.log, "Availability check failure"; "root" => %root, "peer_id" => %peer_id, "error" => ?e); - lookup.handle_availability_check_failure(cx); - lookup.request_block_and_blobs(cx)? - } - BlockError::BlobValidation(_) => { - warn!(self.log, "Blob validation failure"; "root" => %root, "peer_id" => %peer_id); - if let Ok(blob_peer) = lookup.blob_request_state.state.processing_peer() { - cx.report_peer( - blob_peer.to_peer_id(), - PeerAction::MidToleranceError, - "single_block_failure", - ); - lookup - .blob_request_state - .state - .remove_peer_if_useless(blob_peer.as_peer_id()); + match e { + // Internal error. + AvailabilityCheckError::KzgNotInitialized + | AvailabilityCheckError::SszTypes(_) + | AvailabilityCheckError::MissingBlobs + | AvailabilityCheckError::UnorderedBlobs { .. } + | AvailabilityCheckError::StoreError(_) + | AvailabilityCheckError::DecodeError(_) => { + warn!(self.log, "Internal availability check failure"; "root" => %root, "peer_id" => %peer_id, "error" => ?e); + lookup + .block_request_state + .state + .register_failure_downloading(); + lookup + .blob_request_state + .state + .register_failure_downloading(); + lookup.request_block_and_blobs(cx)? + } + + // Invalid block and blob comparison. + AvailabilityCheckError::NumBlobsMismatch { .. } + | AvailabilityCheckError::KzgCommitmentMismatch { .. } + | AvailabilityCheckError::BlockBlobRootMismatch { .. } + | AvailabilityCheckError::BlockBlobSlotMismatch { .. } => { + warn!(self.log, "Availability check failure in consistency"; "root" => %root, "peer_id" => %peer_id, "error" => ?e); + lookup.handle_consistency_failure(cx); + lookup.request_block_and_blobs(cx)? + } + + // Malicious errors. + AvailabilityCheckError::Kzg(_) + | AvailabilityCheckError::BlobIndexInvalid(_) + | AvailabilityCheckError::KzgVerificationFailed => { + warn!(self.log, "Availability check failure"; "root" => %root, "peer_id" => %peer_id, "error" => ?e); + lookup.handle_availability_check_failure(cx); + lookup.request_block_and_blobs(cx)? + } } - lookup - .blob_request_state - .state - .register_failure_processing(); - lookup.request_block_and_blobs(cx)? } other => { warn!(self.log, "Peer sent invalid block in single block lookup"; "root" => %root, "error" => ?other, "peer_id" => %peer_id); diff --git a/beacon_node/network/src/sync/block_lookups/parent_lookup.rs b/beacon_node/network/src/sync/block_lookups/parent_lookup.rs index b94a3be71ea..56c509c1640 100644 --- a/beacon_node/network/src/sync/block_lookups/parent_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/parent_lookup.rs @@ -1,5 +1,5 @@ use super::single_block_lookup::{LookupRequestError, LookupVerifyError, SingleBlockLookup}; -use super::{DownloadedBlocks, PeerShouldHave}; +use super::{DownloadedBlock, PeerShouldHave}; use crate::sync::block_lookups::common::Parent; use crate::sync::block_lookups::common::RequestState; use crate::sync::{manager::SLOT_IMPORT_TOLERANCE, network_context::SyncNetworkContext}; @@ -25,7 +25,7 @@ pub(crate) struct ParentLookup { /// The root of the block triggering this parent request. chain_hash: Hash256, /// The blocks that have currently been downloaded. - downloaded_blocks: Vec>, + downloaded_blocks: Vec>, /// Request of the last parent. pub current_parent_request: SingleBlockLookup, } @@ -186,7 +186,7 @@ impl ParentLookup { /// Verifies that the received block is what we requested. If so, parent lookup now waits for /// the processing result of the block. - pub fn verify_block>( + pub fn verify_response>( &mut self, block: Option, failed_chains: &mut lru_cache::LRUTimeCache, diff --git a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs index fef676c8719..16badf61372 100644 --- a/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/single_block_lookup.rs @@ -172,9 +172,9 @@ impl SingleBlockLookup { downloaded_blobs, } = components; if let Some(block) = downloaded_block { - existing_components.add_unknown_parent_block(block); + existing_components.add_cached_child_block(block); } - existing_components.add_unknown_parent_blobs(downloaded_blobs); + existing_components.add_cached_child_blobs(downloaded_blobs); } else { self.cached_child_components = Some(components); } @@ -283,13 +283,13 @@ impl SingleBlockLookup { /// Penalizes a blob peer if it should have blobs but didn't return them to us. Does not penalize /// a peer who we request blobs from based on seeing a block or blobs over gossip. This may /// have been a benign failure. - pub fn penalize_lazy_blob_peer(&mut self, cx: &SyncNetworkContext) { + pub fn penalize_blob_peer(&mut self, penalize_always: bool, cx: &SyncNetworkContext) { if let Ok(blob_peer) = self.blob_request_state.state.processing_peer() { - if let PeerShouldHave::BlockAndBlobs(blob_peer) = blob_peer { + if penalize_always || matches!(blob_peer, PeerShouldHave::BlockAndBlobs(_)) { cx.report_peer( - blob_peer, + blob_peer.to_peer_id(), PeerAction::MidToleranceError, - "single_block_failure", + "single_blob_failure", ); } self.blob_request_state @@ -301,7 +301,7 @@ impl SingleBlockLookup { /// This failure occurs on download, so register a failure downloading, penalize the peer if /// necessary and clear the blob cache. pub fn handle_consistency_failure(&mut self, cx: &SyncNetworkContext) { - self.penalize_lazy_blob_peer(cx); + self.penalize_blob_peer(false, cx); if let Some(cached_child) = self.cached_child_components.as_mut() { cached_child.clear_blobs(); } @@ -311,7 +311,7 @@ impl SingleBlockLookup { /// This failure occurs after processing, so register a failure processing, penalize the peer if /// necessary and clear the blob cache. pub fn handle_availability_check_failure(&mut self, cx: &SyncNetworkContext) { - self.penalize_lazy_blob_peer(cx); + self.penalize_blob_peer(true, cx); if let Some(cached_child) = self.cached_child_components.as_mut() { cached_child.clear_blobs(); } @@ -411,11 +411,11 @@ impl CachedChildComponents { self.downloaded_blobs = FixedBlobSidecarList::default(); } - pub fn add_unknown_parent_block(&mut self, block: Arc>) { + pub fn add_cached_child_block(&mut self, block: Arc>) { self.downloaded_block = Some(block); } - pub fn add_unknown_parent_blobs(&mut self, blobs: FixedBlobSidecarList) { + pub fn add_cached_child_blobs(&mut self, blobs: FixedBlobSidecarList) { for (index, blob_opt) in self.downloaded_blobs.iter_mut().enumerate() { if let Some(Some(downloaded_blob)) = blobs.get(index) { *blob_opt = Some(downloaded_blob.clone()); diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index f823e127092..579877f4b39 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -660,6 +660,10 @@ impl SyncManager { let block_root = blob.block_root; let parent_root = blob.block_parent_root; let blob_index = blob.index; + if blob_index >= T::EthSpec::max_blobs_per_block() as u64 { + warn!(self.log, "Peer sent blob with invalid index"; "index" => blob_index, "peer_id" => %peer_id); + return; + } let mut blobs = FixedBlobSidecarList::default(); *blobs.index_mut(blob_index as usize) = Some(blob); self.handle_unknown_parent( @@ -776,7 +780,7 @@ impl SyncManager { block_root: Hash256, parent_root: Hash256, slot: Slot, - parent_components: Option>, + child_components: Option>, ) { if self.should_search_for_block(slot, &peer_id) { self.block_lookups.search_parent( @@ -789,7 +793,7 @@ impl SyncManager { if self.should_delay_lookup(slot) { self.block_lookups.search_child_delayed( block_root, - parent_components, + child_components, &[PeerShouldHave::Neither(peer_id)], &mut self.network, ); @@ -802,7 +806,7 @@ impl SyncManager { } else { self.block_lookups.search_child_block( block_root, - parent_components, + child_components, &[PeerShouldHave::Neither(peer_id)], &mut self.network, ); From b8c1e53bb19e62596b03ff1a28537226d413b036 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Mon, 7 Aug 2023 12:56:55 -0400 Subject: [PATCH 18/18] improve block component processed log --- beacon_node/network/src/sync/block_lookups/mod.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index 7943f6649b3..53670e11855 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -832,7 +832,9 @@ impl BlockLookups { }; debug!( self.log, - "Block component processed for single block lookup not present" + "Block component processed for lookup"; + "response_type" => ?R::response_type(), + "result" => ?result, ); match result {