Improve single block/blob logging (#4579)

* remove closure from `check_availability_mayb_import`

* impove logging, add wrapper struct to requested ids

* improve logging

* only log if we're in deneb. Only delay lookup if we're in deneb

* fix bug in missing components check
This commit is contained in:
realbigsean 2023-08-08 18:45:11 -04:00 committed by GitHub
parent efbf906094
commit 02c7a2eaf5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 240 additions and 86 deletions

View File

@ -33,7 +33,6 @@ use crate::execution_payload::{get_execution_payload, NotifyExecutionLayer, Prep
use crate::fork_choice_signal::{ForkChoiceSignalRx, ForkChoiceSignalTx, ForkChoiceWaitResult}; use crate::fork_choice_signal::{ForkChoiceSignalRx, ForkChoiceSignalTx, ForkChoiceWaitResult};
use crate::head_tracker::HeadTracker; use crate::head_tracker::HeadTracker;
use crate::historical_blocks::HistoricalBlockError; use crate::historical_blocks::HistoricalBlockError;
use crate::kzg_utils;
use crate::light_client_finality_update_verification::{ use crate::light_client_finality_update_verification::{
Error as LightClientFinalityUpdateError, VerifiedLightClientFinalityUpdate, Error as LightClientFinalityUpdateError, VerifiedLightClientFinalityUpdate,
}; };
@ -68,6 +67,7 @@ use crate::validator_monitor::{
HISTORIC_EPOCHS as VALIDATOR_MONITOR_HISTORIC_EPOCHS, HISTORIC_EPOCHS as VALIDATOR_MONITOR_HISTORIC_EPOCHS,
}; };
use crate::validator_pubkey_cache::ValidatorPubkeyCache; use crate::validator_pubkey_cache::ValidatorPubkeyCache;
use crate::{kzg_utils, AvailabilityPendingExecutedBlock};
use crate::{metrics, BeaconChainError, BeaconForkChoiceStore, BeaconSnapshot, CachedHead}; use crate::{metrics, BeaconChainError, BeaconForkChoiceStore, BeaconSnapshot, CachedHead};
use eth2::types::{EventKind, SseBlock, SseExtendedPayloadAttributes, SyncDuty}; use eth2::types::{EventKind, SseBlock, SseExtendedPayloadAttributes, SyncDuty};
use execution_layer::{ use execution_layer::{
@ -118,7 +118,7 @@ use task_executor::{ShutdownReason, TaskExecutor};
use tokio_stream::Stream; use tokio_stream::Stream;
use tree_hash::TreeHash; use tree_hash::TreeHash;
use types::beacon_state::CloneConfig; use types::beacon_state::CloneConfig;
use types::blob_sidecar::BlobSidecarList; use types::blob_sidecar::{BlobSidecarList, FixedBlobSidecarList};
use types::consts::deneb::MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS; use types::consts::deneb::MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS;
use types::*; use types::*;
@ -2786,10 +2786,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
self: &Arc<Self>, self: &Arc<Self>,
blob: GossipVerifiedBlob<T>, blob: GossipVerifiedBlob<T>,
) -> Result<AvailabilityProcessingStatus, BlockError<T::EthSpec>> { ) -> Result<AvailabilityProcessingStatus, BlockError<T::EthSpec>> {
self.check_availability_and_maybe_import(blob.slot(), |chain| { self.check_gossip_blob_availability_and_import(blob).await
chain.data_availability_checker.put_gossip_blob(blob)
})
.await
} }
/// Returns `Ok(block_root)` if the given `unverified_block` was successfully verified and /// Returns `Ok(block_root)` if the given `unverified_block` was successfully verified and
@ -2840,12 +2837,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
match executed_block { match executed_block {
ExecutedBlock::Available(block) => self.import_available_block(Box::new(block)).await, ExecutedBlock::Available(block) => self.import_available_block(Box::new(block)).await,
ExecutedBlock::AvailabilityPending(block) => { ExecutedBlock::AvailabilityPending(block) => {
self.check_availability_and_maybe_import(block.block.slot(), |chain| { self.check_block_availability_and_import(block).await
chain
.data_availability_checker
.put_pending_executed_block(block)
})
.await
} }
} }
} }
@ -2934,17 +2926,57 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
} }
} }
/// Accepts a fully-verified, available block and imports it into the chain without performing any /* Import methods */
/// additional verification.
/// Checks if the block is available, and imports immediately if so, otherwise caches the block
/// in the data availability checker.
pub async fn check_block_availability_and_import(
self: &Arc<Self>,
block: AvailabilityPendingExecutedBlock<T::EthSpec>,
) -> Result<AvailabilityProcessingStatus, BlockError<T::EthSpec>> {
let slot = block.block.slot();
let availability = self
.data_availability_checker
.put_pending_executed_block(block)?;
self.process_availability(slot, availability).await
}
/// Checks if the provided blob can make any cached blocks available, and imports immediately
/// if so, otherwise caches the blob in the data availability checker.
pub async fn check_gossip_blob_availability_and_import(
self: &Arc<Self>,
blob: GossipVerifiedBlob<T>,
) -> Result<AvailabilityProcessingStatus, BlockError<T::EthSpec>> {
let slot = blob.slot();
let availability = self.data_availability_checker.put_gossip_blob(blob)?;
self.process_availability(slot, availability).await
}
/// Checks if the provided blobs can make any cached blocks available, and imports immediately
/// if so, otherwise caches the blob in the data availability checker.
pub async fn check_rpc_blob_availability_and_import(
self: &Arc<Self>,
slot: Slot,
block_root: Hash256,
blobs: FixedBlobSidecarList<T::EthSpec>,
) -> Result<AvailabilityProcessingStatus, BlockError<T::EthSpec>> {
let availability = self
.data_availability_checker
.put_rpc_blobs(block_root, blobs)?;
self.process_availability(slot, availability).await
}
/// Imports a fully available block. Otherwise, returns `AvailabilityProcessingStatus::MissingComponents`
/// ///
/// An error is returned if the block was unable to be imported. It may be partially imported /// An error is returned if the block was unable to be imported. It may be partially imported
/// (i.e., this function is not atomic). /// (i.e., this function is not atomic).
pub async fn check_availability_and_maybe_import( async fn process_availability(
self: &Arc<Self>, self: &Arc<Self>,
slot: Slot, slot: Slot,
cache_fn: impl FnOnce(Arc<Self>) -> Result<Availability<T::EthSpec>, AvailabilityCheckError>, availability: Availability<T::EthSpec>,
) -> Result<AvailabilityProcessingStatus, BlockError<T::EthSpec>> { ) -> Result<AvailabilityProcessingStatus, BlockError<T::EthSpec>> {
let availability = cache_fn(self.clone())?;
match availability { match availability {
Availability::Available(block) => { Availability::Available(block) => {
// This is the time since start of the slot where all the components of the block have become available // This is the time since start of the slot where all the components of the block have become available

View File

@ -313,6 +313,16 @@ impl<T: BeaconChainTypes> DataAvailabilityChecker<T> {
.map_or(false, |da_epoch| block_epoch >= da_epoch) .map_or(false, |da_epoch| block_epoch >= da_epoch)
} }
/// Returns `true` if the current epoch is greater than or equal to the `Deneb` epoch.
pub fn is_deneb(&self) -> bool {
self.slot_clock.now().map_or(false, |slot| {
self.spec.deneb_fork_epoch.map_or(false, |deneb_epoch| {
let now_epoch = slot.epoch(T::EthSpec::slots_per_epoch());
now_epoch >= deneb_epoch
})
})
}
/// Persist all in memory components to disk /// Persist all in memory components to disk
pub fn persist_all(&self) -> Result<(), AvailabilityCheckError> { pub fn persist_all(&self) -> Result<(), AvailabilityCheckError> {
self.availability_cache.write_all_to_disk() self.availability_cache.write_all_to_disk()

View File

@ -721,7 +721,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
self.chain.recompute_head_at_current_slot().await; self.chain.recompute_head_at_current_slot().await;
} }
Ok(AvailabilityProcessingStatus::MissingComponents(slot, block_hash)) => { Ok(AvailabilityProcessingStatus::MissingComponents(slot, block_hash)) => {
debug!( trace!(
self.log, self.log,
"Missing block components for gossip verified blob"; "Missing block components for gossip verified blob";
"slot" => %blob_slot, "slot" => %blob_slot,

View File

@ -287,11 +287,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
let result = self let result = self
.chain .chain
.check_availability_and_maybe_import(slot, |chain| { .check_rpc_blob_availability_and_import(slot, block_root, blobs)
chain
.data_availability_checker
.put_rpc_blobs(block_root, blobs)
})
.await; .await;
// Sync handles these results // Sync handles these results

View File

@ -374,7 +374,7 @@ impl<L: Lookup, T: BeaconChainTypes> RequestState<L, T> for BlobRequestState<L,
fn new_request(&self) -> BlobsByRootRequest { fn new_request(&self) -> BlobsByRootRequest {
BlobsByRootRequest { BlobsByRootRequest {
blob_ids: VariableList::from(self.requested_ids.clone()), blob_ids: self.requested_ids.clone().into(),
} }
} }
@ -402,7 +402,7 @@ impl<L: Lookup, T: BeaconChainTypes> RequestState<L, T> for BlobRequestState<L,
Err(LookupVerifyError::UnrequestedBlobId) Err(LookupVerifyError::UnrequestedBlobId)
} else { } else {
// State should remain downloading until we receive the stream terminator. // State should remain downloading until we receive the stream terminator.
self.requested_ids.retain(|id| *id != received_id); self.requested_ids.remove(&received_id);
let blob_index = blob.index; let blob_index = blob.index;
if blob_index >= T::EthSpec::max_blobs_per_block() as u64 { if blob_index >= T::EthSpec::max_blobs_per_block() as u64 {

View File

@ -96,7 +96,7 @@ pub struct BlockLookups<T: BeaconChainTypes> {
single_block_lookups: FnvHashMap<Id, SingleBlockLookup<Current, T>>, single_block_lookups: FnvHashMap<Id, SingleBlockLookup<Current, T>>,
da_checker: Arc<DataAvailabilityChecker<T>>, pub(crate) da_checker: Arc<DataAvailabilityChecker<T>>,
/// The logger for the import manager. /// The logger for the import manager.
log: Logger, log: Logger,
@ -126,10 +126,14 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
cx: &mut SyncNetworkContext<T>, cx: &mut SyncNetworkContext<T>,
) { ) {
let lookup = self.new_current_lookup(block_root, None, &[peer_source], cx); let lookup = self.new_current_lookup(block_root, None, &[peer_source], cx);
if let Some(lookup) = lookup { if let Some(lookup) = lookup {
let msg = "Searching for block";
lookup_creation_logging(msg, &lookup, peer_source, &self.log);
self.trigger_single_lookup(lookup, cx); self.trigger_single_lookup(lookup, cx);
} }
} }
/// Creates a lookup for the block with the given `block_root`. /// Creates a lookup for the block with the given `block_root`.
/// ///
/// The request is not immediately triggered, and should be triggered by a call to /// The request is not immediately triggered, and should be triggered by a call to
@ -142,6 +146,8 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
) { ) {
let lookup = self.new_current_lookup(block_root, None, &[peer_source], cx); let lookup = self.new_current_lookup(block_root, None, &[peer_source], cx);
if let Some(lookup) = lookup { if let Some(lookup) = lookup {
let msg = "Initialized delayed lookup for block";
lookup_creation_logging(msg, &lookup, peer_source, &self.log);
self.add_single_lookup(lookup) self.add_single_lookup(lookup)
} }
} }
@ -155,13 +161,18 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
pub fn search_child_block( pub fn search_child_block(
&mut self, &mut self,
block_root: Hash256, block_root: Hash256,
child_components: Option<CachedChildComponents<T::EthSpec>>, child_components: CachedChildComponents<T::EthSpec>,
peer_source: &[PeerShouldHave], peer_source: PeerShouldHave,
cx: &mut SyncNetworkContext<T>, cx: &mut SyncNetworkContext<T>,
) { ) {
let lookup = self.new_current_lookup(block_root, child_components, peer_source, cx); if child_components.is_missing_components() {
if let Some(lookup) = lookup { let lookup =
self.trigger_single_lookup(lookup, cx); self.new_current_lookup(block_root, Some(child_components), &[peer_source], cx);
if let Some(lookup) = lookup {
let msg = "Searching for components of a block with unknown parent";
lookup_creation_logging(msg, &lookup, peer_source, &self.log);
self.trigger_single_lookup(lookup, cx);
}
} }
} }
@ -175,13 +186,18 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
pub fn search_child_delayed( pub fn search_child_delayed(
&mut self, &mut self,
block_root: Hash256, block_root: Hash256,
child_components: Option<CachedChildComponents<T::EthSpec>>, child_components: CachedChildComponents<T::EthSpec>,
peer_source: &[PeerShouldHave], peer_source: PeerShouldHave,
cx: &mut SyncNetworkContext<T>, cx: &mut SyncNetworkContext<T>,
) { ) {
let lookup = self.new_current_lookup(block_root, child_components, peer_source, cx); if child_components.is_missing_components() {
if let Some(lookup) = lookup { let lookup =
self.add_single_lookup(lookup) self.new_current_lookup(block_root, Some(child_components), &[peer_source], cx);
if let Some(lookup) = lookup {
let msg = "Initialized delayed lookup for block with unknown parent";
lookup_creation_logging(msg, &lookup, peer_source, &self.log);
self.add_single_lookup(lookup)
}
} }
} }
@ -218,6 +234,22 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
pub fn trigger_lookup_by_root(&mut self, block_root: Hash256, cx: &SyncNetworkContext<T>) { pub fn trigger_lookup_by_root(&mut self, block_root: Hash256, cx: &SyncNetworkContext<T>) {
self.single_block_lookups.retain(|_id, lookup| { self.single_block_lookups.retain(|_id, lookup| {
if lookup.block_root() == block_root { if lookup.block_root() == block_root {
if lookup.da_checker.is_deneb() {
let blob_indices = lookup.blob_request_state.requested_ids.indices();
debug!(
self.log,
"Triggering delayed single lookup";
"block" => ?block_root,
"blob_indices" => ?blob_indices
);
} else {
debug!(
self.log,
"Triggering delayed single lookup";
"block" => ?block_root,
);
}
if let Err(e) = lookup.request_block_and_blobs(cx) { if let Err(e) = lookup.request_block_and_blobs(cx) {
debug!(self.log, "Delayed single block lookup failed"; debug!(self.log, "Delayed single block lookup failed";
"error" => ?e, "error" => ?e,
@ -271,13 +303,6 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
return None; return None;
} }
debug!(
self.log,
"Searching for block";
"peer_id" => ?peers,
"block" => ?block_root
);
Some(SingleBlockLookup::new( Some(SingleBlockLookup::new(
block_root, block_root,
child_components, child_components,
@ -583,7 +608,6 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
&mut parent_lookup, &mut parent_lookup,
) { ) {
Ok(()) => { Ok(()) => {
debug!(self.log, "Requesting parent"; &parent_lookup);
self.parent_lookups.push(parent_lookup); self.parent_lookups.push(parent_lookup);
} }
Err(e) => { Err(e) => {
@ -1435,10 +1459,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
Err(e) => { Err(e) => {
self.handle_parent_request_error(&mut parent_lookup, cx, e); self.handle_parent_request_error(&mut parent_lookup, cx, e);
} }
Ok(_) => { Ok(_) => self.parent_lookups.push(parent_lookup),
debug!(self.log, "Requesting parent"; &parent_lookup);
self.parent_lookups.push(parent_lookup)
}
} }
// We remove and add back again requests so we want this updated regardless of outcome. // We remove and add back again requests so we want this updated regardless of outcome.
@ -1460,3 +1481,29 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
self.parent_lookups.drain(..).len() self.parent_lookups.drain(..).len()
} }
} }
fn lookup_creation_logging<L: Lookup, T: BeaconChainTypes>(
msg: &str,
lookup: &SingleBlockLookup<L, T>,
peer_source: PeerShouldHave,
log: &Logger,
) {
let block_root = lookup.block_root();
if lookup.da_checker.is_deneb() {
let blob_indices = lookup.blob_request_state.requested_ids.indices();
debug!(
log,
"{}", msg;
"peer_id" => ?peer_source,
"block" => ?block_root,
"blob_indices" => ?blob_indices
);
} else {
debug!(
log,
"{}", msg;
"peer_id" => ?peer_source,
"block" => ?block_root,
);
}
}

View File

@ -193,11 +193,11 @@ impl<T: BeaconChainTypes> ParentLookup<T> {
) -> Result<Option<R::VerifiedResponseType>, ParentVerifyError> { ) -> Result<Option<R::VerifiedResponseType>, ParentVerifyError> {
let expected_block_root = self.current_parent_request.block_root(); let expected_block_root = self.current_parent_request.block_root();
let request_state = R::request_state_mut(&mut self.current_parent_request); let request_state = R::request_state_mut(&mut self.current_parent_request);
let root_and_block = request_state.verify_response(expected_block_root, block)?; let root_and_verified = 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 // 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. // be dropped and the peer downscored.
if let Some(parent_root) = root_and_block if let Some(parent_root) = root_and_verified
.as_ref() .as_ref()
.and_then(|block| R::get_parent_root(block)) .and_then(|block| R::get_parent_root(block))
{ {
@ -207,7 +207,7 @@ impl<T: BeaconChainTypes> ParentLookup<T> {
} }
} }
Ok(root_and_block) Ok(root_and_verified)
} }
pub fn add_peers(&mut self, peer_source: &[PeerShouldHave]) { pub fn add_peers(&mut self, peer_source: &[PeerShouldHave]) {

View File

@ -5,9 +5,12 @@ use crate::sync::network_context::SyncNetworkContext;
use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::block_verification_types::RpcBlock;
use beacon_chain::data_availability_checker::{AvailabilityCheckError, DataAvailabilityChecker}; use beacon_chain::data_availability_checker::{AvailabilityCheckError, DataAvailabilityChecker};
use beacon_chain::BeaconChainTypes; use beacon_chain::BeaconChainTypes;
use lighthouse_network::rpc::methods::MaxRequestBlobSidecars;
use lighthouse_network::{PeerAction, PeerId}; use lighthouse_network::{PeerAction, PeerId};
use slog::{trace, Logger}; use slog::{trace, Logger};
use ssz_types::VariableList;
use std::collections::HashSet; use std::collections::HashSet;
use std::fmt::Debug;
use std::marker::PhantomData; use std::marker::PhantomData;
use std::sync::Arc; use std::sync::Arc;
use store::Hash256; use store::Hash256;
@ -257,7 +260,7 @@ impl<L: Lookup, T: BeaconChainTypes> SingleBlockLookup<L, T> {
/// Updates this request with the most recent picture of which blobs still need to be requested. /// Updates this request with the most recent picture of which blobs still need to be requested.
pub fn update_blobs_request(&mut self) { pub fn update_blobs_request(&mut self) {
self.blob_request_state.requested_ids = self.missing_blob_ids() self.blob_request_state.requested_ids = self.missing_blob_ids().into()
} }
/// If `unknown_parent_components` is `Some`, we know block components won't hit the data /// If `unknown_parent_components` is `Some`, we know block components won't hit the data
@ -319,12 +322,42 @@ impl<L: Lookup, T: BeaconChainTypes> SingleBlockLookup<L, T> {
} }
} }
#[derive(Clone, Default)]
pub struct RequestedBlobIds(Vec<BlobIdentifier>);
impl From<Vec<BlobIdentifier>> for RequestedBlobIds {
fn from(value: Vec<BlobIdentifier>) -> Self {
Self(value)
}
}
impl Into<VariableList<BlobIdentifier, MaxRequestBlobSidecars>> for RequestedBlobIds {
fn into(self) -> VariableList<BlobIdentifier, MaxRequestBlobSidecars> {
VariableList::from(self.0)
}
}
impl RequestedBlobIds {
pub fn is_empty(&self) -> bool {
self.0.is_empty()
}
pub fn contains(&self, blob_id: &BlobIdentifier) -> bool {
self.0.contains(blob_id)
}
pub fn remove(&mut self, blob_id: &BlobIdentifier) {
self.0.retain(|id| id != blob_id)
}
pub fn indices(&self) -> Vec<u64> {
self.0.iter().map(|id| id.index).collect()
}
}
/// The state of the blob request component of a `SingleBlockLookup`. /// The state of the blob request component of a `SingleBlockLookup`.
pub struct BlobRequestState<L: Lookup, T: EthSpec> { pub struct BlobRequestState<L: Lookup, T: EthSpec> {
/// The latest picture of which blobs still need to be requested. This includes information /// 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 /// from both block/blobs downloaded in the network layer and any blocks/blobs that exist in
/// the data availability checker. /// the data availability checker.
pub requested_ids: Vec<BlobIdentifier>, pub requested_ids: RequestedBlobIds,
/// Where we store blobs until we receive the stream terminator. /// Where we store blobs until we receive the stream terminator.
pub blob_download_queue: FixedBlobSidecarList<T>, pub blob_download_queue: FixedBlobSidecarList<T>,
pub state: SingleLookupRequestState, pub state: SingleLookupRequestState,
@ -430,6 +463,16 @@ impl<E: EthSpec> CachedChildComponents<E> {
.filter_map(|(i, blob_opt)| blob_opt.as_ref().map(|_| i)) .filter_map(|(i, blob_opt)| blob_opt.as_ref().map(|_| i))
.collect::<HashSet<_>>() .collect::<HashSet<_>>()
} }
pub fn is_missing_components(&self) -> bool {
self.downloaded_block
.as_ref()
.map(|block| {
block.num_expected_blobs()
!= self.downloaded_blobs.iter().filter(|b| b.is_some()).count()
})
.unwrap_or(true)
}
} }
/// Object representing the state of a single block or blob lookup request. /// Object representing the state of a single block or blob lookup request.
@ -562,7 +605,7 @@ impl<L: Lookup, T: BeaconChainTypes> slog::Value for SingleBlockLookup<L, T> {
serializer.emit_arguments("hash", &format_args!("{}", self.block_root()))?; serializer.emit_arguments("hash", &format_args!("{}", self.block_root()))?;
serializer.emit_arguments( serializer.emit_arguments(
"blob_ids", "blob_ids",
&format_args!("{:?}", self.blob_request_state.requested_ids), &format_args!("{:?}", self.blob_request_state.requested_ids.indices()),
)?; )?;
serializer.emit_arguments( serializer.emit_arguments(
"block_request_state.state", "block_request_state.state",

View File

@ -1306,8 +1306,8 @@ mod deneb_only {
block_root = child_root; block_root = child_root;
bl.search_child_block( bl.search_child_block(
child_root, child_root,
Some(CachedChildComponents::new(Some(child_block), None)), CachedChildComponents::new(Some(child_block), None),
&[PeerShouldHave::Neither(peer_id)], PeerShouldHave::Neither(peer_id),
&mut cx, &mut cx,
); );
@ -1344,8 +1344,8 @@ mod deneb_only {
*blobs.index_mut(0) = Some(child_blob); *blobs.index_mut(0) = Some(child_blob);
bl.search_child_block( bl.search_child_block(
child_root, child_root,
Some(CachedChildComponents::new(None, Some(blobs))), CachedChildComponents::new(None, Some(blobs)),
&[PeerShouldHave::Neither(peer_id)], PeerShouldHave::Neither(peer_id),
&mut cx, &mut cx,
); );

View File

@ -654,7 +654,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
block_root, block_root,
parent_root, parent_root,
block_slot, block_slot,
Some(block.into()), block.into(),
); );
} }
SyncMessage::UnknownParentBlob(peer_id, blob) => { SyncMessage::UnknownParentBlob(peer_id, blob) => {
@ -673,7 +673,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
block_root, block_root,
parent_root, parent_root,
blob_slot, blob_slot,
Some(CachedChildComponents::new(None, Some(blobs))), CachedChildComponents::new(None, Some(blobs)),
); );
} }
SyncMessage::UnknownBlockHashFromAttestation(peer_id, block_hash) => { SyncMessage::UnknownBlockHashFromAttestation(peer_id, block_hash) => {
@ -782,7 +782,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
block_root: Hash256, block_root: Hash256,
parent_root: Hash256, parent_root: Hash256,
slot: Slot, slot: Slot,
child_components: Option<CachedChildComponents<T::EthSpec>>, child_components: CachedChildComponents<T::EthSpec>,
) { ) {
if self.should_search_for_block(slot, &peer_id) { if self.should_search_for_block(slot, &peer_id) {
self.block_lookups.search_parent( self.block_lookups.search_parent(
@ -796,7 +796,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
self.block_lookups.search_child_delayed( self.block_lookups.search_child_delayed(
block_root, block_root,
child_components, child_components,
&[PeerShouldHave::Neither(peer_id)], PeerShouldHave::Neither(peer_id),
&mut self.network, &mut self.network,
); );
if let Err(e) = self if let Err(e) = self
@ -809,7 +809,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
self.block_lookups.search_child_block( self.block_lookups.search_child_block(
block_root, block_root,
child_components, child_components,
&[PeerShouldHave::Neither(peer_id)], PeerShouldHave::Neither(peer_id),
&mut self.network, &mut self.network,
); );
} }
@ -817,6 +817,10 @@ impl<T: BeaconChainTypes> SyncManager<T> {
} }
fn should_delay_lookup(&mut self, slot: Slot) -> bool { fn should_delay_lookup(&mut self, slot: Slot) -> bool {
if !self.block_lookups.da_checker.is_deneb() {
return false;
}
let maximum_gossip_clock_disparity = self.chain.spec.maximum_gossip_clock_disparity(); let maximum_gossip_clock_disparity = self.chain.spec.maximum_gossip_clock_disparity();
let earliest_slot = self let earliest_slot = self
.chain .chain
@ -1013,28 +1017,44 @@ impl<T: BeaconChainTypes> SyncManager<T> {
RequestId::SingleBlock { .. } => { RequestId::SingleBlock { .. } => {
crit!(self.log, "Single blob received during block request"; "peer_id" => %peer_id ); crit!(self.log, "Single blob received during block request"; "peer_id" => %peer_id );
} }
RequestId::SingleBlob { id } => self RequestId::SingleBlob { id } => {
.block_lookups if let Some(blob) = blob.as_ref() {
.single_lookup_response::<BlobRequestState<Current, T::EthSpec>>( debug!(self.log,
id, "Peer returned blob for single lookup";
peer_id, "peer_id" => %peer_id ,
blob, "blob_id" =>?blob.id()
seen_timestamp, );
&self.network, }
), self.block_lookups
.single_lookup_response::<BlobRequestState<Current, T::EthSpec>>(
id,
peer_id,
blob,
seen_timestamp,
&self.network,
)
}
RequestId::ParentLookup { id: _ } => { RequestId::ParentLookup { id: _ } => {
crit!(self.log, "Single blob received during parent block request"; "peer_id" => %peer_id ); crit!(self.log, "Single blob received during parent block request"; "peer_id" => %peer_id );
} }
RequestId::ParentLookupBlob { id } => self RequestId::ParentLookupBlob { id } => {
.block_lookups if let Some(blob) = blob.as_ref() {
.parent_lookup_response::<BlobRequestState<Parent, T::EthSpec>>( debug!(self.log,
id, "Peer returned blob for parent lookup";
peer_id, "peer_id" => %peer_id ,
blob, "blob_id" =>?blob.id()
seen_timestamp, );
&self.network, }
), self.block_lookups
.parent_lookup_response::<BlobRequestState<Parent, T::EthSpec>>(
id,
peer_id,
blob,
seen_timestamp,
&self.network,
)
}
RequestId::BackFillBlocks { id: _ } => { RequestId::BackFillBlocks { id: _ } => {
crit!(self.log, "Blob received during backfill block request"; "peer_id" => %peer_id ); crit!(self.log, "Blob received during backfill block request"; "peer_id" => %peer_id );
} }

View File

@ -418,11 +418,11 @@ impl<T: BeaconChainTypes> SyncNetworkContext<T> {
}; };
let request_id = RequestId::Sync(sync_id); let request_id = RequestId::Sync(sync_id);
trace!( debug!(
self.log, self.log,
"Sending BlocksByRoot Request"; "Sending BlocksByRoot Request";
"method" => "BlocksByRoot", "method" => "BlocksByRoot",
"count" => request.block_roots().len(), "block_roots" => ?request.block_roots().to_vec(),
"peer" => %peer_id, "peer" => %peer_id,
"lookup_type" => ?lookup_type "lookup_type" => ?lookup_type
); );
@ -448,12 +448,18 @@ impl<T: BeaconChainTypes> SyncNetworkContext<T> {
}; };
let request_id = RequestId::Sync(sync_id); let request_id = RequestId::Sync(sync_id);
if !blob_request.blob_ids.is_empty() { if let Some(block_root) = blob_request.blob_ids.first().map(|id| id.block_root) {
trace!( let indices = blob_request
.blob_ids
.iter()
.map(|id| id.index)
.collect::<Vec<_>>();
debug!(
self.log, self.log,
"Sending BlobsByRoot Request"; "Sending BlobsByRoot Request";
"method" => "BlobsByRoot", "method" => "BlobsByRoot",
"count" => blob_request.blob_ids.len(), "block_root" => ?block_root,
"blob_indices" => ?indices,
"peer" => %blob_peer_id, "peer" => %blob_peer_id,
"lookup_type" => ?lookup_type "lookup_type" => ?lookup_type
); );