From 731b7e7af56b57a20db2002d8c6bd8bfa8f48145 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Mon, 7 Aug 2023 14:16:21 -0400 Subject: [PATCH] Refactor deneb networking (#4561) * Revert "fix merge" This reverts commit 405e95b0ce15409f06504f45c8d93071523e9539. * refactor deneb block processing * cargo fmt * make block and blob single lookups generic * get tests compiling * clean up everything add child component, fix peer scoring and retry logic * smol cleanup and a bugfix * remove ParentLookupReqId * Update beacon_node/network/src/sync/manager.rs Co-authored-by: Jimmy Chen * Update beacon_node/network/src/sync/manager.rs Co-authored-by: Jimmy Chen * update unreachables to crits * Revert "update unreachables to crits" This reverts commit 064bf64dff86b3229316aeed0431c3f4251571a5. * update make request/build request to make more sense * pr feedback * Update beacon_node/network/src/sync/block_lookups/mod.rs Co-authored-by: Jimmy Chen * Update beacon_node/network/src/sync/block_lookups/mod.rs Co-authored-by: Jimmy Chen * more pr feedback, fix availability check error handling * improve block component processed log --------- Co-authored-by: Jimmy Chen --- .../src/block_verification_types.rs | 18 + .../src/data_availability_checker.rs | 1 - .../gossip_methods.rs | 1 - .../src/network_beacon_processor/mod.rs | 4 + .../network_beacon_processor/sync_methods.rs | 4 - beacon_node/network/src/router.rs | 41 +- .../network/src/sync/block_lookups/common.rs | 473 +++++ .../network/src/sync/block_lookups/mod.rs | 1624 ++++++++--------- .../src/sync/block_lookups/parent_lookup.rs | 245 +-- .../sync/block_lookups/single_block_lookup.rs | 1001 +++++----- .../network/src/sync/block_lookups/tests.rs | 506 ++--- beacon_node/network/src/sync/manager.rs | 194 +- beacon_node/network/src/sync/mod.rs | 2 +- .../network/src/sync/network_context.rs | 155 +- 14 files changed, 2257 insertions(+), 2012 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 beded5763e1..0e56de74723 100644 --- a/beacon_node/beacon_chain/src/block_verification_types.rs +++ b/beacon_node/beacon_chain/src/block_verification_types.rs @@ -6,8 +6,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, @@ -73,6 +75,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/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/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/sync_methods.rs b/beacon_node/network/src/network_beacon_processor/sync_methods.rs index b27bf50ffde..b9d9a78f8cf 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)) @@ -249,7 +247,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 @@ -301,7 +298,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/router.rs b/beacon_node/network/src/router.rs index 30a75a91052..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}; @@ -482,15 +482,22 @@ impl Router { ) { let request_id = match request_id { RequestId::Sync(sync_id) => match sync_id { - SyncId::SingleBlock { .. } | SyncId::ParentLookup { .. } => { - unreachable!("Block lookups do not request BBRange requests") + SyncId::SingleBlock { .. } + | SyncId::SingleBlob { .. } + | SyncId::ParentLookup { .. } + | SyncId::ParentLookupBlob { .. } => { + 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!( @@ -548,10 +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 { .. } => { + 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!( @@ -576,15 +591,23 @@ 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 { .. } => { + 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 new file mode 100644 index 00000000000..4f071a04358 --- /dev/null +++ b/beacon_node/network/src/sync/block_lookups/common.rs @@ -0,0 +1,473 @@ +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::CachedChildComponents; +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<(PeerShouldHave, Self::RequestType), LookupRequestError> { + // Verify and construct request. + self.too_many_attempts()?; + let peer = self.get_peer()?; + let request = self.new_request(); + 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(); + 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()); + Ok(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 CachedChildComponents, + ); + + /// 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_reconstructed_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 CachedChildComponents, + ) { + components.add_cached_child_block(verified_response); + } + + fn verified_to_reconstructed( + block: Arc>, + ) -> RpcBlock { + RpcBlock::new_without_blobs(block) + } + + fn send_reconstructed_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 CachedChildComponents, + ) { + components.add_cached_child_blobs(verified_response); + } + + fn verified_to_reconstructed( + blobs: FixedBlobSidecarList, + ) -> FixedBlobSidecarList { + blobs + } + + fn send_reconstructed_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 7c4703e1e0c..53670e11855 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -1,91 +1,52 @@ -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::common::LookupType; +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::validator_monitor::timestamp_now; use beacon_chain::{AvailabilityProcessingStatus, BeaconChainTypes, BlockError}; +pub use common::Current; +pub use common::Lookup; +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::CachedChildComponents; +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::{BlobSidecar, Slot}; +use types::Slot; +pub mod common; pub(crate) mod delayed_lookup; mod parent_lookup; mod single_block_lookup; #[cfg(test)] mod tests; -pub type DownloadedBlocks = (Hash256, RpcBlock); -pub type RootBlockTuple = (Hash256, Arc>); -pub type RootBlobsTuple = (Hash256, FixedBlobSidecarList); +pub type DownloadedBlock = (Hash256, RpcBlock); const FAILED_CHAINS_CACHE_EXPIRY_SECONDS: u64 = 60; -const SINGLE_BLOCK_LOOKUP_MAX_ATTEMPTS: u8 = 3; - -pub(crate) 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: Vec>, - - da_checker: Arc>, - - /// The logger for the import manager. - 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, -} +pub const SINGLE_BLOCK_LOOKUP_MAX_ATTEMPTS: u8 = 3; /// 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. @@ -124,11 +85,21 @@ 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, +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 { @@ -154,7 +125,7 @@ impl BlockLookups { peer_source: PeerShouldHave, cx: &mut 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 +134,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: &mut SyncNetworkContext, + ) { + let lookup = self.new_current_lookup(block_root, None, &[peer_source], cx); if let Some(lookup) = lookup { self.add_single_lookup(lookup) } @@ -172,18 +148,18 @@ 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>, + child_components: Option>, peer_source: &[PeerShouldHave], cx: &mut SyncNetworkContext, ) { - let lookup = self.search_block_with(block_root, parent_components, peer_source); + let lookup = self.new_current_lookup(block_root, child_components, peer_source, cx); if let Some(lookup) = lookup { self.trigger_single_lookup(lookup, cx); } @@ -199,10 +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.search_block_with(block_root, parent_components, peer_source); + let lookup = self.new_current_lookup(block_root, child_components, peer_source, cx); if let Some(lookup) = lookup { self.add_single_lookup(lookup) } @@ -211,21 +188,25 @@ 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() - { - 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, + ); + } } } - pub fn add_single_lookup( - &mut self, - single_block_lookup: SingleBlockLookup, - ) { - self.single_block_lookups.push(single_block_lookup); + /// 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); metrics::set_gauge( &metrics::SYNC_SINGLE_BLOCK_LOOKUPS, @@ -233,42 +214,40 @@ impl BlockLookups { ); } - pub fn trigger_lookup_by_root( - &mut self, - block_root: Hash256, - cx: &mut 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)?; - 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(|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 /// constructed. - pub fn search_block_with( + pub fn new_current_lookup( &mut self, block_root: Hash256, - parent_components: Option>, + child_components: Option>, peers: &[PeerShouldHave], - ) -> Option> { + cx: &mut 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 { - lookup.add_unknown_parent_components(components); + if let Some(components) = child_components { + lookup.add_child_components(components); } return None; } @@ -301,9 +280,10 @@ impl BlockLookups { Some(SingleBlockLookup::new( block_root, - parent_components, + child_components, peers, self.da_checker.clone(), + cx.next_id(), )) } @@ -345,72 +325,83 @@ impl BlockLookups { // we are already processing this block, ignore it. return; } - let parent_lookup = ParentLookup::new( block_root, parent_root, peer_source, self.da_checker.clone(), + cx, ); - self.request_parent_block_and_blobs(parent_lookup, cx); + self.request_parent(parent_lookup, cx); } /* Lookup responses */ - pub fn single_block_lookup_response( + /// Get a single block lookup by its ID. This method additionally ensures the `req_counter` + /// 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, - id: Id, + 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, 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. + pub fn single_lookup_response>( + &mut self, + lookup_id: SingleLookupReqId, peer_id: PeerId, - block: Option>>, + response: Option, seen_timestamp: Duration, - cx: &mut SyncNetworkContext, + cx: &SyncNetworkContext, ) { - let stream_terminator = block.is_none().into(); - let log = self.log.clone(); + let id = lookup_id.id; + let response_type = R::response_type(); - let Some((has_pending_parent_request, request_ref)) = self.find_single_lookup_request(id, stream_terminator, ResponseType::Block) else { + 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"; + "response_type" => ?response_type, + ); + } 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()); - }; + let expected_block_root = lookup.block_root(); - 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 - } + match self.single_lookup_response_inner::(peer_id, response, seen_timestamp, cx, lookup) + { + Ok(lookup) => { + self.single_block_lookups.insert(id, lookup); + } + Err(e) => { + debug!(self.log, + "Single lookup request failed"; + "error" => ?e, + "block_root" => ?expected_block_root, + ); } - Ok(None) => ShouldRemoveLookup::False, - Err(e) => handle_block_lookup_verify_error( - request_ref, - ResponseType::Block, - peer_id, - e, - cx, - &log, - ), - }; - - if matches!(should_remove, ShouldRemoveLookup::True) { - self.single_block_lookups - .retain(|req| req.id.block_request_id != Some(id)); } metrics::set_gauge( @@ -419,196 +410,186 @@ impl BlockLookups { ); } - pub fn single_blob_lookup_response( - &mut self, - id: Id, + /// 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, - blob: Option>>, + response: Option, seen_timestamp: Duration, - cx: &mut SyncNetworkContext, - ) { - let stream_terminator = blob.is_none().into(); - + cx: &SyncNetworkContext, + 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(verified_response)) => { + self.handle_verified_response::( + seen_timestamp, + cx, + BlockProcessType::SingleBlock { id: lookup.id }, + verified_response, + &mut lookup, + )?; + } + Ok(None) => {} + Err(e) => { + 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); + } else { + let msg = e.into(); + cx.report_peer(peer_id, PeerAction::LowToleranceError, msg); + }; - let Some((has_pending_parent_requests, request_ref)) = - self.find_single_lookup_request(id, stream_terminator, ResponseType::Blob) else { - return; - }; + request_state.register_failure_downloading(); + lookup.request_block_and_blobs(cx)?; + } + } + Ok(lookup) + } - 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 - } - } else { - // These are the correct blobs, send them for processing - match self.send_blobs_for_processing( + fn handle_verified_response>( + &self, + seen_timestamp: Duration, + cx: &SyncNetworkContext, + process_type: BlockProcessType, + 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 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, - blobs, + block, seen_timestamp, - BlockProcessType::SingleBlock { id }, + process_type, cx, - ) { - Ok(()) => ShouldRemoveLookup::False, - Err(()) => ShouldRemoveLookup::True, - } + )? } } - Ok(None) => ShouldRemoveLookup::False, - Err(e) => handle_block_lookup_verify_error( - request_ref, - ResponseType::Blob, - peer_id, - e, + 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_blob_peer(false, cx); + lookup + .blob_request_state + .state + .register_failure_downloading(); + } + lookup.request_block_and_blobs(cx)?; + } + CachedChild::NotRequired => R::send_reconstructed_for_processing( + id, + self, + block_root, + R::verified_to_reconstructed(verified_response), + seen_timestamp, cx, - &log, - ), - }; - - if matches!(should_remove, ShouldRemoveLookup::True) { - self.single_block_lookups - .retain(|req| req.id.blob_request_id != Some(id)); + )?, + 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)?; + } } - - metrics::set_gauge( - &metrics::SYNC_SINGLE_BLOCK_LOOKUPS, - self.single_block_lookups.len() as i64, - ); + Ok(()) } - /// 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( + /// 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, - 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 - }); + 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 lookup.is_none() && matches!(stream_terminator, StreamTerminator::False) { - warn!( - self.log, - "Block returned for single block lookup not present"; - "response_type" => ?response_type, - ); + 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; } - lookup + Some(parent_lookup) } /// Process a response received from a parent lookup request. - pub fn parent_lookup_response( + pub fn parent_lookup_response>( &mut self, - id: Id, + id: SingleLookupReqId, peer_id: PeerId, - block: Option>>, + response: 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)) - { - 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, block))) => { - parent_lookup.add_current_request_block(block); - 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 { - 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) - } - } - } - 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, ResponseType::Block, e, cx) + self.handle_parent_request_error(&mut parent_lookup, cx, e); } - }; + } metrics::set_gauge( &metrics::SYNC_PARENT_BLOCK_LOOKUPS, @@ -616,72 +597,42 @@ impl BlockLookups { ); } - pub fn parent_lookup_blob_response( + /// Consolidates error handling for `parent_lookup_response`. An `Err` here should always mean + /// the lookup is dropped. + fn parent_lookup_response_inner>( &mut self, - id: Id, peer_id: PeerId, - blob: Option>>, + response: 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); + cx: &SyncNetworkContext, + parent_lookup: &mut ParentLookup, + ) -> Result<(), RequestError> { + match parent_lookup.verify_response::(response, &mut self.failed_chains) { + Ok(Some(verified_response)) => { + self.handle_verified_response::( + seen_timestamp, + cx, + BlockProcessType::ParentLookup { + chain_hash: parent_lookup.chain_hash(), + }, + verified_response, + &mut parent_lookup.current_parent_request, + )?; } - return; + Ok(None) => {} + Err(e) => self.handle_parent_verify_error::(peer_id, parent_lookup, e, cx)?, }; - - 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() - { - 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 { - self.parent_lookups.push(parent_lookup) - } - } - Ok(None) => { - // Waiting for more blobs to arrive - self.parent_lookups.push(parent_lookup); - } - Err(e) => { - self.handle_parent_verify_error(peer_id, parent_lookup, ResponseType::Blob, e, cx) - } - }; - - metrics::set_gauge( - &metrics::SYNC_PARENT_BLOCK_LOOKUPS, - self.parent_lookups.len() as i64, - ); + Ok(()) } - fn handle_parent_verify_error( + /// 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, - response_type: ResponseType, + parent_lookup: &mut ParentLookup, e: ParentVerifyError, - cx: &mut SyncNetworkContext, - ) { + cx: &SyncNetworkContext, + ) -> Result<(), RequestError> { match e { ParentVerifyError::RootMismatch | ParentVerifyError::NoBlockReturned @@ -699,10 +650,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), - }; + parent_lookup.request_parent(cx)?; } ParentVerifyError::PreviousFailure { parent_root } => { debug!( @@ -724,13 +672,49 @@ 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); + 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 } } } @@ -738,100 +722,89 @@ 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(|_, req| { + let should_drop_lookup = + req.should_drop_lookup_on_disconnected_peer(peer_id, cx, &self.log); + + !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_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, + id: SingleLookupReqId, 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)) - { - let mut parent_lookup = self.parent_lookups.remove(pos); - parent_lookup.block_download_failed(); - trace!(self.log, "Parent lookup block request failed"; &parent_lookup, "error" => msg); - - self.request_parent_block(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); - 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(parent_lookup, cx); - 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( + /// 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: &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, + 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 block_root = lookup.block_root(); + let request_state = R::request_state_mut(&mut lookup); + let response_type = R::response_type(); + trace!(log, + "Single lookup failed"; + "block_root" => ?block_root, + "error" => msg, + "peer_id" => %peer_id, + "response_type" => ?response_type + ); + 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, ); - - matches!(should_remove_block, ShouldRemoveLookup::False) - && matches!(should_remove_blob, ShouldRemoveLookup::False) - }); + self.single_block_lookups.remove(&id); + } metrics::set_gauge( &metrics::SYNC_SINGLE_BLOCK_LOOKUPS, @@ -841,48 +814,44 @@ 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, ) { - 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(mut lookup) = self.single_block_lookups.remove(&target_id) else { + return; + }; - let root = request_ref.block_request_state.requested_block_root; - let peer_id = request_ref.processing_peer(response_type); + 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 lookup"; + "response_type" => ?R::response_type(), + "result" => ?result, + ); - 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 } AvailabilityProcessingStatus::MissingComponents(_, _block_root) => { - should_remove_missing_components(request_ref, response_type, cx, &self.log) + 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); + } + } } }, BlockProcessingResult::Ignored => { @@ -893,101 +862,155 @@ impl BlockLookups { "Single block processing was ignored, cpu might be overloaded"; "action" => "dropping single block request" ); - ShouldRemoveLookup::True } BlockProcessingResult::Err(e) => { - trace!(self.log, "Single block processing failed"; "block" => %root, "error" => %e); - match e { - BlockError::BlockIsAlreadyKnown => { - // No error here - ShouldRemoveLookup::True + match self.handle_single_lookup_block_error(cx, lookup, peer_id, e) { + Ok(Some(lookup)) => { + self.single_block_lookups.insert(target_id, lookup); } - BlockError::BeaconChainError(e) => { - // Internal error - error!(self.log, "Beacon chain error processing single block"; "block_root" => %root, "error" => ?e); - ShouldRemoveLookup::True + Ok(None) => { + // Drop without an additional error. } - 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 + Err(e) => { + // Drop with an additional error. + warn!(self.log, "Single block lookup failed"; "block" => %root, "error" => ?e); } - 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 - ); - ShouldRemoveLookup::True + } + } + }; + } + + /// 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: &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_blob_peer(false, 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) => { + 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)? } - BlockError::AvailabilityCheck( - AvailabilityCheckError::KzgVerificationFailed, - ) - | BlockError::AvailabilityCheck(AvailabilityCheckError::Kzg(_)) => { - warn!(self.log, "Blob validation failure"; "root" => %root, "peer_id" => %peer_id); - if let Ok(blob_peer) = request_ref.processing_peer(ResponseType::Blob) { - 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 - } + + // 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)? } - 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) { - cx.report_peer( - block_peer.to_peer_id(), - PeerAction::MidToleranceError, - "single_block_failure", - ); - // Try it again if possible. - retry_request_after_failure( - request_ref, - ResponseType::Block, - block_peer.as_peer_id(), - cx, - &self.log, - ) - } else { - ShouldRemoveLookup::False - } + // 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)? } } } - }; + 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", + ); - if matches!(should_remove_lookup, ShouldRemoveLookup::True) { - self.single_block_lookups.remove(index); + // Try it again if possible. + lookup + .block_request_state + .state + .register_failure_processing(); + lookup.request_block_and_blobs(cx)? + } + } } - - metrics::set_gauge( - &metrics::SYNC_SINGLE_BLOCK_LOOKUPS, - self.single_block_lookups.len() as i64, - ); + Ok(Some(lookup)) } pub fn parent_block_processed( &mut self, chain_hash: Hash256, result: BlockProcessingResult, - response_type: ResponseType, cx: &mut SyncNetworkContext, ) { let index = self @@ -1001,15 +1024,6 @@ 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, - }; - match &result { BlockProcessingResult::Ok(status) => match status { AvailabilityProcessingStatus::Imported(block_root) => { @@ -1037,11 +1051,36 @@ 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); - self.request_parent_block_and_blobs(parent_lookup, cx); + self.request_parent(parent_lookup, cx); } BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(_)) | BlockProcessingResult::Err(BlockError::BlockIsAlreadyKnown { .. }) => { @@ -1056,17 +1095,11 @@ impl BlockLookups { ); } }; - let (chain_hash, mut blocks, hashes, block_request) = + let (chain_hash, 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() - } else { - None - } - }) { - blocks.push(child_block); - }; + + let blocks = self.add_child_block_to_chain(chain_hash, blocks, cx); + let process_id = ChainSegmentProcessId::ParentLookup(chain_hash); match beacon_processor.send_chain_segment(process_id, blocks) { @@ -1096,7 +1129,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. @@ -1115,35 +1148,121 @@ 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 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); + } + } + } + } 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: &mut SyncNetworkContext, + 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.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, @@ -1155,41 +1274,59 @@ 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_root() == chain_hash).then_some(*id)) else { + warn!(self.log, "No id found for single block lookup"; "chain_hash" => %chain_hash); + return; + }; + + 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; + }; + 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, - Duration::from_secs(0), //TODO(sean) pipe this through + timestamp_now(), BlockProcessType::SingleBlock { id }, cx, ) .is_err() { // Remove to avoid inconsistencies - self.single_block_lookups.remove(index); + 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 let Err(e) = lookup.request_block_and_blobs(cx) { + debug!(self.log, + "Failed to request block and blobs, dropping lookup"; + "error" => ?e + ); + self.single_block_lookups.remove(&id); } } } @@ -1199,9 +1336,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") } } @@ -1219,13 +1354,13 @@ impl BlockLookups { /* Helper functions */ fn send_block_for_processing( - &mut self, + &self, block_root: Hash256, block: RpcBlock, duration: Duration, process_type: BlockProcessType, - cx: &mut SyncNetworkContext, - ) -> Result<(), ()> { + cx: &SyncNetworkContext, + ) -> Result<(), LookupRequestError> { match cx.beacon_processor_if_enabled() { Some(beacon_processor) => { trace!(self.log, "Sending block for processing"; "block" => ?block_root, "process" => ?process_type); @@ -1240,14 +1375,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", + )) } } } @@ -1258,12 +1397,8 @@ impl BlockLookups { blobs: FixedBlobSidecarList, duration: Duration, process_type: BlockProcessType, - cx: &mut SyncNetworkContext, - ) -> Result<(), ()> { - let blob_count = blobs.iter().filter(|b| b.is_some()).count(); - if blob_count == 0 { - return Ok(()); - } + cx: &SyncNetworkContext, + ) -> 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); @@ -1275,92 +1410,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", + )) } } } - 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); - } + /// 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); - 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) - } + match response { Err(e) => { - self.handle_response(parent_lookup, cx, Err(e), ResponseType::Block); - } - } - } - - fn handle_response( - &mut self, - parent_lookup: ParentLookup, - cx: &mut SyncNetworkContext, - result: Result<(), parent_lookup::RequestError>, - response_type: ResponseType, - ) { - match result { - 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(response_type) { - 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(response_type) { - 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 - } - } + self.handle_parent_request_error(&mut parent_lookup, cx, e); } Ok(_) => { debug!(self.log, "Requesting parent"; &parent_lookup); @@ -1387,176 +1460,3 @@ impl BlockLookups { self.parent_lookups.drain(..).len() } } - -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 -} 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..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,18 +1,17 @@ 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::{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}; 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}; /// 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; @@ -26,9 +25,9 @@ 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, + pub current_parent_request: SingleBlockLookup, } #[derive(Debug, PartialEq, Eq, IntoStaticStr)] @@ -63,9 +62,15 @@ impl ParentLookup { parent_root: Hash256, peer_id: PeerShouldHave, da_checker: Arc>, + cx: &mut 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.next_id(), + ); Self { chain_hash: block_root, @@ -85,116 +90,53 @@ 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(()) - } - - pub fn check_block_peer_disconnected(&mut self, peer_id: &PeerId) -> Result<(), ()> { self.current_parent_request - .block_request_state - .state - .check_peer_disconnected(peer_id) + .request_block_and_blobs(cx) + .map_err(Into::into) } - pub fn check_blob_peer_disconnected(&mut self, peer_id: &PeerId) -> Result<(), ()> { + pub fn check_peer_disconnected(&mut self, peer_id: &PeerId) -> Result<(), ()> { self.current_parent_request - .blob_request_state + .block_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; - 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 = - Some(UnknownParentComponents::default()); + .update_requested_parent_block(next_parent) } - 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 block_processing_peer(&self) -> Result { + self.current_parent_request + .block_request_state + .state + .processing_peer() } - pub fn pending_blob_response(&self, req_id: BlobRequestId) -> bool { - self.current_parent_request.id.blob_request_id == Some(req_id) + 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. @@ -205,7 +147,7 @@ impl ParentLookup { Hash256, Vec>, Vec, - SingleBlockLookup, + SingleBlockLookup, ) { let ParentLookup { chain_hash, @@ -227,73 +169,40 @@ 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 .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(); } - 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_response>( &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_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()) + .and_then(|block| R::get_parent_root(block)) { 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 +210,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 +255,7 @@ impl From for RequestError { RequestError::TooManyAttempts { cannot_process } } E::NoPeers => RequestError::NoPeers, + E::SendFailed(msg) => RequestError::SendFailed(msg), } } } 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..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 @@ -1,199 +1,19 @@ -use crate::sync::block_lookups::{BlobRequestId, BlockRequestId, RootBlobsTuple, RootBlockTuple}; +use super::PeerShouldHave; +use crate::sync::block_lookups::common::{Lookup, RequestState}; +use crate::sync::block_lookups::Id; 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 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::ops::IndexMut; +use std::marker::PhantomData; use std::sync::Arc; use store::Hash256; use strum::IntoStaticStr; use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; -use types::{BlobSidecar, EthSpec, SignedBeaconBlock}; - -use super::{PeerShouldHave, ResponseType}; - -pub struct SingleBlockLookup { - pub id: LookupId, - 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, -} - -#[derive(Default, Clone)] -pub struct LookupId { - pub block_request_id: Option, - pub blob_request_id: Option, -} - -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, -} - -impl BlobRequestState { - pub fn new(peer_source: &[PeerShouldHave]) -> Self { - Self { - requested_ids: <_>::default(), - blob_download_queue: <_>::default(), - state: SingleLookupRequestState::new(peer_source), - } - } -} - -pub struct BlockRequestState { - pub requested_block_root: Hash256, - pub state: SingleLookupRequestState, -} - -impl BlockRequestState { - pub fn new(block_root: Hash256, peers: &[PeerShouldHave]) -> Self { - Self { - requested_block_root: block_root, - state: SingleLookupRequestState::new(peers), - } - } -} - -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), - } - } - - 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), - } - } -} - -/// 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 -/// won't pass validation and therefore won't make it into the cache. -#[derive(Default)] -pub struct UnknownParentComponents { - pub downloaded_block: Option>>, - 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>>, - blobs: Option>, - ) -> Self { - Self { - downloaded_block: block, - downloaded_blobs: blobs.unwrap_or_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) { - *blob_opt = Some(downloaded_blob.clone()); - } - } - } - pub fn downloaded_indices(&self) -> HashSet { - self.downloaded_blobs - .iter() - .enumerate() - .filter_map(|(i, blob_opt)| blob_opt.as_ref().map(|_| i)) - .collect::>() - } -} - -/// Object representing the state of a single block or blob lookup request. -#[derive(PartialEq, Eq, Debug)] -pub struct SingleLookupRequestState { - /// State of this request. - pub state: State, - /// Peers that should have this block or blob. - pub available_peers: HashSet, - /// Peers that mar or may not have this block or blob. - pub potential_peers: HashSet, - /// 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, - /// How many times have we attempted to download this block or blob. - failed_downloading: u8, - pub component_processed: bool, -} +use types::{EthSpec, SignedBeaconBlock}; #[derive(Debug, PartialEq, Eq)] pub enum State { @@ -225,454 +45,421 @@ pub enum LookupRequestError { cannot_process: bool, }, NoPeers, + SendFailed(&'static str), +} + +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. + pub cached_child_components: Option>, } -impl SingleBlockLookup { +impl SingleBlockLookup { pub fn new( requested_block_root: Hash256, - unknown_parent_components: Option>, + unknown_parent_components: Option>, peers: &[PeerShouldHave], da_checker: Arc>, + id: Id, ) -> Self { Self { - id: <_>::default(), + id, block_request_state: BlockRequestState::new(requested_block_root, peers), blob_request_state: BlobRequestState::new(peers), da_checker, - unknown_parent_components, - triggered: false, + cached_child_components: unknown_parent_components, } } + /// Get the block root that is being requested. + pub fn block_root(&self) -> Hash256 { + self.block_request_state.requested_block_root + } + + /// Check the block root matches the requested block root. pub fn is_for_block(&self, block_root: Hash256) -> bool { - self.block_request_state.requested_block_root == block_root + self.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: &mut SyncNetworkContext) -> 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 - }; + /// 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(CachedChildComponents::default()); + } - 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 - }; + /// 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 + } - if block_request_id.is_none() && blob_request_id.is_none() { - return Err(()); - } + /// 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, + 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(()); + } + let id = self.id; + 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: + /// + /// 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 + }; - self.id = LookupId { - block_request_id, - blob_request_id, - }; - Ok(()) - } + if !self.missing_blob_ids().is_empty() { + return CachedChild::DownloadIncomplete; + } - 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() + match RpcBlock::new_from_fixed(block.clone(), components.downloaded_blobs.clone()) { + Ok(rpc_block) => CachedChild::Ok(rpc_block), + Err(e) => CachedChild::Err(e), + } } 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 - } - }) + CachedChild::NotRequired + } } - pub fn add_unknown_parent_components( + /// 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, - components: UnknownParentComponents, - ) { - if let Some(ref mut existing_components) = self.unknown_parent_components { - let UnknownParentComponents { + 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 + } + } + + /// Add a child component to the lookup request. Merges with any existing child components. + pub fn add_child_components(&mut self, components: CachedChildComponents) { + if let Some(ref mut existing_components) = self.cached_child_components { + let CachedChildComponents { downloaded_block, 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.unknown_parent_components = Some(components); + self.cached_child_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(), - }) + + /// 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); + } + } } } - 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, - }) - } + /// 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 } - /// Verifies if the received block matches the requested one. - /// Returns the block for processing if the response is what we expected. - pub fn verify_block( - &mut self, - block: Option>>, - ) -> 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) - } - }, - } + /// 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 } - pub fn verify_blob( + /// 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, - blob: Option>>, - ) -> Result>, LookupVerifyError> { - match self.blob_request_state.state.state { - State::AwaitingDownload => { - self.blob_request_state.state.register_failure_downloading(); - Err(LookupVerifyError::ExtraBlobsReturned) + 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; } - 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) - } - }, } + false } - 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))) + /// 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 { - Err(LookupRequestError::NoPeers) + self.da_checker.has_block(&self.block_root()) } } - pub fn request_blobs( - &mut self, - ) -> Result, LookupRequestError> { + /// 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() + } - if self.blob_request_state.requested_ids.is_empty() { - return Ok(None); - } + /// 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() + } - 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))) + /// 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 { - Err(LookupRequestError::NoPeers) + self.da_checker + .get_missing_blob_ids_checking_cache(self.block_root()) + .unwrap_or_default() } } - 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, + /// 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_blob_peer(&mut self, penalize_always: bool, cx: &SyncNetworkContext) { + if let Ok(blob_peer) = self.blob_request_state.state.processing_peer() { + if penalize_always || matches!(blob_peer, PeerShouldHave::BlockAndBlobs(_)) { + cx.report_peer( + blob_peer.to_peer_id(), + PeerAction::MidToleranceError, + "single_blob_failure", + ); + } + self.blob_request_state + .state + .remove_peer_if_useless(blob_peer.as_peer_id()); } } - 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 - } + /// 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_blob_peer(false, 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 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)), + /// 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_blob_peer(true, cx); + if let Some(cached_child) = self.cached_child_components.as_mut() { + cached_child.clear_blobs(); } + self.blob_request_state.state.register_failure_processing() } +} - 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 }; - } +/// 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, + pub state: SingleLookupRequestState, + _phantom: PhantomData, +} + +impl BlobRequestState { + pub fn new(peer_source: &[PeerShouldHave]) -> Self { + Self { + requested_ids: <_>::default(), + blob_download_queue: <_>::default(), + state: SingleLookupRequestState::new(peer_source), + _phantom: PhantomData, } } +} - 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); - } - } +/// The state of the block request component of a `SingleBlockLookup`. +pub struct BlockRequestState { + pub requested_block_root: Hash256, + pub state: SingleLookupRequestState, + _phantom: PhantomData, +} + +impl BlockRequestState { + pub fn new(block_root: Hash256, peers: &[PeerShouldHave]) -> Self { + Self { + requested_block_root: block_root, + state: SingleLookupRequestState::new(peers), + _phantom: PhantomData, } } +} - pub fn processing_peer(&self, response_type: ResponseType) -> Result { - match response_type { - ResponseType::Block => self.block_request_state.state.processing_peer(), - ResponseType::Blob => self.blob_request_state.state.processing_peer(), - } +/// 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 `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 CachedChildComponents { + pub downloaded_block: Option>>, + pub downloaded_blobs: FixedBlobSidecarList, +} + +impl From> for CachedChildComponents { + 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) } +} - 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(), +impl CachedChildComponents { + pub fn new( + block: Option>>, + blobs: Option>, + ) -> Self { + Self { + downloaded_block: block, + downloaded_blobs: blobs.unwrap_or_default(), } } - pub fn both_components_processed(&self) -> bool { - self.block_request_state.state.component_processed - && self.blob_request_state.state.component_processed + pub fn clear_blobs(&mut self) { + self.downloaded_blobs = FixedBlobSidecarList::default(); + } + + pub fn add_cached_child_block(&mut self, block: Arc>) { + self.downloaded_block = Some(block); } - 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 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()); + } } } + + pub fn downloaded_indices(&self) -> HashSet { + self.downloaded_blobs + .iter() + .enumerate() + .filter_map(|(i, blob_opt)| blob_opt.as_ref().map(|_| i)) + .collect::>() + } +} + +/// Object representing the state of a single block or blob lookup request. +#[derive(PartialEq, Eq, Debug)] +pub struct SingleLookupRequestState { + /// State of this request. + pub state: State, + /// Peers that should have this block or blob. + pub available_peers: HashSet, + /// Peers that mar or may not have this block or blob. + pub potential_peers: HashSet, + /// Peers from which we have requested this block. + pub used_peers: HashSet, + /// How many times have we attempted to process this block or blob. + pub failed_processing: u8, + /// How many times have we attempted to download this block or blob. + 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, + /// 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 { +impl SingleLookupRequestState { pub fn new(peers: &[PeerShouldHave]) -> Self { let mut available_peers = HashSet::default(); let mut potential_peers = HashSet::default(); @@ -693,7 +480,9 @@ impl SingleLookupRequestState { used_peers: HashSet::default(), failed_processing: 0, failed_downloading: 0, + component_downloaded: false, component_processed: false, + req_counter: 0, } } @@ -715,11 +504,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); @@ -740,6 +531,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) @@ -748,14 +541,8 @@ 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(()), - } - } - + /// 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); @@ -763,9 +550,7 @@ impl SingleLookupRequestState { } } -impl slog::Value - for SingleBlockLookup -{ +impl slog::Value for SingleBlockLookup { fn serialize( &self, _record: &slog::Record, @@ -773,10 +558,8 @@ impl slog::Value serializer: &mut dyn slog::Serializer, ) -> slog::Result { serializer.emit_str("request", key)?; - serializer.emit_arguments( - "hash", - &format_args!("{}", self.block_request_state.requested_block_root), - )?; + serializer.emit_arguments("lookup_type", &format_args!("{:?}", L::lookup_type()))?; + serializer.emit_arguments("hash", &format_args!("{}", self.block_root()))?; serializer.emit_arguments( "blob_ids", &format_args!("{:?}", self.blob_request_state.requested_ids), @@ -793,7 +576,7 @@ impl slog::Value } } -impl slog::Value for SingleLookupRequestState { +impl slog::Value for SingleLookupRequestState { fn serialize( &self, record: &slog::Record, @@ -821,6 +604,8 @@ impl slog::Value for SingleLookupRequestState, 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()); @@ -861,15 +666,30 @@ 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(); + sl.block_request_state.state.state = State::Downloading { peer_id }; + + 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(); @@ -887,25 +707,42 @@ 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(); + sl.block_request_state.state.state = State::Downloading { peer_id }; + + 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 e7d1dd442db..ecc1cdc8e13 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}; @@ -20,7 +21,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>; @@ -155,7 +157,7 @@ impl TestRig { } #[track_caller] - fn expect_block_request(&mut self, response_type: ResponseType) -> Id { + fn expect_lookup_request(&mut self, response_type: ResponseType) -> SingleLookupReqId { match response_type { ResponseType::Block => match self.network_rx.try_recv() { Ok(NetworkMessage::SendRequest { @@ -171,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); @@ -181,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 { @@ -195,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), }, @@ -295,16 +297,22 @@ 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 // 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, + &cx, + ); rig.expect_empty_network(); rig.expect_block_process(response_type); @@ -313,11 +321,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_block_lookup_response(id, peer_id, None, D, &mut cx); - bl.single_block_component_processed( - id, + bl.single_lookup_response::>(id, peer_id, None, D, &cx); + bl.single_block_component_processed::>( + id.id, BlockProcessingResult::Ok(AvailabilityProcessingStatus::Imported(block_root)), - response_type, &mut cx, ); rig.expect_empty_network(); @@ -338,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_block_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 + rig.expect_lookup_request(response_type); // it should be retried } #[test] @@ -366,21 +373,27 @@ 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. 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, + &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_block_lookup_response(id, peer_id, None, D, &mut cx); + bl.single_lookup_response::>(id, peer_id, None, D, &cx); rig.expect_empty_network(); } @@ -398,16 +411,21 @@ 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. - bl.single_block_lookup_failed(id, &peer_id, &mut cx, RPCError::UnsupportedProtocol); - rig.expect_block_request(response_type); + bl.single_block_lookup_failed::>( + id, + &peer_id, + &cx, + RPCError::UnsupportedProtocol, + ); + rig.expect_lookup_request(response_type); rig.expect_empty_network(); } @@ -429,16 +447,22 @@ 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 // 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, + &cx, + ); rig.expect_empty_network(); rig.expect_block_process(response_type); @@ -447,10 +471,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( - id, + bl.single_block_component_processed::>( + id.id, BlockError::ParentUnknown(block.into()).into(), - response_type, &mut cx, ); assert_eq!(bl.single_block_lookups.len(), 1); @@ -491,22 +514,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 +562,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 +627,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 +682,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 +694,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 +746,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 +759,23 @@ 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); + + // 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(); } } @@ -764,10 +823,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 +835,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 +890,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 +911,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 +968,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, ) } @@ -962,16 +1033,22 @@ 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 // 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, + &cx, + ); rig.expect_empty_network(); rig.expect_block_process(response_type); @@ -980,9 +1057,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_block_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.id, + BlockProcessingResult::Ignored, + &mut cx, + ); rig.expect_empty_network(); assert_eq!(bl.single_block_lookups.len(), 0); } @@ -1015,17 +1096,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 +1174,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,12 +1219,13 @@ 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); } mod deneb_only { use super::*; + use crate::sync::block_lookups::common::ResponseType; use beacon_chain::data_availability_checker::AvailabilityCheckError; use std::ops::IndexMut; use std::str::FromStr; @@ -1156,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, } @@ -1202,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 => { @@ -1223,12 +1306,12 @@ mod deneb_only { block_root = child_root; bl.search_child_block( child_root, - Some(UnknownParentComponents::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); @@ -1261,13 +1344,13 @@ mod deneb_only { *blobs.index_mut(0) = Some(child_blob); bl.search_child_block( child_root, - Some(UnknownParentComponents::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); @@ -1281,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) } }; @@ -1307,12 +1390,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 +1404,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 +1438,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_block_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 +1455,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 +1490,63 @@ mod deneb_only { } fn empty_block_response(mut self) -> Self { - self.bl.single_block_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").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 +1556,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 +1569,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 +1579,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 +1586,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").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::AvailabilityCheck( - AvailabilityCheckError::KzgVerificationFailed, - )), - ResponseType::Blob, - &mut self.cx, - ); + self.bl + .single_block_component_processed::>( + self.blob_req_id.expect("blob request id").id, + BlockProcessingResult::Err(BlockError::AvailabilityCheck( + AvailabilityCheckError::KzgVerificationFailed, + )), + &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").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").id, + BlockProcessingResult::Ok(AvailabilityProcessingStatus::MissingComponents( + self.slot, + self.block_root, + )), + &mut self.cx, + ); assert_eq!(self.bl.single_block_lookups.len(), 1); self } @@ -1556,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 5e8fc4a4e9e..579877f4b39 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -41,10 +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; -pub use crate::sync::block_lookups::ResponseType; -use crate::sync::block_lookups::UnknownParentComponents; +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; @@ -83,13 +83,25 @@ pub const DELAY_QUEUE_CHANNEL_SIZE: usize = 128; pub type Id = u32; +#[derive(Debug, Hash, PartialEq, Eq, Clone, Copy)] +pub struct SingleLookupReqId { + 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 }, - /// Request searching for a block's parent. The id is the chain - ParentLookup { 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 }, /// Backfill request that is composed by both a block range request and a blob range request. @@ -100,10 +112,6 @@ pub enum RequestId { 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 { @@ -166,7 +174,6 @@ pub enum SyncMessage { BlockComponentProcessed { process_type: BlockProcessType, result: BlockProcessingResult, - response_type: ResponseType, }, } @@ -174,6 +181,7 @@ pub enum SyncMessage { #[derive(Debug, Clone)] pub enum BlockProcessType { SingleBlock { id: Id }, + SingleBlob { id: Id }, ParentLookup { chain_hash: Hash256 }, } @@ -324,16 +332,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, + &self.network, + error, + ); + } + RequestId::SingleBlob { id } => { + self.block_lookups + .single_block_lookup_failed::>( + id, + &peer_id, + &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, + &self.network, + error, + ); + } + RequestId::ParentLookupBlob { id } => { + self.block_lookups + .parent_lookup_failed::>( + id, + peer_id, + &self.network, + error, + ); } RequestId::BackFillBlocks { id } => { if let Some(batch_id) = self @@ -628,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( @@ -635,7 +671,7 @@ impl SyncManager { block_root, parent_root, blob_slot, - Some(UnknownParentComponents::new(None, Some(blobs))), + Some(CachedChildComponents::new(None, Some(blobs))), ); } SyncMessage::UnknownBlockHashFromAttestation(peer_id, block_hash) => { @@ -652,8 +688,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), + &mut self.network, + ); if let Err(e) = self .delayed_lookups .try_send(DelayedLookupMessage::MissingComponents(block_root)) @@ -670,16 +709,9 @@ impl SyncManager { } } } - SyncMessage::MissingGossipBlockComponentsDelayed(block_root) => { - if self - .block_lookups - .trigger_lookup_by_root(block_root, &mut 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); } @@ -691,14 +723,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, response_type, &mut self.network), + .single_block_component_processed::>( + id, + result, + &mut self.network, + ), + BlockProcessType::SingleBlob { id } => self + .block_lookups + .single_block_component_processed::>( + id, + result, + &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) => { @@ -727,7 +769,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), }, } } @@ -738,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( @@ -751,8 +793,9 @@ 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, ); if let Err(e) = self .delayed_lookups @@ -763,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, ); @@ -883,20 +926,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 { .. } => { + crit!(self.log, "Block received during blob 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, "Block received during parent blob request"; "peer_id" => %peer_id ); + } RequestId::BackFillBlocks { id } => { let is_stream_terminator = block.is_none(); if let Some(batch_id) = self @@ -954,20 +1007,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 { .. } => { + 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, + &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, + &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/mod.rs b/beacon_node/network/src/sync/mod.rs index 1dd33bd31c8..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::UnknownParentComponents; +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 d635dd2ea18..df48005e473 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::{BlobRequestId, BlockRequestId}; +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; @@ -62,7 +63,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. @@ -118,11 +119,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!( @@ -408,21 +405,26 @@ impl SyncNetworkContext { } } - /// Sends a blocks by root request for a parent request. - pub fn single_block_lookup_request( - &mut self, + pub fn block_lookup_request( + &self, + id: SingleLookupReqId, peer_id: PeerId, request: BlocksByRootRequest, - ) -> Result { - let id = self.next_id(); - let request_id = RequestId::Sync(SyncRequestId::SingleBlock { 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 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 { @@ -430,82 +432,39 @@ impl SyncNetworkContext { request: Request::BlocksByRoot(request), request_id, })?; - Ok(id) + 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, - peer_id: PeerId, - request: BlocksByRootRequest, - ) -> Result { - let id = self.next_id(); - let request_id = RequestId::Sync(SyncRequestId::ParentLookup { id }); - - trace!( - self.log, - "Sending parent 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) - } - - /// 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 blob_lookup_request( + &self, + id: SingleLookupReqId, + blob_peer_id: PeerId, + blob_request: BlobsByRootRequest, + lookup_type: LookupType, + ) -> Result<(), &'static str> { + let sync_id = match lookup_type { + LookupType::Current => SyncRequestId::SingleBlob { id }, + LookupType::Parent => SyncRequestId::ParentLookupBlob { id }, + }; + let request_id = RequestId::Sync(sync_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, + "lookup_type" => ?lookup_type + ); - 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 +491,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 +506,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 +515,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,7 +531,7 @@ impl SyncNetworkContext { &self.network_beacon_processor } - fn next_id(&mut self) -> Id { + pub(crate) fn next_id(&mut self) -> Id { let id = self.request_id; self.request_id += 1; id @@ -587,7 +546,7 @@ 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)] @@ -596,16 +555,14 @@ impl SyncNetworkContext { 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 } } }