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:
parent
efbf906094
commit
02c7a2eaf5
@ -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
|
||||||
|
@ -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()
|
||||||
|
@ -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,
|
||||||
|
@ -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
|
||||||
|
@ -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 {
|
||||||
|
@ -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,
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
@ -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]) {
|
||||||
|
@ -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",
|
||||||
|
@ -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,
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -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 );
|
||||||
}
|
}
|
||||||
|
@ -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
|
||||||
);
|
);
|
||||||
|
Loading…
Reference in New Issue
Block a user