From 3a7ea056a5ec0528fde0bd98a3ead7ad205f185b Mon Sep 17 00:00:00 2001 From: realbigsean Date: Wed, 13 Sep 2023 16:56:46 -0400 Subject: [PATCH 01/47] add processing and processed caching to the DA checker --- Cargo.lock | 1 + beacon_node/beacon_chain/src/beacon_chain.rs | 49 +++++++- .../beacon_chain/src/blob_verification.rs | 22 +++- .../src/data_availability_checker.rs | 49 +++++++- .../overflow_lru_cache.rs | 53 ++++++--- .../processing_cache.rs | 77 +++++++++++++ beacon_node/beacon_chain/src/metrics.rs | 12 +- .../beacon_chain/tests/block_verification.rs | 8 +- beacon_node/http_api/src/publish_blocks.rs | 2 +- beacon_node/network/Cargo.toml | 1 + .../gossip_methods.rs | 96 +++++++++++----- .../src/network_beacon_processor/mod.rs | 6 + .../network_beacon_processor/sync_methods.rs | 25 ++-- .../src/network_beacon_processor/tests.rs | 5 + beacon_node/network/src/router.rs | 5 + .../src/sync/block_lookups/delayed_lookup.rs | 23 ++-- .../network/src/sync/block_lookups/mod.rs | 32 ------ beacon_node/network/src/sync/manager.rs | 108 +++--------------- 18 files changed, 369 insertions(+), 205 deletions(-) create mode 100644 beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs diff --git a/Cargo.lock b/Cargo.lock index 3d664151d13..553a67e4a70 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5252,6 +5252,7 @@ dependencies = [ "lighthouse_metrics", "lighthouse_network", "logging", + "lru 0.7.8", "lru_cache", "matches", "operation_pool", diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index fb2fc582200..2312cf8d95c 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -2791,13 +2791,52 @@ impl BeaconChain { .map_err(BeaconChainError::TokioJoin)? } - pub async fn process_blob( + pub async fn process_gossip_blob( self: &Arc, blob: GossipVerifiedBlob, ) -> Result> { - self.check_gossip_blob_availability_and_import(blob).await + todo!() + // self.data_availability_checker.notify_blob(); + // self.check_gossip_blob_availability_and_import(blob) + // .await + // .map_err(|e| { + // self.data_availability_checker.remove_notified_blob(); + // e + // }) } + pub async fn process_rpc_blobs( + self: &Arc, + slot: Slot, + block_root: Hash256, + blobs: FixedBlobSidecarList, + ) -> Result> { + todo!(); + // self.data_availability_checker.notify_blob(); + // self.check_rpc_blob_availability_and_import(slot, block_root, blobs) + // .await + // .map_err(|e| { + // self.data_availability_checker.remove_notified_blob(block_root); + // e + // }) + } + + pub async fn process_block_with_early_caching>( + self: &Arc, + block_root: Hash256, + unverified_block: B, + notify_execution_layer: NotifyExecutionLayer, + ) -> Result> { + todo!(); + + // self.data_availability_checker.notify_block(); + // self.process_block(block_root, unverified_block, notify_execution_layer) + // .await + // .map_err(|e| { + // self.data_availability_checker.remove_notified_block(); + // e + // }) + } /// Returns `Ok(block_root)` if the given `unverified_block` was successfully verified and /// imported into the chain. /// @@ -2961,7 +3000,7 @@ impl BeaconChain { /// 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( + async fn check_block_availability_and_import( self: &Arc, block: AvailabilityPendingExecutedBlock, ) -> Result> { @@ -2974,7 +3013,7 @@ impl BeaconChain { /// 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( + async fn check_gossip_blob_availability_and_import( self: &Arc, blob: GossipVerifiedBlob, ) -> Result> { @@ -2986,7 +3025,7 @@ impl BeaconChain { /// 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( + async fn check_rpc_blob_availability_and_import( self: &Arc, slot: Slot, block_root: Hash256, diff --git a/beacon_node/beacon_chain/src/blob_verification.rs b/beacon_node/beacon_chain/src/blob_verification.rs index ed07e9176aa..2247e020c00 100644 --- a/beacon_node/beacon_chain/src/blob_verification.rs +++ b/beacon_node/beacon_chain/src/blob_verification.rs @@ -9,12 +9,13 @@ use crate::beacon_chain::{ }; use crate::data_availability_checker::AvailabilityCheckError; use crate::kzg_utils::{validate_blob, validate_blobs}; -use crate::BeaconChainError; +use crate::{metrics, BeaconChainError}; use kzg::Kzg; use slog::{debug, warn}; use ssz_derive::{Decode, Encode}; use ssz_types::VariableList; use std::borrow::Cow; +use tree_hash::TreeHash; use types::blob_sidecar::BlobIdentifier; use types::{ BeaconState, BeaconStateError, BlobSidecar, BlobSidecarList, ChainSpec, CloneConfig, EthSpec, @@ -147,6 +148,7 @@ pub type GossipVerifiedBlobList = VariableList< /// the p2p network. #[derive(Debug)] pub struct GossipVerifiedBlob { + blob_root: Hash256, blob: SignedBlobSidecar, } @@ -198,6 +200,8 @@ pub fn validate_blob_sidecar_for_gossip( }); } + let blob_root = get_blob_root(&signed_blob_sidecar); + // Verify that the sidecar is not from a future slot. let latest_permissible_slot = chain .slot_clock @@ -387,7 +391,7 @@ pub fn validate_blob_sidecar_for_gossip( .ok_or_else(|| GossipBlobError::UnknownValidator(proposer_index as u64))?; signed_blob_sidecar.verify_signature( - None, + Some(blob_root), pubkey, &fork, chain.genesis_validators_root, @@ -424,6 +428,7 @@ pub fn validate_blob_sidecar_for_gossip( } Ok(GossipVerifiedBlob { + blob_root, blob: signed_blob_sidecar, }) } @@ -562,3 +567,16 @@ pub fn verify_kzg_for_blob_list( Err(AvailabilityCheckError::KzgVerificationFailed) } } + +/// Returns the canonical root of the given `blob`. +/// +/// Use this function to ensure that we report the blob hashing time Prometheus metric. +pub fn get_blob_root(blob: &SignedBlobSidecar) -> Hash256 { + let blob_root_timer = metrics::start_timer(&metrics::BLOCK_PROCESSING_BLOB_ROOT); + + let blob_root = blob.message.tree_hash_root(); + + metrics::stop_timer(blob_root_timer); + + blob_root +} diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index 080addb3a78..b38e8684f2e 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -5,7 +5,7 @@ use crate::block_verification_types::{ AvailabilityPendingExecutedBlock, AvailableExecutedBlock, RpcBlock, }; use crate::data_availability_checker::overflow_lru_cache::OverflowLRUCache; -use crate::{BeaconChain, BeaconChainTypes, BeaconStore}; +use crate::{BeaconChain, BeaconChainTypes, BeaconStore, GossipVerifiedBlock}; use kzg::Error as KzgError; use kzg::Kzg; use slog::{debug, error}; @@ -22,6 +22,7 @@ use types::consts::deneb::MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS; use types::{BlobSidecarList, ChainSpec, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot}; mod overflow_lru_cache; +mod processing_cache; /// The LRU Cache stores `PendingComponents` which can store up to /// `MAX_BLOBS_PER_BLOCK = 6` blobs each. A `BlobSidecar` is 0.131256 MB. So @@ -283,6 +284,52 @@ impl DataAvailabilityChecker { block_within_da_period && block_has_kzg_commitments } + pub fn notify_block(&self, block: &GossipVerifiedBlock) -> bool { + todo!() + } + + pub fn notify_blob(&self, block_root: Hash256, blob: &BlobSidecar) -> bool { + todo!() + } + + pub fn remove_notified_block(&self, block_root: Hash256) -> bool { + todo!() + } + + pub fn remove_notified_blob(&self, block_root: Hash256, blob_index: usize) -> bool { + todo!() + } + + pub fn get_delayed_lookups(&self, slot: Slot) -> Vec { + todo!() + } + + pub fn should_delay_lookup(&self, slot: Slot) -> bool { + if !self.is_deneb() { + return false; + } + + let maximum_gossip_clock_disparity = self.spec.maximum_gossip_clock_disparity(); + let earliest_slot = self + .slot_clock + .now_with_past_tolerance(maximum_gossip_clock_disparity); + let latest_slot = self + .slot_clock + .now_with_future_tolerance(maximum_gossip_clock_disparity); + if let (Some(earliest_slot), Some(latest_slot)) = (earliest_slot, latest_slot) { + let msg_for_current_slot = slot >= earliest_slot && slot <= latest_slot; + let delay_threshold_unmet = self + .slot_clock + .millis_from_current_slot_start() + .map_or(false, |millis_into_slot| { + millis_into_slot < self.slot_clock.single_lookup_delay() + }); + msg_for_current_slot && delay_threshold_unmet + } else { + false + } + } + /// The epoch at which we require a data availability check in block processing. /// `None` if the `Deneb` fork is disabled. pub fn data_availability_boundary(&self) -> Option { diff --git a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs index f7bbb861c93..8ef04480c6a 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs @@ -32,6 +32,7 @@ use crate::blob_verification::KzgVerifiedBlob; use crate::block_verification_types::{ AsBlock, AvailabilityPendingExecutedBlock, AvailableExecutedBlock, }; +use crate::data_availability_checker::processing_cache::ProcessingCache; use crate::data_availability_checker::{make_available, Availability, AvailabilityCheckError}; use crate::store::{DBColumn, KeyValueStore}; use crate::BeaconChainTypes; @@ -40,11 +41,12 @@ use parking_lot::{Mutex, RwLock, RwLockUpgradableReadGuard, RwLockWriteGuard}; use ssz::{Decode, Encode}; use ssz_derive::{Decode, Encode}; use ssz_types::FixedVector; +use std::collections::HashMap; use std::{collections::HashSet, sync::Arc}; use types::blob_sidecar::BlobIdentifier; -use types::{BlobSidecar, Epoch, EthSpec, Hash256, SignedBeaconBlock}; +use types::{BlobSidecar, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot}; -type MissingBlobInfo = (Option>>, HashSet); +pub(crate) type MissingBlobInfo = (Option>>, HashSet); /// This represents the components of a partially available block /// @@ -291,6 +293,7 @@ impl OverflowStore { /// This data stores the *critical* data that we need to keep in memory /// protected by the RWLock struct Critical { + pub processing_cache: ProcessingCache, /// This is the LRU cache of pending components pub in_memory: LruCache>, /// This holds all the roots of the blocks for which we have @@ -301,6 +304,7 @@ struct Critical { impl Critical { pub fn new(capacity: usize) -> Self { Self { + processing_cache: <_>::default(), in_memory: LruCache::new(capacity), store_keys: HashSet::new(), } @@ -347,6 +351,7 @@ impl Critical { self.store_keys.insert(lru_key); } } + self.processing_cache.remove_processing(block_root); self.in_memory.put(block_root, pending_components); Ok(()) } @@ -407,7 +412,9 @@ impl OverflowLRUCache { /// Returns whether or not a block is in the cache (in memory or on disk) pub fn has_block(&self, block_root: &Hash256) -> bool { let read_lock = self.critical.read(); - if read_lock + if read_lock.processing_cache.has_block(block_root) { + true + } else if read_lock .in_memory .peek(block_root) .map_or(false, |cache| cache.executed_block.is_some()) @@ -427,17 +434,29 @@ impl OverflowLRUCache { /// Fetch the missing blob info for a block without affecting the LRU ordering pub fn get_missing_blob_info(&self, block_root: Hash256) -> MissingBlobInfo { let read_lock = self.critical.read(); - if let Some(cache) = read_lock.in_memory.peek(&block_root) { - cache.get_missing_blob_info() - } else if read_lock.store_keys.contains(&block_root) { - drop(read_lock); - // return default if there's an error reading from the store - match self.overflow_store.load_pending_components(block_root) { - Ok(Some(pending_components)) => pending_components.get_missing_blob_info(), - _ => Default::default(), + + match ( + read_lock.in_memory.peek(&block_root), + read_lock.processing_cache.peek(&block_root), + ) { + (Some(in_memory), Some(processing)) => { + //TODO: merge the two views + todo!() + } + (Some(in_memory), None) => in_memory.get_missing_blob_info(), + (None, Some(processing)) => processing.get_missing_blob_info(), + (None, None) => { + if read_lock.store_keys.contains(&block_root) { + drop(read_lock); + // return default if there's an error reading from the store + match self.overflow_store.load_pending_components(block_root) { + Ok(Some(pending_components)) => pending_components.get_missing_blob_info(), + _ => Default::default(), + } + } else { + Default::default() + } } - } else { - Default::default() } } @@ -481,6 +500,9 @@ impl OverflowLRUCache { if let Some(maybe_verified_blob) = pending_components.verified_blobs.get_mut(blob_index) { + //TODO: it's possible we get multiple blobs for the same block root + index + // if we don't yet have the block we don't know which is valid so we need to + // cache both *maybe_verified_blob = Some(kzg_verified_blob.clone()) } else { return Err(AvailabilityCheckError::BlobIndexInvalid(blob_index as u64)); @@ -575,6 +597,7 @@ impl OverflowLRUCache { mut pending_components: PendingComponents, executed_block: AvailabilityPendingExecutedBlock, ) -> Result, AvailabilityCheckError> { + let block_root = executed_block.import_data.block_root; if pending_components.has_all_blobs(&executed_block) { let num_blobs_expected = executed_block.num_blobs_expected(); let AvailabilityPendingExecutedBlock { @@ -592,6 +615,7 @@ impl OverflowLRUCache { }; let available_block = make_available(block, verified_blobs)?; + write_lock.processing_cache.put_processed(block_root); Ok(Availability::Available(Box::new( AvailableExecutedBlock::new( available_block, @@ -600,7 +624,6 @@ impl OverflowLRUCache { ), ))) } else { - let block_root = executed_block.import_data.block_root; let _ = pending_components.executed_block.insert(executed_block); write_lock.put_pending_components( block_root, @@ -638,6 +661,8 @@ impl OverflowLRUCache { self.maintain_threshold(threshold, cutoff_epoch)?; // clean up any keys on the disk that shouldn't be there self.prune_disk(cutoff_epoch)?; + //TODO: fix + self.critical.write().processing_cache.prune(Slot::new(0)); Ok(()) } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs new file mode 100644 index 00000000000..43b1356fc63 --- /dev/null +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -0,0 +1,77 @@ +use crate::blob_verification::GossipVerifiedBlob; +use crate::data_availability_checker::overflow_lru_cache::MissingBlobInfo; +use crate::data_availability_checker::{Availability, AvailabilityCheckError}; +use crate::GossipVerifiedBlock; +use kzg::KzgCommitment; +use parking_lot::{Mutex, RwLock}; +use std::collections::{HashMap, HashSet}; +use std::sync::Arc; +use types::beacon_block_body::KzgCommitments; +use types::blob_sidecar::BlobIdentifier; +use types::{BlobSidecar, EthSpec, Hash256, Slot}; + +/// This cache is used only for gossip and single lookups, to give req/resp a view of what we have +/// and what we require. This cache serves a slightly different purpose than gossip caches. It +/// tracks all unique messages we're currently processing, or have already processed. This should +/// be used in conjunction with the `data_availability_cache` to have a full view of processing +/// statuses. +/// +/// Components should be atomically removed when being added to the data availability cache. +/// +/// Components should be atomically inserted into the `processed_cache` when removed from the +/// `data_availability_cache` for import (removed as `Available`). +#[derive(Default)] +pub struct ProcessingCache { + //TODO: Fnv hash map? lru cache? + processing_cache: HashMap>, + processed_cache: HashSet, +} + +#[derive(Default)] +pub struct SimplifiedPendingComponents { + /// Blobs required for a block can only be known if we have seen the block. So `Some` here + /// means we've seen it, a `None` means we haven't. The `kzg_commitments` value is also + /// necessary to verify the . + kzg_commitments: Option>, + /// This is an array of optional blob tree hash roots, each index in the array corresponding + /// to the blob index. On insertion, a collision at an index here when `required_blobs` is + /// `None` means we need to construct an entirely new `Data` entry. This is because we have + /// no way of knowing which blob is the correct one until we see the block. + processing_blobs: Vec, +} + +impl SimplifiedPendingComponents { + pub fn get_missing_blob_info(&self) -> MissingBlobInfo { + todo!() + } +} + +impl ProcessingCache { + pub fn put_processed(&mut self, block_root: Hash256) -> bool { + self.processed_cache.insert(block_root) + } + + pub fn has_block(&self, block_root: &Hash256) -> bool { + self.processed_cache.contains(block_root) + || self + .processing_cache + .get(block_root) + .map_or(false, |b| b.kzg_commitments.is_some()) + } + + pub fn peek(&self, block_root: &Hash256) -> Option<&SimplifiedPendingComponents> { + self.processing_cache.get(block_root) + } + + pub fn get_missing_blob_ids(&self) -> Vec { + todo!() + } + + pub fn remove_processing(&mut self, block_root: Hash256) { + todo!() + } + + pub fn prune(&mut self, slot: Slot) { + todo!() + } +} diff --git a/beacon_node/beacon_chain/src/metrics.rs b/beacon_node/beacon_chain/src/metrics.rs index 991b7b675d1..a23bcdc0b55 100644 --- a/beacon_node/beacon_chain/src/metrics.rs +++ b/beacon_node/beacon_chain/src/metrics.rs @@ -40,6 +40,10 @@ lazy_static! { "beacon_block_processing_block_root_seconds", "Time spent calculating the block root when processing a block." ); + pub static ref BLOCK_PROCESSING_BLOB_ROOT: Result = try_create_histogram( + "beacon_block_processing_blob_root_seconds", + "Time spent calculating the blob root when processing a block." + ); pub static ref BLOCK_PROCESSING_DB_READ: Result = try_create_histogram( "beacon_block_processing_db_read_seconds", "Time spent loading block and state from DB for block processing" @@ -282,6 +286,11 @@ lazy_static! { "Count of times the early attester cache returns an attestation" ); +} + +// Second lazy-static block is used to account for macro recursion limit. +lazy_static! { + /* * Attestation Production */ @@ -301,10 +310,7 @@ lazy_static! { "attestation_production_cache_prime_seconds", "Time spent loading a new state from the disk due to a cache miss" ); -} -// Second lazy-static block is used to account for macro recursion limit. -lazy_static! { /* * Fork Choice */ diff --git a/beacon_node/beacon_chain/tests/block_verification.rs b/beacon_node/beacon_chain/tests/block_verification.rs index 010f763759e..5490dc88584 100644 --- a/beacon_node/beacon_chain/tests/block_verification.rs +++ b/beacon_node/beacon_chain/tests/block_verification.rs @@ -877,7 +877,7 @@ async fn block_gossip_verification() { harness .chain - .process_blob(gossip_verified) + .process_gossip_blob(gossip_verified) .await .expect("should import valid gossip verified blob"); } @@ -1143,7 +1143,11 @@ async fn verify_block_for_gossip_slashing_detection() { .chain .verify_blob_sidecar_for_gossip(blob, blob_index) .unwrap(); - harness.chain.process_blob(verified_blob).await.unwrap(); + harness + .chain + .process_gossip_blob(verified_blob) + .await + .unwrap(); } } harness diff --git a/beacon_node/http_api/src/publish_blocks.rs b/beacon_node/http_api/src/publish_blocks.rs index 8666a18dd8b..7c08bbacc22 100644 --- a/beacon_node/http_api/src/publish_blocks.rs +++ b/beacon_node/http_api/src/publish_blocks.rs @@ -177,7 +177,7 @@ pub async fn publish_block NetworkBeaconProcessor { self.log, "Unknown parent hash for blob"; "action" => "requesting parent", - "blob_root" => %blob.block_root, + "block_root" => %blob.block_root, "parent_root" => %blob.block_parent_root ); self.send_sync_message(SyncMessage::UnknownParentBlob(peer_id, blob)); @@ -732,10 +732,16 @@ impl NetworkBeaconProcessor { // This value is not used presently, but it might come in handy for debugging. _seen_duration: Duration, ) { - let blob_root = verified_blob.block_root(); + let block_root = verified_blob.block_root(); let blob_slot = verified_blob.slot(); let blob_index = verified_blob.id().index; - match self.chain.process_blob(verified_blob).await { + + let delay_lookup = self + .chain + .data_availability_checker + .should_delay_lookup(blob_slot); + + match self.chain.process_gossip_blob(verified_blob).await { Ok(AvailabilityProcessingStatus::Imported(hash)) => { // Note: Reusing block imported metric here metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL); @@ -746,24 +752,40 @@ impl NetworkBeaconProcessor { ); self.chain.recompute_head_at_current_slot().await; } - Ok(AvailabilityProcessingStatus::MissingComponents(slot, block_hash)) => { - trace!( - self.log, - "Missing block components for gossip verified blob"; - "slot" => %blob_slot, - "blob_index" => %blob_index, - "blob_root" => %blob_root, - ); - self.send_sync_message(SyncMessage::MissingGossipBlockComponents( - slot, peer_id, block_hash, - )); + Ok(AvailabilityProcessingStatus::MissingComponents(slot, block_root)) => { + if delay_lookup { + let mut guard = self.delayed_lookup_peers.lock(); + if let Some(peers) = guard.get_mut(&block_root) { + peers.push(peer_id); + } else { + guard.push(block_root, vec![peer_id]); + } + trace!( + self.log, + "Processed blob, delaying lookup for other components"; + "slot" => %blob_slot, + "blob_index" => %blob_index, + "block_root" => %block_root, + ); + } else { + trace!( + self.log, + "Missing block components for gossip verified blob"; + "slot" => %blob_slot, + "blob_index" => %blob_index, + "block_root" => %block_root, + ); + self.send_sync_message(SyncMessage::MissingGossipBlockComponents( + peer_id, block_root, + )); + } } Err(err) => { debug!( self.log, "Invalid gossip blob"; "outcome" => ?err, - "block root" => ?blob_root, + "block root" => ?block_root, "block slot" => blob_slot, "blob index" => blob_index, ); @@ -1112,14 +1134,14 @@ impl NetworkBeaconProcessor { let block = verified_block.block.block_cloned(); let block_root = verified_block.block_root; + let delay_lookup = self + .chain + .data_availability_checker + .should_delay_lookup(verified_block.block.slot()); + let result = self .chain - .process_block( - block_root, - verified_block, - NotifyExecutionLayer::Yes, - || Ok(()), - ) + .process_block_with_early_caching(block_root, verified_block, NotifyExecutionLayer::Yes) .await; match &result { @@ -1151,12 +1173,31 @@ impl NetworkBeaconProcessor { self.chain.recompute_head_at_current_slot().await; } Ok(AvailabilityProcessingStatus::MissingComponents(slot, block_root)) => { - // make rpc request for blob - self.send_sync_message(SyncMessage::MissingGossipBlockComponents( - *slot, - peer_id, - *block_root, - )); + if delay_lookup { + let mut guard = self.delayed_lookup_peers.lock(); + if let Some(peers) = guard.get_mut(block_root) { + peers.push(peer_id); + } else { + guard.push(*block_root, vec![peer_id]); + } + trace!( + self.log, + "Processed block, delaying lookup for other components"; + "slot" => slot, + "block_root" => %block_root, + ); + } else { + trace!( + self.log, + "Missing block components for gossip verified block"; + "slot" => slot, + "block_root" => %block_root, + ); + self.send_sync_message(SyncMessage::MissingGossipBlockComponents( + peer_id, + *block_root, + )); + } } Err(BlockError::ParentUnknown(block)) => { // Inform the sync manager to find parents for this block @@ -1237,6 +1278,7 @@ impl NetworkBeaconProcessor { }; if let Err(e) = &result { + //TODO: drop from processing cache self.maybe_store_invalid_block( &invalid_block_storage, block_root, diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 88d11940547..1208cc1c3fd 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -18,8 +18,11 @@ use lighthouse_network::{ rpc::{BlocksByRangeRequest, BlocksByRootRequest, LightClientBootstrapRequest, StatusMessage}, Client, MessageId, NetworkGlobals, PeerId, PeerRequestId, }; +use lru::LruCache; +use parking_lot::Mutex; use slog::{debug, Logger}; use slot_clock::ManualSlotClock; +use std::collections::HashMap; use std::path::PathBuf; use std::sync::Arc; use std::time::Duration; @@ -40,6 +43,7 @@ mod sync_methods; mod tests; pub(crate) const FUTURE_SLOT_TOLERANCE: u64 = 1; +pub const DELAYED_PEER_CACHE_SIZE: usize = 16; /// Defines if and where we will store the SSZ files of invalid blocks. #[derive(Clone)] @@ -60,6 +64,7 @@ pub struct NetworkBeaconProcessor { pub reprocess_tx: mpsc::Sender, pub network_globals: Arc>, pub invalid_block_storage: InvalidBlockStorage, + pub delayed_lookup_peers: Mutex>>, pub executor: TaskExecutor, pub log: Logger, } @@ -668,6 +673,7 @@ impl NetworkBeaconProcessor> { reprocess_tx: work_reprocessing_tx, network_globals, invalid_block_storage: InvalidBlockStorage::Disabled, + delayed_lookup_peers: Mutex::new(LruCache::new(DELAYED_PEER_CACHE_SIZE)), executor: runtime.task_executor.clone(), log, }; 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 07332628a45..46d8d754fb1 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -25,7 +25,7 @@ use std::time::Duration; use std::time::{SystemTime, UNIX_EPOCH}; use tokio::sync::mpsc; use types::blob_sidecar::FixedBlobSidecarList; -use types::{Epoch, Hash256}; +use types::{Epoch, Hash256, Slot}; /// Id associated to a batch processing request, either a sync batch or a parent lookup. #[derive(Clone, Debug, PartialEq)] @@ -214,7 +214,7 @@ impl NetworkBeaconProcessor { let result = self .chain - .process_block(block_root, block, NotifyExecutionLayer::Yes, || Ok(())) + .process_block_with_early_caching(block_root, block, NotifyExecutionLayer::Yes) .await; metrics::inc_counter(&metrics::BEACON_PROCESSOR_RPC_BLOCK_IMPORTED_TOTAL); @@ -286,10 +286,7 @@ impl NetworkBeaconProcessor { return; }; - let result = self - .chain - .check_rpc_blob_availability_and_import(slot, block_root, blobs) - .await; + let result = self.chain.process_rpc_blobs(slot, block_root, blobs).await; // Sync handles these results self.send_sync_message(SyncMessage::BlockComponentProcessed { @@ -298,8 +295,20 @@ impl NetworkBeaconProcessor { }); } - pub fn send_delayed_lookup(&self, block_root: Hash256) { - self.send_sync_message(SyncMessage::MissingGossipBlockComponentsDelayed(block_root)) + pub fn poll_delayed_lookups(&self, slot: Slot) { + let lookups = self + .chain + .data_availability_checker + .get_delayed_lookups(slot); + for block_root in lookups { + if let Some(peer_ids) = self.delayed_lookup_peers.lock().pop(&block_root) { + for peer_id in peer_ids { + self.send_sync_message(SyncMessage::MissingGossipBlockComponents( + peer_id, block_root, + )); + } + } + } } /// Attempt to import the chain segment (`blocks`) to the beacon chain, informing the sync diff --git a/beacon_node/network/src/network_beacon_processor/tests.rs b/beacon_node/network/src/network_beacon_processor/tests.rs index 993551441f3..903d1ac6127 100644 --- a/beacon_node/network/src/network_beacon_processor/tests.rs +++ b/beacon_node/network/src/network_beacon_processor/tests.rs @@ -1,6 +1,7 @@ #![cfg(not(debug_assertions))] // Tests are too slow in debug. #![cfg(test)] +use crate::network_beacon_processor::DELAYED_PEER_CACHE_SIZE; use crate::{ network_beacon_processor::{ ChainSegmentProcessId, DuplicateCache, InvalidBlockStorage, NetworkBeaconProcessor, @@ -22,7 +23,10 @@ use lighthouse_network::{ types::{EnrAttestationBitfield, EnrSyncCommitteeBitfield}, Client, MessageId, NetworkGlobals, PeerId, Response, }; +use lru::LruCache; +use parking_lot::Mutex; use slot_clock::SlotClock; +use std::collections::HashMap; use std::iter::Iterator; use std::sync::Arc; use std::time::Duration; @@ -226,6 +230,7 @@ impl TestRig { reprocess_tx: work_reprocessing_tx.clone(), network_globals: network_globals.clone(), invalid_block_storage: InvalidBlockStorage::Disabled, + delayed_lookup_peers: Mutex::new(LruCache::new(DELAYED_PEER_CACHE_SIZE)), executor: executor.clone(), log: log.clone(), }; diff --git a/beacon_node/network/src/router.rs b/beacon_node/network/src/router.rs index 86181c347d3..179aef8a96c 100644 --- a/beacon_node/network/src/router.rs +++ b/beacon_node/network/src/router.rs @@ -21,6 +21,8 @@ use lighthouse_network::{ MessageId, NetworkGlobals, PeerId, PeerRequestId, PubsubMessage, Request, Response, }; use logging::TimeLatch; +use lru::LruCache; +use parking_lot::Mutex; use slog::{crit, debug, o, trace}; use slog::{error, warn}; use std::sync::Arc; @@ -109,6 +111,9 @@ impl Router { reprocess_tx: beacon_processor_reprocess_tx, network_globals: network_globals.clone(), invalid_block_storage, + delayed_lookup_peers: Mutex::new(LruCache::new( + crate::network_beacon_processor::DELAYED_PEER_CACHE_SIZE, + )), executor: executor.clone(), log: log.clone(), }; diff --git a/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs b/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs index 55e9e49db30..507aa9d1591 100644 --- a/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs @@ -1,6 +1,6 @@ use crate::network_beacon_processor::NetworkBeaconProcessor; use beacon_chain::{BeaconChain, BeaconChainTypes}; -use slog::crit; +use slog::{crit, error, trace}; use slot_clock::SlotClock; use std::sync::Arc; use tokio::sync::mpsc; @@ -8,12 +8,6 @@ use tokio::time::interval_at; use tokio::time::Instant; use types::Hash256; -#[derive(Debug)] -pub enum DelayedLookupMessage { - /// A lookup for all components of a block or blob seen over gossip. - MissingComponents(Hash256), -} - /// This service is responsible for collecting lookup messages and sending them back to sync /// for processing after a short delay. /// @@ -34,7 +28,6 @@ pub enum DelayedLookupMessage { pub fn spawn_delayed_lookup_service( executor: &task_executor::TaskExecutor, beacon_chain: Arc>, - mut delayed_lookups_recv: mpsc::Receiver, beacon_processor: Arc>, log: slog::Logger, ) { @@ -66,15 +59,13 @@ pub fn spawn_delayed_lookup_service( let mut interval = interval_at(interval_start, slot_duration); loop { interval.tick().await; - while let Ok(msg) = delayed_lookups_recv.try_recv() { - match msg { - DelayedLookupMessage::MissingComponents(block_root) => { - beacon_processor - .send_delayed_lookup(block_root) - } - } + let Some(slot) = beacon_chain.slot_clock.now_or_genesis() else { + error!(log, "Skipping delayed lookup poll, unable to read slot clock"); + continue + }; + trace!(log, "Polling delayed lookups for slot: {slot}"); + beacon_processor.poll_delayed_lookups(slot) } - } }, "delayed_lookups", ); diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index 2aa337e2f7c..35f29a71378 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -230,38 +230,6 @@ impl BlockLookups { ); } - /// 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 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) { - debug!(self.log, "Delayed single block lookup failed"; - "error" => ?e, - "block_root" => ?block_root, - ); - return false; - } - } - true - }); - } - /// Searches for a single block hash. If the blocks parent is unknown, a chain of blocks is /// constructed. pub fn new_current_lookup( diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 88ce0b36197..6749abf9f80 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -43,7 +43,6 @@ use crate::service::NetworkMessage; use crate::status::ToStatusMessage; use crate::sync::block_lookups::common::{Current, Parent}; use crate::sync::block_lookups::delayed_lookup; -use crate::sync::block_lookups::delayed_lookup::DelayedLookupMessage; use crate::sync::block_lookups::{BlobRequestState, BlockRequestState, CachedChildComponents}; use crate::sync::range_sync::ByRangeRequestType; use beacon_chain::block_verification_types::AsBlock; @@ -76,9 +75,6 @@ use types::{BlobSidecar, EthSpec, Hash256, SignedBeaconBlock, Slot}; /// gossip if no peers are further than this range ahead of us that we have not already downloaded /// blocks for. pub const SLOT_IMPORT_TOLERANCE: usize = 32; -/// The maximum number of messages the delay queue can handle in a single slot before messages are -/// dropped. -pub const DELAY_QUEUE_CHANNEL_SIZE: usize = 128; pub type Id = u32; @@ -148,10 +144,7 @@ pub enum SyncMessage { /// /// We will either attempt to find the block matching the unknown hash immediately or queue a lookup, /// which will then trigger the request when we receive `MissingGossipBlockComponentsDelayed`. - MissingGossipBlockComponents(Slot, PeerId, Hash256), - - /// This message triggers a request for missing block components after a delay. - MissingGossipBlockComponentsDelayed(Hash256), + MissingGossipBlockComponents(PeerId, Hash256), /// A peer has disconnected. Disconnect(PeerId), @@ -228,8 +221,6 @@ pub struct SyncManager { block_lookups: BlockLookups, - delayed_lookups: mpsc::Sender, - /// The logger for the import manager. log: Logger, } @@ -249,8 +240,6 @@ pub fn spawn( MAX_REQUEST_BLOCKS >= T::EthSpec::slots_per_epoch() * EPOCHS_PER_BATCH, "Max blocks that can be requested in a single batch greater than max allowed blocks in a single request" ); - let (delayed_lookups_send, delayed_lookups_recv) = - mpsc::channel::(DELAY_QUEUE_CHANNEL_SIZE); // create an instance of the SyncManager let network_globals = beacon_processor.network_globals.clone(); @@ -269,18 +258,11 @@ pub fn spawn( beacon_chain.data_availability_checker.clone(), log.clone(), ), - delayed_lookups: delayed_lookups_send, log: log.clone(), }; let log_clone = log.clone(); - delayed_lookup::spawn_delayed_lookup_service( - &executor, - beacon_chain, - delayed_lookups_recv, - beacon_processor, - log, - ); + delayed_lookup::spawn_delayed_lookup_service(&executor, beacon_chain, beacon_processor, log); // spawn the sync manager thread debug!(log_clone, "Sync Manager started"); @@ -686,34 +668,16 @@ impl SyncManager { ); } } - SyncMessage::MissingGossipBlockComponents(slot, peer_id, block_root) => { + SyncMessage::MissingGossipBlockComponents(peer_id, block_root) => { // 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), - &mut self.network, - ); - if let Err(e) = self - .delayed_lookups - .try_send(DelayedLookupMessage::MissingComponents(block_root)) - { - warn!(self.log, "Delayed lookup dropped for block referenced by a blob"; - "block_root" => ?block_root, "error" => ?e); - } - } else { - self.block_lookups.search_block( - block_root, - PeerShouldHave::Neither(peer_id), - &mut self.network, - ) - } + self.block_lookups.search_block( + block_root, + PeerShouldHave::Neither(peer_id), + &mut self.network, + ) } } - SyncMessage::MissingGossipBlockComponentsDelayed(block_root) => self - .block_lookups - .trigger_lookup_by_root(block_root, &self.network), SyncMessage::Disconnect(peer_id) => { self.peer_disconnect(&peer_id); } @@ -792,56 +756,12 @@ impl SyncManager { peer_id, &mut self.network, ); - if self.should_delay_lookup(slot) { - self.block_lookups.search_child_delayed( - block_root, - child_components, - PeerShouldHave::Neither(peer_id), - &mut self.network, - ); - if let Err(e) = self - .delayed_lookups - .try_send(DelayedLookupMessage::MissingComponents(block_root)) - { - warn!(self.log, "Delayed lookups dropped for block"; "block_root" => ?block_root, "error" => ?e); - } - } else { - self.block_lookups.search_child_block( - block_root, - child_components, - PeerShouldHave::Neither(peer_id), - &mut self.network, - ); - } - } - } - - 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 earliest_slot = self - .chain - .slot_clock - .now_with_past_tolerance(maximum_gossip_clock_disparity); - let latest_slot = self - .chain - .slot_clock - .now_with_future_tolerance(maximum_gossip_clock_disparity); - if let (Some(earliest_slot), Some(latest_slot)) = (earliest_slot, latest_slot) { - let msg_for_current_slot = slot >= earliest_slot && slot <= latest_slot; - let delay_threshold_unmet = self - .chain - .slot_clock - .millis_from_current_slot_start() - .map_or(false, |millis_into_slot| { - millis_into_slot < self.chain.slot_clock.single_lookup_delay() - }); - msg_for_current_slot && delay_threshold_unmet - } else { - false + self.block_lookups.search_child_block( + block_root, + child_components, + PeerShouldHave::Neither(peer_id), + &mut self.network, + ); } } From 6ed602f21a5955b6616571670a8cda1ca6e8dd25 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 14 Sep 2023 10:45:30 -0400 Subject: [PATCH 02/47] move processing cache out of critical cache --- beacon_node/beacon_chain/src/beacon_chain.rs | 47 ++++++++-------- .../src/data_availability_checker.rs | 4 +- .../overflow_lru_cache.rs | 53 +++++++------------ .../processing_cache.rs | 12 ++++- 4 files changed, 53 insertions(+), 63 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 2312cf8d95c..30c190f13ac 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -2795,14 +2795,13 @@ impl BeaconChain { self: &Arc, blob: GossipVerifiedBlob, ) -> Result> { - todo!() - // self.data_availability_checker.notify_blob(); - // self.check_gossip_blob_availability_and_import(blob) - // .await - // .map_err(|e| { - // self.data_availability_checker.remove_notified_blob(); - // e - // }) + self.data_availability_checker.notify_blob(); + self.check_gossip_blob_availability_and_import(blob) + .await + .map_err(|e| { + self.data_availability_checker.remove_notified_blob(); + e + }) } pub async fn process_rpc_blobs( @@ -2811,14 +2810,14 @@ impl BeaconChain { block_root: Hash256, blobs: FixedBlobSidecarList, ) -> Result> { - todo!(); - // self.data_availability_checker.notify_blob(); - // self.check_rpc_blob_availability_and_import(slot, block_root, blobs) - // .await - // .map_err(|e| { - // self.data_availability_checker.remove_notified_blob(block_root); - // e - // }) + self.data_availability_checker.notify_blob(); + self.check_rpc_blob_availability_and_import(slot, block_root, blobs) + .await + .map_err(|e| { + self.data_availability_checker + .remove_notified_blob(block_root); + e + }) } pub async fn process_block_with_early_caching>( @@ -2827,15 +2826,13 @@ impl BeaconChain { unverified_block: B, notify_execution_layer: NotifyExecutionLayer, ) -> Result> { - todo!(); - - // self.data_availability_checker.notify_block(); - // self.process_block(block_root, unverified_block, notify_execution_layer) - // .await - // .map_err(|e| { - // self.data_availability_checker.remove_notified_block(); - // e - // }) + self.data_availability_checker.notify_block(); + self.process_block(block_root, unverified_block, notify_execution_layer) + .await + .map_err(|e| { + self.data_availability_checker.remove_notified_block(); + e + }) } /// Returns `Ok(block_root)` if the given `unverified_block` was successfully verified and /// imported into the chain. diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index b38e8684f2e..7cfa9f761cb 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -5,6 +5,7 @@ use crate::block_verification_types::{ AvailabilityPendingExecutedBlock, AvailableExecutedBlock, RpcBlock, }; use crate::data_availability_checker::overflow_lru_cache::OverflowLRUCache; +use crate::data_availability_checker::processing_cache::ProcessingCache; use crate::{BeaconChain, BeaconChainTypes, BeaconStore, GossipVerifiedBlock}; use kzg::Error as KzgError; use kzg::Kzg; @@ -85,6 +86,7 @@ impl From for AvailabilityCheckError { /// `DataAvailabilityChecker` is responsible for KZG verification of block components as well as /// checking whether a "availability check" is required at all. pub struct DataAvailabilityChecker { + processing_cache: ProcessingCache, availability_cache: Arc>, slot_clock: T::SlotClock, kzg: Option::Kzg>>>, @@ -285,7 +287,7 @@ impl DataAvailabilityChecker { } pub fn notify_block(&self, block: &GossipVerifiedBlock) -> bool { - todo!() + self.availability_cache.notify_block(block) } pub fn notify_blob(&self, block_root: Hash256, blob: &BlobSidecar) -> bool { diff --git a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs index 8ef04480c6a..bf9736e0ac0 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs @@ -32,7 +32,6 @@ use crate::blob_verification::KzgVerifiedBlob; use crate::block_verification_types::{ AsBlock, AvailabilityPendingExecutedBlock, AvailableExecutedBlock, }; -use crate::data_availability_checker::processing_cache::ProcessingCache; use crate::data_availability_checker::{make_available, Availability, AvailabilityCheckError}; use crate::store::{DBColumn, KeyValueStore}; use crate::BeaconChainTypes; @@ -43,10 +42,14 @@ use ssz_derive::{Decode, Encode}; use ssz_types::FixedVector; use std::collections::HashMap; use std::{collections::HashSet, sync::Arc}; +use types::beacon_block_body::KzgCommitments; use types::blob_sidecar::BlobIdentifier; -use types::{BlobSidecar, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot}; +use types::{BlobSidecar, Epoch, EthSpec, Hash256}; -pub(crate) type MissingBlobInfo = (Option>>, HashSet); +pub(crate) type MissingBlobInfo = ( + Option>, + FixedVector>, ::MaxBlobsPerBlock>, +); /// This represents the components of a partially available block /// @@ -293,7 +296,6 @@ impl OverflowStore { /// This data stores the *critical* data that we need to keep in memory /// protected by the RWLock struct Critical { - pub processing_cache: ProcessingCache, /// This is the LRU cache of pending components pub in_memory: LruCache>, /// This holds all the roots of the blocks for which we have @@ -304,7 +306,6 @@ struct Critical { impl Critical { pub fn new(capacity: usize) -> Self { Self { - processing_cache: <_>::default(), in_memory: LruCache::new(capacity), store_keys: HashSet::new(), } @@ -351,7 +352,6 @@ impl Critical { self.store_keys.insert(lru_key); } } - self.processing_cache.remove_processing(block_root); self.in_memory.put(block_root, pending_components); Ok(()) } @@ -412,9 +412,7 @@ impl OverflowLRUCache { /// Returns whether or not a block is in the cache (in memory or on disk) pub fn has_block(&self, block_root: &Hash256) -> bool { let read_lock = self.critical.read(); - if read_lock.processing_cache.has_block(block_root) { - true - } else if read_lock + if read_lock .in_memory .peek(block_root) .map_or(false, |cache| cache.executed_block.is_some()) @@ -434,29 +432,17 @@ impl OverflowLRUCache { /// Fetch the missing blob info for a block without affecting the LRU ordering pub fn get_missing_blob_info(&self, block_root: Hash256) -> MissingBlobInfo { let read_lock = self.critical.read(); - - match ( - read_lock.in_memory.peek(&block_root), - read_lock.processing_cache.peek(&block_root), - ) { - (Some(in_memory), Some(processing)) => { - //TODO: merge the two views - todo!() - } - (Some(in_memory), None) => in_memory.get_missing_blob_info(), - (None, Some(processing)) => processing.get_missing_blob_info(), - (None, None) => { - if read_lock.store_keys.contains(&block_root) { - drop(read_lock); - // return default if there's an error reading from the store - match self.overflow_store.load_pending_components(block_root) { - Ok(Some(pending_components)) => pending_components.get_missing_blob_info(), - _ => Default::default(), - } - } else { - Default::default() - } + if let Some(cache) = read_lock.in_memory.peek(&block_root) { + cache.get_missing_blob_info() + } else if read_lock.store_keys.contains(&block_root) { + drop(read_lock); + // return default if there's an error reading from the store + match self.overflow_store.load_pending_components(block_root) { + Ok(Some(pending_components)) => pending_components.get_missing_blob_info(), + _ => Default::default(), } + } else { + Default::default() } } @@ -597,7 +583,6 @@ impl OverflowLRUCache { mut pending_components: PendingComponents, executed_block: AvailabilityPendingExecutedBlock, ) -> Result, AvailabilityCheckError> { - let block_root = executed_block.import_data.block_root; if pending_components.has_all_blobs(&executed_block) { let num_blobs_expected = executed_block.num_blobs_expected(); let AvailabilityPendingExecutedBlock { @@ -615,7 +600,6 @@ impl OverflowLRUCache { }; let available_block = make_available(block, verified_blobs)?; - write_lock.processing_cache.put_processed(block_root); Ok(Availability::Available(Box::new( AvailableExecutedBlock::new( available_block, @@ -624,6 +608,7 @@ impl OverflowLRUCache { ), ))) } else { + let block_root = executed_block.import_data.block_root; let _ = pending_components.executed_block.insert(executed_block); write_lock.put_pending_components( block_root, @@ -661,8 +646,6 @@ impl OverflowLRUCache { self.maintain_threshold(threshold, cutoff_epoch)?; // clean up any keys on the disk that shouldn't be there self.prune_disk(cutoff_epoch)?; - //TODO: fix - self.critical.write().processing_cache.prune(Slot::new(0)); Ok(()) } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs index 43b1356fc63..4c8b0cfac21 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -4,6 +4,7 @@ use crate::data_availability_checker::{Availability, AvailabilityCheckError}; use crate::GossipVerifiedBlock; use kzg::KzgCommitment; use parking_lot::{Mutex, RwLock}; +use ssz_types::FixedVector; use std::collections::{HashMap, HashSet}; use std::sync::Arc; use types::beacon_block_body::KzgCommitments; @@ -47,8 +48,15 @@ impl SimplifiedPendingComponents { } impl ProcessingCache { - pub fn put_processed(&mut self, block_root: Hash256) -> bool { - self.processed_cache.insert(block_root) + pub fn put_processing_block(&mut self, block_root: Hash256) { + self.processing_cache.insert(block_root); + } + pub fn put_processing_blob(&mut self, block_root: Hash256) { + self.processing_cache.insert(block_root); + } + + pub fn put_processed(&mut self, block_root: Hash256) { + self.processed_cache.insert(block_root); } pub fn has_block(&self, block_root: &Hash256) -> bool { From 904e5936c981a7ffa2297eac7506e94934902f7a Mon Sep 17 00:00:00 2001 From: realbigsean Date: Mon, 18 Sep 2023 18:17:20 -0400 Subject: [PATCH 03/47] get it compiling --- beacon_node/beacon_chain/src/beacon_chain.rs | 75 +++-- .../beacon_chain/src/blob_verification.rs | 3 + .../src/block_verification_types.rs | 14 - .../src/data_availability_checker.rs | 166 +++-------- .../availability_view.rs | 161 ++++++++++ .../overflow_lru_cache.rs | 281 +++++------------- .../processing_cache.rs | 130 ++++---- .../beacon_chain/tests/block_verification.rs | 2 +- .../gossip_methods.rs | 7 +- .../network/src/sync/block_lookups/mod.rs | 67 +---- .../sync/block_lookups/single_block_lookup.rs | 84 ++++-- consensus/types/src/beacon_block_body.rs | 2 + consensus/types/src/blob_sidecar.rs | 13 + consensus/types/src/signed_beacon_block.rs | 50 ++-- 14 files changed, 526 insertions(+), 529 deletions(-) create mode 100644 beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 30c190f13ac..2640844baba 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -118,6 +118,7 @@ use store::{ use task_executor::{ShutdownReason, TaskExecutor}; use tokio_stream::Stream; use tree_hash::TreeHash; +use types::beacon_block_body::KzgCommitmentOpts; use types::beacon_state::CloneConfig; use types::blob_sidecar::{BlobSidecarList, FixedBlobSidecarList}; use types::sidecar::BlobItems; @@ -2795,13 +2796,15 @@ impl BeaconChain { self: &Arc, blob: GossipVerifiedBlob, ) -> Result> { - self.data_availability_checker.notify_blob(); - self.check_gossip_blob_availability_and_import(blob) - .await - .map_err(|e| { - self.data_availability_checker.remove_notified_blob(); - e - }) + let block_root = blob.block_root(); + let mut commitments = KzgCommitmentOpts::::default(); + commitments + .get_mut(blob.as_blob().index as usize) + .map(|b| *b = Some(blob.as_blob().kzg_commitment)); + self.data_availability_checker + .notify_blob_commitments(block_root, commitments); + let r = self.check_gossip_blob_availability_and_import(blob).await; + self.remove_notified(&block_root, r) } pub async fn process_rpc_blobs( @@ -2810,14 +2813,33 @@ impl BeaconChain { block_root: Hash256, blobs: FixedBlobSidecarList, ) -> Result> { - self.data_availability_checker.notify_blob(); - self.check_rpc_blob_availability_and_import(slot, block_root, blobs) - .await - .map_err(|e| { - self.data_availability_checker - .remove_notified_blob(block_root); - e - }) + let mut commitments = KzgCommitmentOpts::::default(); + for blob in &blobs { + if let Some(blob) = blob { + commitments + .get_mut(blob.index as usize) + .map(|b| *b = Some(blob.kzg_commitment)); + } + } + self.data_availability_checker + .notify_blob_commitments(block_root, commitments); + let r = self + .check_rpc_blob_availability_and_import(slot, block_root, blobs) + .await; + self.remove_notified(&block_root, r) + } + + fn remove_notified( + &self, + block_root: &Hash256, + r: Result>, + ) -> Result> { + let has_missing_components = + matches!(r, Ok(AvailabilityProcessingStatus::MissingComponents(_, _))); + if !has_missing_components { + self.data_availability_checker.remove_notified(&block_root); + } + r } pub async fn process_block_with_early_caching>( @@ -2826,14 +2848,23 @@ impl BeaconChain { unverified_block: B, notify_execution_layer: NotifyExecutionLayer, ) -> Result> { - self.data_availability_checker.notify_block(); - self.process_block(block_root, unverified_block, notify_execution_layer) - .await - .map_err(|e| { - self.data_availability_checker.remove_notified_block(); - e + if let Ok(commitments) = unverified_block + .block() + .message() + .body() + .blob_kzg_commitments() + { + self.data_availability_checker + .notify_block_commitments(block_root, commitments.clone()); + }; + let r = self + .process_block(block_root, unverified_block, notify_execution_layer, || { + Ok(()) }) + .await; + self.remove_notified(&block_root, r) } + /// Returns `Ok(block_root)` if the given `unverified_block` was successfully verified and /// imported into the chain. /// @@ -5286,7 +5317,7 @@ impl BeaconChain { return Ok(()); } - // Fetch payoad attributes from the execution layer's cache, or compute them from scratch + // Fetch payload attributes from the execution layer's cache, or compute them from scratch // if no matching entry is found. This saves recomputing the withdrawals which can take // considerable time to compute if a state load is required. let head_root = forkchoice_update_params.head_root; diff --git a/beacon_node/beacon_chain/src/blob_verification.rs b/beacon_node/beacon_chain/src/blob_verification.rs index 2247e020c00..dd7fbc23998 100644 --- a/beacon_node/beacon_chain/src/blob_verification.rs +++ b/beacon_node/beacon_chain/src/blob_verification.rs @@ -169,6 +169,9 @@ impl GossipVerifiedBlob { pub fn to_blob(self) -> Arc> { self.blob.message } + pub fn as_blob(&self) -> &BlobSidecar { + &self.blob.message + } pub fn signed_blob(&self) -> SignedBlobSidecar { self.blob.clone() } diff --git a/beacon_node/beacon_chain/src/block_verification_types.rs b/beacon_node/beacon_chain/src/block_verification_types.rs index b20bb4f0ef8..1b8458e4477 100644 --- a/beacon_node/beacon_chain/src/block_verification_types.rs +++ b/beacon_node/beacon_chain/src/block_verification_types.rs @@ -242,20 +242,6 @@ impl AvailabilityPendingExecutedBlock { .blob_kzg_commitments() .map_or(0, |commitments| commitments.len()) } - - pub fn get_all_blob_ids(&self) -> Vec { - let block_root = self.import_data.block_root; - self.block - .get_filtered_blob_ids(Some(block_root), |_, _| true) - } - - pub fn get_filtered_blob_ids( - &self, - filter: impl Fn(usize, Hash256) -> bool, - ) -> Vec { - self.block - .get_filtered_blob_ids(Some(self.import_data.block_root), filter) - } } #[derive(Debug, PartialEq, Encode, Decode, Clone)] diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index 7cfa9f761cb..f205f4fa4a9 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -1,27 +1,29 @@ -use crate::blob_verification::{ - verify_kzg_for_blob, verify_kzg_for_blob_list, GossipVerifiedBlob, KzgVerifiedBlob, -}; +use crate::blob_verification::{verify_kzg_for_blob, verify_kzg_for_blob_list, GossipVerifiedBlob}; use crate::block_verification_types::{ AvailabilityPendingExecutedBlock, AvailableExecutedBlock, RpcBlock, }; +pub use crate::data_availability_checker::availability_view::AvailabilityView; use crate::data_availability_checker::overflow_lru_cache::OverflowLRUCache; use crate::data_availability_checker::processing_cache::ProcessingCache; -use crate::{BeaconChain, BeaconChainTypes, BeaconStore, GossipVerifiedBlock}; +use crate::{BeaconChain, BeaconChainTypes, BeaconStore}; use kzg::Error as KzgError; use kzg::Kzg; +use parking_lot::RwLock; +pub use processing_cache::ProcessingInfo; use slog::{debug, error}; use slot_clock::SlotClock; -use ssz_types::{Error, VariableList}; -use std::collections::HashSet; +use ssz_types::Error; use std::fmt; use std::fmt::Debug; use std::sync::Arc; use strum::IntoStaticStr; use task_executor::TaskExecutor; +use types::beacon_block_body::{KzgCommitmentOpts, KzgCommitments}; use types::blob_sidecar::{BlobIdentifier, BlobSidecar, FixedBlobSidecarList}; use types::consts::deneb::MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS; use types::{BlobSidecarList, ChainSpec, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot}; +mod availability_view; mod overflow_lru_cache; mod processing_cache; @@ -37,30 +39,16 @@ pub enum AvailabilityCheckError { Kzg(KzgError), KzgNotInitialized, KzgVerificationFailed, + Unexpected, SszTypes(ssz_types::Error), - NumBlobsMismatch { - num_kzg_commitments: usize, - num_blobs: usize, - }, MissingBlobs, - KzgCommitmentMismatch { - blob_index: u64, - }, BlobIndexInvalid(u64), - UnorderedBlobs { - blob_index: u64, - expected_index: u64, - }, StoreError(store::Error), DecodeError(ssz::DecodeError), - BlockBlobRootMismatch { + InconsistentBlobBlockRoots { block_root: Hash256, blob_block_root: Hash256, }, - BlockBlobSlotMismatch { - block_slot: Slot, - blob_slot: Slot, - }, } impl From for AvailabilityCheckError { @@ -86,7 +74,7 @@ impl From for AvailabilityCheckError { /// `DataAvailabilityChecker` is responsible for KZG verification of block components as well as /// checking whether a "availability check" is required at all. pub struct DataAvailabilityChecker { - processing_cache: ProcessingCache, + processing_cache: RwLock>, availability_cache: Arc>, slot_clock: T::SlotClock, kzg: Option::Kzg>>>, @@ -123,6 +111,7 @@ impl DataAvailabilityChecker { ) -> Result { let overflow_cache = OverflowLRUCache::new(OVERFLOW_LRU_CAPACITY, store)?; Ok(Self { + processing_cache: <_>::default(), availability_cache: Arc::new(overflow_cache), slot_clock, kzg, @@ -132,7 +121,7 @@ impl DataAvailabilityChecker { /// Checks if the given block root is cached. pub fn has_block(&self, block_root: &Hash256) -> bool { - self.availability_cache.has_block(block_root) + self.processing_cache.read().has_block(block_root) } /// Checks which blob ids are still required for a given block root, taking any cached @@ -141,8 +130,8 @@ impl DataAvailabilityChecker { &self, block_root: Hash256, ) -> Option> { - let (block, blob_indices) = self.availability_cache.get_missing_blob_info(block_root); - self.get_missing_blob_ids(block_root, block.as_ref(), Some(blob_indices)) + let guard = self.processing_cache.read(); + self.get_missing_blob_ids(block_root, guard.get(&block_root)?) } /// A `None` indicates blobs are not required. @@ -152,31 +141,12 @@ impl DataAvailabilityChecker { pub fn get_missing_blob_ids( &self, block_root: Hash256, - block_opt: Option<&Arc>>, - blobs_opt: Option>, + processing_info: &ProcessingInfo, ) -> Option> { let epoch = self.slot_clock.now()?.epoch(T::EthSpec::slots_per_epoch()); - self.da_check_required_for_epoch(epoch).then(|| { - block_opt - .map(|block| { - block.get_filtered_blob_ids(Some(block_root), |i, _| { - blobs_opt.as_ref().map_or(true, |blobs| !blobs.contains(&i)) - }) - }) - .unwrap_or_else(|| { - let mut blob_ids = Vec::with_capacity(T::EthSpec::max_blobs_per_block()); - for i in 0..T::EthSpec::max_blobs_per_block() { - if blobs_opt.as_ref().map_or(true, |blobs| !blobs.contains(&i)) { - blob_ids.push(BlobIdentifier { - block_root, - index: i as u64, - }); - } - } - blob_ids - }) - }) + self.da_check_required_for_epoch(epoch) + .then(|| processing_info.get_missing_blob_ids(block_root)) } /// Get a blob from the availability cache. @@ -286,24 +256,38 @@ impl DataAvailabilityChecker { block_within_da_period && block_has_kzg_commitments } - pub fn notify_block(&self, block: &GossipVerifiedBlock) -> bool { - self.availability_cache.notify_block(block) - } - - pub fn notify_blob(&self, block_root: Hash256, blob: &BlobSidecar) -> bool { - todo!() + pub fn notify_block_commitments( + &self, + block_root: Hash256, + commitments: KzgCommitments, + ) { + self.processing_cache + .write() + .entry(block_root) + .or_insert_with(ProcessingInfo::default) + .merge_block(commitments); } - pub fn remove_notified_block(&self, block_root: Hash256) -> bool { - todo!() + pub fn notify_blob_commitments( + &self, + block_root: Hash256, + blobs: KzgCommitmentOpts, + ) { + self.processing_cache + .write() + .entry(block_root) + .or_insert_with(ProcessingInfo::default) + .merge_blobs(blobs); } - pub fn remove_notified_blob(&self, block_root: Hash256, blob_index: usize) -> bool { - todo!() + pub fn remove_notified(&self, block_root: &Hash256) { + self.processing_cache.write().remove(block_root) } pub fn get_delayed_lookups(&self, slot: Slot) -> Vec { - todo!() + self.processing_cache + .read() + .incomplete_lookups_for_slot(slot) } pub fn should_delay_lookup(&self, slot: Slot) -> bool { @@ -370,23 +354,6 @@ impl DataAvailabilityChecker { } } -/// Verifies an `SignedBeaconBlock` against a set of KZG verified blobs. -/// This does not check whether a block *should* have blobs, these checks should have been -/// completed when producing the `AvailabilityPendingBlock`. -pub fn make_available( - block: Arc>, - blobs: Vec>, -) -> Result, AvailabilityCheckError> { - let blobs = VariableList::new(blobs.into_iter().map(|blob| blob.to_blob()).collect())?; - - consistency_checks(&block, &blobs)?; - - Ok(AvailableBlock { - block, - blobs: Some(blobs), - }) -} - /// Makes the following checks to ensure that the list of blobs correspond block: /// /// * Check that a block is post-deneb @@ -394,57 +361,22 @@ pub fn make_available( /// * Checks that the index, slot, root and kzg_commitment in the block match the blobs in the correct order /// /// Returns `Ok(())` if all consistency checks pass and an error otherwise. -pub fn consistency_checks( - block: &SignedBeaconBlock, - blobs: &[Arc>], +pub fn consistency_checks( + block: &SignedBeaconBlock, + blobs: &[Arc>], ) -> Result<(), AvailabilityCheckError> { let Ok(block_kzg_commitments) = block.message().body().blob_kzg_commitments() else { return Ok(()); }; - if blobs.len() != block_kzg_commitments.len() { - return Err(AvailabilityCheckError::NumBlobsMismatch { - num_kzg_commitments: block_kzg_commitments.len(), - num_blobs: blobs.len(), - }); - } - if block_kzg_commitments.is_empty() { return Ok(()); } - let block_root = blobs - .first() - .map(|blob| blob.block_root) - .unwrap_or(block.canonical_root()); - for (index, (block_commitment, blob)) in - block_kzg_commitments.iter().zip(blobs.iter()).enumerate() - { - let index = index as u64; - if index != blob.index { - return Err(AvailabilityCheckError::UnorderedBlobs { - blob_index: blob.index, - expected_index: index, - }); - } - if block_root != blob.block_root { - return Err(AvailabilityCheckError::BlockBlobRootMismatch { - block_root, - blob_block_root: blob.block_root, - }); - } - if block.slot() != blob.slot { - return Err(AvailabilityCheckError::BlockBlobSlotMismatch { - block_slot: block.slot(), - blob_slot: blob.slot, - }); - } - if *block_commitment != blob.kzg_commitment { - return Err(AvailabilityCheckError::KzgCommitmentMismatch { - blob_index: blob.index, - }); - } + if blobs.len() != block_kzg_commitments.len() { + return Err(AvailabilityCheckError::MissingBlobs); } + Ok(()) } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs new file mode 100644 index 00000000000..7d743a1225f --- /dev/null +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -0,0 +1,161 @@ +use crate::blob_verification::KzgVerifiedBlob; +use crate::data_availability_checker::overflow_lru_cache::PendingComponents; +use crate::data_availability_checker::processing_cache::ProcessingCache; +use crate::data_availability_checker::ProcessingInfo; +use crate::AvailabilityPendingExecutedBlock; +use bls::Hash256; +use kzg::KzgCommitment; +use ssz_types::FixedVector; +use types::beacon_block_body::{KzgCommitmentOpts, KzgCommitments}; +use types::EthSpec; + +/// This trait is meant to ensure we maintain the following invariants across caches used in +/// availability checking: +/// +/// 1. Never clobber blobs when adding new blobs. +/// 2. When adding a block, evict all blobs whose KZG commitments do not match the block's. +pub trait AvailabilityView { + type BlockType; + type BlobType: Clone; + + fn block_exists(&self) -> bool; + fn blob_exists(&self, blob_index: u64) -> bool; + fn num_expected_blobs(&self) -> usize; + fn num_received_blobs(&self) -> usize; + fn insert_block(&mut self, block: Self::BlockType); + fn insert_blob_at_index(&mut self, blob_index: u64, blob: &Self::BlobType); + fn blob_to_commitment(blob: &Self::BlobType) -> KzgCommitment; + fn get_cached_blob_commitments_mut( + &mut self, + ) -> &mut FixedVector, E::MaxBlobsPerBlock>; + fn get_block_commitment_at_index(&self, blob_index: u64) -> Option; + /// validate the index + /// only insert if: + /// 1. the blob entry is empty and there is no block + /// 2. the block exists and the commitment matches + fn merge_blobs(&mut self, blobs: FixedVector, E::MaxBlobsPerBlock>) { + for (index, blob) in blobs.into_iter().enumerate() { + let Some(blob) = blob else { continue }; + let commitment = Self::blob_to_commitment(blob); + + let index = index as u64; + + if let Some(block_commitment) = self.get_block_commitment_at_index(index) { + if block_commitment == commitment { + self.insert_blob_at_index(index, blob) + } + } else { + if !self.blob_exists(index) { + self.insert_blob_at_index(index, blob) + } + } + } + } + fn merge_block(&mut self, block: Self::BlockType) { + self.insert_block(block); + let mut cached = self.get_cached_blob_commitments_mut(); + let mut reinsert = FixedVector::default(); + for (index, cached_blob) in cached.iter_mut().enumerate() { + // Take the existing blobs and re-insert them. + reinsert + .get_mut(index) + .map(|blob| *blob = cached_blob.take()); + } + + self.merge_blobs(reinsert) + } + fn is_available(&self) -> bool { + self.block_exists() && self.num_expected_blobs() == self.num_received_blobs() + } +} + +impl AvailabilityView for ProcessingInfo { + type BlockType = KzgCommitments; + type BlobType = KzgCommitment; + + fn block_exists(&self) -> bool { + self.kzg_commitments.is_some() + } + + fn blob_exists(&self, blob_index: u64) -> bool { + self.processing_blobs + .get(blob_index as usize) + .map(|b| b.is_some()) + .unwrap_or(false) + } + + fn num_expected_blobs(&self) -> usize { + self.kzg_commitments.as_ref().map_or(0, |c| c.len()) + } + + fn num_received_blobs(&self) -> usize { + self.processing_blobs.iter().flatten().count() + } + + fn insert_block(&mut self, block: Self::BlockType) { + let _ = self.kzg_commitments.insert(block); + } + + fn insert_blob_at_index(&mut self, blob_index: u64, blob: &Self::BlobType) { + self.processing_blobs + .get_mut(blob_index as usize) + .map(|blob| *blob = blob.clone()); + } + + fn blob_to_commitment(blob: &Self::BlobType) -> KzgCommitment { + todo!() + } + + fn get_cached_blob_commitments_mut( + &mut self, + ) -> &mut FixedVector, E::MaxBlobsPerBlock> { + todo!() + } + + fn get_block_commitment_at_index(&self, blob_index: u64) -> Option { + todo!() + } +} + +impl AvailabilityView for PendingComponents { + type BlockType = AvailabilityPendingExecutedBlock; + type BlobType = KzgVerifiedBlob; + + fn block_exists(&self) -> bool { + todo!() + } + + fn blob_exists(&self, blob_index: u64) -> bool { + todo!() + } + + fn num_expected_blobs(&self) -> usize { + todo!() + } + + fn num_received_blobs(&self) -> usize { + todo!() + } + + fn insert_block(&mut self, block: Self::BlockType) { + todo!() + } + + fn insert_blob_at_index(&mut self, blob_index: u64, blob: &Self::BlobType) { + todo!() + } + + fn blob_to_commitment(blob: &Self::BlobType) -> KzgCommitment { + todo!() + } + + fn get_cached_blob_commitments_mut( + &mut self, + ) -> &mut FixedVector, E::MaxBlobsPerBlock> { + todo!() + } + + fn get_block_commitment_at_index(&self, blob_index: u64) -> Option { + todo!() + } +} diff --git a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs index bf9736e0ac0..f7e09495a21 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs @@ -30,73 +30,74 @@ use crate::beacon_chain::BeaconStore; use crate::blob_verification::KzgVerifiedBlob; use crate::block_verification_types::{ - AsBlock, AvailabilityPendingExecutedBlock, AvailableExecutedBlock, + AsBlock, AvailabilityPendingExecutedBlock, AvailableBlock, AvailableExecutedBlock, }; -use crate::data_availability_checker::{make_available, Availability, AvailabilityCheckError}; +use crate::data_availability_checker::availability_view::AvailabilityView; +use crate::data_availability_checker::{Availability, AvailabilityCheckError}; use crate::store::{DBColumn, KeyValueStore}; use crate::BeaconChainTypes; +use kzg::KzgCommitment; use lru::LruCache; use parking_lot::{Mutex, RwLock, RwLockUpgradableReadGuard, RwLockWriteGuard}; use ssz::{Decode, Encode}; use ssz_derive::{Decode, Encode}; -use ssz_types::FixedVector; +use ssz_types::{FixedVector, VariableList}; use std::collections::HashMap; use std::{collections::HashSet, sync::Arc}; use types::beacon_block_body::KzgCommitments; -use types::blob_sidecar::BlobIdentifier; +use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; use types::{BlobSidecar, Epoch, EthSpec, Hash256}; -pub(crate) type MissingBlobInfo = ( - Option>, - FixedVector>, ::MaxBlobsPerBlock>, -); - /// This represents the components of a partially available block /// /// The blobs are all gossip and kzg verified. /// The block has completed all verifications except the availability check. -#[derive(Encode, Decode, Clone)] +#[derive(Encode, Decode, Clone, Default)] pub struct PendingComponents { verified_blobs: FixedVector>, T::MaxBlobsPerBlock>, executed_block: Option>, } impl PendingComponents { - pub fn new_from_blobs(blobs: &[KzgVerifiedBlob]) -> Self { - let mut verified_blobs = FixedVector::<_, _>::default(); - for blob in blobs { - if let Some(mut_maybe_blob) = verified_blobs.get_mut(blob.blob_index() as usize) { - *mut_maybe_blob = Some(blob.clone()); - } - } - - Self { + /// Verifies an `SignedBeaconBlock` against a set of KZG verified blobs. + /// This does not check whether a block *should* have blobs, these checks should have been + /// completed when producing the `AvailabilityPendingBlock`. + pub fn make_available(self) -> Result, AvailabilityCheckError> { + let Self { verified_blobs, - executed_block: None, - } - } + executed_block, + } = self; - pub fn new_from_block(block: AvailabilityPendingExecutedBlock) -> Self { - Self { - verified_blobs: <_>::default(), - executed_block: Some(block), - } - } + let Some(executed_block) = executed_block else { + return Err(AvailabilityCheckError::Unexpected); + }; + let num_blobs_expected = executed_block.num_blobs_expected(); + let Some(verified_blobs) = verified_blobs + .into_iter() + .cloned() + .map(|b| b.map(|b| b.to_blob())) + .take(num_blobs_expected) + .collect::>>() + else { + return Err(AvailabilityCheckError::Unexpected); + }; + let verified_blobs = VariableList::new(verified_blobs)?; - /// Returns `true` if the cache has all blobs corresponding to the - /// kzg commitments in the block. - pub fn has_all_blobs(&self, block: &AvailabilityPendingExecutedBlock) -> bool { - for i in 0..block.num_blobs_expected() { - if self - .verified_blobs - .get(i) - .map(|maybe_blob| maybe_blob.is_none()) - .unwrap_or(true) - { - return false; - } - } - true + let AvailabilityPendingExecutedBlock { + block, + import_data, + payload_verification_outcome, + } = executed_block; + + // Run consistency checks on the total + //TODO: move the full block consistency checks to `is_complete` + let available_block = AvailableBlock { + block, + blobs: Some(verified_blobs), + }; + Ok(Availability::Available(Box::new( + AvailableExecutedBlock::new(available_block, import_data, payload_verification_outcome), + ))) } pub fn empty() -> Self { @@ -121,20 +122,6 @@ impl PendingComponents { None }) } - - pub fn get_missing_blob_info(&self) -> MissingBlobInfo { - let block_opt = self - .executed_block - .as_ref() - .map(|block| block.block.clone()); - let blobs = self - .verified_blobs - .iter() - .enumerate() - .filter_map(|(i, maybe_blob)| maybe_blob.as_ref().map(|_| i)) - .collect::>(); - (block_opt, blobs) - } } /// Blocks and blobs are stored in the database sequentially so that it's @@ -409,43 +396,6 @@ impl OverflowLRUCache { }) } - /// Returns whether or not a block is in the cache (in memory or on disk) - pub fn has_block(&self, block_root: &Hash256) -> bool { - let read_lock = self.critical.read(); - if read_lock - .in_memory - .peek(block_root) - .map_or(false, |cache| cache.executed_block.is_some()) - { - true - } else if read_lock.store_keys.contains(block_root) { - drop(read_lock); - // If there's some kind of error reading from the store, we should just return false - self.overflow_store - .load_block(block_root) - .map_or(false, |maybe_block| maybe_block.is_some()) - } else { - false - } - } - - /// Fetch the missing blob info for a block without affecting the LRU ordering - pub fn get_missing_blob_info(&self, block_root: Hash256) -> MissingBlobInfo { - let read_lock = self.critical.read(); - if let Some(cache) = read_lock.in_memory.peek(&block_root) { - cache.get_missing_blob_info() - } else if read_lock.store_keys.contains(&block_root) { - drop(read_lock); - // return default if there's an error reading from the store - match self.overflow_store.load_pending_components(block_root) { - Ok(Some(pending_components)) => pending_components.get_missing_blob_info(), - _ => Default::default(), - } - } else { - Default::default() - } - } - /// Fetch a blob from the cache without affecting the LRU ordering pub fn peek_blob( &self, @@ -467,60 +417,43 @@ impl OverflowLRUCache { block_root: Hash256, kzg_verified_blobs: &[KzgVerifiedBlob], ) -> Result, AvailabilityCheckError> { + let mut fixed_blobs = FixedVector::default(); + + // Initial check to ensure all provided blobs have a consistent block root. for blob in kzg_verified_blobs { let blob_block_root = blob.block_root(); if blob_block_root != block_root { - return Err(AvailabilityCheckError::BlockBlobRootMismatch { + return Err(AvailabilityCheckError::InconsistentBlobBlockRoots { block_root, blob_block_root, }); } + fixed_blobs + .get_mut(blob.blob_index() as usize) + .map(|blob_opt| *blob_opt = Some(blob.clone())); } + let mut write_lock = self.critical.write(); - let availability = if let Some(mut pending_components) = - write_lock.pop_pending_components(block_root, &self.overflow_store)? - { - for kzg_verified_blob in kzg_verified_blobs { - let blob_index = kzg_verified_blob.blob_index() as usize; - if let Some(maybe_verified_blob) = - pending_components.verified_blobs.get_mut(blob_index) - { - //TODO: it's possible we get multiple blobs for the same block root + index - // if we don't yet have the block we don't know which is valid so we need to - // cache both - *maybe_verified_blob = Some(kzg_verified_blob.clone()) - } else { - return Err(AvailabilityCheckError::BlobIndexInvalid(blob_index as u64)); - } - } + // Grab existing entry or create a new entry. + let mut pending_components = write_lock + .pop_pending_components(block_root, &self.overflow_store)? + .unwrap_or_default(); - if let Some(executed_block) = pending_components.executed_block.take() { - self.check_block_availability_maybe_cache( - write_lock, - pending_components, - executed_block, - )? - } else { - write_lock.put_pending_components( - block_root, - pending_components, - &self.overflow_store, - )?; - Availability::MissingComponents(block_root) - } + // Merge in the blobs. + pending_components.merge_blobs(fixed_blobs); + + if pending_components.is_available() { + //TODO: add the constructor to make available? + pending_components.make_available() } else { - // not in memory or store -> put new in memory - let new_pending_components = PendingComponents::new_from_blobs(kzg_verified_blobs); write_lock.put_pending_components( block_root, - new_pending_components, + pending_components, &self.overflow_store, )?; - Availability::MissingComponents(block_root) - }; - - Ok(availability) + Ok(Availability::MissingComponents(block_root)) + } } /// Check if we have all the blobs for a block. If we do, return the Availability variant that @@ -532,90 +465,24 @@ impl OverflowLRUCache { let mut write_lock = self.critical.write(); let block_root = executed_block.import_data.block_root; - let availability = - match write_lock.pop_pending_components(block_root, &self.overflow_store)? { - Some(pending_components) => self.check_block_availability_maybe_cache( - write_lock, - pending_components, - executed_block, - )?, - None => { - let all_blob_ids = executed_block.get_all_blob_ids(); - if all_blob_ids.is_empty() { - // no blobs for this block, we can import it - let AvailabilityPendingExecutedBlock { - block, - import_data, - payload_verification_outcome, - } = executed_block; - let available_block = make_available(block, vec![])?; - return Ok(Availability::Available(Box::new( - AvailableExecutedBlock::new( - available_block, - import_data, - payload_verification_outcome, - ), - ))); - } - let new_pending_components = PendingComponents::new_from_block(executed_block); - write_lock.put_pending_components( - block_root, - new_pending_components, - &self.overflow_store, - )?; - Availability::MissingComponents(block_root) - } - }; + // Grab existing entry or create a new entry. + let mut pending_components = write_lock + .pop_pending_components(block_root, &self.overflow_store)? + .unwrap_or_default(); - Ok(availability) - } + // Merge in the block. + pending_components.merge_block(executed_block); - /// Checks if the provided `executed_block` contains all required blobs to be considered an - /// `AvailableBlock` based on blobs that are cached. - /// - /// Returns an error if there was an error when matching the block commitments against blob commitments. - /// - /// Returns `Ok(Availability::Available(_))` if all blobs for the block are present in cache. - /// Returns `Ok(Availability::MissingComponents(_))` if all corresponding blobs have not been received in the cache. - fn check_block_availability_maybe_cache( - &self, - mut write_lock: RwLockWriteGuard>, - mut pending_components: PendingComponents, - executed_block: AvailabilityPendingExecutedBlock, - ) -> Result, AvailabilityCheckError> { - if pending_components.has_all_blobs(&executed_block) { - let num_blobs_expected = executed_block.num_blobs_expected(); - let AvailabilityPendingExecutedBlock { - block, - import_data, - payload_verification_outcome, - } = executed_block; - - let Some(verified_blobs) = Vec::from(pending_components.verified_blobs) - .into_iter() - .take(num_blobs_expected) - .collect::>>() - else { - return Ok(Availability::MissingComponents(import_data.block_root)); - }; - - let available_block = make_available(block, verified_blobs)?; - Ok(Availability::Available(Box::new( - AvailableExecutedBlock::new( - available_block, - import_data, - payload_verification_outcome, - ), - ))) + // Check if we have all components and entire set is consistent. + if pending_components.is_available() { + //TODO: add the constructor to make available? + pending_components.make_available() } else { - let block_root = executed_block.import_data.block_root; - let _ = pending_components.executed_block.insert(executed_block); write_lock.put_pending_components( block_root, pending_components, &self.overflow_store, )?; - Ok(Availability::MissingComponents(block_root)) } } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs index 4c8b0cfac21..a0b91834433 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -1,15 +1,12 @@ -use crate::blob_verification::GossipVerifiedBlob; -use crate::data_availability_checker::overflow_lru_cache::MissingBlobInfo; -use crate::data_availability_checker::{Availability, AvailabilityCheckError}; -use crate::GossipVerifiedBlock; -use kzg::KzgCommitment; -use parking_lot::{Mutex, RwLock}; +use crate::data_availability_checker::AvailabilityView; use ssz_types::FixedVector; -use std::collections::{HashMap, HashSet}; +use std::collections::hash_map::Entry; +use std::collections::HashMap; use std::sync::Arc; -use types::beacon_block_body::KzgCommitments; -use types::blob_sidecar::BlobIdentifier; -use types::{BlobSidecar, EthSpec, Hash256, Slot}; +use types::beacon_block_body::{KzgCommitmentOpts, KzgCommitments}; +use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; +use types::signed_beacon_block::get_missing_blob_ids; +use types::{EthSpec, Hash256, SignedBeaconBlock, Slot}; /// This cache is used only for gossip and single lookups, to give req/resp a view of what we have /// and what we require. This cache serves a slightly different purpose than gossip caches. It @@ -23,63 +20,86 @@ use types::{BlobSidecar, EthSpec, Hash256, Slot}; /// `data_availability_cache` for import (removed as `Available`). #[derive(Default)] pub struct ProcessingCache { - //TODO: Fnv hash map? lru cache? - processing_cache: HashMap>, - processed_cache: HashSet, + processing_cache: HashMap>, +} + +impl ProcessingCache { + pub fn get(&self, block_root: &Hash256) -> Option<&ProcessingInfo> { + self.processing_cache.get(block_root) + } + pub fn entry(&mut self, block_root: Hash256) -> Entry<'_, Hash256, ProcessingInfo> { + self.processing_cache.entry(block_root) + } + pub fn remove(&mut self, block_root: &Hash256) { + self.processing_cache.remove(block_root); + } + pub fn has_block(&self, block_root: &Hash256) -> bool { + self.processing_cache + .get(block_root) + .map_or(false, |b| b.kzg_commitments.is_some()) + } + pub fn incomplete_lookups_for_slot(&self, slot: Slot) -> Vec { + let mut incomplete_lookups = vec![]; + for (&block_root, info) in self.processing_cache.iter() { + if info.slot == slot { + if info.kzg_commitments.is_none() { + incomplete_lookups.push(block_root); + } + let missing_blob_ids = info.get_missing_blob_ids(block_root); + if !missing_blob_ids.is_empty() { + incomplete_lookups.push(block_root); + } + } + } + incomplete_lookups + } } #[derive(Default)] -pub struct SimplifiedPendingComponents { +pub struct ProcessingInfo { + slot: Slot, /// Blobs required for a block can only be known if we have seen the block. So `Some` here /// means we've seen it, a `None` means we haven't. The `kzg_commitments` value is also /// necessary to verify the . - kzg_commitments: Option>, + pub kzg_commitments: Option>, /// This is an array of optional blob tree hash roots, each index in the array corresponding /// to the blob index. On insertion, a collision at an index here when `required_blobs` is /// `None` means we need to construct an entirely new `Data` entry. This is because we have /// no way of knowing which blob is the correct one until we see the block. - processing_blobs: Vec, + pub processing_blobs: KzgCommitmentOpts, } -impl SimplifiedPendingComponents { - pub fn get_missing_blob_info(&self) -> MissingBlobInfo { - todo!() - } -} - -impl ProcessingCache { - pub fn put_processing_block(&mut self, block_root: Hash256) { - self.processing_cache.insert(block_root); - } - pub fn put_processing_blob(&mut self, block_root: Hash256) { - self.processing_cache.insert(block_root); - } - - pub fn put_processed(&mut self, block_root: Hash256) { - self.processed_cache.insert(block_root); - } - - pub fn has_block(&self, block_root: &Hash256) -> bool { - self.processed_cache.contains(block_root) - || self - .processing_cache - .get(block_root) - .map_or(false, |b| b.kzg_commitments.is_some()) - } - - pub fn peek(&self, block_root: &Hash256) -> Option<&SimplifiedPendingComponents> { - self.processing_cache.get(block_root) - } - - pub fn get_missing_blob_ids(&self) -> Vec { - todo!() +impl ProcessingInfo { + pub fn from_parts( + block: Option<&Arc>>, + blobs: &FixedBlobSidecarList, + ) -> Option { + let block_slot = block.map(|block| block.message().slot()); + let blob_slot = blobs.iter().find_map(|b| b.as_ref()).map(|b| b.slot); + let slot = block_slot.or(blob_slot)?; + let block_commitments = block.map(|block| { + block + .message() + .body() + .blob_kzg_commitments() + .cloned() + .unwrap_or_default() + }); + let blobs = blobs + .iter() + .map(|blob_opt| blob_opt.as_ref().map(|blob| blob.kzg_commitment)) + .collect::>(); + Some(Self { + slot, + kzg_commitments: block_commitments, + processing_blobs: FixedVector::new(blobs).ok()?, + }) } - - pub fn remove_processing(&mut self, block_root: Hash256) { - todo!() - } - - pub fn prune(&mut self, slot: Slot) { - todo!() + pub fn get_missing_blob_ids(&self, block_root: Hash256) -> Vec { + get_missing_blob_ids::( + block_root, + self.kzg_commitments.as_ref(), + &self.processing_blobs, + ) } } diff --git a/beacon_node/beacon_chain/tests/block_verification.rs b/beacon_node/beacon_chain/tests/block_verification.rs index 5490dc88584..26178e86d4a 100644 --- a/beacon_node/beacon_chain/tests/block_verification.rs +++ b/beacon_node/beacon_chain/tests/block_verification.rs @@ -868,7 +868,7 @@ async fn block_gossip_verification() { .await .expect("should import valid gossip verified block"); if let Some(blobs) = blobs_opt { - for blob in blobs { + for blob in &blobs { let blob_index = blob.message.index; let gossip_verified = harness .chain 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 433a39feeb8..b66c2011cf9 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -1223,6 +1223,7 @@ impl NetworkBeaconProcessor { Err(BlockError::AvailabilityCheck(err)) => { match err { AvailabilityCheckError::KzgNotInitialized + | AvailabilityCheckError::Unexpected | AvailabilityCheckError::SszTypes(_) | AvailabilityCheckError::MissingBlobs | AvailabilityCheckError::StoreError(_) @@ -1235,12 +1236,8 @@ impl NetworkBeaconProcessor { } AvailabilityCheckError::Kzg(_) | AvailabilityCheckError::KzgVerificationFailed - | AvailabilityCheckError::NumBlobsMismatch { .. } | AvailabilityCheckError::BlobIndexInvalid(_) - | AvailabilityCheckError::UnorderedBlobs { .. } - | AvailabilityCheckError::BlockBlobRootMismatch { .. } - | AvailabilityCheckError::BlockBlobSlotMismatch { .. } - | AvailabilityCheckError::KzgCommitmentMismatch { .. } => { + | AvailabilityCheckError::InconsistentBlobBlockRoots { .. } => { // Note: we cannot penalize the peer that sent us the block // over gossip here because these errors imply either an issue // with: diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index 35f29a71378..9b84a64057a 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -12,7 +12,9 @@ use crate::sync::block_lookups::single_block_lookup::{ }; use crate::sync::manager::{Id, SingleLookupReqId}; use beacon_chain::block_verification_types::{AsBlock, RpcBlock}; -use beacon_chain::data_availability_checker::{AvailabilityCheckError, DataAvailabilityChecker}; +use beacon_chain::data_availability_checker::{ + AvailabilityCheckError, AvailabilityView, DataAvailabilityChecker, +}; use beacon_chain::validator_monitor::timestamp_now; use beacon_chain::{AvailabilityProcessingStatus, BeaconChainTypes, BlockError}; pub use common::Current; @@ -127,6 +129,7 @@ impl BlockLookups { ) { let lookup = self.new_current_lookup(block_root, None, &[peer_source], cx); + // TODO: can get rid of some of this if let Some(lookup) = lookup { let msg = "Searching for block"; lookup_creation_logging(msg, &lookup, peer_source, &self.log); @@ -134,24 +137,6 @@ impl BlockLookups { } } - /// 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 - /// `trigger_lookup_by_root`. - 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 { - let msg = "Initialized delayed lookup for block"; - lookup_creation_logging(msg, &lookup, peer_source, &self.log); - self.add_single_lookup(lookup) - } - } - /// 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 its parent and therefore can't fully validate it and store it in the data @@ -165,7 +150,7 @@ impl BlockLookups { peer_source: PeerShouldHave, cx: &mut SyncNetworkContext, ) { - if child_components.is_missing_components() { + if !child_components.is_available() { let lookup = self.new_current_lookup(block_root, Some(child_components), &[peer_source], cx); if let Some(lookup) = lookup { @@ -176,31 +161,6 @@ 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 - /// availability cache. - /// - /// The request is not immediately triggered, and should be triggered by a call to - /// `trigger_lookup_by_root`. - pub fn search_child_delayed( - &mut self, - block_root: Hash256, - child_components: CachedChildComponents, - peer_source: PeerShouldHave, - cx: &mut SyncNetworkContext, - ) { - if child_components.is_missing_components() { - let 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) - } - } - } - /// Attempts to trigger the request matching the given `block_root`. pub fn trigger_single_lookup( &mut self, @@ -943,9 +903,9 @@ impl BlockLookups { AvailabilityCheckError::KzgNotInitialized | AvailabilityCheckError::SszTypes(_) | AvailabilityCheckError::MissingBlobs - | AvailabilityCheckError::UnorderedBlobs { .. } | AvailabilityCheckError::StoreError(_) - | AvailabilityCheckError::DecodeError(_) => { + | AvailabilityCheckError::DecodeError(_) + | AvailabilityCheckError::Unexpected => { warn!(self.log, "Internal availability check failure"; "root" => %root, "peer_id" => %peer_id, "error" => ?e); lookup .block_request_state @@ -958,20 +918,11 @@ impl BlockLookups { lookup.request_block_and_blobs(cx)? } - // Invalid block and blob comparison. - AvailabilityCheckError::NumBlobsMismatch { .. } - | AvailabilityCheckError::KzgCommitmentMismatch { .. } - | AvailabilityCheckError::BlockBlobRootMismatch { .. } - | AvailabilityCheckError::BlockBlobSlotMismatch { .. } => { - warn!(self.log, "Availability check failure in consistency"; "root" => %root, "peer_id" => %peer_id, "error" => ?e); - lookup.handle_consistency_failure(cx); - lookup.request_block_and_blobs(cx)? - } - // Malicious errors. AvailabilityCheckError::Kzg(_) | AvailabilityCheckError::BlobIndexInvalid(_) - | AvailabilityCheckError::KzgVerificationFailed => { + | AvailabilityCheckError::KzgVerificationFailed + | AvailabilityCheckError::InconsistentBlobBlockRoots { .. } => { 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)? 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 a6ec24ee204..2fc3f9fba5f 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 @@ -3,12 +3,14 @@ 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::{AvailabilityCheckError, DataAvailabilityChecker}; +use beacon_chain::data_availability_checker::{ + AvailabilityCheckError, AvailabilityView, DataAvailabilityChecker, ProcessingInfo, +}; use beacon_chain::BeaconChainTypes; use lighthouse_network::rpc::methods::MaxRequestBlobSidecars; use lighthouse_network::{PeerAction, PeerId}; use slog::{trace, Logger}; -use ssz_types::VariableList; +use ssz_types::{FixedVector, VariableList}; use std::collections::HashSet; use std::fmt::Debug; use std::marker::PhantomData; @@ -16,7 +18,7 @@ use std::sync::Arc; use store::Hash256; use strum::IntoStaticStr; use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; -use types::{EthSpec, SignedBeaconBlock}; +use types::{BlobSidecar, EthSpec, KzgCommitment, SignedBeaconBlock}; #[derive(Debug, PartialEq, Eq)] pub enum State { @@ -267,18 +269,14 @@ impl SingleBlockLookup { /// 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 { + let block_root = self.block_root(); 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), - ) + .get_missing_blob_ids(block_root, &components.processing_info()) .unwrap_or_default() } else { self.da_checker - .get_missing_blob_ids_checking_cache(self.block_root()) + .get_missing_blob_ids_checking_cache(block_root) .unwrap_or_default() } } @@ -445,33 +443,59 @@ impl CachedChildComponents { } pub fn add_cached_child_block(&mut self, block: Arc>) { - self.downloaded_block = Some(block); + self.merge_block(block) } 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()); - } - } + self.merge_blobs(blobs) } - pub fn downloaded_indices(&self) -> HashSet { - self.downloaded_blobs - .iter() - .enumerate() - .filter_map(|(i, blob_opt)| blob_opt.as_ref().map(|_| i)) - .collect::>() + pub fn processing_info(&self) -> ProcessingInfo { + ProcessingInfo::from_parts(self.downloaded_block.as_ref(), &self.downloaded_blobs) + .unwrap_or_default() } +} - 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) +impl AvailabilityView for CachedChildComponents { + type BlockType = Arc>; + type BlobType = Arc>; + + fn block_exists(&self) -> bool { + todo!() + } + + fn blob_exists(&self, blob_index: u64) -> bool { + todo!() + } + + fn num_expected_blobs(&self) -> usize { + todo!() + } + + fn num_received_blobs(&self) -> usize { + todo!() + } + + fn insert_block(&mut self, block: Self::BlockType) { + todo!() + } + + fn insert_blob_at_index(&mut self, blob_index: u64, blob: &Self::BlobType) { + todo!() + } + + fn blob_to_commitment(blob: &Self::BlobType) -> KzgCommitment { + todo!() + } + + fn get_cached_blob_commitments_mut( + &mut self, + ) -> &mut FixedVector, E::MaxBlobsPerBlock> { + todo!() + } + + fn get_block_commitment_at_index(&self, blob_index: u64) -> Option { + todo!() } } diff --git a/consensus/types/src/beacon_block_body.rs b/consensus/types/src/beacon_block_body.rs index 61400a8b4b7..d5a220a5d9f 100644 --- a/consensus/types/src/beacon_block_body.rs +++ b/consensus/types/src/beacon_block_body.rs @@ -11,6 +11,8 @@ use tree_hash_derive::TreeHash; pub type KzgCommitments = VariableList::MaxBlobCommitmentsPerBlock>; +pub type KzgCommitmentOpts = + FixedVector, ::MaxBlobsPerBlock>; /// The body of a `BeaconChain` block, containing operations. /// diff --git a/consensus/types/src/blob_sidecar.rs b/consensus/types/src/blob_sidecar.rs index a48c38421fa..e72bc8f7c2e 100644 --- a/consensus/types/src/blob_sidecar.rs +++ b/consensus/types/src/blob_sidecar.rs @@ -25,6 +25,19 @@ pub struct BlobIdentifier { pub index: u64, } +impl BlobIdentifier { + pub fn get_all_blob_ids(block_root: Hash256) -> Vec { + let mut blob_ids = Vec::with_capacity(E::max_blobs_per_block()); + for i in 0..E::max_blobs_per_block() { + blob_ids.push(BlobIdentifier { + block_root, + index: i as u64, + }); + } + blob_ids + } +} + impl PartialOrd for BlobIdentifier { fn partial_cmp(&self, other: &Self) -> Option { self.index.partial_cmp(&other.index) diff --git a/consensus/types/src/signed_beacon_block.rs b/consensus/types/src/signed_beacon_block.rs index c52ba11d278..e2f37266b8e 100644 --- a/consensus/types/src/signed_beacon_block.rs +++ b/consensus/types/src/signed_beacon_block.rs @@ -1,3 +1,4 @@ +use crate::beacon_block_body::{KzgCommitmentOpts, KzgCommitments}; use crate::blob_sidecar::BlobIdentifier; use crate::*; use bls::Signature; @@ -257,29 +258,38 @@ impl> SignedBeaconBlock .map(|c| c.len()) .unwrap_or(0) } +} - pub fn get_expected_blob_ids(&self, block_root: Option) -> Vec { - self.get_filtered_blob_ids(block_root, |_, _| true) - } - - /// If the filter returns `true` the id for the corresponding index and root will be included. - pub fn get_filtered_blob_ids( - &self, - block_root: Option, - filter: impl Fn(usize, Hash256) -> bool, - ) -> Vec { - let block_root = block_root.unwrap_or_else(|| self.canonical_root()); - let num_blobs_expected = self.num_expected_blobs(); - let mut blob_ids = Vec::with_capacity(num_blobs_expected); - for i in 0..num_blobs_expected { - if filter(i, block_root) { - blob_ids.push(BlobIdentifier { - block_root, - index: i as u64, - }); +pub fn get_missing_blob_ids( + block_root: Hash256, + required_commitments: Option<&KzgCommitments>, + buffered_commitments: &KzgCommitmentOpts, +) -> Vec { + match required_commitments { + Some(required_commitments) => { + let num_blobs_expected = required_commitments.len(); + let mut blob_ids = Vec::with_capacity(num_blobs_expected); + + // Zip here will always limit the number of iterations to the size of + // `required_commitments` because `buffered_commitments` will always be populated + // with `Option` values up to `MAX_BLOBS_PER_BLOCK`. + for (index, (req_commitment, buffered_commitment_opt)) in required_commitments + .into_iter() + .zip(buffered_commitments.iter()) + .enumerate() + { + if buffered_commitment_opt.map_or(true, |buffered_commitment| { + buffered_commitment != *req_commitment + }) { + blob_ids.push(BlobIdentifier { + block_root, + index: index as u64, + }); + } } + blob_ids } - blob_ids + None => BlobIdentifier::get_all_blob_ids::(block_root), } } From 654485ff99448ed15ce6c676662ea381f150be8c Mon Sep 17 00:00:00 2001 From: realbigsean Date: Tue, 19 Sep 2023 13:04:59 -0400 Subject: [PATCH 04/47] fix lints --- .../beacon_chain/src/blob_verification.rs | 2 - .../availability_view.rs | 52 ++++++++++++------- .../overflow_lru_cache.rs | 28 ++-------- .../processing_cache.rs | 1 - .../gossip_methods.rs | 2 +- .../src/network_beacon_processor/mod.rs | 1 - .../src/sync/block_lookups/delayed_lookup.rs | 2 - .../sync/block_lookups/single_block_lookup.rs | 27 ++++++---- beacon_node/network/src/sync/manager.rs | 1 - 9 files changed, 56 insertions(+), 60 deletions(-) diff --git a/beacon_node/beacon_chain/src/blob_verification.rs b/beacon_node/beacon_chain/src/blob_verification.rs index dd7fbc23998..870fe57b8b7 100644 --- a/beacon_node/beacon_chain/src/blob_verification.rs +++ b/beacon_node/beacon_chain/src/blob_verification.rs @@ -148,7 +148,6 @@ pub type GossipVerifiedBlobList = VariableList< /// the p2p network. #[derive(Debug)] pub struct GossipVerifiedBlob { - blob_root: Hash256, blob: SignedBlobSidecar, } @@ -431,7 +430,6 @@ pub fn validate_blob_sidecar_for_gossip( } Ok(GossipVerifiedBlob { - blob_root, blob: signed_blob_sidecar, }) } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index 7d743a1225f..fe3af21e63b 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -1,12 +1,10 @@ use crate::blob_verification::KzgVerifiedBlob; use crate::data_availability_checker::overflow_lru_cache::PendingComponents; -use crate::data_availability_checker::processing_cache::ProcessingCache; use crate::data_availability_checker::ProcessingInfo; use crate::AvailabilityPendingExecutedBlock; -use bls::Hash256; use kzg::KzgCommitment; use ssz_types::FixedVector; -use types::beacon_block_body::{KzgCommitmentOpts, KzgCommitments}; +use types::beacon_block_body::{ KzgCommitments}; use types::EthSpec; /// This trait is meant to ensure we maintain the following invariants across caches used in @@ -53,7 +51,7 @@ pub trait AvailabilityView { } fn merge_block(&mut self, block: Self::BlockType) { self.insert_block(block); - let mut cached = self.get_cached_blob_commitments_mut(); + let cached = self.get_cached_blob_commitments_mut(); let mut reinsert = FixedVector::default(); for (index, cached_blob) in cached.iter_mut().enumerate() { // Take the existing blobs and re-insert them. @@ -97,23 +95,25 @@ impl AvailabilityView for ProcessingInfo { } fn insert_blob_at_index(&mut self, blob_index: u64, blob: &Self::BlobType) { - self.processing_blobs - .get_mut(blob_index as usize) - .map(|blob| *blob = blob.clone()); + if let Some(b) = self.processing_blobs.get_mut(blob_index as usize) { + *b = Some(blob.clone()); + } } fn blob_to_commitment(blob: &Self::BlobType) -> KzgCommitment { - todo!() + *blob } fn get_cached_blob_commitments_mut( &mut self, ) -> &mut FixedVector, E::MaxBlobsPerBlock> { - todo!() + &mut self.processing_blobs } fn get_block_commitment_at_index(&self, blob_index: u64) -> Option { - todo!() + self.processing_blobs + .get(blob_index as usize) + .and_then(|b| b.clone()) } } @@ -122,40 +122,54 @@ impl AvailabilityView for PendingComponents { type BlobType = KzgVerifiedBlob; fn block_exists(&self) -> bool { - todo!() + self.executed_block.is_some() } fn blob_exists(&self, blob_index: u64) -> bool { - todo!() + self.verified_blobs + .get(blob_index as usize) + .map(|b| b.is_some()) + .unwrap_or(false) } fn num_expected_blobs(&self) -> usize { - todo!() + self.executed_block + .as_ref() + .map_or(0, |b| b.num_blobs_expected()) } fn num_received_blobs(&self) -> usize { - todo!() + self.verified_blobs.iter().flatten().count() } fn insert_block(&mut self, block: Self::BlockType) { - todo!() + self.executed_block = Some(block); } fn insert_blob_at_index(&mut self, blob_index: u64, blob: &Self::BlobType) { - todo!() + if let Some(b) = self.verified_blobs.get_mut(blob_index as usize) { + *b = Some(blob.clone()); + } } fn blob_to_commitment(blob: &Self::BlobType) -> KzgCommitment { - todo!() + blob.as_blob().kzg_commitment } fn get_cached_blob_commitments_mut( &mut self, ) -> &mut FixedVector, E::MaxBlobsPerBlock> { - todo!() + &mut self.verified_blobs } fn get_block_commitment_at_index(&self, blob_index: u64) -> Option { - todo!() + self.executed_block.as_ref().and_then(|b| { + b.block + .message() + .body() + .blob_kzg_commitments() + .ok() + .and_then(|c| c.get(blob_index as usize).cloned()) + }) } } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs index f7e09495a21..5a348df2f85 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs @@ -36,16 +36,13 @@ use crate::data_availability_checker::availability_view::AvailabilityView; use crate::data_availability_checker::{Availability, AvailabilityCheckError}; use crate::store::{DBColumn, KeyValueStore}; use crate::BeaconChainTypes; -use kzg::KzgCommitment; use lru::LruCache; -use parking_lot::{Mutex, RwLock, RwLockUpgradableReadGuard, RwLockWriteGuard}; +use parking_lot::{Mutex, RwLock, RwLockUpgradableReadGuard, }; use ssz::{Decode, Encode}; use ssz_derive::{Decode, Encode}; use ssz_types::{FixedVector, VariableList}; -use std::collections::HashMap; use std::{collections::HashSet, sync::Arc}; -use types::beacon_block_body::KzgCommitments; -use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; +use types::blob_sidecar::{BlobIdentifier, }; use types::{BlobSidecar, Epoch, EthSpec, Hash256}; /// This represents the components of a partially available block @@ -54,8 +51,8 @@ use types::{BlobSidecar, Epoch, EthSpec, Hash256}; /// The block has completed all verifications except the availability check. #[derive(Encode, Decode, Clone, Default)] pub struct PendingComponents { - verified_blobs: FixedVector>, T::MaxBlobsPerBlock>, - executed_block: Option>, + pub verified_blobs: FixedVector>, T::MaxBlobsPerBlock>, + pub executed_block: Option>, } impl PendingComponents { @@ -237,23 +234,6 @@ impl OverflowStore { Ok(disk_keys) } - /// Load a single block from the database (ignoring blobs) - pub fn load_block( - &self, - block_root: &Hash256, - ) -> Result>, AvailabilityCheckError> { - let key = OverflowKey::from_block_root(*block_root); - - self.0 - .hot_db - .get_bytes(DBColumn::OverflowLRUCache.as_str(), &key.as_ssz_bytes())? - .map(|block_bytes| { - AvailabilityPendingExecutedBlock::from_ssz_bytes(block_bytes.as_slice()) - }) - .transpose() - .map_err(|e| e.into()) - } - /// Load a single blob from the database pub fn load_blob( &self, diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs index a0b91834433..af365b4937a 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -1,4 +1,3 @@ -use crate::data_availability_checker::AvailabilityView; use ssz_types::FixedVector; use std::collections::hash_map::Entry; use std::collections::HashMap; 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 b66c2011cf9..b2361a4f7b2 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -752,7 +752,7 @@ impl NetworkBeaconProcessor { ); self.chain.recompute_head_at_current_slot().await; } - Ok(AvailabilityProcessingStatus::MissingComponents(slot, block_root)) => { + Ok(AvailabilityProcessingStatus::MissingComponents(_slot, block_root)) => { if delay_lookup { let mut guard = self.delayed_lookup_peers.lock(); if let Some(peers) = guard.get_mut(&block_root) { diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 1208cc1c3fd..2419db6c283 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -22,7 +22,6 @@ use lru::LruCache; use parking_lot::Mutex; use slog::{debug, Logger}; use slot_clock::ManualSlotClock; -use std::collections::HashMap; use std::path::PathBuf; use std::sync::Arc; use std::time::Duration; diff --git a/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs b/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs index 507aa9d1591..74d21fc4226 100644 --- a/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs @@ -3,10 +3,8 @@ use beacon_chain::{BeaconChain, BeaconChainTypes}; use slog::{crit, error, trace}; use slot_clock::SlotClock; use std::sync::Arc; -use tokio::sync::mpsc; use tokio::time::interval_at; use tokio::time::Instant; -use types::Hash256; /// This service is responsible for collecting lookup messages and sending them back to sync /// for processing after a short delay. 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 2fc3f9fba5f..158b70d2eda 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 @@ -461,41 +461,50 @@ impl AvailabilityView for CachedChildComponents { type BlobType = Arc>; fn block_exists(&self) -> bool { - todo!() + self.downloaded_block.is_some() } fn blob_exists(&self, blob_index: u64) -> bool { - todo!() + self.downloaded_blobs + .get(blob_index as usize) + .map(|b| b.is_some()) + .unwrap_or(false) } fn num_expected_blobs(&self) -> usize { - todo!() + self.downloaded_block + .as_ref() + .map_or(0, |b| b.num_expected_blobs()) } fn num_received_blobs(&self) -> usize { - todo!() + self.downloaded_blobs.iter().flatten().count() } fn insert_block(&mut self, block: Self::BlockType) { - todo!() + self.downloaded_block = Some(block); } fn insert_blob_at_index(&mut self, blob_index: u64, blob: &Self::BlobType) { - todo!() + if let Some(b) = self.downloaded_blobs.get_mut(blob_index as usize) { + *b = Some(blob.clone()); + } } fn blob_to_commitment(blob: &Self::BlobType) -> KzgCommitment { - todo!() + blob.kzg_commitment } fn get_cached_blob_commitments_mut( &mut self, ) -> &mut FixedVector, E::MaxBlobsPerBlock> { - todo!() + &mut self.downloaded_blobs } fn get_block_commitment_at_index(&self, blob_index: u64) -> Option { - todo!() + self.downloaded_blobs + .get(blob_index as usize) + .and_then(|b| b.as_ref().map(|b| b.kzg_commitment)) } } diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 6749abf9f80..9d0c8b5ab5b 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -57,7 +57,6 @@ use lighthouse_network::types::{NetworkGlobals, SyncState}; use lighthouse_network::SyncInfo; use lighthouse_network::{PeerAction, PeerId}; use slog::{crit, debug, error, info, trace, warn, Logger}; -use slot_clock::SlotClock; use std::boxed::Box; use std::ops::IndexMut; use std::ops::Sub; From 0ad729cbcf67a433a3c09e725fe76f1b45c4ee2c Mon Sep 17 00:00:00 2001 From: realbigsean Date: Tue, 19 Sep 2023 13:17:05 -0400 Subject: [PATCH 05/47] add docs to `AvailabilityView` --- .../availability_view.rs | 65 ++++++++++++++++--- .../overflow_lru_cache.rs | 4 +- 2 files changed, 58 insertions(+), 11 deletions(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index fe3af21e63b..25294ab219d 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -4,33 +4,71 @@ use crate::data_availability_checker::ProcessingInfo; use crate::AvailabilityPendingExecutedBlock; use kzg::KzgCommitment; use ssz_types::FixedVector; -use types::beacon_block_body::{ KzgCommitments}; +use types::beacon_block_body::KzgCommitments; use types::EthSpec; -/// This trait is meant to ensure we maintain the following invariants across caches used in -/// availability checking: +/// Defines an interface for managing data availability with two key invariants: +/// 1. Blobs won't be clobbered if we've yet to see the corresponding block. +/// 2. On block insertion, any non-matching blob commitments are evicted. /// -/// 1. Never clobber blobs when adding new blobs. -/// 2. When adding a block, evict all blobs whose KZG commitments do not match the block's. +/// Types implementing this trait can be used for validating and managing availability +/// of blocks and blobs in a cache-like data structure. pub trait AvailabilityView { + /// The type representing a block in the implementation. type BlockType; + + /// The type representing a blob in the implementation. Must implement `Clone`. type BlobType: Clone; + /// Checks if a block exists in the cache. + /// + /// Returns: + /// - `true` if a block exists. + /// - `false` otherwise. fn block_exists(&self) -> bool; + + /// Checks if a blob exists at the given index in the cache. + /// + /// Returns: + /// - `true` if a blob exists at the given index. + /// - `false` otherwise. fn blob_exists(&self, blob_index: u64) -> bool; + + /// Returns the number of blobs that are expected to be present. Returns 0 if we don't have a + /// block. + /// + /// This corresponds to the number of commitments that are present in a block. fn num_expected_blobs(&self) -> usize; + + /// Returns the number of blobs that have been received and are stored in the cache. fn num_received_blobs(&self) -> usize; + + /// Inserts a block into the cache. fn insert_block(&mut self, block: Self::BlockType); + + /// Inserts a blob at a specific index in the cache. + /// + /// Existing blob at the index will be replaced. fn insert_blob_at_index(&mut self, blob_index: u64, blob: &Self::BlobType); + + /// Converts a blob to its KZG commitment. fn blob_to_commitment(blob: &Self::BlobType) -> KzgCommitment; + + /// Provides mutable access to the underlying blob commitments cache. fn get_cached_blob_commitments_mut( &mut self, ) -> &mut FixedVector, E::MaxBlobsPerBlock>; + + /// Retrieves the KZG commitment of the blob stored at the given index in the block. + /// + /// Returns `None` if no blob is present at the index. fn get_block_commitment_at_index(&self, blob_index: u64) -> Option; - /// validate the index - /// only insert if: - /// 1. the blob entry is empty and there is no block - /// 2. the block exists and the commitment matches + + /// Merges a given set of blobs into the cache. + /// + /// Blobs are only inserted if: + /// 1. The blob entry at the index is empty and no block exists. + /// 2. The block exists and its commitment matches the blob's commitment. fn merge_blobs(&mut self, blobs: FixedVector, E::MaxBlobsPerBlock>) { for (index, blob) in blobs.into_iter().enumerate() { let Some(blob) = blob else { continue }; @@ -49,6 +87,10 @@ pub trait AvailabilityView { } } } + + /// Inserts a new block and revalidates the existing blobs against it. + /// + /// Blobs that don't match the new block's commitments are evicted. fn merge_block(&mut self, block: Self::BlockType) { self.insert_block(block); let cached = self.get_cached_blob_commitments_mut(); @@ -62,6 +104,11 @@ pub trait AvailabilityView { self.merge_blobs(reinsert) } + + /// Checks if the block and all of its expected blobs are available in the cache. + /// + /// Returns `true` if both the block exists and the number of received blobs matches the number + /// of expected blobs. fn is_available(&self) -> bool { self.block_exists() && self.num_expected_blobs() == self.num_received_blobs() } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs index 5a348df2f85..5b00ab555eb 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs @@ -37,12 +37,12 @@ use crate::data_availability_checker::{Availability, AvailabilityCheckError}; use crate::store::{DBColumn, KeyValueStore}; use crate::BeaconChainTypes; use lru::LruCache; -use parking_lot::{Mutex, RwLock, RwLockUpgradableReadGuard, }; +use parking_lot::{Mutex, RwLock, RwLockUpgradableReadGuard}; use ssz::{Decode, Encode}; use ssz_derive::{Decode, Encode}; use ssz_types::{FixedVector, VariableList}; use std::{collections::HashSet, sync::Arc}; -use types::blob_sidecar::{BlobIdentifier, }; +use types::blob_sidecar::BlobIdentifier; use types::{BlobSidecar, Epoch, EthSpec, Hash256}; /// This represents the components of a partially available block From e25c609fcfb17a86ea30b60a0118059ce0247936 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Tue, 19 Sep 2023 16:24:57 -0400 Subject: [PATCH 06/47] some self review --- .../src/block_verification_types.rs | 20 +++++++-- .../src/data_availability_checker.rs | 41 ++++--------------- .../overflow_lru_cache.rs | 4 -- .../processing_cache.rs | 36 ++++++---------- .../gossip_methods.rs | 1 + .../network/src/sync/block_lookups/mod.rs | 1 + 6 files changed, 39 insertions(+), 64 deletions(-) diff --git a/beacon_node/beacon_chain/src/block_verification_types.rs b/beacon_node/beacon_chain/src/block_verification_types.rs index 1b8458e4477..78c8fc628fd 100644 --- a/beacon_node/beacon_chain/src/block_verification_types.rs +++ b/beacon_node/beacon_chain/src/block_verification_types.rs @@ -3,7 +3,7 @@ use crate::block_verification::BlockError; use crate::data_availability_checker::AvailabilityCheckError; pub use crate::data_availability_checker::{AvailableBlock, MaybeAvailableBlock}; use crate::eth1_finalization_cache::Eth1FinalizationData; -use crate::{data_availability_checker, GossipVerifiedBlock, PayloadVerificationOutcome}; +use crate::{GossipVerifiedBlock, PayloadVerificationOutcome}; use derivative::Derivative; use ssz_derive::{Decode, Encode}; use ssz_types::VariableList; @@ -65,8 +65,22 @@ impl RpcBlock { block: Arc>, blobs: Option>, ) -> Result { - if let Some(blobs) = blobs.as_ref() { - data_availability_checker::consistency_checks(&block, blobs)?; + if let (Some(blobs), Ok(block_commitments)) = ( + blobs.as_ref(), + block.message().body().blob_kzg_commitments(), + ) { + if blobs.len() != block_commitments.len() { + return Err(AvailabilityCheckError::MissingBlobs); + } + for (blob, &block_commitment) in blobs.iter().zip(block_commitments.iter()) { + let blob_commitment = blob.kzg_commitment; + if blob_commitment != block_commitment { + return Err(AvailabilityCheckError::KzgCommitmentMismatch { + block_commitment, + blob_commitment, + }); + } + } } let inner = match blobs { Some(blobs) => RpcBlockInner::BlockAndBlobs(block, blobs), diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index f205f4fa4a9..e905eb5ddc3 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -6,8 +6,8 @@ pub use crate::data_availability_checker::availability_view::AvailabilityView; use crate::data_availability_checker::overflow_lru_cache::OverflowLRUCache; use crate::data_availability_checker::processing_cache::ProcessingCache; use crate::{BeaconChain, BeaconChainTypes, BeaconStore}; -use kzg::Error as KzgError; use kzg::Kzg; +use kzg::{Error as KzgError, KzgCommitment}; use parking_lot::RwLock; pub use processing_cache::ProcessingInfo; use slog::{debug, error}; @@ -39,6 +39,10 @@ pub enum AvailabilityCheckError { Kzg(KzgError), KzgNotInitialized, KzgVerificationFailed, + KzgCommitmentMismatch { + blob_commitment: KzgCommitment, + block_commitment: KzgCommitment, + }, Unexpected, SszTypes(ssz_types::Error), MissingBlobs, @@ -248,12 +252,7 @@ impl DataAvailabilityChecker { /// blobs?". fn blobs_required_for_block(&self, block: &SignedBeaconBlock) -> bool { let block_within_da_period = self.da_check_required_for_epoch(block.epoch()); - let block_has_kzg_commitments = block - .message() - .body() - .blob_kzg_commitments() - .map_or(false, |commitments| !commitments.is_empty()); - block_within_da_period && block_has_kzg_commitments + block.num_expected_blobs() > 0 && block_within_da_period } pub fn notify_block_commitments( @@ -287,7 +286,7 @@ impl DataAvailabilityChecker { pub fn get_delayed_lookups(&self, slot: Slot) -> Vec { self.processing_cache .read() - .incomplete_lookups_for_slot(slot) + .blocks_with_missing_components(slot) } pub fn should_delay_lookup(&self, slot: Slot) -> bool { @@ -354,32 +353,6 @@ impl DataAvailabilityChecker { } } -/// Makes the following checks to ensure that the list of blobs correspond block: -/// -/// * Check that a block is post-deneb -/// * Checks that the number of blobs is equal to the length of kzg commitments in the list -/// * Checks that the index, slot, root and kzg_commitment in the block match the blobs in the correct order -/// -/// Returns `Ok(())` if all consistency checks pass and an error otherwise. -pub fn consistency_checks( - block: &SignedBeaconBlock, - blobs: &[Arc>], -) -> Result<(), AvailabilityCheckError> { - let Ok(block_kzg_commitments) = block.message().body().blob_kzg_commitments() else { - return Ok(()); - }; - - if block_kzg_commitments.is_empty() { - return Ok(()); - } - - if blobs.len() != block_kzg_commitments.len() { - return Err(AvailabilityCheckError::MissingBlobs); - } - - Ok(()) -} - pub fn start_availability_cache_maintenance_service( executor: TaskExecutor, chain: Arc>, diff --git a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs index 5b00ab555eb..a0cb7cd7954 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs @@ -86,8 +86,6 @@ impl PendingComponents { payload_verification_outcome, } = executed_block; - // Run consistency checks on the total - //TODO: move the full block consistency checks to `is_complete` let available_block = AvailableBlock { block, blobs: Some(verified_blobs), @@ -424,7 +422,6 @@ impl OverflowLRUCache { pending_components.merge_blobs(fixed_blobs); if pending_components.is_available() { - //TODO: add the constructor to make available? pending_components.make_available() } else { write_lock.put_pending_components( @@ -455,7 +452,6 @@ impl OverflowLRUCache { // Check if we have all components and entire set is consistent. if pending_components.is_available() { - //TODO: add the constructor to make available? pending_components.make_available() } else { write_lock.put_pending_components( diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs index af365b4937a..f6bb61045b6 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -1,3 +1,4 @@ +use crate::data_availability_checker::AvailabilityView; use ssz_types::FixedVector; use std::collections::hash_map::Entry; use std::collections::HashMap; @@ -7,16 +8,10 @@ use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; use types::signed_beacon_block::get_missing_blob_ids; use types::{EthSpec, Hash256, SignedBeaconBlock, Slot}; -/// This cache is used only for gossip and single lookups, to give req/resp a view of what we have -/// and what we require. This cache serves a slightly different purpose than gossip caches. It -/// tracks all unique messages we're currently processing, or have already processed. This should -/// be used in conjunction with the `data_availability_cache` to have a full view of processing -/// statuses. -/// -/// Components should be atomically removed when being added to the data availability cache. -/// -/// Components should be atomically inserted into the `processed_cache` when removed from the -/// `data_availability_cache` for import (removed as `Available`). +/// This cache is used only for gossip blocks/blobs and single block/blob lookups, to give req/resp +/// a view of what we have and what we require. This cache serves a slightly different purpose than +/// gossip caches because it allows us to process duplicate blobs that are valid in gossip. +/// See `AvailabilityView`'s trait definition. #[derive(Default)] pub struct ProcessingCache { processing_cache: HashMap>, @@ -37,20 +32,16 @@ impl ProcessingCache { .get(block_root) .map_or(false, |b| b.kzg_commitments.is_some()) } - pub fn incomplete_lookups_for_slot(&self, slot: Slot) -> Vec { - let mut incomplete_lookups = vec![]; + pub fn blocks_with_missing_components(&self, slot: Slot) -> Vec { + let mut roots_missing_components = vec![]; for (&block_root, info) in self.processing_cache.iter() { if info.slot == slot { - if info.kzg_commitments.is_none() { - incomplete_lookups.push(block_root); - } - let missing_blob_ids = info.get_missing_blob_ids(block_root); - if !missing_blob_ids.is_empty() { - incomplete_lookups.push(block_root); + if !info.is_available() { + roots_missing_components.push(block_root); } } } - incomplete_lookups + roots_missing_components } } @@ -58,11 +49,10 @@ impl ProcessingCache { pub struct ProcessingInfo { slot: Slot, /// Blobs required for a block can only be known if we have seen the block. So `Some` here - /// means we've seen it, a `None` means we haven't. The `kzg_commitments` value is also - /// necessary to verify the . + /// means we've seen it, a `None` means we haven't. The `kzg_commitments` value helps us figure + /// out whether incoming blobs actually match the block. pub kzg_commitments: Option>, - /// This is an array of optional blob tree hash roots, each index in the array corresponding - /// to the blob index. On insertion, a collision at an index here when `required_blobs` is + /// On insertion, a collision at an index here when `required_blobs` is /// `None` means we need to construct an entirely new `Data` entry. This is because we have /// no way of knowing which blob is the correct one until we see the block. pub processing_blobs: KzgCommitmentOpts, 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 b2361a4f7b2..22d8b999868 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -1236,6 +1236,7 @@ impl NetworkBeaconProcessor { } AvailabilityCheckError::Kzg(_) | AvailabilityCheckError::KzgVerificationFailed + | AvailabilityCheckError::KzgCommitmentMismatch { .. } | AvailabilityCheckError::BlobIndexInvalid(_) | AvailabilityCheckError::InconsistentBlobBlockRoots { .. } => { // Note: we cannot penalize the peer that sent us the block diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index 9b84a64057a..b206143c2de 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -921,6 +921,7 @@ impl BlockLookups { // Malicious errors. AvailabilityCheckError::Kzg(_) | AvailabilityCheckError::BlobIndexInvalid(_) + | AvailabilityCheckError::KzgCommitmentMismatch { .. } | AvailabilityCheckError::KzgVerificationFailed | AvailabilityCheckError::InconsistentBlobBlockRoots { .. } => { warn!(self.log, "Availability check failure"; "root" => %root, "peer_id" => %peer_id, "error" => ?e); From 77afea30844fdcf9d47019dc9649cd08e8e407a2 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Tue, 19 Sep 2023 17:02:31 -0400 Subject: [PATCH 07/47] fix lints --- beacon_node/beacon_chain/src/beacon_chain.rs | 16 +++++++--------- .../availability_view.rs | 16 +++++++--------- .../overflow_lru_cache.rs | 6 +++--- .../processing_cache.rs | 6 ++---- 4 files changed, 19 insertions(+), 25 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 2640844baba..322c753bb6b 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -2798,9 +2798,9 @@ impl BeaconChain { ) -> Result> { let block_root = blob.block_root(); let mut commitments = KzgCommitmentOpts::::default(); - commitments - .get_mut(blob.as_blob().index as usize) - .map(|b| *b = Some(blob.as_blob().kzg_commitment)); + if let Some(commitment_opt) = commitments.get_mut(blob.as_blob().index as usize) { + *commitment_opt = Some(blob.as_blob().kzg_commitment); + } self.data_availability_checker .notify_blob_commitments(block_root, commitments); let r = self.check_gossip_blob_availability_and_import(blob).await; @@ -2814,11 +2814,9 @@ impl BeaconChain { blobs: FixedBlobSidecarList, ) -> Result> { let mut commitments = KzgCommitmentOpts::::default(); - for blob in &blobs { - if let Some(blob) = blob { - commitments - .get_mut(blob.index as usize) - .map(|b| *b = Some(blob.kzg_commitment)); + for blob in blobs.iter().flatten() { + if let Some(commitment) = commitments.get_mut(blob.index as usize) { + *commitment = Some(blob.kzg_commitment); } } self.data_availability_checker @@ -2837,7 +2835,7 @@ impl BeaconChain { let has_missing_components = matches!(r, Ok(AvailabilityProcessingStatus::MissingComponents(_, _))); if !has_missing_components { - self.data_availability_checker.remove_notified(&block_root); + self.data_availability_checker.remove_notified(block_root); } r } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index 25294ab219d..b1e1772c55e 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -80,10 +80,8 @@ pub trait AvailabilityView { if block_commitment == commitment { self.insert_blob_at_index(index, blob) } - } else { - if !self.blob_exists(index) { - self.insert_blob_at_index(index, blob) - } + } else if !self.blob_exists(index) { + self.insert_blob_at_index(index, blob) } } } @@ -97,9 +95,9 @@ pub trait AvailabilityView { let mut reinsert = FixedVector::default(); for (index, cached_blob) in cached.iter_mut().enumerate() { // Take the existing blobs and re-insert them. - reinsert - .get_mut(index) - .map(|blob| *blob = cached_blob.take()); + if let Some(blob) = reinsert.get_mut(index) { + *blob = cached_blob.take() + } } self.merge_blobs(reinsert) @@ -143,7 +141,7 @@ impl AvailabilityView for ProcessingInfo { fn insert_blob_at_index(&mut self, blob_index: u64, blob: &Self::BlobType) { if let Some(b) = self.processing_blobs.get_mut(blob_index as usize) { - *b = Some(blob.clone()); + *b = Some(*blob); } } @@ -160,7 +158,7 @@ impl AvailabilityView for ProcessingInfo { fn get_block_commitment_at_index(&self, blob_index: u64) -> Option { self.processing_blobs .get(blob_index as usize) - .and_then(|b| b.clone()) + .and_then(|b| *b) } } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs index a0cb7cd7954..c422bcf2483 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs @@ -406,9 +406,9 @@ impl OverflowLRUCache { blob_block_root, }); } - fixed_blobs - .get_mut(blob.blob_index() as usize) - .map(|blob_opt| *blob_opt = Some(blob.clone())); + if let Some(blob_opt) = fixed_blobs.get_mut(blob.blob_index() as usize) { + *blob_opt = Some(blob.clone()); + } } let mut write_lock = self.critical.write(); diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs index f6bb61045b6..3fabef5b088 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -35,10 +35,8 @@ impl ProcessingCache { pub fn blocks_with_missing_components(&self, slot: Slot) -> Vec { let mut roots_missing_components = vec![]; for (&block_root, info) in self.processing_cache.iter() { - if info.slot == slot { - if !info.is_available() { - roots_missing_components.push(block_root); - } + if info.slot == slot && !info.is_available() { + roots_missing_components.push(block_root); } } roots_missing_components From 2402b7911f9a0907974793cf5b748bf1bb28087c Mon Sep 17 00:00:00 2001 From: realbigsean Date: Tue, 19 Sep 2023 18:03:47 -0400 Subject: [PATCH 08/47] fix beacon chain tests --- beacon_node/beacon_chain/tests/block_verification.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/beacon_chain/tests/block_verification.rs b/beacon_node/beacon_chain/tests/block_verification.rs index 26178e86d4a..5490dc88584 100644 --- a/beacon_node/beacon_chain/tests/block_verification.rs +++ b/beacon_node/beacon_chain/tests/block_verification.rs @@ -868,7 +868,7 @@ async fn block_gossip_verification() { .await .expect("should import valid gossip verified block"); if let Some(blobs) = blobs_opt { - for blob in &blobs { + for blob in blobs { let blob_index = blob.message.index; let gossip_verified = harness .chain From aeeddfc0a05a98ab0b6cb32abd08923558992454 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Tue, 19 Sep 2023 18:06:49 -0400 Subject: [PATCH 09/47] cargo fmt --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 33cec25a0d8..4a525144f8a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -753,7 +753,7 @@ version = "0.66.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f2b84e06fc203107bfbad243f4aba2af864eb7db3b1cf46ea0a023b0b433d2a7" dependencies = [ - "bitflags 2.3.3", + "bitflags 2.4.0", "cexpr", "clang-sys", "lazy_static", @@ -6166,7 +6166,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6c64d9ba0963cdcea2e1b2230fbae2bab30eb25a174be395c41e764bfb65dd62" dependencies = [ "proc-macro2", - "syn 2.0.28", + "syn 2.0.29", ] [[package]] From 91b81e9a314367d5e2c926df5fddcf0f17a867b8 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Wed, 20 Sep 2023 18:09:07 -0400 Subject: [PATCH 10/47] make availability view easier to implement, start on testing --- .../src/block_verification_types.rs | 4 + .../src/data_availability_checker.rs | 1 + .../availability_view.rs | 453 ++++++++++++++---- .../processing_cache.rs | 2 +- beacon_node/beacon_chain/src/test_utils.rs | 71 ++- .../sync/block_lookups/single_block_lookup.rs | 60 +-- .../network/src/sync/block_lookups/tests.rs | 73 +-- 7 files changed, 444 insertions(+), 220 deletions(-) diff --git a/beacon_node/beacon_chain/src/block_verification_types.rs b/beacon_node/beacon_chain/src/block_verification_types.rs index 78c8fc628fd..46e4dba6140 100644 --- a/beacon_node/beacon_chain/src/block_verification_types.rs +++ b/beacon_node/beacon_chain/src/block_verification_types.rs @@ -249,6 +249,10 @@ impl AvailabilityPendingExecutedBlock { } } + pub fn as_block(&self) -> &SignedBeaconBlock { + &self.block + } + pub fn num_blobs_expected(&self) -> usize { self.block .message() diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index e905eb5ddc3..eb4cee612eb 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -5,6 +5,7 @@ use crate::block_verification_types::{ pub use crate::data_availability_checker::availability_view::AvailabilityView; use crate::data_availability_checker::overflow_lru_cache::OverflowLRUCache; use crate::data_availability_checker::processing_cache::ProcessingCache; +pub use crate::impl_availability_view; use crate::{BeaconChain, BeaconChainTypes, BeaconStore}; use kzg::Kzg; use kzg::{Error as KzgError, KzgCommitment}; diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index b1e1772c55e..e2188d5cd7c 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -1,11 +1,18 @@ use crate::blob_verification::KzgVerifiedBlob; +use crate::block_verification_types::AsBlock; use crate::data_availability_checker::overflow_lru_cache::PendingComponents; +use crate::data_availability_checker::processing_cache::ProcessingCache; use crate::data_availability_checker::ProcessingInfo; +use crate::test_utils::{generate_rand_block_and_blobs, NumBlobs}; use crate::AvailabilityPendingExecutedBlock; -use kzg::KzgCommitment; +use eth2_network_config::get_trusted_setup; +use kzg::{KzgCommitment, TrustedSetup}; +use slasher::test_utils::E; use ssz_types::FixedVector; +use std::sync::Arc; use types::beacon_block_body::KzgCommitments; -use types::EthSpec; +use types::test_utils::TestRandom; +use types::{BlobSidecar, EthSpec, ForkName, MainnetEthSpec, SignedBeaconBlock}; /// Defines an interface for managing data availability with two key invariants: /// 1. Blobs won't be clobbered if we've yet to see the corresponding block. @@ -15,54 +22,75 @@ use types::EthSpec; /// of blocks and blobs in a cache-like data structure. pub trait AvailabilityView { /// The type representing a block in the implementation. - type BlockType; + type BlockType: GetCommitments; /// The type representing a blob in the implementation. Must implement `Clone`. - type BlobType: Clone; + type BlobType: Clone + GetCommitment; + + /// Returns an immutable reference to the cached block. + fn get_cached_block(&self) -> &Option; + + /// Returns an immutable reference to the fixed vector of cached blobs. + fn get_cached_blobs(&self) -> &FixedVector, E::MaxBlobsPerBlock>; + + /// Returns a mutable reference to the cached block. + fn get_cached_block_mut(&mut self) -> &mut Option; + + /// Returns a mutable reference to the fixed vector of cached blobs. + fn get_cached_blobs_mut( + &mut self, + ) -> &mut FixedVector, E::MaxBlobsPerBlock>; /// Checks if a block exists in the cache. /// /// Returns: /// - `true` if a block exists. /// - `false` otherwise. - fn block_exists(&self) -> bool; + fn block_exists(&self) -> bool { + self.get_cached_block().is_some() + } /// Checks if a blob exists at the given index in the cache. /// /// Returns: /// - `true` if a blob exists at the given index. /// - `false` otherwise. - fn blob_exists(&self, blob_index: u64) -> bool; + fn blob_exists(&self, blob_index: u64) -> bool { + self.get_cached_blobs() + .get(blob_index as usize) + .map(|b| b.is_some()) + .unwrap_or(false) + } /// Returns the number of blobs that are expected to be present. Returns 0 if we don't have a /// block. /// /// This corresponds to the number of commitments that are present in a block. - fn num_expected_blobs(&self) -> usize; + fn num_expected_blobs(&self) -> usize { + self.get_cached_block() + .as_ref() + .and_then(|b| b.get_commitments()) + .map_or(0, |c| c.len()) + } /// Returns the number of blobs that have been received and are stored in the cache. - fn num_received_blobs(&self) -> usize; + fn num_received_blobs(&self) -> usize { + self.get_cached_blobs().iter().flatten().count() + } /// Inserts a block into the cache. - fn insert_block(&mut self, block: Self::BlockType); + fn insert_block(&mut self, block: Self::BlockType) { + *self.get_cached_block_mut() = Some(block) + } /// Inserts a blob at a specific index in the cache. /// /// Existing blob at the index will be replaced. - fn insert_blob_at_index(&mut self, blob_index: u64, blob: &Self::BlobType); - - /// Converts a blob to its KZG commitment. - fn blob_to_commitment(blob: &Self::BlobType) -> KzgCommitment; - - /// Provides mutable access to the underlying blob commitments cache. - fn get_cached_blob_commitments_mut( - &mut self, - ) -> &mut FixedVector, E::MaxBlobsPerBlock>; - - /// Retrieves the KZG commitment of the blob stored at the given index in the block. - /// - /// Returns `None` if no blob is present at the index. - fn get_block_commitment_at_index(&self, blob_index: u64) -> Option; + fn insert_blob_at_index(&mut self, blob_index: u64, blob: &Self::BlobType) { + if let Some(b) = self.get_cached_blobs_mut().get_mut(blob_index as usize) { + *b = Some(blob.clone()); + } + } /// Merges a given set of blobs into the cache. /// @@ -72,13 +100,17 @@ pub trait AvailabilityView { fn merge_blobs(&mut self, blobs: FixedVector, E::MaxBlobsPerBlock>) { for (index, blob) in blobs.into_iter().enumerate() { let Some(blob) = blob else { continue }; - let commitment = Self::blob_to_commitment(blob); + let commitment = *blob.get_commitment(); let index = index as u64; - if let Some(block_commitment) = self.get_block_commitment_at_index(index) { - if block_commitment == commitment { - self.insert_blob_at_index(index, blob) + if let Some(block_commitments) = self.get_cached_block_mut() { + if let Some(block_commitment) = block_commitments.get_commitments() { + if let Some(&bc) = block_commitment.get(index as usize) { + if bc == commitment { + self.insert_blob_at_index(index, blob) + } + } } } else if !self.blob_exists(index) { self.insert_blob_at_index(index, blob) @@ -91,12 +123,14 @@ pub trait AvailabilityView { /// Blobs that don't match the new block's commitments are evicted. fn merge_block(&mut self, block: Self::BlockType) { self.insert_block(block); - let cached = self.get_cached_blob_commitments_mut(); + let cached = self.get_cached_blobs_mut(); let mut reinsert = FixedVector::default(); for (index, cached_blob) in cached.iter_mut().enumerate() { // Take the existing blobs and re-insert them. if let Some(blob) = reinsert.get_mut(index) { - *blob = cached_blob.take() + if let Some(cached_blob) = cached_blob.take() { + *blob = Some(cached_blob); + } } } @@ -112,109 +146,324 @@ pub trait AvailabilityView { } } -impl AvailabilityView for ProcessingInfo { - type BlockType = KzgCommitments; - type BlobType = KzgCommitment; +/// Implements the `AvailabilityView` trait for a given struct. +/// +/// - `$struct_name`: The name of the struct for which to implement `AvailabilityView`. +/// - `$block_type`: The type to use for `BlockType` in the `AvailabilityView` trait. +/// - `$blob_type`: The type to use for `BlobType` in the `AvailabilityView` trait. +/// - `$block_field`: The field name in the struct that holds the cached block. +/// - `$blob_field`: The field name in the struct that holds the cached blobs. +#[macro_export] +macro_rules! impl_availability_view { + ($struct_name:ident, $block_type:ty, $blob_type:ty, $block_field:ident, $blob_field:ident) => { + impl AvailabilityView for $struct_name { + type BlockType = $block_type; + type BlobType = $blob_type; + + fn get_cached_block(&self) -> &Option { + &self.$block_field + } - fn block_exists(&self) -> bool { - self.kzg_commitments.is_some() - } + fn get_cached_blobs( + &self, + ) -> &FixedVector, E::MaxBlobsPerBlock> { + &self.$blob_field + } - fn blob_exists(&self, blob_index: u64) -> bool { - self.processing_blobs - .get(blob_index as usize) - .map(|b| b.is_some()) - .unwrap_or(false) - } + fn get_cached_block_mut(&mut self) -> &mut Option { + &mut self.$block_field + } - fn num_expected_blobs(&self) -> usize { - self.kzg_commitments.as_ref().map_or(0, |c| c.len()) + fn get_cached_blobs_mut( + &mut self, + ) -> &mut FixedVector, E::MaxBlobsPerBlock> { + &mut self.$blob_field + } + } + }; +} + +impl_availability_view!( + ProcessingInfo, + KzgCommitments, + KzgCommitment, + kzg_commitments, + processing_blobs +); + +impl_availability_view!( + PendingComponents, + AvailabilityPendingExecutedBlock, + KzgVerifiedBlob, + executed_block, + verified_blobs +); + +pub trait GetCommitments { + fn get_commitments(&self) -> Option>; +} + +pub trait GetCommitment { + fn get_commitment(&self) -> &KzgCommitment; +} + +// These implementations are required to implement `AvailabilityView` for `ProcessingInfo`. +impl GetCommitments for KzgCommitments { + fn get_commitments(&self) -> Option> { + Some(self.clone()) + } +} +impl GetCommitment for KzgCommitment { + fn get_commitment(&self) -> &KzgCommitment { + self } +} - fn num_received_blobs(&self) -> usize { - self.processing_blobs.iter().flatten().count() +// These implementations are required to implement `AvailabilityView` for `PendingComponents`. +impl GetCommitments for AvailabilityPendingExecutedBlock { + fn get_commitments(&self) -> Option> { + self.as_block() + .message() + .body() + .blob_kzg_commitments() + .ok() + .cloned() + } +} +impl GetCommitment for KzgVerifiedBlob { + fn get_commitment(&self) -> &KzgCommitment { + &self.as_blob().kzg_commitment } +} - fn insert_block(&mut self, block: Self::BlockType) { - let _ = self.kzg_commitments.insert(block); +// These implementations are required to implement `AvailabilityView` for `CachedChildComponents`. +impl GetCommitments for Arc> { + fn get_commitments(&self) -> Option> { + self.message().body().blob_kzg_commitments().ok().cloned() + } +} +impl GetCommitment for Arc> { + fn get_commitment(&self) -> &KzgCommitment { + &self.kzg_commitment } +} - fn insert_blob_at_index(&mut self, blob_index: u64, blob: &Self::BlobType) { - if let Some(b) = self.processing_blobs.get_mut(blob_index as usize) { - *b = Some(*blob); +#[cfg(test)] +mod tests { + use super::*; + use crate::test_utils::{generate_rand_block_and_blobs, NumBlobs}; + use eth2_network_config::get_trusted_setup; + use kzg::{Kzg, TrustedSetup}; + use rand::rngs::StdRng; + use rand::SeedableRng; + use slog::b; + use types::test_utils::TestRandom; + use types::ForkName; + + type E = MainnetEthSpec; + fn pre_setup() -> ( + SignedBeaconBlock, + FixedVector>, ::MaxBlobsPerBlock>, + FixedVector>, ::MaxBlobsPerBlock>, + ) { + let trusted_setup: TrustedSetup = + serde_json::from_reader(get_trusted_setup::<::Kzg>()).unwrap(); + let kzg = Kzg::new_from_trusted_setup(trusted_setup).unwrap(); + + let mut rng = StdRng::seed_from_u64(0xDEADBEEF0BAD5EEDu64); + let (block, blobs_vec) = + generate_rand_block_and_blobs::(ForkName::Deneb, NumBlobs::Random, &kzg, &mut rng); + let mut blobs: FixedVector<_, ::MaxBlobsPerBlock> = FixedVector::default(); + + for blob in blobs_vec { + if let Some(b) = blobs.get_mut(blob.index as usize) { + *b = Some(blob); + } } - } - fn blob_to_commitment(blob: &Self::BlobType) -> KzgCommitment { - *blob - } + let mut invalid_blobs: FixedVector< + Option>, + ::MaxBlobsPerBlock, + > = FixedVector::default(); + for (index, blob) in blobs.iter().enumerate() { + let mut invalid_blob_opt = blob.clone(); + if let Some(invalid_blob) = invalid_blob_opt.as_mut() { + invalid_blob.kzg_commitment = KzgCommitment::random_for_test(&mut rng); + } + *invalid_blobs.get_mut(index).unwrap() = invalid_blob_opt; + } - fn get_cached_blob_commitments_mut( - &mut self, - ) -> &mut FixedVector, E::MaxBlobsPerBlock> { - &mut self.processing_blobs + (block, blobs, invalid_blobs) } - fn get_block_commitment_at_index(&self, blob_index: u64) -> Option { - self.processing_blobs - .get(blob_index as usize) - .and_then(|b| *b) + fn setup_processing_info( + block: SignedBeaconBlock, + valid_blobs: FixedVector>, ::MaxBlobsPerBlock>, + invalid_blobs: FixedVector>, ::MaxBlobsPerBlock>, + ) -> ( + KzgCommitments, + FixedVector, ::MaxBlobsPerBlock>, + FixedVector, ::MaxBlobsPerBlock>, + ) { + let commitments = block + .message() + .body() + .blob_kzg_commitments() + .unwrap() + .clone(); + let blobs = FixedVector::from( + valid_blobs + .iter() + .map(|blob_opt| blob_opt.as_ref().map(|blob| blob.kzg_commitment)) + .collect::>(), + ); + let invalid_blobs = FixedVector::from( + invalid_blobs + .iter() + .map(|blob_opt| blob_opt.as_ref().map(|blob| blob.kzg_commitment)) + .collect::>(), + ); + (commitments, blobs, invalid_blobs) } -} -impl AvailabilityView for PendingComponents { - type BlockType = AvailabilityPendingExecutedBlock; - type BlobType = KzgVerifiedBlob; + fn setup_pending_components( + block: SignedBeaconBlock, + valid_blobs: FixedVector>, ::MaxBlobsPerBlock>, + invalid_blobs: FixedVector>, ::MaxBlobsPerBlock>, + ) -> ( + AvailabilityPendingExecutedBlock, + FixedVector>, ::MaxBlobsPerBlock>, + FixedVector>, ::MaxBlobsPerBlock>, + ) { + let commitments = block + .message() + .body() + .blob_kzg_commitments() + .unwrap() + .clone(); + let blobs = FixedVector::from( + valid_blobs + .iter() + .map(|blob_opt| blob_opt.as_ref().map(|blob| blob.kzg_commitment)) + .collect::>(), + ); + let invalid_blobs = FixedVector::from( + invalid_blobs + .iter() + .map(|blob_opt| blob_opt.as_ref().map(|blob| blob.kzg_commitment)) + .collect::>(), + ); + (commitments, blobs, invalid_blobs) + } - fn block_exists(&self) -> bool { - self.executed_block.is_some() + fn assert_cache_consistent(cache: ProcessingInfo) { + if let Some(cached_block_commitments) = cache.kzg_commitments { + for (index, (block_commitment, blob_commitment)) in cached_block_commitments + .iter() + .zip(cache.processing_blobs.iter()) + .enumerate() + { + assert_eq!(Some(*block_commitment), *blob_commitment); + } + } else { + panic!() + } } - fn blob_exists(&self, blob_index: u64) -> bool { - self.verified_blobs - .get(blob_index as usize) - .map(|b| b.is_some()) - .unwrap_or(false) + fn assert_empty_blob_cache(cache: ProcessingInfo) { + for blob in cache.processing_blobs.iter() { + assert!(blob.is_none()); + } } - fn num_expected_blobs(&self) -> usize { - self.executed_block - .as_ref() - .map_or(0, |b| b.num_blobs_expected()) + #[test] + fn valid_block_invalid_blobs_valid_blobs() { + let (block_commitments, blobs, random_blobs) = pre_setup(); + let (block_commitments, blobs, random_blobs) = + setup_processing_info(block_commitments, blobs, random_blobs); + + let mut cache = ProcessingInfo::::default(); + + cache.merge_block(block_commitments); + cache.merge_blobs(random_blobs); + cache.merge_blobs(blobs); + + assert_cache_consistent(cache); } - fn num_received_blobs(&self) -> usize { - self.verified_blobs.iter().flatten().count() + #[test] + fn invalid_blobs_block_valid_blobs() { + let (block_commitments, blobs, random_blobs) = pre_setup(); + let (block_commitments, blobs, random_blobs) = + setup_processing_info(block_commitments, blobs, random_blobs); + + let mut cache = ProcessingInfo::::default(); + + cache.merge_blobs(random_blobs); + cache.merge_block(block_commitments); + cache.merge_blobs(blobs); + + assert_cache_consistent(cache); } - fn insert_block(&mut self, block: Self::BlockType) { - self.executed_block = Some(block); + #[test] + fn invalid_blobs_valid_blobs_block() { + let (block_commitments, blobs, random_blobs) = pre_setup(); + let (block_commitments, blobs, random_blobs) = + setup_processing_info(block_commitments, blobs, random_blobs); + + let mut cache = ProcessingInfo::::default(); + + cache.merge_blobs(random_blobs); + cache.merge_blobs(blobs); + cache.merge_block(block_commitments); + + // The random blobs should be pruned + assert_empty_blob_cache(cache); } - fn insert_blob_at_index(&mut self, blob_index: u64, blob: &Self::BlobType) { - if let Some(b) = self.verified_blobs.get_mut(blob_index as usize) { - *b = Some(blob.clone()); - } + #[test] + fn block_valid_blobs_invalid_blobs() { + let (block_commitments, blobs, random_blobs) = pre_setup(); + let (block_commitments, blobs, random_blobs) = + setup_processing_info(block_commitments, blobs, random_blobs); + + let mut cache = ProcessingInfo::::default(); + + cache.merge_block(block_commitments); + cache.merge_blobs(blobs); + cache.merge_blobs(random_blobs); + + assert_cache_consistent(cache); } - fn blob_to_commitment(blob: &Self::BlobType) -> KzgCommitment { - blob.as_blob().kzg_commitment + #[test] + fn valid_blobs_block_invalid_blobs() { + let (block_commitments, blobs, random_blobs) = pre_setup(); + let (block_commitments, blobs, random_blobs) = + setup_processing_info(block_commitments, blobs, random_blobs); + + let mut cache = ProcessingInfo::::default(); + + cache.merge_blobs(blobs); + cache.merge_block(block_commitments); + cache.merge_blobs(random_blobs); + + assert_cache_consistent(cache); } - fn get_cached_blob_commitments_mut( - &mut self, - ) -> &mut FixedVector, E::MaxBlobsPerBlock> { - &mut self.verified_blobs - } - - fn get_block_commitment_at_index(&self, blob_index: u64) -> Option { - self.executed_block.as_ref().and_then(|b| { - b.block - .message() - .body() - .blob_kzg_commitments() - .ok() - .and_then(|c| c.get(blob_index as usize).cloned()) - }) + #[test] + fn valid_blobs_invalid_blobs_block() { + let (block_commitments, blobs, random_blobs) = pre_setup(); + let (block_commitments, blobs, random_blobs) = + setup_processing_info(block_commitments, blobs, random_blobs); + + let mut cache = ProcessingInfo::::default(); + + cache.merge_blobs(blobs); + cache.merge_blobs(random_blobs); + cache.merge_block(block_commitments); + + assert_cache_consistent(cache); } } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs index 3fabef5b088..9ddf5dbb8cf 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -43,7 +43,7 @@ impl ProcessingCache { } } -#[derive(Default)] +#[derive(Default, Debug)] pub struct ProcessingInfo { slot: Slot, /// Blobs required for a block can only be known if we have seen the block. So `Some` here diff --git a/beacon_node/beacon_chain/src/test_utils.rs b/beacon_node/beacon_chain/src/test_utils.rs index e7d459afe4f..44b5b20c0ee 100644 --- a/beacon_node/beacon_chain/src/test_utils.rs +++ b/beacon_node/beacon_chain/src/test_utils.rs @@ -28,7 +28,7 @@ use execution_layer::{ use futures::channel::mpsc::Receiver; pub use genesis::{interop_genesis_state_with_eth1, DEFAULT_ETH1_BLOCK_HASH}; use int_to_bytes::int_to_bytes32; -use kzg::{Kzg, TrustedSetup}; +use kzg::{Kzg, KzgPreset, TrustedSetup}; use merkle_proof::MerkleTree; use operation_pool::ReceivedPreCapella; use parking_lot::RwLockWriteGuard; @@ -38,6 +38,7 @@ use rand::Rng; use rand::SeedableRng; use rayon::prelude::*; use sensitive_url::SensitiveUrl; +use slasher::test_utils::E; use slog::{o, Drain, Logger}; use slog_async::Async; use slog_term::{FullFormat, TermDecorator}; @@ -60,6 +61,7 @@ use task_executor::{test_utils::TestRuntime, ShutdownReason}; use tree_hash::TreeHash; use types::sync_selection_proof::SyncSelectionProof; pub use types::test_utils::generate_deterministic_keypairs; +use types::test_utils::{TestRandom, XorShiftRng}; use types::{typenum::U4294967296, *}; // 4th September 2019 @@ -2503,3 +2505,70 @@ pub fn build_log(level: slog::Level, enabled: bool) -> Logger { Logger::root(drain.filter(|_| false).fuse(), o!()) } } + +pub enum NumBlobs { + Random, + None, +} + +//TODO: port michael's changes to Rng from store updates +pub fn generate_rand_block_and_blobs( + fork_name: ForkName, + num_blobs: NumBlobs, + kzg: &Kzg, + rng: &mut impl Rng, +) -> (SignedBeaconBlock>, Vec>) { + let inner = map_fork_name!(fork_name, BeaconBlock, <_>::random_for_test(rng)); + let mut block = SignedBeaconBlock::from_block(inner, types::Signature::random_for_test(rng)); + let mut blob_sidecars = vec![]; + if let Ok(message) = block.message_deneb_mut() { + // get random number between 0 and Max Blobs + let payload: &mut FullPayloadDeneb = &mut message.body.execution_payload; + let num_blobs = match num_blobs { + NumBlobs::Random => { + let mut num_blobs = rand::random::() % E::max_blobs_per_block(); + if num_blobs == 0 { + num_blobs += 1; + } + num_blobs + } + NumBlobs::None => 0, + }; + let (bundle, transactions) = + execution_layer::test_utils::generate_random_blobs::(num_blobs, kzg).unwrap(); + + payload.execution_payload.transactions = <_>::default(); + for tx in Vec::from(transactions) { + payload.execution_payload.transactions.push(tx).unwrap(); + } + message.body.blob_kzg_commitments = bundle.commitments.clone(); + + let eth2::types::BlobsBundle { + commitments, + proofs, + blobs, + } = bundle; + + let block_root = block.canonical_root(); + + for (index, ((blob, kzg_commitment), kzg_proof)) in blobs + .into_iter() + .zip(commitments.into_iter()) + .zip(proofs.into_iter()) + .enumerate() + { + blob_sidecars.push(BlobSidecar { + block_root, + index: index as u64, + slot: block.slot(), + block_parent_root: block.parent_root(), + proposer_index: block.message().proposer_index(), + blob: blob.clone(), + kzg_commitment, + kzg_proof, + }); + } + } + + (block, blob_sidecars) +} 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 158b70d2eda..7818c8a25e5 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 @@ -6,7 +6,7 @@ use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::data_availability_checker::{ AvailabilityCheckError, AvailabilityView, DataAvailabilityChecker, ProcessingInfo, }; -use beacon_chain::BeaconChainTypes; +use beacon_chain::{impl_availability_view, BeaconChainTypes}; use lighthouse_network::rpc::methods::MaxRequestBlobSidecars; use lighthouse_network::{PeerAction, PeerId}; use slog::{trace, Logger}; @@ -456,57 +456,13 @@ impl CachedChildComponents { } } -impl AvailabilityView for CachedChildComponents { - type BlockType = Arc>; - type BlobType = Arc>; - - fn block_exists(&self) -> bool { - self.downloaded_block.is_some() - } - - fn blob_exists(&self, blob_index: u64) -> bool { - self.downloaded_blobs - .get(blob_index as usize) - .map(|b| b.is_some()) - .unwrap_or(false) - } - - fn num_expected_blobs(&self) -> usize { - self.downloaded_block - .as_ref() - .map_or(0, |b| b.num_expected_blobs()) - } - - fn num_received_blobs(&self) -> usize { - self.downloaded_blobs.iter().flatten().count() - } - - fn insert_block(&mut self, block: Self::BlockType) { - self.downloaded_block = Some(block); - } - - fn insert_blob_at_index(&mut self, blob_index: u64, blob: &Self::BlobType) { - if let Some(b) = self.downloaded_blobs.get_mut(blob_index as usize) { - *b = Some(blob.clone()); - } - } - - fn blob_to_commitment(blob: &Self::BlobType) -> KzgCommitment { - blob.kzg_commitment - } - - fn get_cached_blob_commitments_mut( - &mut self, - ) -> &mut FixedVector, E::MaxBlobsPerBlock> { - &mut self.downloaded_blobs - } - - fn get_block_commitment_at_index(&self, blob_index: u64) -> Option { - self.downloaded_blobs - .get(blob_index as usize) - .and_then(|b| b.as_ref().map(|b| b.kzg_commitment)) - } -} +impl_availability_view!( + CachedChildComponents, + Arc>, + Arc>, + downloaded_block, + downloaded_blobs +); /// Object representing the state of a single block or blob lookup request. #[derive(PartialEq, Eq, Debug)] diff --git a/beacon_node/network/src/sync/block_lookups/tests.rs b/beacon_node/network/src/sync/block_lookups/tests.rs index 422de8e0c5b..1ba158af130 100644 --- a/beacon_node/network/src/sync/block_lookups/tests.rs +++ b/beacon_node/network/src/sync/block_lookups/tests.rs @@ -10,7 +10,9 @@ 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}; +use beacon_chain::test_utils::{ + build_log, generate_rand_block_and_blobs, BeaconChainHarness, EphemeralHarnessType, NumBlobs, +}; use beacon_processor::WorkEvent; use lighthouse_network::rpc::RPCResponseErrorCode; use lighthouse_network::{NetworkGlobals, Request}; @@ -20,8 +22,8 @@ use tokio::sync::mpsc; use types::{ map_fork_name, map_fork_name_with, test_utils::{SeedableRng, TestRandom, XorShiftRng}, - BeaconBlock, BlobSidecar, EthSpec, ForkName, FullPayloadDeneb, MinimalEthSpec as E, - SignedBeaconBlock, + BeaconBlock, BlobSidecar, EthSpec, ForkName, FullPayload, FullPayloadDeneb, + MinimalEthSpec as E, MinimalEthSpec, SignedBeaconBlock, }; type T = Witness, E, MemoryStore, MemoryStore>; @@ -35,11 +37,6 @@ struct TestRig { const D: Duration = Duration::new(0, 0); -enum NumBlobs { - Random, - None, -} - impl TestRig { fn test_setup(enable_log: bool) -> (BlockLookups, SyncNetworkContext, Self) { let log = build_log(slog::Level::Debug, enable_log); @@ -96,63 +93,10 @@ impl TestRig { fork_name: ForkName, num_blobs: NumBlobs, ) -> (SignedBeaconBlock, Vec>) { - let inner = map_fork_name!(fork_name, BeaconBlock, <_>::random_for_test(&mut self.rng)); - let mut block = - SignedBeaconBlock::from_block(inner, types::Signature::random_for_test(&mut self.rng)); - let mut blob_sidecars = vec![]; - if let Ok(message) = block.message_deneb_mut() { - // get random number between 0 and Max Blobs - let payload: &mut FullPayloadDeneb = &mut message.body.execution_payload; - let num_blobs = match num_blobs { - NumBlobs::Random => { - let mut num_blobs = rand::random::() % E::max_blobs_per_block(); - if num_blobs == 0 { - num_blobs += 1; - } - num_blobs - } - NumBlobs::None => 0, - }; - let (bundle, transactions) = execution_layer::test_utils::generate_random_blobs::( - num_blobs, - self.harness.chain.kzg.as_ref().unwrap(), - ) - .unwrap(); - - payload.execution_payload.transactions = <_>::default(); - for tx in Vec::from(transactions) { - payload.execution_payload.transactions.push(tx).unwrap(); - } - message.body.blob_kzg_commitments = bundle.commitments.clone(); - - let eth2::types::BlobsBundle { - commitments, - proofs, - blobs, - } = bundle; - - let block_root = block.canonical_root(); - - for (index, ((blob, kzg_commitment), kzg_proof)) in blobs - .into_iter() - .zip(commitments.into_iter()) - .zip(proofs.into_iter()) - .enumerate() - { - blob_sidecars.push(BlobSidecar { - block_root, - index: index as u64, - slot: block.slot(), - block_parent_root: block.parent_root(), - proposer_index: block.message().proposer_index(), - blob: blob.clone(), - kzg_commitment, - kzg_proof, - }); - } - } + let kzg = self.harness.chain.kzg.as_ref().unwrap(); + let rng = &mut self.rng; - (block, blob_sidecars) + generate_rand_block_and_blobs::(fork_name, num_blobs, kzg.as_ref(), rng) } #[track_caller] @@ -1226,6 +1170,7 @@ mod deneb_only { use super::*; use crate::sync::block_lookups::common::ResponseType; use beacon_chain::data_availability_checker::AvailabilityCheckError; + use beacon_chain::test_utils::NumBlobs; use std::ops::IndexMut; use std::str::FromStr; From 420e7cf8f4858076e4554b779526dfe83e57ab25 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Wed, 20 Sep 2023 20:17:16 -0400 Subject: [PATCH 11/47] move child component cache and finish test --- .../beacon_chain/src/blob_verification.rs | 9 + .../src/data_availability_checker.rs | 7 +- .../availability_view.rs | 282 ++++++++++++------ .../child_component_cache.rs | 54 ++++ .../network/src/sync/block_lookups/common.rs | 8 +- .../network/src/sync/block_lookups/mod.rs | 6 +- .../sync/block_lookups/single_block_lookup.rs | 77 +---- .../network/src/sync/block_lookups/tests.rs | 4 +- beacon_node/network/src/sync/manager.rs | 7 +- beacon_node/network/src/sync/mod.rs | 1 - 10 files changed, 278 insertions(+), 177 deletions(-) create mode 100644 beacon_node/beacon_chain/src/data_availability_checker/child_component_cache.rs diff --git a/beacon_node/beacon_chain/src/blob_verification.rs b/beacon_node/beacon_chain/src/blob_verification.rs index 962ff7963b0..7b02aa34038 100644 --- a/beacon_node/beacon_chain/src/blob_verification.rs +++ b/beacon_node/beacon_chain/src/blob_verification.rs @@ -473,6 +473,15 @@ impl KzgVerifiedBlob { } } +#[cfg(test)] +impl KzgVerifiedBlob { + pub fn new(blob: BlobSidecar) -> Self { + Self { + blob: Arc::new(blob), + } + } +} + /// Complete kzg verification for a `GossipVerifiedBlob`. /// /// Returns an error if the kzg verification check fails. diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index eb4cee612eb..f5bdb5994ab 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -2,10 +2,12 @@ use crate::blob_verification::{verify_kzg_for_blob, verify_kzg_for_blob_list, Go use crate::block_verification_types::{ AvailabilityPendingExecutedBlock, AvailableExecutedBlock, RpcBlock, }; -pub use crate::data_availability_checker::availability_view::AvailabilityView; +pub use crate::data_availability_checker::availability_view::{ + AvailabilityView, GetCommitment, GetCommitments, +}; +pub use crate::data_availability_checker::child_component_cache::ChildComponentCache; use crate::data_availability_checker::overflow_lru_cache::OverflowLRUCache; use crate::data_availability_checker::processing_cache::ProcessingCache; -pub use crate::impl_availability_view; use crate::{BeaconChain, BeaconChainTypes, BeaconStore}; use kzg::Kzg; use kzg::{Error as KzgError, KzgCommitment}; @@ -25,6 +27,7 @@ use types::consts::deneb::MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS; use types::{BlobSidecarList, ChainSpec, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot}; mod availability_view; +mod child_component_cache; mod overflow_lru_cache; mod processing_cache; diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index e2188d5cd7c..ca7604975b5 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -1,13 +1,12 @@ +use super::child_component_cache::ChildComponentCache; use crate::blob_verification::KzgVerifiedBlob; use crate::block_verification_types::AsBlock; use crate::data_availability_checker::overflow_lru_cache::PendingComponents; -use crate::data_availability_checker::processing_cache::ProcessingCache; use crate::data_availability_checker::ProcessingInfo; use crate::test_utils::{generate_rand_block_and_blobs, NumBlobs}; use crate::AvailabilityPendingExecutedBlock; use eth2_network_config::get_trusted_setup; use kzg::{KzgCommitment, TrustedSetup}; -use slasher::test_utils::E; use ssz_types::FixedVector; use std::sync::Arc; use types::beacon_block_body::KzgCommitments; @@ -199,6 +198,14 @@ impl_availability_view!( verified_blobs ); +impl_availability_view!( + ChildComponentCache, + Arc>, + Arc>, + downloaded_block, + downloaded_blobs +); + pub trait GetCommitments { fn get_commitments(&self) -> Option>; } @@ -249,19 +256,23 @@ impl GetCommitment for Arc> { } #[cfg(test)] -mod tests { +pub mod tests { use super::*; + use crate::block_verification_types::BlockImportData; + use crate::eth1_finalization_cache::Eth1FinalizationData; use crate::test_utils::{generate_rand_block_and_blobs, NumBlobs}; + use crate::PayloadVerificationOutcome; use eth2_network_config::get_trusted_setup; + use fork_choice::PayloadVerificationStatus; use kzg::{Kzg, TrustedSetup}; use rand::rngs::StdRng; use rand::SeedableRng; - use slog::b; + use state_processing::ConsensusContext; use types::test_utils::TestRandom; - use types::ForkName; + use types::{BeaconState, ChainSpec, ForkName, Slot}; type E = MainnetEthSpec; - fn pre_setup() -> ( + pub fn pre_setup() -> ( SignedBeaconBlock, FixedVector>, ::MaxBlobsPerBlock>, FixedVector>, ::MaxBlobsPerBlock>, @@ -296,7 +307,7 @@ mod tests { (block, blobs, invalid_blobs) } - fn setup_processing_info( + pub fn setup_processing_info( block: SignedBeaconBlock, valid_blobs: FixedVector>, ::MaxBlobsPerBlock>, invalid_blobs: FixedVector>, ::MaxBlobsPerBlock>, @@ -326,7 +337,7 @@ mod tests { (commitments, blobs, invalid_blobs) } - fn setup_pending_components( + pub fn setup_pending_components( block: SignedBeaconBlock, valid_blobs: FixedVector>, ::MaxBlobsPerBlock>, invalid_blobs: FixedVector>, ::MaxBlobsPerBlock>, @@ -335,135 +346,212 @@ mod tests { FixedVector>, ::MaxBlobsPerBlock>, FixedVector>, ::MaxBlobsPerBlock>, ) { - let commitments = block - .message() - .body() - .blob_kzg_commitments() - .unwrap() - .clone(); let blobs = FixedVector::from( valid_blobs .iter() - .map(|blob_opt| blob_opt.as_ref().map(|blob| blob.kzg_commitment)) + .map(|blob_opt| { + blob_opt + .as_ref() + .map(|blob| KzgVerifiedBlob::new(blob.clone())) + }) .collect::>(), ); let invalid_blobs = FixedVector::from( invalid_blobs .iter() - .map(|blob_opt| blob_opt.as_ref().map(|blob| blob.kzg_commitment)) + .map(|blob_opt| { + blob_opt + .as_ref() + .map(|blob| KzgVerifiedBlob::new(blob.clone())) + }) .collect::>(), ); - (commitments, blobs, invalid_blobs) + let dummy_parent = block.clone_as_blinded(); + let block = AvailabilityPendingExecutedBlock { + block: Arc::new(block), + import_data: BlockImportData { + block_root: Default::default(), + state: BeaconState::new(0, Default::default(), &ChainSpec::minimal()), + parent_block: dummy_parent, + parent_eth1_finalization_data: Eth1FinalizationData { + eth1_data: Default::default(), + eth1_deposit_index: 0, + }, + confirmed_state_roots: vec![], + consensus_context: ConsensusContext::new(Slot::new(0)), + }, + payload_verification_outcome: PayloadVerificationOutcome { + payload_verification_status: PayloadVerificationStatus::Verified, + is_valid_merge_transition_block: false, + }, + }; + (block, blobs, invalid_blobs) } - fn assert_cache_consistent(cache: ProcessingInfo) { - if let Some(cached_block_commitments) = cache.kzg_commitments { - for (index, (block_commitment, blob_commitment)) in cached_block_commitments - .iter() - .zip(cache.processing_blobs.iter()) - .enumerate() - { - assert_eq!(Some(*block_commitment), *blob_commitment); + pub fn setup_child_components( + block: SignedBeaconBlock, + valid_blobs: FixedVector>, ::MaxBlobsPerBlock>, + invalid_blobs: FixedVector>, ::MaxBlobsPerBlock>, + ) -> ( + Arc>, + FixedVector>>, ::MaxBlobsPerBlock>, + FixedVector>>, ::MaxBlobsPerBlock>, + ) { + let blobs = FixedVector::from( + valid_blobs + .into_iter() + .map(|blob_opt| blob_opt.clone().map(|blob| Arc::new(blob))) + .collect::>(), + ); + let invalid_blobs = FixedVector::from( + invalid_blobs + .into_iter() + .map(|blob_opt| blob_opt.clone().map(|blob| Arc::new(blob))) + .collect::>(), + ); + (Arc::new(block), blobs, invalid_blobs) + } + + pub fn assert_cache_consistent>(cache: A) { + if let Some(cached_block) = cache.get_cached_block() { + if let Some(cached_block_commitments) = cached_block.get_commitments() { + for (index, (block_commitment, blob_commitment_opt)) in cached_block_commitments + .iter() + .zip(cache.get_cached_blobs().iter()) + .enumerate() + { + let blob_commitment = blob_commitment_opt + .as_ref() + .map(|b| *b.get_commitment()) + .unwrap(); + assert_eq!(*block_commitment, blob_commitment); + } + } else { + panic!("Cached block has no commitments") } } else { panic!() } } - fn assert_empty_blob_cache(cache: ProcessingInfo) { - for blob in cache.processing_blobs.iter() { + pub fn assert_empty_blob_cache>(cache: A) { + for blob in cache.get_cached_blobs().iter() { assert!(blob.is_none()); } } - #[test] - fn valid_block_invalid_blobs_valid_blobs() { - let (block_commitments, blobs, random_blobs) = pre_setup(); - let (block_commitments, blobs, random_blobs) = - setup_processing_info(block_commitments, blobs, random_blobs); - - let mut cache = ProcessingInfo::::default(); + #[macro_export] + macro_rules! generate_tests { + ($module_name:ident, $type_name:ty, $block_field:ident, $blob_field:ident, $setup_fn:ident) => { + mod $module_name { + use super::*; - cache.merge_block(block_commitments); - cache.merge_blobs(random_blobs); - cache.merge_blobs(blobs); + #[test] + fn valid_block_invalid_blobs_valid_blobs() { + let (block_commitments, blobs, random_blobs) = pre_setup(); + let (block_commitments, blobs, random_blobs) = + $setup_fn(block_commitments, blobs, random_blobs); - assert_cache_consistent(cache); - } + let mut cache = <$type_name>::default(); + cache.merge_block(block_commitments); + cache.merge_blobs(random_blobs); + cache.merge_blobs(blobs); - #[test] - fn invalid_blobs_block_valid_blobs() { - let (block_commitments, blobs, random_blobs) = pre_setup(); - let (block_commitments, blobs, random_blobs) = - setup_processing_info(block_commitments, blobs, random_blobs); + assert_cache_consistent(cache); + } - let mut cache = ProcessingInfo::::default(); + #[test] + fn invalid_blobs_block_valid_blobs() { + let (block_commitments, blobs, random_blobs) = pre_setup(); + let (block_commitments, blobs, random_blobs) = + $setup_fn(block_commitments, blobs, random_blobs); - cache.merge_blobs(random_blobs); - cache.merge_block(block_commitments); - cache.merge_blobs(blobs); + let mut cache = <$type_name>::default(); + cache.merge_blobs(random_blobs); + cache.merge_block(block_commitments); + cache.merge_blobs(blobs); - assert_cache_consistent(cache); - } + assert_cache_consistent(cache); + } - #[test] - fn invalid_blobs_valid_blobs_block() { - let (block_commitments, blobs, random_blobs) = pre_setup(); - let (block_commitments, blobs, random_blobs) = - setup_processing_info(block_commitments, blobs, random_blobs); + #[test] + fn invalid_blobs_valid_blobs_block() { + let (block_commitments, blobs, random_blobs) = pre_setup(); + let (block_commitments, blobs, random_blobs) = + $setup_fn(block_commitments, blobs, random_blobs); - let mut cache = ProcessingInfo::::default(); + let mut cache = <$type_name>::default(); + cache.merge_blobs(random_blobs); + cache.merge_blobs(blobs); + cache.merge_block(block_commitments); - cache.merge_blobs(random_blobs); - cache.merge_blobs(blobs); - cache.merge_block(block_commitments); + assert_empty_blob_cache(cache); + } - // The random blobs should be pruned - assert_empty_blob_cache(cache); - } + #[test] + fn block_valid_blobs_invalid_blobs() { + let (block_commitments, blobs, random_blobs) = pre_setup(); + let (block_commitments, blobs, random_blobs) = + $setup_fn(block_commitments, blobs, random_blobs); - #[test] - fn block_valid_blobs_invalid_blobs() { - let (block_commitments, blobs, random_blobs) = pre_setup(); - let (block_commitments, blobs, random_blobs) = - setup_processing_info(block_commitments, blobs, random_blobs); + let mut cache = <$type_name>::default(); + cache.merge_block(block_commitments); + cache.merge_blobs(blobs); + cache.merge_blobs(random_blobs); - let mut cache = ProcessingInfo::::default(); + assert_cache_consistent(cache); + } - cache.merge_block(block_commitments); - cache.merge_blobs(blobs); - cache.merge_blobs(random_blobs); + #[test] + fn valid_blobs_block_invalid_blobs() { + let (block_commitments, blobs, random_blobs) = pre_setup(); + let (block_commitments, blobs, random_blobs) = + $setup_fn(block_commitments, blobs, random_blobs); - assert_cache_consistent(cache); - } + let mut cache = <$type_name>::default(); + cache.merge_blobs(blobs); + cache.merge_block(block_commitments); + cache.merge_blobs(random_blobs); - #[test] - fn valid_blobs_block_invalid_blobs() { - let (block_commitments, blobs, random_blobs) = pre_setup(); - let (block_commitments, blobs, random_blobs) = - setup_processing_info(block_commitments, blobs, random_blobs); + assert_cache_consistent(cache); + } - let mut cache = ProcessingInfo::::default(); + #[test] + fn valid_blobs_invalid_blobs_block() { + let (block_commitments, blobs, random_blobs) = pre_setup(); + let (block_commitments, blobs, random_blobs) = + $setup_fn(block_commitments, blobs, random_blobs); - cache.merge_blobs(blobs); - cache.merge_block(block_commitments); - cache.merge_blobs(random_blobs); + let mut cache = <$type_name>::default(); + cache.merge_blobs(blobs); + cache.merge_blobs(random_blobs); + cache.merge_block(block_commitments); - assert_cache_consistent(cache); + assert_cache_consistent(cache); + } + } + }; } - #[test] - fn valid_blobs_invalid_blobs_block() { - let (block_commitments, blobs, random_blobs) = pre_setup(); - let (block_commitments, blobs, random_blobs) = - setup_processing_info(block_commitments, blobs, random_blobs); - - let mut cache = ProcessingInfo::::default(); - - cache.merge_blobs(blobs); - cache.merge_blobs(random_blobs); - cache.merge_block(block_commitments); - - assert_cache_consistent(cache); - } + generate_tests!( + processing_info_tests, + ProcessingInfo::, + kzg_commitments, + processing_blobs, + setup_processing_info + ); + generate_tests!( + pending_components_tests, + PendingComponents, + executed_block, + verified_blobs, + setup_pending_components + ); + generate_tests!( + child_component_tests, + ChildComponentCache::, + downloaded_block, + downloaded_blobs, + setup_child_components + ); } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/child_component_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/child_component_cache.rs new file mode 100644 index 00000000000..c0139c4267e --- /dev/null +++ b/beacon_node/beacon_chain/src/data_availability_checker/child_component_cache.rs @@ -0,0 +1,54 @@ +use crate::block_verification_types::RpcBlock; +use crate::data_availability_checker::{AvailabilityView, ProcessingInfo}; +use std::sync::Arc; +use types::blob_sidecar::FixedBlobSidecarList; +use types::{EthSpec, SignedBeaconBlock}; + +/// 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 ChildComponentCache { + pub downloaded_block: Option>>, + pub downloaded_blobs: FixedBlobSidecarList, +} + +impl From> for ChildComponentCache { + 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 ChildComponentCache { + pub fn new( + block: Option>>, + blobs: Option>, + ) -> Self { + Self { + downloaded_block: block, + downloaded_blobs: blobs.unwrap_or_default(), + } + } + + pub fn clear_blobs(&mut self) { + self.downloaded_blobs = FixedBlobSidecarList::default(); + } + + pub fn add_cached_child_block(&mut self, block: Arc>) { + self.merge_block(block) + } + + pub fn add_cached_child_blobs(&mut self, blobs: FixedBlobSidecarList) { + self.merge_blobs(blobs) + } + + pub fn processing_info(&self) -> ProcessingInfo { + ProcessingInfo::from_parts(self.downloaded_block.as_ref(), &self.downloaded_blobs) + .unwrap_or_default() + } +} diff --git a/beacon_node/network/src/sync/block_lookups/common.rs b/beacon_node/network/src/sync/block_lookups/common.rs index 05cf0a9b5cb..91e217cdca3 100644 --- a/beacon_node/network/src/sync/block_lookups/common.rs +++ b/beacon_node/network/src/sync/block_lookups/common.rs @@ -8,8 +8,8 @@ use crate::sync::block_lookups::{ }; 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::data_availability_checker::ChildComponentCache; use beacon_chain::{get_block_root, BeaconChainTypes}; use lighthouse_network::rpc::methods::BlobsByRootRequest; use lighthouse_network::rpc::BlocksByRootRequest; @@ -222,7 +222,7 @@ pub trait RequestState { /// triggered by `UnknownParent` errors. fn add_to_child_components( verified_response: Self::VerifiedResponseType, - components: &mut CachedChildComponents, + components: &mut ChildComponentCache, ); /// Convert a verified response to the type we send to the beacon processor. @@ -326,7 +326,7 @@ impl RequestState for BlockRequestState fn add_to_child_components( verified_response: Arc>, - components: &mut CachedChildComponents, + components: &mut ChildComponentCache, ) { components.add_cached_child_block(verified_response); } @@ -432,7 +432,7 @@ impl RequestState for BlobRequestState, - components: &mut CachedChildComponents, + components: &mut ChildComponentCache, ) { components.add_cached_child_blobs(verified_response); } diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index b206143c2de..104306aee5e 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -12,6 +12,7 @@ use crate::sync::block_lookups::single_block_lookup::{ }; use crate::sync::manager::{Id, SingleLookupReqId}; use beacon_chain::block_verification_types::{AsBlock, RpcBlock}; +pub use beacon_chain::data_availability_checker::ChildComponentCache; use beacon_chain::data_availability_checker::{ AvailabilityCheckError, AvailabilityView, DataAvailabilityChecker, }; @@ -25,7 +26,6 @@ use fnv::FnvHashMap; use lighthouse_network::rpc::RPCError; use lighthouse_network::{PeerAction, PeerId}; use lru_cache::LRUTimeCache; -pub use single_block_lookup::CachedChildComponents; pub use single_block_lookup::{BlobRequestState, BlockRequestState}; use slog::{debug, error, trace, warn, Logger}; use smallvec::SmallVec; @@ -146,7 +146,7 @@ impl BlockLookups { pub fn search_child_block( &mut self, block_root: Hash256, - child_components: CachedChildComponents, + child_components: ChildComponentCache, peer_source: PeerShouldHave, cx: &mut SyncNetworkContext, ) { @@ -195,7 +195,7 @@ impl BlockLookups { pub fn new_current_lookup( &mut self, block_root: Hash256, - child_components: Option>, + child_components: Option>, peers: &[PeerShouldHave], cx: &mut SyncNetworkContext, ) -> Option> { 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 7818c8a25e5..09cfd431f0b 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 @@ -3,8 +3,9 @@ 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::ChildComponentCache; use beacon_chain::data_availability_checker::{ - AvailabilityCheckError, AvailabilityView, DataAvailabilityChecker, ProcessingInfo, + AvailabilityCheckError, AvailabilityView, DataAvailabilityChecker, }; use beacon_chain::{impl_availability_view, BeaconChainTypes}; use lighthouse_network::rpc::methods::MaxRequestBlobSidecars; @@ -18,7 +19,7 @@ use std::sync::Arc; use store::Hash256; use strum::IntoStaticStr; use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; -use types::{BlobSidecar, EthSpec, KzgCommitment, SignedBeaconBlock}; +use types::{BlobSidecar, EthSpec, SignedBeaconBlock}; #[derive(Debug, PartialEq, Eq)] pub enum State { @@ -60,13 +61,13 @@ pub struct SingleBlockLookup { pub da_checker: Arc>, /// Only necessary for requests triggered by an `UnknownBlockParent` or `UnknownBlockParent` /// because any blocks or blobs without parents won't hit the data availability cache. - pub cached_child_components: Option>, + pub cached_child_components: Option>, } impl SingleBlockLookup { pub fn new( requested_block_root: Hash256, - unknown_parent_components: Option>, + unknown_parent_components: Option>, peers: &[PeerShouldHave], da_checker: Arc>, id: Id, @@ -96,7 +97,7 @@ impl SingleBlockLookup { self.block_request_state.requested_block_root = block_root; self.block_request_state.state.state = State::AwaitingDownload; self.blob_request_state.state.state = State::AwaitingDownload; - self.cached_child_components = Some(CachedChildComponents::default()); + self.cached_child_components = Some(ChildComponentCache::default()); } /// Get all unique peers across block and blob requests. @@ -170,9 +171,9 @@ impl SingleBlockLookup { } /// Add a child component to the lookup request. Merges with any existing child components. - pub fn add_child_components(&mut self, components: CachedChildComponents) { + pub fn add_child_components(&mut self, components: ChildComponentCache) { if let Some(ref mut existing_components) = self.cached_child_components { - let CachedChildComponents { + let ChildComponentCache { downloaded_block, downloaded_blobs, } = components; @@ -406,64 +407,6 @@ pub enum CachedChild { /// 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) - } -} - -impl CachedChildComponents { - pub fn new( - block: Option>>, - blobs: Option>, - ) -> Self { - Self { - downloaded_block: block, - downloaded_blobs: blobs.unwrap_or_default(), - } - } - - pub fn clear_blobs(&mut self) { - self.downloaded_blobs = FixedBlobSidecarList::default(); - } - - pub fn add_cached_child_block(&mut self, block: Arc>) { - self.merge_block(block) - } - - pub fn add_cached_child_blobs(&mut self, blobs: FixedBlobSidecarList) { - self.merge_blobs(blobs) - } - - pub fn processing_info(&self) -> ProcessingInfo { - ProcessingInfo::from_parts(self.downloaded_block.as_ref(), &self.downloaded_blobs) - .unwrap_or_default() - } -} - -impl_availability_view!( - CachedChildComponents, - Arc>, - Arc>, - downloaded_block, - downloaded_blobs -); - /// Object representing the state of a single block or blob lookup request. #[derive(PartialEq, Eq, Debug)] pub struct SingleLookupRequestState { @@ -639,7 +582,11 @@ mod tests { use crate::sync::block_lookups::common::LookupType; use crate::sync::block_lookups::common::{Lookup, RequestState}; use beacon_chain::builder::Witness; + use beacon_chain::data_availability_checker::{ + AvailabilityView, GetCommitment, GetCommitments, + }; use beacon_chain::eth1_chain::CachingEth1Backend; + use beacon_chain::generate_tests; use sloggers::null::NullLoggerBuilder; use sloggers::Build; use slot_clock::{SlotClock, TestingSlotClock}; diff --git a/beacon_node/network/src/sync/block_lookups/tests.rs b/beacon_node/network/src/sync/block_lookups/tests.rs index 1ba158af130..201054f49d8 100644 --- a/beacon_node/network/src/sync/block_lookups/tests.rs +++ b/beacon_node/network/src/sync/block_lookups/tests.rs @@ -1250,7 +1250,7 @@ mod deneb_only { block_root = child_root; bl.search_child_block( child_root, - CachedChildComponents::new(Some(child_block), None), + ChildComponentCache::new(Some(child_block), None), PeerShouldHave::Neither(peer_id), &mut cx, ); @@ -1288,7 +1288,7 @@ mod deneb_only { *blobs.index_mut(0) = Some(child_blob); bl.search_child_block( child_root, - CachedChildComponents::new(None, Some(blobs)), + ChildComponentCache::new(None, Some(blobs)), PeerShouldHave::Neither(peer_id), &mut cx, ); diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 9d0c8b5ab5b..3cdd2633b4e 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -43,10 +43,11 @@ 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::{BlobRequestState, BlockRequestState, CachedChildComponents}; +use crate::sync::block_lookups::{BlobRequestState, BlockRequestState}; use crate::sync::range_sync::ByRangeRequestType; use beacon_chain::block_verification_types::AsBlock; use beacon_chain::block_verification_types::RpcBlock; +use beacon_chain::data_availability_checker::ChildComponentCache; use beacon_chain::{ AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes, BlockError, EngineState, }; @@ -654,7 +655,7 @@ impl SyncManager { block_root, parent_root, blob_slot, - CachedChildComponents::new(None, Some(blobs)), + ChildComponentCache::new(None, Some(blobs)), ); } SyncMessage::UnknownBlockHashFromAttestation(peer_id, block_hash) => { @@ -745,7 +746,7 @@ impl SyncManager { block_root: Hash256, parent_root: Hash256, slot: Slot, - child_components: CachedChildComponents, + child_components: ChildComponentCache, ) { if self.should_search_for_block(slot, &peer_id) { self.block_lookups.search_parent( diff --git a/beacon_node/network/src/sync/mod.rs b/beacon_node/network/src/sync/mod.rs index b6ed1b3c3d5..7b244bceceb 100644 --- a/beacon_node/network/src/sync/mod.rs +++ b/beacon_node/network/src/sync/mod.rs @@ -9,6 +9,5 @@ mod network_context; mod peer_sync_info; mod range_sync; -pub use block_lookups::CachedChildComponents; pub use manager::{BatchProcessResult, SyncMessage}; pub use range_sync::{BatchOperationOutcome, ChainId}; From 787eb375bf5ade15aeaa799c19d9b933b7facc96 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Wed, 20 Sep 2023 20:20:25 -0400 Subject: [PATCH 12/47] cargo fix --- .../src/data_availability_checker/availability_view.rs | 9 +++------ beacon_node/beacon_chain/src/test_utils.rs | 5 ++--- .../src/sync/block_lookups/single_block_lookup.rs | 8 ++++---- 3 files changed, 9 insertions(+), 13 deletions(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index ca7604975b5..182520de639 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -3,15 +3,12 @@ use crate::blob_verification::KzgVerifiedBlob; use crate::block_verification_types::AsBlock; use crate::data_availability_checker::overflow_lru_cache::PendingComponents; use crate::data_availability_checker::ProcessingInfo; -use crate::test_utils::{generate_rand_block_and_blobs, NumBlobs}; use crate::AvailabilityPendingExecutedBlock; -use eth2_network_config::get_trusted_setup; -use kzg::{KzgCommitment, TrustedSetup}; +use kzg::{KzgCommitment}; use ssz_types::FixedVector; use std::sync::Arc; use types::beacon_block_body::KzgCommitments; -use types::test_utils::TestRandom; -use types::{BlobSidecar, EthSpec, ForkName, MainnetEthSpec, SignedBeaconBlock}; +use types::{BlobSidecar, EthSpec, SignedBeaconBlock}; /// Defines an interface for managing data availability with two key invariants: /// 1. Blobs won't be clobbered if we've yet to see the corresponding block. @@ -269,7 +266,7 @@ pub mod tests { use rand::SeedableRng; use state_processing::ConsensusContext; use types::test_utils::TestRandom; - use types::{BeaconState, ChainSpec, ForkName, Slot}; + use types::{BeaconState, ChainSpec, ForkName, MainnetEthSpec, Slot}; type E = MainnetEthSpec; pub fn pre_setup() -> ( diff --git a/beacon_node/beacon_chain/src/test_utils.rs b/beacon_node/beacon_chain/src/test_utils.rs index 44b5b20c0ee..04720cae1b8 100644 --- a/beacon_node/beacon_chain/src/test_utils.rs +++ b/beacon_node/beacon_chain/src/test_utils.rs @@ -28,7 +28,7 @@ use execution_layer::{ use futures::channel::mpsc::Receiver; pub use genesis::{interop_genesis_state_with_eth1, DEFAULT_ETH1_BLOCK_HASH}; use int_to_bytes::int_to_bytes32; -use kzg::{Kzg, KzgPreset, TrustedSetup}; +use kzg::{Kzg, TrustedSetup}; use merkle_proof::MerkleTree; use operation_pool::ReceivedPreCapella; use parking_lot::RwLockWriteGuard; @@ -38,7 +38,6 @@ use rand::Rng; use rand::SeedableRng; use rayon::prelude::*; use sensitive_url::SensitiveUrl; -use slasher::test_utils::E; use slog::{o, Drain, Logger}; use slog_async::Async; use slog_term::{FullFormat, TermDecorator}; @@ -61,7 +60,7 @@ use task_executor::{test_utils::TestRuntime, ShutdownReason}; use tree_hash::TreeHash; use types::sync_selection_proof::SyncSelectionProof; pub use types::test_utils::generate_deterministic_keypairs; -use types::test_utils::{TestRandom, XorShiftRng}; +use types::test_utils::{TestRandom}; use types::{typenum::U4294967296, *}; // 4th September 2019 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 09cfd431f0b..adc254dd178 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 @@ -5,13 +5,13 @@ use crate::sync::network_context::SyncNetworkContext; use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::data_availability_checker::ChildComponentCache; use beacon_chain::data_availability_checker::{ - AvailabilityCheckError, AvailabilityView, DataAvailabilityChecker, + AvailabilityCheckError, DataAvailabilityChecker, }; -use beacon_chain::{impl_availability_view, BeaconChainTypes}; +use beacon_chain::{BeaconChainTypes}; use lighthouse_network::rpc::methods::MaxRequestBlobSidecars; use lighthouse_network::{PeerAction, PeerId}; use slog::{trace, Logger}; -use ssz_types::{FixedVector, VariableList}; +use ssz_types::{VariableList}; use std::collections::HashSet; use std::fmt::Debug; use std::marker::PhantomData; @@ -19,7 +19,7 @@ use std::sync::Arc; use store::Hash256; use strum::IntoStaticStr; use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; -use types::{BlobSidecar, EthSpec, SignedBeaconBlock}; +use types::{EthSpec}; #[derive(Debug, PartialEq, Eq)] pub enum State { From c5263b67ac2db68f8cdc4560a6adc843545cd9e2 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Wed, 20 Sep 2023 20:22:20 -0400 Subject: [PATCH 13/47] cargo fix --- .../src/data_availability_checker/availability_view.rs | 3 +-- .../network/src/sync/block_lookups/single_block_lookup.rs | 4 ---- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index 182520de639..48eaefcedd6 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -412,10 +412,9 @@ pub mod tests { pub fn assert_cache_consistent>(cache: A) { if let Some(cached_block) = cache.get_cached_block() { if let Some(cached_block_commitments) = cached_block.get_commitments() { - for (index, (block_commitment, blob_commitment_opt)) in cached_block_commitments + for (block_commitment, blob_commitment_opt) in cached_block_commitments .iter() .zip(cache.get_cached_blobs().iter()) - .enumerate() { let blob_commitment = blob_commitment_opt .as_ref() 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 adc254dd178..edc4da4c330 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 @@ -582,11 +582,7 @@ mod tests { use crate::sync::block_lookups::common::LookupType; use crate::sync::block_lookups::common::{Lookup, RequestState}; use beacon_chain::builder::Witness; - use beacon_chain::data_availability_checker::{ - AvailabilityView, GetCommitment, GetCommitments, - }; use beacon_chain::eth1_chain::CachingEth1Backend; - use beacon_chain::generate_tests; use sloggers::null::NullLoggerBuilder; use sloggers::Build; use slot_clock::{SlotClock, TestingSlotClock}; From 90e3e4ed1f6364b07cce38c97dece01f4f789846 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Wed, 20 Sep 2023 20:24:59 -0400 Subject: [PATCH 14/47] cargo fix --- beacon_node/network/src/sync/block_lookups/tests.rs | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/beacon_node/network/src/sync/block_lookups/tests.rs b/beacon_node/network/src/sync/block_lookups/tests.rs index 201054f49d8..4af6ccd27ee 100644 --- a/beacon_node/network/src/sync/block_lookups/tests.rs +++ b/beacon_node/network/src/sync/block_lookups/tests.rs @@ -20,10 +20,8 @@ use slot_clock::{ManualSlotClock, SlotClock, TestingSlotClock}; use store::MemoryStore; use tokio::sync::mpsc; use types::{ - map_fork_name, map_fork_name_with, - test_utils::{SeedableRng, TestRandom, XorShiftRng}, - BeaconBlock, BlobSidecar, EthSpec, ForkName, FullPayload, FullPayloadDeneb, - MinimalEthSpec as E, MinimalEthSpec, SignedBeaconBlock, + test_utils::{SeedableRng, XorShiftRng}, BlobSidecar, EthSpec, ForkName, + MinimalEthSpec as E, SignedBeaconBlock, }; type T = Witness, E, MemoryStore, MemoryStore>; From 2534dff692e1c53947f1da7513e0d325a53b423e Mon Sep 17 00:00:00 2001 From: realbigsean Date: Wed, 20 Sep 2023 20:31:50 -0400 Subject: [PATCH 15/47] fmt and lint --- .../availability_view.rs | 51 +++++++++++-------- beacon_node/beacon_chain/src/test_utils.rs | 2 +- .../sync/block_lookups/single_block_lookup.rs | 10 ++-- .../network/src/sync/block_lookups/tests.rs | 4 +- 4 files changed, 37 insertions(+), 30 deletions(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index 48eaefcedd6..90ad34f3473 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -4,7 +4,7 @@ use crate::block_verification_types::AsBlock; use crate::data_availability_checker::overflow_lru_cache::PendingComponents; use crate::data_availability_checker::ProcessingInfo; use crate::AvailabilityPendingExecutedBlock; -use kzg::{KzgCommitment}; +use kzg::KzgCommitment; use ssz_types::FixedVector; use std::sync::Arc; use types::beacon_block_body::KzgCommitments; @@ -269,11 +269,14 @@ pub mod tests { use types::{BeaconState, ChainSpec, ForkName, MainnetEthSpec, Slot}; type E = MainnetEthSpec; - pub fn pre_setup() -> ( + + type Setup = ( SignedBeaconBlock, FixedVector>, ::MaxBlobsPerBlock>, FixedVector>, ::MaxBlobsPerBlock>, - ) { + ); + + pub fn pre_setup() -> Setup { let trusted_setup: TrustedSetup = serde_json::from_reader(get_trusted_setup::<::Kzg>()).unwrap(); let kzg = Kzg::new_from_trusted_setup(trusted_setup).unwrap(); @@ -304,15 +307,17 @@ pub mod tests { (block, blobs, invalid_blobs) } + type ProcessingInfoSetup = ( + KzgCommitments, + FixedVector, ::MaxBlobsPerBlock>, + FixedVector, ::MaxBlobsPerBlock>, + ); + pub fn setup_processing_info( block: SignedBeaconBlock, valid_blobs: FixedVector>, ::MaxBlobsPerBlock>, invalid_blobs: FixedVector>, ::MaxBlobsPerBlock>, - ) -> ( - KzgCommitments, - FixedVector, ::MaxBlobsPerBlock>, - FixedVector, ::MaxBlobsPerBlock>, - ) { + ) -> ProcessingInfoSetup { let commitments = block .message() .body() @@ -334,15 +339,17 @@ pub mod tests { (commitments, blobs, invalid_blobs) } + type PendingComponentsSetup = ( + AvailabilityPendingExecutedBlock, + FixedVector>, ::MaxBlobsPerBlock>, + FixedVector>, ::MaxBlobsPerBlock>, + ); + pub fn setup_pending_components( block: SignedBeaconBlock, valid_blobs: FixedVector>, ::MaxBlobsPerBlock>, invalid_blobs: FixedVector>, ::MaxBlobsPerBlock>, - ) -> ( - AvailabilityPendingExecutedBlock, - FixedVector>, ::MaxBlobsPerBlock>, - FixedVector>, ::MaxBlobsPerBlock>, - ) { + ) -> PendingComponentsSetup { let blobs = FixedVector::from( valid_blobs .iter() @@ -385,25 +392,27 @@ pub mod tests { (block, blobs, invalid_blobs) } + type ChildComponentsSetup = ( + Arc>, + FixedVector>>, ::MaxBlobsPerBlock>, + FixedVector>>, ::MaxBlobsPerBlock>, + ); + pub fn setup_child_components( block: SignedBeaconBlock, valid_blobs: FixedVector>, ::MaxBlobsPerBlock>, invalid_blobs: FixedVector>, ::MaxBlobsPerBlock>, - ) -> ( - Arc>, - FixedVector>>, ::MaxBlobsPerBlock>, - FixedVector>>, ::MaxBlobsPerBlock>, - ) { + ) -> ChildComponentsSetup { let blobs = FixedVector::from( valid_blobs .into_iter() - .map(|blob_opt| blob_opt.clone().map(|blob| Arc::new(blob))) + .map(|blob_opt| blob_opt.clone().map(Arc::new)) .collect::>(), ); let invalid_blobs = FixedVector::from( invalid_blobs .into_iter() - .map(|blob_opt| blob_opt.clone().map(|blob| Arc::new(blob))) + .map(|blob_opt| blob_opt.clone().map(Arc::new)) .collect::>(), ); (Arc::new(block), blobs, invalid_blobs) @@ -426,7 +435,7 @@ pub mod tests { panic!("Cached block has no commitments") } } else { - panic!() + panic!("No cached block") } } diff --git a/beacon_node/beacon_chain/src/test_utils.rs b/beacon_node/beacon_chain/src/test_utils.rs index 04720cae1b8..3439e34cab9 100644 --- a/beacon_node/beacon_chain/src/test_utils.rs +++ b/beacon_node/beacon_chain/src/test_utils.rs @@ -60,7 +60,7 @@ use task_executor::{test_utils::TestRuntime, ShutdownReason}; use tree_hash::TreeHash; use types::sync_selection_proof::SyncSelectionProof; pub use types::test_utils::generate_deterministic_keypairs; -use types::test_utils::{TestRandom}; +use types::test_utils::TestRandom; use types::{typenum::U4294967296, *}; // 4th September 2019 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 edc4da4c330..976dbd8e6bc 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 @@ -4,14 +4,12 @@ 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::ChildComponentCache; -use beacon_chain::data_availability_checker::{ - AvailabilityCheckError, DataAvailabilityChecker, -}; -use beacon_chain::{BeaconChainTypes}; +use beacon_chain::data_availability_checker::{AvailabilityCheckError, DataAvailabilityChecker}; +use beacon_chain::BeaconChainTypes; use lighthouse_network::rpc::methods::MaxRequestBlobSidecars; use lighthouse_network::{PeerAction, PeerId}; use slog::{trace, Logger}; -use ssz_types::{VariableList}; +use ssz_types::VariableList; use std::collections::HashSet; use std::fmt::Debug; use std::marker::PhantomData; @@ -19,7 +17,7 @@ use std::sync::Arc; use store::Hash256; use strum::IntoStaticStr; use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; -use types::{EthSpec}; +use types::EthSpec; #[derive(Debug, PartialEq, Eq)] pub enum State { diff --git a/beacon_node/network/src/sync/block_lookups/tests.rs b/beacon_node/network/src/sync/block_lookups/tests.rs index 4af6ccd27ee..00d1c8778ed 100644 --- a/beacon_node/network/src/sync/block_lookups/tests.rs +++ b/beacon_node/network/src/sync/block_lookups/tests.rs @@ -20,8 +20,8 @@ use slot_clock::{ManualSlotClock, SlotClock, TestingSlotClock}; use store::MemoryStore; use tokio::sync::mpsc; use types::{ - test_utils::{SeedableRng, XorShiftRng}, BlobSidecar, EthSpec, ForkName, - MinimalEthSpec as E, SignedBeaconBlock, + test_utils::{SeedableRng, XorShiftRng}, + BlobSidecar, EthSpec, ForkName, MinimalEthSpec as E, SignedBeaconBlock, }; type T = Witness, E, MemoryStore, MemoryStore>; From 65fa03384888a234006485d7749264c1f6776969 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 21 Sep 2023 11:41:44 -0400 Subject: [PATCH 16/47] make blob commitments not optional, rename some caches, add missing blobs struct --- beacon_node/beacon_chain/src/beacon_chain.rs | 1 + beacon_node/beacon_chain/src/builder.rs | 2 +- .../src/data_availability_checker.rs | 155 +++++++++++++++--- .../availability_view.rs | 90 +++++----- .../child_component_cache.rs | 17 +- .../processing_cache.rs | 46 +----- .../network/src/sync/block_lookups/common.rs | 8 +- .../src/sync/block_lookups/parent_lookup.rs | 2 +- .../sync/block_lookups/single_block_lookup.rs | 100 +++++------ consensus/types/src/signed_beacon_block.rs | 33 ---- 10 files changed, 242 insertions(+), 212 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 02ecddf05a0..788db2a1c2f 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -2796,6 +2796,7 @@ impl BeaconChain { self: &Arc, blob: GossipVerifiedBlob, ) -> Result> { + // TODO: fewer places where we initialize this fixed vec let block_root = blob.block_root(); let mut commitments = KzgCommitmentOpts::::default(); if let Some(commitment_opt) = commitments.get_mut(blob.as_blob().index as usize) { diff --git a/beacon_node/beacon_chain/src/builder.rs b/beacon_node/beacon_chain/src/builder.rs index a31a364cce9..42d9fffce0c 100644 --- a/beacon_node/beacon_chain/src/builder.rs +++ b/beacon_node/beacon_chain/src/builder.rs @@ -916,7 +916,7 @@ where validator_monitor: RwLock::new(validator_monitor), genesis_backfill_slot, data_availability_checker: Arc::new( - DataAvailabilityChecker::new(slot_clock, kzg.clone(), store, self.spec) + DataAvailabilityChecker::new(slot_clock, kzg.clone(), store, &log, self.spec) .map_err(|e| format!("Error initializing DataAvailabiltyChecker: {:?}", e))?, ), kzg, diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index f5bdb5994ab..cb46cae4757 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -9,13 +9,15 @@ pub use crate::data_availability_checker::child_component_cache::ChildComponentC use crate::data_availability_checker::overflow_lru_cache::OverflowLRUCache; use crate::data_availability_checker::processing_cache::ProcessingCache; use crate::{BeaconChain, BeaconChainTypes, BeaconStore}; +use futures::StreamExt; use kzg::Kzg; use kzg::{Error as KzgError, KzgCommitment}; use parking_lot::RwLock; -pub use processing_cache::ProcessingInfo; -use slog::{debug, error}; +pub use processing_cache::ProcessingView; +use slasher::test_utils::E; +use slog::{debug, error, Logger}; use slot_clock::SlotClock; -use ssz_types::Error; +use ssz_types::{Error, VariableList}; use std::fmt; use std::fmt::Debug; use std::sync::Arc; @@ -86,6 +88,7 @@ pub struct DataAvailabilityChecker { availability_cache: Arc>, slot_clock: T::SlotClock, kzg: Option::Kzg>>>, + log: Logger, spec: ChainSpec, } @@ -115,6 +118,7 @@ impl DataAvailabilityChecker { slot_clock: T::SlotClock, kzg: Option::Kzg>>>, store: BeaconStore, + log: &Logger, spec: ChainSpec, ) -> Result { let overflow_cache = OverflowLRUCache::new(OVERFLOW_LRU_CAPACITY, store)?; @@ -122,6 +126,7 @@ impl DataAvailabilityChecker { processing_cache: <_>::default(), availability_cache: Arc::new(overflow_cache), slot_clock, + log: log.clone(), kzg, spec, }) @@ -132,29 +137,83 @@ impl DataAvailabilityChecker { self.processing_cache.read().has_block(block_root) } - /// Checks which blob ids are still required for a given block root, taking any cached - /// components into consideration. - pub fn get_missing_blob_ids_checking_cache( - &self, - block_root: Hash256, - ) -> Option> { - let guard = self.processing_cache.read(); - self.get_missing_blob_ids(block_root, guard.get(&block_root)?) + /// Get the processing info for a block. + pub fn get_processing_view(&self, block_root: Hash256) -> Option> { + self.processing_cache.read().get(&block_root).cloned() } /// A `None` indicates blobs are not required. /// /// If there's no block, all possible ids will be returned that don't exist in the given blobs. /// If there no blobs, all possible ids will be returned. - pub fn get_missing_blob_ids( + pub fn get_missing_blob_ids>( &self, block_root: Hash256, - processing_info: &ProcessingInfo, - ) -> Option> { - let epoch = self.slot_clock.now()?.epoch(T::EthSpec::slots_per_epoch()); + availability_view: &V, + ) -> MissingBlobs { + let Some(current_slot) = self.slot_clock.now_or_genesis() else { + error!( + self.log, + "Failed to read slot clock when checking for missing blob ids" + ); + return MissingBlobs::BlobsNotRequired; + }; - self.da_check_required_for_epoch(epoch) - .then(|| processing_info.get_missing_blob_ids(block_root)) + let current_epoch = current_slot.epoch(T::EthSpec::slots_per_epoch()); + + if self.da_check_required_for_epoch(current_epoch) { + match availability_view.get_cached_block() { + Some(cached_block) => { + let block_commitments = cached_block.get_commitments(); + let blob_commitments = availability_view.get_cached_blobs(); + + let num_blobs_expected = block_commitments.len(); + let mut blob_ids = Vec::with_capacity(num_blobs_expected); + + // Zip here will always limit the number of iterations to the size of + // `block_commitment` because `blob_commitments` will always be populated + // with `Option` values up to `MAX_BLOBS_PER_BLOCK`. + for (index, (block_commitment, blob_commitment_opt)) in block_commitments + .into_iter() + .zip(blob_commitments.iter()) + .enumerate() + { + // Always add a missing blob. + let Some(blob_commitment) = blob_commitment_opt else { + blob_ids.push(BlobIdentifier { + block_root, + index: index as u64, + }); + continue; + }; + + let blob_commitment = *blob_commitment.get_commitment(); + + // Check for consistency, but this shouldn't happen, an availability view + // should guaruntee consistency. + if blob_commitment != block_commitment { + error!(self.log, + "Inconsistent availability view"; + "block_root" => ?block_root, + "block_commitment" => ?block_commitment, + "blob_commitment" => ?blob_commitment, + "index" => index + ); + blob_ids.push(BlobIdentifier { + block_root, + index: index as u64, + }); + } + } + MissingBlobs::KnownMissing(blob_ids) + } + None => { + MissingBlobs::PossibleMissing(BlobIdentifier::get_all_blob_ids::(block_root)) + } + } + } else { + MissingBlobs::BlobsNotRequired + } } /// Get a blob from the availability cache. @@ -267,7 +326,7 @@ impl DataAvailabilityChecker { self.processing_cache .write() .entry(block_root) - .or_insert_with(ProcessingInfo::default) + .or_insert_with(ProcessingView::default) .merge_block(commitments); } @@ -279,7 +338,7 @@ impl DataAvailabilityChecker { self.processing_cache .write() .entry(block_root) - .or_insert_with(ProcessingInfo::default) + .or_insert_with(ProcessingView::default) .merge_blobs(blobs); } @@ -476,3 +535,61 @@ pub enum MaybeAvailableBlock { /// This variant is not fully available and requires blobs to become fully available. AvailabilityPending(Arc>), } + +#[derive(Debug, Clone)] +pub enum MissingBlobs { + /// We know for certain these blobs are missing. + KnownMissing(Vec), + /// We think these blobs might be missing. + PossibleMissing(Vec), + /// Blobs are not required. + BlobsNotRequired, +} + +impl MissingBlobs { + pub fn new_without_block(block_root: Hash256, is_deneb: bool) -> Self { + if is_deneb { + MissingBlobs::PossibleMissing(BlobIdentifier::get_all_blob_ids::(block_root)) + } else { + MissingBlobs::BlobsNotRequired + } + } + pub fn is_empty(&self) -> bool { + match self { + MissingBlobs::KnownMissing(v) => v.is_empty(), + MissingBlobs::PossibleMissing(v) => v.is_empty(), + MissingBlobs::BlobsNotRequired => true, + } + } + pub fn contains(&self, blob_id: &BlobIdentifier) -> bool { + match self { + MissingBlobs::KnownMissing(v) => v.contains(blob_id), + MissingBlobs::PossibleMissing(v) => v.contains(blob_id), + MissingBlobs::BlobsNotRequired => false, + } + } + pub fn remove(&mut self, blob_id: &BlobIdentifier) { + match self { + MissingBlobs::KnownMissing(v) => v.retain(|id| id != blob_id), + MissingBlobs::PossibleMissing(v) => v.retain(|id| id != blob_id), + MissingBlobs::BlobsNotRequired => {} + } + } + pub fn indices(&self) -> Vec { + match self { + MissingBlobs::KnownMissing(v) => v.into_iter().map(|id| id.index).collect(), + MissingBlobs::PossibleMissing(v) => v.into_iter().map(|id| id.index).collect(), + MissingBlobs::BlobsNotRequired => vec![], + } + } +} + +impl Into> for MissingBlobs { + fn into(self) -> Vec { + match self { + MissingBlobs::KnownMissing(v) => v, + MissingBlobs::PossibleMissing(v) => v, + MissingBlobs::BlobsNotRequired => vec![], + } + } +} diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index 90ad34f3473..a386776cb0b 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -2,9 +2,11 @@ use super::child_component_cache::ChildComponentCache; use crate::blob_verification::KzgVerifiedBlob; use crate::block_verification_types::AsBlock; use crate::data_availability_checker::overflow_lru_cache::PendingComponents; -use crate::data_availability_checker::ProcessingInfo; +use crate::data_availability_checker::ProcessingView; use crate::AvailabilityPendingExecutedBlock; use kzg::KzgCommitment; +use slasher::test_utils::E; +use slog::b; use ssz_types::FixedVector; use std::sync::Arc; use types::beacon_block_body::KzgCommitments; @@ -58,15 +60,14 @@ pub trait AvailabilityView { .unwrap_or(false) } - /// Returns the number of blobs that are expected to be present. Returns 0 if we don't have a + /// Returns the number of blobs that are expected to be present. Returns `None` if we don't have a /// block. /// /// This corresponds to the number of commitments that are present in a block. - fn num_expected_blobs(&self) -> usize { + fn num_expected_blobs(&self) -> Option { self.get_cached_block() .as_ref() - .and_then(|b| b.get_commitments()) - .map_or(0, |c| c.len()) + .map(|b| b.get_commitments().len()) } /// Returns the number of blobs that have been received and are stored in the cache. @@ -100,12 +101,11 @@ pub trait AvailabilityView { let index = index as u64; - if let Some(block_commitments) = self.get_cached_block_mut() { - if let Some(block_commitment) = block_commitments.get_commitments() { - if let Some(&bc) = block_commitment.get(index as usize) { - if bc == commitment { - self.insert_blob_at_index(index, blob) - } + if let Some(cached_block) = self.get_cached_block() { + let block_commitment = cached_block.get_commitments(); + if let Some(&bc) = block_commitment.get(index as usize) { + if bc == commitment { + self.insert_blob_at_index(index, blob) } } } else if !self.blob_exists(index) { @@ -138,7 +138,11 @@ pub trait AvailabilityView { /// Returns `true` if both the block exists and the number of received blobs matches the number /// of expected blobs. fn is_available(&self) -> bool { - self.block_exists() && self.num_expected_blobs() == self.num_received_blobs() + if let Some(num_expected_blobs) = self.num_expected_blobs() { + num_expected_blobs == self.num_received_blobs() + } else { + false + } } } @@ -180,7 +184,7 @@ macro_rules! impl_availability_view { } impl_availability_view!( - ProcessingInfo, + ProcessingView, KzgCommitments, KzgCommitment, kzg_commitments, @@ -204,17 +208,17 @@ impl_availability_view!( ); pub trait GetCommitments { - fn get_commitments(&self) -> Option>; + fn get_commitments(&self) -> KzgCommitments; } pub trait GetCommitment { fn get_commitment(&self) -> &KzgCommitment; } -// These implementations are required to implement `AvailabilityView` for `ProcessingInfo`. +// These implementations are required to implement `AvailabilityView` for `ProcessingView`. impl GetCommitments for KzgCommitments { - fn get_commitments(&self) -> Option> { - Some(self.clone()) + fn get_commitments(&self) -> KzgCommitments { + self.clone() } } impl GetCommitment for KzgCommitment { @@ -225,13 +229,13 @@ impl GetCommitment for KzgCommitment { // These implementations are required to implement `AvailabilityView` for `PendingComponents`. impl GetCommitments for AvailabilityPendingExecutedBlock { - fn get_commitments(&self) -> Option> { + fn get_commitments(&self) -> KzgCommitments { self.as_block() .message() .body() .blob_kzg_commitments() - .ok() .cloned() + .unwrap_or_default() } } impl GetCommitment for KzgVerifiedBlob { @@ -242,8 +246,13 @@ impl GetCommitment for KzgVerifiedBlob { // These implementations are required to implement `AvailabilityView` for `CachedChildComponents`. impl GetCommitments for Arc> { - fn get_commitments(&self) -> Option> { - self.message().body().blob_kzg_commitments().ok().cloned() + fn get_commitments(&self) -> KzgCommitments { + self.message() + .body() + .blob_kzg_commitments() + .ok() + .cloned() + .unwrap_or_default() } } impl GetCommitment for Arc> { @@ -307,17 +316,17 @@ pub mod tests { (block, blobs, invalid_blobs) } - type ProcessingInfoSetup = ( + type ProcessingViewSetup = ( KzgCommitments, FixedVector, ::MaxBlobsPerBlock>, FixedVector, ::MaxBlobsPerBlock>, ); - pub fn setup_processing_info( + pub fn setup_processing_view( block: SignedBeaconBlock, valid_blobs: FixedVector>, ::MaxBlobsPerBlock>, invalid_blobs: FixedVector>, ::MaxBlobsPerBlock>, - ) -> ProcessingInfoSetup { + ) -> ProcessingViewSetup { let commitments = block .message() .body() @@ -418,28 +427,25 @@ pub mod tests { (Arc::new(block), blobs, invalid_blobs) } - pub fn assert_cache_consistent>(cache: A) { + pub fn assert_cache_consistent>(cache: V) { if let Some(cached_block) = cache.get_cached_block() { - if let Some(cached_block_commitments) = cached_block.get_commitments() { - for (block_commitment, blob_commitment_opt) in cached_block_commitments - .iter() - .zip(cache.get_cached_blobs().iter()) - { - let blob_commitment = blob_commitment_opt - .as_ref() - .map(|b| *b.get_commitment()) - .unwrap(); - assert_eq!(*block_commitment, blob_commitment); - } - } else { - panic!("Cached block has no commitments") + let cached_block_commitments = cached_block.get_commitments(); + for (block_commitment, blob_commitment_opt) in cached_block_commitments + .iter() + .zip(cache.get_cached_blobs().iter()) + { + let blob_commitment = blob_commitment_opt + .as_ref() + .map(|b| *b.get_commitment()) + .unwrap(); + assert_eq!(*block_commitment, blob_commitment); } } else { panic!("No cached block") } } - pub fn assert_empty_blob_cache>(cache: A) { + pub fn assert_empty_blob_cache>(cache: V) { for blob in cache.get_cached_blobs().iter() { assert!(blob.is_none()); } @@ -539,11 +545,11 @@ pub mod tests { } generate_tests!( - processing_info_tests, - ProcessingInfo::, + processing_view_tests, + ProcessingView::, kzg_commitments, processing_blobs, - setup_processing_info + setup_processing_view ); generate_tests!( pending_components_tests, diff --git a/beacon_node/beacon_chain/src/data_availability_checker/child_component_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/child_component_cache.rs index c0139c4267e..400abec401f 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/child_component_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/child_component_cache.rs @@ -1,5 +1,5 @@ use crate::block_verification_types::RpcBlock; -use crate::data_availability_checker::{AvailabilityView, ProcessingInfo}; +use crate::data_availability_checker::{AvailabilityView, ProcessingView}; use std::sync::Arc; use types::blob_sidecar::FixedBlobSidecarList; use types::{EthSpec, SignedBeaconBlock}; @@ -29,10 +29,14 @@ impl ChildComponentCache { block: Option>>, blobs: Option>, ) -> Self { - Self { - downloaded_block: block, - downloaded_blobs: blobs.unwrap_or_default(), + let mut cache = Self::default(); + if let Some(block) = block { + cache.merge_block(block); } + if let Some(blobs) = blobs { + cache.merge_blobs(blobs); + } + cache } pub fn clear_blobs(&mut self) { @@ -46,9 +50,4 @@ impl ChildComponentCache { pub fn add_cached_child_blobs(&mut self, blobs: FixedBlobSidecarList) { self.merge_blobs(blobs) } - - pub fn processing_info(&self) -> ProcessingInfo { - ProcessingInfo::from_parts(self.downloaded_block.as_ref(), &self.downloaded_blobs) - .unwrap_or_default() - } } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs index 9ddf5dbb8cf..9abf69ac011 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -5,7 +5,6 @@ use std::collections::HashMap; use std::sync::Arc; use types::beacon_block_body::{KzgCommitmentOpts, KzgCommitments}; use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; -use types::signed_beacon_block::get_missing_blob_ids; use types::{EthSpec, Hash256, SignedBeaconBlock, Slot}; /// This cache is used only for gossip blocks/blobs and single block/blob lookups, to give req/resp @@ -14,14 +13,14 @@ use types::{EthSpec, Hash256, SignedBeaconBlock, Slot}; /// See `AvailabilityView`'s trait definition. #[derive(Default)] pub struct ProcessingCache { - processing_cache: HashMap>, + processing_cache: HashMap>, } impl ProcessingCache { - pub fn get(&self, block_root: &Hash256) -> Option<&ProcessingInfo> { + pub fn get(&self, block_root: &Hash256) -> Option<&ProcessingView> { self.processing_cache.get(block_root) } - pub fn entry(&mut self, block_root: Hash256) -> Entry<'_, Hash256, ProcessingInfo> { + pub fn entry(&mut self, block_root: Hash256) -> Entry<'_, Hash256, ProcessingView> { self.processing_cache.entry(block_root) } pub fn remove(&mut self, block_root: &Hash256) { @@ -43,8 +42,8 @@ impl ProcessingCache { } } -#[derive(Default, Debug)] -pub struct ProcessingInfo { +#[derive(Default, Debug, Clone)] +pub struct ProcessingView { slot: Slot, /// Blobs required for a block can only be known if we have seen the block. So `Some` here /// means we've seen it, a `None` means we haven't. The `kzg_commitments` value helps us figure @@ -55,38 +54,3 @@ pub struct ProcessingInfo { /// no way of knowing which blob is the correct one until we see the block. pub processing_blobs: KzgCommitmentOpts, } - -impl ProcessingInfo { - pub fn from_parts( - block: Option<&Arc>>, - blobs: &FixedBlobSidecarList, - ) -> Option { - let block_slot = block.map(|block| block.message().slot()); - let blob_slot = blobs.iter().find_map(|b| b.as_ref()).map(|b| b.slot); - let slot = block_slot.or(blob_slot)?; - let block_commitments = block.map(|block| { - block - .message() - .body() - .blob_kzg_commitments() - .cloned() - .unwrap_or_default() - }); - let blobs = blobs - .iter() - .map(|blob_opt| blob_opt.as_ref().map(|blob| blob.kzg_commitment)) - .collect::>(); - Some(Self { - slot, - kzg_commitments: block_commitments, - processing_blobs: FixedVector::new(blobs).ok()?, - }) - } - pub fn get_missing_blob_ids(&self, block_root: Hash256) -> Vec { - get_missing_blob_ids::( - block_root, - self.kzg_commitments.as_ref(), - &self.processing_blobs, - ) - } -} diff --git a/beacon_node/network/src/sync/block_lookups/common.rs b/beacon_node/network/src/sync/block_lookups/common.rs index 91e217cdca3..f1abe16dddc 100644 --- a/beacon_node/network/src/sync/block_lookups/common.rs +++ b/beacon_node/network/src/sync/block_lookups/common.rs @@ -19,7 +19,7 @@ use ssz_types::VariableList; use std::ops::IndexMut; use std::sync::Arc; use std::time::Duration; -use types::blob_sidecar::FixedBlobSidecarList; +use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; use types::{BlobSidecar, EthSpec, Hash256, SignedBeaconBlock}; #[derive(Debug, Copy, Clone)] @@ -375,9 +375,9 @@ impl RequestState for BlobRequestState; fn new_request(&self) -> BlobsByRootRequest { - BlobsByRootRequest { - blob_ids: self.requested_ids.clone().into(), - } + let blob_id_vec: Vec = self.requested_ids.clone().into(); + let blob_ids = VariableList::from(blob_id_vec); + BlobsByRootRequest { blob_ids } } fn make_request( 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 93f2615c0ff..7aa674c57fd 100644 --- a/beacon_node/network/src/sync/block_lookups/parent_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/parent_lookup.rs @@ -178,7 +178,7 @@ impl ParentLookup { .blob_request_state .state .register_failure_processing(); - if let Some(components) = self.current_parent_request.cached_child_components.as_mut() { + if let Some(components) = self.current_parent_request.child_component_cache.as_mut() { components.downloaded_block = None; components.downloaded_blobs = <_>::default(); } 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 976dbd8e6bc..913a445d089 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 @@ -3,8 +3,10 @@ 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::ChildComponentCache; -use beacon_chain::data_availability_checker::{AvailabilityCheckError, DataAvailabilityChecker}; +use beacon_chain::data_availability_checker::{ + AvailabilityCheckError, DataAvailabilityChecker, MissingBlobs, +}; +use beacon_chain::data_availability_checker::{AvailabilityView, ChildComponentCache}; use beacon_chain::BeaconChainTypes; use lighthouse_network::rpc::methods::MaxRequestBlobSidecars; use lighthouse_network::{PeerAction, PeerId}; @@ -59,23 +61,24 @@ pub struct SingleBlockLookup { pub da_checker: Arc>, /// Only necessary for requests triggered by an `UnknownBlockParent` or `UnknownBlockParent` /// because any blocks or blobs without parents won't hit the data availability cache. - pub cached_child_components: Option>, + pub child_component_cache: Option>, } impl SingleBlockLookup { pub fn new( requested_block_root: Hash256, - unknown_parent_components: Option>, + child_component_cache: Option>, peers: &[PeerShouldHave], da_checker: Arc>, id: Id, ) -> Self { + let is_deneb = da_checker.is_deneb(); Self { id, block_request_state: BlockRequestState::new(requested_block_root, peers), - blob_request_state: BlobRequestState::new(peers), + blob_request_state: BlobRequestState::new(requested_block_root, peers, is_deneb), da_checker, - cached_child_components: unknown_parent_components, + child_component_cache, } } @@ -95,7 +98,7 @@ impl SingleBlockLookup { self.block_request_state.requested_block_root = block_root; self.block_request_state.state.state = State::AwaitingDownload; self.blob_request_state.state.state = State::AwaitingDownload; - self.cached_child_components = Some(ChildComponentCache::default()); + self.child_component_cache = Some(ChildComponentCache::default()); } /// Get all unique peers across block and blob requests. @@ -135,7 +138,7 @@ impl SingleBlockLookup { /// 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() { + if let Some(components) = self.child_component_cache.as_ref() { let Some(block) = components.downloaded_block.as_ref() else { return CachedChild::DownloadIncomplete; }; @@ -160,8 +163,8 @@ impl SingleBlockLookup { &mut self, verified_response: R::VerifiedResponseType, ) -> CachedChild { - if let Some(cached_child_components) = self.cached_child_components.as_mut() { - R::add_to_child_components(verified_response, cached_child_components); + if let Some(child_component_cache) = self.child_component_cache.as_mut() { + R::add_to_child_components(verified_response, child_component_cache); self.get_cached_child_block() } else { CachedChild::NotRequired @@ -170,7 +173,7 @@ impl SingleBlockLookup { /// Add a child component to the lookup request. Merges with any existing child components. pub fn add_child_components(&mut self, components: ChildComponentCache) { - if let Some(ref mut existing_components) = self.cached_child_components { + if let Some(ref mut existing_components) = self.child_component_cache { let ChildComponentCache { downloaded_block, downloaded_blobs, @@ -180,7 +183,7 @@ impl SingleBlockLookup { } existing_components.add_cached_child_blobs(downloaded_blobs); } else { - self.cached_child_components = Some(components); + self.child_component_cache = Some(components); } } @@ -244,8 +247,8 @@ impl SingleBlockLookup { /// 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() + if let Some(components) = self.child_component_cache.as_ref() { + components.block_exists() } else { self.da_checker.has_block(&self.block_root()) } @@ -261,22 +264,24 @@ impl SingleBlockLookup { /// 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().into() + self.blob_request_state.requested_ids = self.missing_blob_ids(); } - /// 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 `child_component_cache` is `Some`, we know block components won't hit the data + /// availability cache, so we don't check its processing cache unless `child_component_cache` + /// is `None`. + pub(crate) fn missing_blob_ids(&self) -> MissingBlobs { let block_root = self.block_root(); - if let Some(components) = self.cached_child_components.as_ref() { - self.da_checker - .get_missing_blob_ids(block_root, &components.processing_info()) - .unwrap_or_default() + if let Some(components) = self.child_component_cache.as_ref() { + self.da_checker.get_missing_blob_ids(block_root, components) } else { + let Some(processing_availability_view) = + self.da_checker.get_processing_view(block_root) + else { + return MissingBlobs::new_without_block(block_root, self.da_checker.is_deneb()); + }; self.da_checker - .get_missing_blob_ids_checking_cache(block_root) - .unwrap_or_default() + .get_missing_blob_ids(block_root, &processing_availability_view) } } @@ -302,7 +307,7 @@ impl SingleBlockLookup { /// 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() { + if let Some(cached_child) = self.child_component_cache.as_mut() { cached_child.clear_blobs(); } self.blob_request_state.state.register_failure_downloading() @@ -312,49 +317,19 @@ impl SingleBlockLookup { /// 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() { + if let Some(cached_child) = self.child_component_cache.as_mut() { cached_child.clear_blobs(); } self.blob_request_state.state.register_failure_processing() } } -#[derive(Clone, Default)] -pub struct RequestedBlobIds(Vec); - -impl From> for RequestedBlobIds { - fn from(value: Vec) -> Self { - Self(value) - } -} - -impl Into> for RequestedBlobIds { - fn into(self) -> VariableList { - 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 { - self.0.iter().map(|id| id.index).collect() - } -} - /// 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: RequestedBlobIds, + pub requested_ids: MissingBlobs, /// Where we store blobs until we receive the stream terminator. pub blob_download_queue: FixedBlobSidecarList, pub state: SingleLookupRequestState, @@ -362,9 +337,10 @@ pub struct BlobRequestState { } impl BlobRequestState { - pub fn new(peer_source: &[PeerShouldHave]) -> Self { + pub fn new(block_root: Hash256, peer_source: &[PeerShouldHave], is_deneb: bool) -> Self { + let default_ids = MissingBlobs::new_without_block(block_root, is_deneb); Self { - requested_ids: <_>::default(), + requested_ids: default_ids, blob_download_queue: <_>::default(), state: SingleLookupRequestState::new(peer_source), _phantom: PhantomData, @@ -636,7 +612,7 @@ mod tests { let store = HotColdDB::open_ephemeral(StoreConfig::default(), ChainSpec::minimal(), log) .expect("store"); let da_checker = Arc::new( - DataAvailabilityChecker::new(slot_clock, None, store.into(), spec) + DataAvailabilityChecker::new(slot_clock, None, store.into(), &log, spec) .expect("data availability checker"), ); let mut sl = SingleBlockLookup::::new( @@ -676,7 +652,7 @@ mod tests { .expect("store"); let da_checker = Arc::new( - DataAvailabilityChecker::new(slot_clock, None, store.into(), spec) + DataAvailabilityChecker::new(slot_clock, None, store.into(), &log, spec) .expect("data availability checker"), ); diff --git a/consensus/types/src/signed_beacon_block.rs b/consensus/types/src/signed_beacon_block.rs index e2f37266b8e..5aee63de60c 100644 --- a/consensus/types/src/signed_beacon_block.rs +++ b/consensus/types/src/signed_beacon_block.rs @@ -260,39 +260,6 @@ impl> SignedBeaconBlock } } -pub fn get_missing_blob_ids( - block_root: Hash256, - required_commitments: Option<&KzgCommitments>, - buffered_commitments: &KzgCommitmentOpts, -) -> Vec { - match required_commitments { - Some(required_commitments) => { - let num_blobs_expected = required_commitments.len(); - let mut blob_ids = Vec::with_capacity(num_blobs_expected); - - // Zip here will always limit the number of iterations to the size of - // `required_commitments` because `buffered_commitments` will always be populated - // with `Option` values up to `MAX_BLOBS_PER_BLOCK`. - for (index, (req_commitment, buffered_commitment_opt)) in required_commitments - .into_iter() - .zip(buffered_commitments.iter()) - .enumerate() - { - if buffered_commitment_opt.map_or(true, |buffered_commitment| { - buffered_commitment != *req_commitment - }) { - blob_ids.push(BlobIdentifier { - block_root, - index: index as u64, - }); - } - } - blob_ids - } - None => BlobIdentifier::get_all_blob_ids::(block_root), - } -} - // We can convert pre-Bellatrix blocks without payloads into blocks with payloads. impl From>> for SignedBeaconBlockBase> From bc23dafe660d007fb932a6caa6dd31983a50e431 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 21 Sep 2023 11:55:51 -0400 Subject: [PATCH 17/47] Update beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs Co-authored-by: ethDreamer <37123614+ethDreamer@users.noreply.github.com> --- .../src/data_availability_checker/processing_cache.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs index 9abf69ac011..673eb3f4414 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -29,7 +29,7 @@ impl ProcessingCache { pub fn has_block(&self, block_root: &Hash256) -> bool { self.processing_cache .get(block_root) - .map_or(false, |b| b.kzg_commitments.is_some()) + .map_or(false, |b| b.block_exists()) } pub fn blocks_with_missing_components(&self, slot: Slot) -> Vec { let mut roots_missing_components = vec![]; From e07dd7ce483ab706b77da6fd689ba3098f22d301 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 21 Sep 2023 12:43:58 -0400 Subject: [PATCH 18/47] marks review feedback and other general cleanup --- beacon_node/beacon_chain/src/beacon_chain.rs | 17 +++++-- .../src/data_availability_checker.rs | 14 +++--- .../availability_view.rs | 47 +++++++------------ .../overflow_lru_cache.rs | 15 ++---- .../processing_cache.rs | 21 ++++++++- 5 files changed, 60 insertions(+), 54 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 788db2a1c2f..57b477382e9 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -87,6 +87,7 @@ use operation_pool::{AttestationRef, OperationPool, PersistedOperationPool, Rece use parking_lot::{Mutex, RwLock}; use proto_array::{DoNotReOrg, ProposerHeadError}; use safe_arith::SafeArith; +use slasher::test_utils::block; use slasher::Slasher; use slog::{crit, debug, error, info, trace, warn, Logger}; use slot_clock::SlotClock; @@ -2802,8 +2803,11 @@ impl BeaconChain { if let Some(commitment_opt) = commitments.get_mut(blob.as_blob().index as usize) { *commitment_opt = Some(blob.as_blob().kzg_commitment); } - self.data_availability_checker - .notify_blob_commitments(block_root, commitments); + self.data_availability_checker.notify_blob_commitments( + blob.as_blob().slot, + block_root, + commitments, + ); let r = self.check_gossip_blob_availability_and_import(blob).await; self.remove_notified(&block_root, r) } @@ -2821,7 +2825,7 @@ impl BeaconChain { } } self.data_availability_checker - .notify_blob_commitments(block_root, commitments); + .notify_blob_commitments(slot, block_root, commitments); let r = self .check_rpc_blob_availability_and_import(slot, block_root, blobs) .await; @@ -2853,8 +2857,11 @@ impl BeaconChain { .body() .blob_kzg_commitments() { - self.data_availability_checker - .notify_block_commitments(block_root, commitments.clone()); + self.data_availability_checker.notify_block_commitments( + unverified_block.block().slot(), + block_root, + commitments.clone(), + ); }; let r = self .process_block(block_root, unverified_block, notify_execution_layer, || { diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index cb46cae4757..7ccc3491b45 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -10,6 +10,7 @@ use crate::data_availability_checker::overflow_lru_cache::OverflowLRUCache; use crate::data_availability_checker::processing_cache::ProcessingCache; use crate::{BeaconChain, BeaconChainTypes, BeaconStore}; use futures::StreamExt; +use itertools::Itertools; use kzg::Kzg; use kzg::{Error as KzgError, KzgCommitment}; use parking_lot::RwLock; @@ -240,7 +241,7 @@ impl DataAvailabilityChecker { return Err(AvailabilityCheckError::KzgNotInitialized); }; self.availability_cache - .put_kzg_verified_blobs(block_root, &verified_blobs) + .put_kzg_verified_blobs(block_root, verified_blobs) } /// This first validates the KZG commitments included in the blob sidecar. @@ -261,7 +262,7 @@ impl DataAvailabilityChecker { }; self.availability_cache - .put_kzg_verified_blobs(kzg_verified_blob.block_root(), &[kzg_verified_blob]) + .put_kzg_verified_blobs(kzg_verified_blob.block_root(), vec![kzg_verified_blob]) } /// Check if we have all the blobs for a block. Returns `Availability` which has information @@ -314,31 +315,32 @@ impl DataAvailabilityChecker { /// Determines the blob requirements for a block. Answers the question: "Does this block require /// blobs?". fn blobs_required_for_block(&self, block: &SignedBeaconBlock) -> bool { - let block_within_da_period = self.da_check_required_for_epoch(block.epoch()); - block.num_expected_blobs() > 0 && block_within_da_period + block.num_expected_blobs() > 0 && self.da_check_required_for_epoch(block.epoch()) } pub fn notify_block_commitments( &self, + slot: Slot, block_root: Hash256, commitments: KzgCommitments, ) { self.processing_cache .write() .entry(block_root) - .or_insert_with(ProcessingView::default) + .or_insert_with(|| ProcessingView::new(slot)) .merge_block(commitments); } pub fn notify_blob_commitments( &self, + slot: Slot, block_root: Hash256, blobs: KzgCommitmentOpts, ) { self.processing_cache .write() .entry(block_root) - .or_insert_with(ProcessingView::default) + .or_insert_with(|| ProcessingView::new(slot)) .merge_blobs(blobs); } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index a386776cb0b..0e8120abae0 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -53,9 +53,9 @@ pub trait AvailabilityView { /// Returns: /// - `true` if a blob exists at the given index. /// - `false` otherwise. - fn blob_exists(&self, blob_index: u64) -> bool { + fn blob_exists(&self, blob_index: usize) -> bool { self.get_cached_blobs() - .get(blob_index as usize) + .get(blob_index) .map(|b| b.is_some()) .unwrap_or(false) } @@ -83,9 +83,9 @@ pub trait AvailabilityView { /// Inserts a blob at a specific index in the cache. /// /// Existing blob at the index will be replaced. - fn insert_blob_at_index(&mut self, blob_index: u64, blob: &Self::BlobType) { - if let Some(b) = self.get_cached_blobs_mut().get_mut(blob_index as usize) { - *b = Some(blob.clone()); + fn insert_blob_at_index(&mut self, blob_index: usize, blob: Self::BlobType) { + if let Some(b) = self.get_cached_blobs_mut().get_mut(blob_index) { + *b = Some(blob); } } @@ -95,16 +95,14 @@ pub trait AvailabilityView { /// 1. The blob entry at the index is empty and no block exists. /// 2. The block exists and its commitment matches the blob's commitment. fn merge_blobs(&mut self, blobs: FixedVector, E::MaxBlobsPerBlock>) { - for (index, blob) in blobs.into_iter().enumerate() { + for (index, blob) in blobs.to_vec().into_iter().enumerate() { let Some(blob) = blob else { continue }; let commitment = *blob.get_commitment(); - let index = index as u64; - if let Some(cached_block) = self.get_cached_block() { - let block_commitment = cached_block.get_commitments(); - if let Some(&bc) = block_commitment.get(index as usize) { - if bc == commitment { + let block_commitment_opt = cached_block.get_commitments().get(index).copied(); + if let Some(block_commitment) = block_commitment_opt { + if block_commitment == commitment { self.insert_blob_at_index(index, blob) } } @@ -119,18 +117,9 @@ pub trait AvailabilityView { /// Blobs that don't match the new block's commitments are evicted. fn merge_block(&mut self, block: Self::BlockType) { self.insert_block(block); - let cached = self.get_cached_blobs_mut(); let mut reinsert = FixedVector::default(); - for (index, cached_blob) in cached.iter_mut().enumerate() { - // Take the existing blobs and re-insert them. - if let Some(blob) = reinsert.get_mut(index) { - if let Some(cached_blob) = cached_blob.take() { - *blob = Some(cached_blob); - } - } - } - - self.merge_blobs(reinsert) + std::mem::swap(self.get_cached_blobs_mut(), &mut reinsert); + self.merge_blobs(reinsert); } /// Checks if the block and all of its expected blobs are available in the cache. @@ -430,15 +419,11 @@ pub mod tests { pub fn assert_cache_consistent>(cache: V) { if let Some(cached_block) = cache.get_cached_block() { let cached_block_commitments = cached_block.get_commitments(); - for (block_commitment, blob_commitment_opt) in cached_block_commitments - .iter() - .zip(cache.get_cached_blobs().iter()) - { - let blob_commitment = blob_commitment_opt - .as_ref() - .map(|b| *b.get_commitment()) - .unwrap(); - assert_eq!(*block_commitment, blob_commitment); + for index in 0..E::max_blobs_per_block() { + let block_commitment = cached_block_commitments.get(index).copied(); + let blob_commitment_opt = cache.get_cached_blobs().get(index).unwrap(); + let blob_commitment = blob_commitment_opt.as_ref().map(|b| *b.get_commitment()); + assert_eq!(block_commitment, blob_commitment); } } else { panic!("No cached block") diff --git a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs index c25f534c5b7..06182820928 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs @@ -95,13 +95,6 @@ impl PendingComponents { ))) } - pub fn empty() -> Self { - Self { - verified_blobs: <_>::default(), - executed_block: None, - } - } - pub fn epoch(&self) -> Option { self.executed_block .as_ref() @@ -203,14 +196,14 @@ impl OverflowStore { match OverflowKey::from_ssz_bytes(&key_bytes)? { OverflowKey::Block(_) => { maybe_pending_components - .get_or_insert_with(PendingComponents::empty) + .get_or_insert_with(PendingComponents::default) .executed_block = Some(AvailabilityPendingExecutedBlock::from_ssz_bytes( value_bytes.as_slice(), )?); } OverflowKey::Blob(_, index) => { *maybe_pending_components - .get_or_insert_with(PendingComponents::empty) + .get_or_insert_with(PendingComponents::default) .verified_blobs .get_mut(index as usize) .ok_or(AvailabilityCheckError::BlobIndexInvalid(index as u64))? = @@ -393,7 +386,7 @@ impl OverflowLRUCache { pub fn put_kzg_verified_blobs( &self, block_root: Hash256, - kzg_verified_blobs: &[KzgVerifiedBlob], + kzg_verified_blobs: Vec>, ) -> Result, AvailabilityCheckError> { let mut fixed_blobs = FixedVector::default(); @@ -407,7 +400,7 @@ impl OverflowLRUCache { }); } if let Some(blob_opt) = fixed_blobs.get_mut(blob.blob_index() as usize) { - *blob_opt = Some(blob.clone()); + *blob_opt = Some(blob); } } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs index 9abf69ac011..b96ec158c5e 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -42,7 +42,7 @@ impl ProcessingCache { } } -#[derive(Default, Debug, Clone)] +#[derive(Debug, Clone)] pub struct ProcessingView { slot: Slot, /// Blobs required for a block can only be known if we have seen the block. So `Some` here @@ -54,3 +54,22 @@ pub struct ProcessingView { /// no way of knowing which blob is the correct one until we see the block. pub processing_blobs: KzgCommitmentOpts, } + +impl ProcessingView { + pub fn new(slot: Slot) -> Self { + Self { + slot, + kzg_commitments: None, + processing_blobs: KzgCommitmentOpts::::default(), + } + } + + #[cfg(test)] + pub fn default() -> Self { + Self { + slot: Slot::new(0), + kzg_commitments: None, + processing_blobs: KzgCommitmentOpts::::default(), + } + } +} From a1f6b17450d8287bd1df728066bd28871bfd5a36 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 21 Sep 2023 12:46:42 -0400 Subject: [PATCH 19/47] cargo fix --- beacon_node/beacon_chain/src/beacon_chain.rs | 1 - beacon_node/beacon_chain/src/data_availability_checker.rs | 2 +- .../src/data_availability_checker/availability_view.rs | 2 -- .../src/data_availability_checker/child_component_cache.rs | 2 +- .../src/data_availability_checker/processing_cache.rs | 5 +---- .../network/src/sync/block_lookups/single_block_lookup.rs | 4 +--- consensus/types/src/signed_beacon_block.rs | 2 -- 7 files changed, 4 insertions(+), 14 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 57b477382e9..eab6659a49b 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -87,7 +87,6 @@ use operation_pool::{AttestationRef, OperationPool, PersistedOperationPool, Rece use parking_lot::{Mutex, RwLock}; use proto_array::{DoNotReOrg, ProposerHeadError}; use safe_arith::SafeArith; -use slasher::test_utils::block; use slasher::Slasher; use slog::{crit, debug, error, info, trace, warn, Logger}; use slot_clock::SlotClock; diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index 7ccc3491b45..1b73820d10d 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -18,7 +18,7 @@ pub use processing_cache::ProcessingView; use slasher::test_utils::E; use slog::{debug, error, Logger}; use slot_clock::SlotClock; -use ssz_types::{Error, VariableList}; +use ssz_types::{Error}; use std::fmt; use std::fmt::Debug; use std::sync::Arc; diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index 0e8120abae0..ad1f48f4f6d 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -5,8 +5,6 @@ use crate::data_availability_checker::overflow_lru_cache::PendingComponents; use crate::data_availability_checker::ProcessingView; use crate::AvailabilityPendingExecutedBlock; use kzg::KzgCommitment; -use slasher::test_utils::E; -use slog::b; use ssz_types::FixedVector; use std::sync::Arc; use types::beacon_block_body::KzgCommitments; diff --git a/beacon_node/beacon_chain/src/data_availability_checker/child_component_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/child_component_cache.rs index 400abec401f..886c508e6ae 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/child_component_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/child_component_cache.rs @@ -1,5 +1,5 @@ use crate::block_verification_types::RpcBlock; -use crate::data_availability_checker::{AvailabilityView, ProcessingView}; +use crate::data_availability_checker::{AvailabilityView}; use std::sync::Arc; use types::blob_sidecar::FixedBlobSidecarList; use types::{EthSpec, SignedBeaconBlock}; diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs index b96ec158c5e..ee5631f6d55 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -1,11 +1,8 @@ use crate::data_availability_checker::AvailabilityView; -use ssz_types::FixedVector; use std::collections::hash_map::Entry; use std::collections::HashMap; -use std::sync::Arc; use types::beacon_block_body::{KzgCommitmentOpts, KzgCommitments}; -use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; -use types::{EthSpec, Hash256, SignedBeaconBlock, Slot}; +use types::{EthSpec, Hash256, Slot}; /// This cache is used only for gossip blocks/blobs and single block/blob lookups, to give req/resp /// a view of what we have and what we require. This cache serves a slightly different purpose than 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 913a445d089..49cb29452c8 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 @@ -8,17 +8,15 @@ use beacon_chain::data_availability_checker::{ }; use beacon_chain::data_availability_checker::{AvailabilityView, ChildComponentCache}; use beacon_chain::BeaconChainTypes; -use lighthouse_network::rpc::methods::MaxRequestBlobSidecars; use lighthouse_network::{PeerAction, PeerId}; use slog::{trace, Logger}; -use ssz_types::VariableList; use std::collections::HashSet; use std::fmt::Debug; use std::marker::PhantomData; use std::sync::Arc; use store::Hash256; use strum::IntoStaticStr; -use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList}; +use types::blob_sidecar::{FixedBlobSidecarList}; use types::EthSpec; #[derive(Debug, PartialEq, Eq)] diff --git a/consensus/types/src/signed_beacon_block.rs b/consensus/types/src/signed_beacon_block.rs index 5aee63de60c..2234e38f083 100644 --- a/consensus/types/src/signed_beacon_block.rs +++ b/consensus/types/src/signed_beacon_block.rs @@ -1,5 +1,3 @@ -use crate::beacon_block_body::{KzgCommitmentOpts, KzgCommitments}; -use crate::blob_sidecar::BlobIdentifier; use crate::*; use bls::Signature; use derivative::Derivative; From d6862e7f914aa8719ee8c516d78633f36eb9812d Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 21 Sep 2023 12:51:57 -0400 Subject: [PATCH 20/47] improve availability view docs --- .../src/data_availability_checker/availability_view.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index ad1f48f4f6d..2eb21e63c0b 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -11,7 +11,10 @@ use types::beacon_block_body::KzgCommitments; use types::{BlobSidecar, EthSpec, SignedBeaconBlock}; /// Defines an interface for managing data availability with two key invariants: -/// 1. Blobs won't be clobbered if we've yet to see the corresponding block. +/// +/// 1. If we haven't seen a block yet, we will insert the first blob for a given (block_root, index) +/// but we won't insert subsequent blobs for the same (block_root, index) if they have a different +/// commitment. /// 2. On block insertion, any non-matching blob commitments are evicted. /// /// Types implementing this trait can be used for validating and managing availability From 3986d336d9b267dc7a98ad3f6560b47004ac4de2 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 21 Sep 2023 13:22:20 -0400 Subject: [PATCH 21/47] some renames --- .../src/data_availability_checker.rs | 17 +++++---- .../availability_view.rs | 12 +++--- ...component_cache.rs => child_components.rs} | 8 ++-- .../processing_cache.rs | 10 ++--- .../network/src/sync/block_lookups/common.rs | 8 ++-- .../network/src/sync/block_lookups/mod.rs | 6 +-- .../src/sync/block_lookups/parent_lookup.rs | 2 +- .../sync/block_lookups/single_block_lookup.rs | 38 +++++++++---------- .../network/src/sync/block_lookups/tests.rs | 4 +- beacon_node/network/src/sync/manager.rs | 6 +-- 10 files changed, 57 insertions(+), 54 deletions(-) rename beacon_node/beacon_chain/src/data_availability_checker/{child_component_cache.rs => child_components.rs} (88%) diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index 1b73820d10d..84aea134c1b 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -5,7 +5,7 @@ use crate::block_verification_types::{ pub use crate::data_availability_checker::availability_view::{ AvailabilityView, GetCommitment, GetCommitments, }; -pub use crate::data_availability_checker::child_component_cache::ChildComponentCache; +pub use crate::data_availability_checker::child_components::ChildComponents; use crate::data_availability_checker::overflow_lru_cache::OverflowLRUCache; use crate::data_availability_checker::processing_cache::ProcessingCache; use crate::{BeaconChain, BeaconChainTypes, BeaconStore}; @@ -14,11 +14,11 @@ use itertools::Itertools; use kzg::Kzg; use kzg::{Error as KzgError, KzgCommitment}; use parking_lot::RwLock; -pub use processing_cache::ProcessingView; +pub use processing_cache::ProcessingComponents; use slasher::test_utils::E; use slog::{debug, error, Logger}; use slot_clock::SlotClock; -use ssz_types::{Error}; +use ssz_types::Error; use std::fmt; use std::fmt::Debug; use std::sync::Arc; @@ -30,7 +30,7 @@ use types::consts::deneb::MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS; use types::{BlobSidecarList, ChainSpec, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot}; mod availability_view; -mod child_component_cache; +mod child_components; mod overflow_lru_cache; mod processing_cache; @@ -139,7 +139,10 @@ impl DataAvailabilityChecker { } /// Get the processing info for a block. - pub fn get_processing_view(&self, block_root: Hash256) -> Option> { + pub fn get_processing_view( + &self, + block_root: Hash256, + ) -> Option> { self.processing_cache.read().get(&block_root).cloned() } @@ -327,7 +330,7 @@ impl DataAvailabilityChecker { self.processing_cache .write() .entry(block_root) - .or_insert_with(|| ProcessingView::new(slot)) + .or_insert_with(|| ProcessingComponents::new(slot)) .merge_block(commitments); } @@ -340,7 +343,7 @@ impl DataAvailabilityChecker { self.processing_cache .write() .entry(block_root) - .or_insert_with(|| ProcessingView::new(slot)) + .or_insert_with(|| ProcessingComponents::new(slot)) .merge_blobs(blobs); } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index 2eb21e63c0b..2bffb18fcd4 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -1,8 +1,8 @@ -use super::child_component_cache::ChildComponentCache; +use super::child_components::ChildComponents; use crate::blob_verification::KzgVerifiedBlob; use crate::block_verification_types::AsBlock; use crate::data_availability_checker::overflow_lru_cache::PendingComponents; -use crate::data_availability_checker::ProcessingView; +use crate::data_availability_checker::ProcessingComponents; use crate::AvailabilityPendingExecutedBlock; use kzg::KzgCommitment; use ssz_types::FixedVector; @@ -174,7 +174,7 @@ macro_rules! impl_availability_view { } impl_availability_view!( - ProcessingView, + ProcessingComponents, KzgCommitments, KzgCommitment, kzg_commitments, @@ -190,7 +190,7 @@ impl_availability_view!( ); impl_availability_view!( - ChildComponentCache, + ChildComponents, Arc>, Arc>, downloaded_block, @@ -532,7 +532,7 @@ pub mod tests { generate_tests!( processing_view_tests, - ProcessingView::, + ProcessingComponents::, kzg_commitments, processing_blobs, setup_processing_view @@ -546,7 +546,7 @@ pub mod tests { ); generate_tests!( child_component_tests, - ChildComponentCache::, + ChildComponents::, downloaded_block, downloaded_blobs, setup_child_components diff --git a/beacon_node/beacon_chain/src/data_availability_checker/child_component_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/child_components.rs similarity index 88% rename from beacon_node/beacon_chain/src/data_availability_checker/child_component_cache.rs rename to beacon_node/beacon_chain/src/data_availability_checker/child_components.rs index 886c508e6ae..f52f4ac62b4 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/child_component_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/child_components.rs @@ -1,5 +1,5 @@ use crate::block_verification_types::RpcBlock; -use crate::data_availability_checker::{AvailabilityView}; +use crate::data_availability_checker::AvailabilityView; use std::sync::Arc; use types::blob_sidecar::FixedBlobSidecarList; use types::{EthSpec, SignedBeaconBlock}; @@ -9,12 +9,12 @@ use types::{EthSpec, SignedBeaconBlock}; /// 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 ChildComponentCache { +pub struct ChildComponents { pub downloaded_block: Option>>, pub downloaded_blobs: FixedBlobSidecarList, } -impl From> for ChildComponentCache { +impl From> for ChildComponents { fn from(value: RpcBlock) -> Self { let (block, blobs) = value.deconstruct(); let fixed_blobs = blobs.map(|blobs| { @@ -24,7 +24,7 @@ impl From> for ChildComponentCache { } } -impl ChildComponentCache { +impl ChildComponents { pub fn new( block: Option>>, blobs: Option>, diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs index ddf2009a1c4..14a3999fdd1 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -10,14 +10,14 @@ use types::{EthSpec, Hash256, Slot}; /// See `AvailabilityView`'s trait definition. #[derive(Default)] pub struct ProcessingCache { - processing_cache: HashMap>, + processing_cache: HashMap>, } impl ProcessingCache { - pub fn get(&self, block_root: &Hash256) -> Option<&ProcessingView> { + pub fn get(&self, block_root: &Hash256) -> Option<&ProcessingComponents> { self.processing_cache.get(block_root) } - pub fn entry(&mut self, block_root: Hash256) -> Entry<'_, Hash256, ProcessingView> { + pub fn entry(&mut self, block_root: Hash256) -> Entry<'_, Hash256, ProcessingComponents> { self.processing_cache.entry(block_root) } pub fn remove(&mut self, block_root: &Hash256) { @@ -40,7 +40,7 @@ impl ProcessingCache { } #[derive(Debug, Clone)] -pub struct ProcessingView { +pub struct ProcessingComponents { slot: Slot, /// Blobs required for a block can only be known if we have seen the block. So `Some` here /// means we've seen it, a `None` means we haven't. The `kzg_commitments` value helps us figure @@ -52,7 +52,7 @@ pub struct ProcessingView { pub processing_blobs: KzgCommitmentOpts, } -impl ProcessingView { +impl ProcessingComponents { pub fn new(slot: Slot) -> Self { Self { slot, diff --git a/beacon_node/network/src/sync/block_lookups/common.rs b/beacon_node/network/src/sync/block_lookups/common.rs index f1abe16dddc..7f34d1836bf 100644 --- a/beacon_node/network/src/sync/block_lookups/common.rs +++ b/beacon_node/network/src/sync/block_lookups/common.rs @@ -9,7 +9,7 @@ use crate::sync::block_lookups::{ use crate::sync::manager::{BlockProcessType, Id, SingleLookupReqId}; use crate::sync::network_context::SyncNetworkContext; use beacon_chain::block_verification_types::RpcBlock; -use beacon_chain::data_availability_checker::ChildComponentCache; +use beacon_chain::data_availability_checker::ChildComponents; use beacon_chain::{get_block_root, BeaconChainTypes}; use lighthouse_network::rpc::methods::BlobsByRootRequest; use lighthouse_network::rpc::BlocksByRootRequest; @@ -222,7 +222,7 @@ pub trait RequestState { /// triggered by `UnknownParent` errors. fn add_to_child_components( verified_response: Self::VerifiedResponseType, - components: &mut ChildComponentCache, + components: &mut ChildComponents, ); /// Convert a verified response to the type we send to the beacon processor. @@ -326,7 +326,7 @@ impl RequestState for BlockRequestState fn add_to_child_components( verified_response: Arc>, - components: &mut ChildComponentCache, + components: &mut ChildComponents, ) { components.add_cached_child_block(verified_response); } @@ -432,7 +432,7 @@ impl RequestState for BlobRequestState, - components: &mut ChildComponentCache, + components: &mut ChildComponents, ) { components.add_cached_child_blobs(verified_response); } diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index 104306aee5e..3c9b64516e1 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -12,7 +12,7 @@ use crate::sync::block_lookups::single_block_lookup::{ }; use crate::sync::manager::{Id, SingleLookupReqId}; use beacon_chain::block_verification_types::{AsBlock, RpcBlock}; -pub use beacon_chain::data_availability_checker::ChildComponentCache; +pub use beacon_chain::data_availability_checker::ChildComponents; use beacon_chain::data_availability_checker::{ AvailabilityCheckError, AvailabilityView, DataAvailabilityChecker, }; @@ -146,7 +146,7 @@ impl BlockLookups { pub fn search_child_block( &mut self, block_root: Hash256, - child_components: ChildComponentCache, + child_components: ChildComponents, peer_source: PeerShouldHave, cx: &mut SyncNetworkContext, ) { @@ -195,7 +195,7 @@ impl BlockLookups { pub fn new_current_lookup( &mut self, block_root: Hash256, - child_components: Option>, + child_components: Option>, peers: &[PeerShouldHave], cx: &mut SyncNetworkContext, ) -> Option> { 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 7aa674c57fd..5671e2a062a 100644 --- a/beacon_node/network/src/sync/block_lookups/parent_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/parent_lookup.rs @@ -178,7 +178,7 @@ impl ParentLookup { .blob_request_state .state .register_failure_processing(); - if let Some(components) = self.current_parent_request.child_component_cache.as_mut() { + if let Some(components) = self.current_parent_request.child_components.as_mut() { components.downloaded_block = None; components.downloaded_blobs = <_>::default(); } 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 49cb29452c8..6d72294cbfd 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 @@ -6,7 +6,7 @@ use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::data_availability_checker::{ AvailabilityCheckError, DataAvailabilityChecker, MissingBlobs, }; -use beacon_chain::data_availability_checker::{AvailabilityView, ChildComponentCache}; +use beacon_chain::data_availability_checker::{AvailabilityView, ChildComponents}; use beacon_chain::BeaconChainTypes; use lighthouse_network::{PeerAction, PeerId}; use slog::{trace, Logger}; @@ -16,7 +16,7 @@ use std::marker::PhantomData; use std::sync::Arc; use store::Hash256; use strum::IntoStaticStr; -use types::blob_sidecar::{FixedBlobSidecarList}; +use types::blob_sidecar::FixedBlobSidecarList; use types::EthSpec; #[derive(Debug, PartialEq, Eq)] @@ -59,13 +59,13 @@ pub struct SingleBlockLookup { pub da_checker: Arc>, /// Only necessary for requests triggered by an `UnknownBlockParent` or `UnknownBlockParent` /// because any blocks or blobs without parents won't hit the data availability cache. - pub child_component_cache: Option>, + pub child_components: Option>, } impl SingleBlockLookup { pub fn new( requested_block_root: Hash256, - child_component_cache: Option>, + child_components: Option>, peers: &[PeerShouldHave], da_checker: Arc>, id: Id, @@ -76,7 +76,7 @@ impl SingleBlockLookup { block_request_state: BlockRequestState::new(requested_block_root, peers), blob_request_state: BlobRequestState::new(requested_block_root, peers, is_deneb), da_checker, - child_component_cache, + child_components, } } @@ -96,7 +96,7 @@ impl SingleBlockLookup { self.block_request_state.requested_block_root = block_root; self.block_request_state.state.state = State::AwaitingDownload; self.blob_request_state.state.state = State::AwaitingDownload; - self.child_component_cache = Some(ChildComponentCache::default()); + self.child_components = Some(ChildComponents::default()); } /// Get all unique peers across block and blob requests. @@ -136,7 +136,7 @@ impl SingleBlockLookup { /// 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.child_component_cache.as_ref() { + if let Some(components) = self.child_components.as_ref() { let Some(block) = components.downloaded_block.as_ref() else { return CachedChild::DownloadIncomplete; }; @@ -161,8 +161,8 @@ impl SingleBlockLookup { &mut self, verified_response: R::VerifiedResponseType, ) -> CachedChild { - if let Some(child_component_cache) = self.child_component_cache.as_mut() { - R::add_to_child_components(verified_response, child_component_cache); + if let Some(child_components) = self.child_components.as_mut() { + R::add_to_child_components(verified_response, child_components); self.get_cached_child_block() } else { CachedChild::NotRequired @@ -170,9 +170,9 @@ impl SingleBlockLookup { } /// Add a child component to the lookup request. Merges with any existing child components. - pub fn add_child_components(&mut self, components: ChildComponentCache) { - if let Some(ref mut existing_components) = self.child_component_cache { - let ChildComponentCache { + pub fn add_child_components(&mut self, components: ChildComponents) { + if let Some(ref mut existing_components) = self.child_components { + let ChildComponents { downloaded_block, downloaded_blobs, } = components; @@ -181,7 +181,7 @@ impl SingleBlockLookup { } existing_components.add_cached_child_blobs(downloaded_blobs); } else { - self.child_component_cache = Some(components); + self.child_components = Some(components); } } @@ -245,7 +245,7 @@ impl SingleBlockLookup { /// Returns `true` if the block has already been downloaded. pub(crate) fn block_already_downloaded(&self) -> bool { - if let Some(components) = self.child_component_cache.as_ref() { + if let Some(components) = self.child_components.as_ref() { components.block_exists() } else { self.da_checker.has_block(&self.block_root()) @@ -265,12 +265,12 @@ impl SingleBlockLookup { self.blob_request_state.requested_ids = self.missing_blob_ids(); } - /// If `child_component_cache` is `Some`, we know block components won't hit the data - /// availability cache, so we don't check its processing cache unless `child_component_cache` + /// If `child_components` is `Some`, we know block components won't hit the data + /// availability cache, so we don't check its processing cache unless `child_components` /// is `None`. pub(crate) fn missing_blob_ids(&self) -> MissingBlobs { let block_root = self.block_root(); - if let Some(components) = self.child_component_cache.as_ref() { + if let Some(components) = self.child_components.as_ref() { self.da_checker.get_missing_blob_ids(block_root, components) } else { let Some(processing_availability_view) = @@ -305,7 +305,7 @@ impl SingleBlockLookup { /// 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.child_component_cache.as_mut() { + if let Some(cached_child) = self.child_components.as_mut() { cached_child.clear_blobs(); } self.blob_request_state.state.register_failure_downloading() @@ -315,7 +315,7 @@ impl SingleBlockLookup { /// 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.child_component_cache.as_mut() { + if let Some(cached_child) = self.child_components.as_mut() { cached_child.clear_blobs(); } self.blob_request_state.state.register_failure_processing() diff --git a/beacon_node/network/src/sync/block_lookups/tests.rs b/beacon_node/network/src/sync/block_lookups/tests.rs index 00d1c8778ed..b94d3ea48eb 100644 --- a/beacon_node/network/src/sync/block_lookups/tests.rs +++ b/beacon_node/network/src/sync/block_lookups/tests.rs @@ -1248,7 +1248,7 @@ mod deneb_only { block_root = child_root; bl.search_child_block( child_root, - ChildComponentCache::new(Some(child_block), None), + ChildComponents::new(Some(child_block), None), PeerShouldHave::Neither(peer_id), &mut cx, ); @@ -1286,7 +1286,7 @@ mod deneb_only { *blobs.index_mut(0) = Some(child_blob); bl.search_child_block( child_root, - ChildComponentCache::new(None, Some(blobs)), + ChildComponents::new(None, Some(blobs)), PeerShouldHave::Neither(peer_id), &mut cx, ); diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 3cdd2633b4e..cf90f9033eb 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -47,7 +47,7 @@ use crate::sync::block_lookups::{BlobRequestState, BlockRequestState}; use crate::sync::range_sync::ByRangeRequestType; use beacon_chain::block_verification_types::AsBlock; use beacon_chain::block_verification_types::RpcBlock; -use beacon_chain::data_availability_checker::ChildComponentCache; +use beacon_chain::data_availability_checker::ChildComponents; use beacon_chain::{ AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes, BlockError, EngineState, }; @@ -655,7 +655,7 @@ impl SyncManager { block_root, parent_root, blob_slot, - ChildComponentCache::new(None, Some(blobs)), + ChildComponents::new(None, Some(blobs)), ); } SyncMessage::UnknownBlockHashFromAttestation(peer_id, block_hash) => { @@ -746,7 +746,7 @@ impl SyncManager { block_root: Hash256, parent_root: Hash256, slot: Slot, - child_components: ChildComponentCache, + child_components: ChildComponents, ) { if self.should_search_for_block(slot, &peer_id) { self.block_lookups.search_parent( From a868a6c55c20dfe5ca75ab79625db30d4397b557 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 21 Sep 2023 13:50:15 -0400 Subject: [PATCH 22/47] some renames and docs --- .../src/data_availability_checker.rs | 24 +++++++++++++++---- .../processing_cache.rs | 2 +- .../network_beacon_processor/sync_methods.rs | 6 ++--- 3 files changed, 24 insertions(+), 8 deletions(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index 84aea134c1b..4efb8fc0e13 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -315,12 +315,15 @@ impl DataAvailabilityChecker { } } - /// Determines the blob requirements for a block. Answers the question: "Does this block require - /// blobs?". + /// Determines the blob requirements for a block. If the block is pre-deneb, no blobs are required. + /// If the block's epoch is from prior to the data availability boundary, no blobs are required. fn blobs_required_for_block(&self, block: &SignedBeaconBlock) -> bool { block.num_expected_blobs() > 0 && self.da_check_required_for_epoch(block.epoch()) } + /// Adds block commitments to the processing cache. These commitments are unverified but caching + /// them here is useful to avoid duplicate downloads of blocks, as well as understanding + /// our blob download requirements. pub fn notify_block_commitments( &self, slot: Slot, @@ -334,6 +337,10 @@ impl DataAvailabilityChecker { .merge_block(commitments); } + + /// Adds blob commitments to the processing cache. These commitments are unverified but caching + /// them here is useful to avoid duplicate downloads of blobs, as well as understanding + /// our block and blob download requirements. pub fn notify_blob_commitments( &self, slot: Slot, @@ -347,16 +354,25 @@ impl DataAvailabilityChecker { .merge_blobs(blobs); } + /// Clears the block and all blobs from the processing cache for a give root if they exist. pub fn remove_notified(&self, block_root: &Hash256) { self.processing_cache.write().remove(block_root) } - pub fn get_delayed_lookups(&self, slot: Slot) -> Vec { + /// Gather all block roots for which we are not currently processing all components for the + /// given slot. + pub fn incomplete_processing_components(&self, slot: Slot) -> Vec { self.processing_cache .read() - .blocks_with_missing_components(slot) + .incomplete_processing_components(slot) } + /// Determines whether we are at least the `single_lookup_delay` duration into the given slot. + /// If we are not currently in the Deneb fork, this delay is not considered. + /// + /// The `single_lookup_delay` is the duration we wait for a blocks or blobs to arrive over + /// gossip before making single block or blob requests. This is to minimize the number of + /// single lookup requests we end up making. pub fn should_delay_lookup(&self, slot: Slot) -> bool { if !self.is_deneb() { return false; diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs index 14a3999fdd1..6e3f4652259 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -28,7 +28,7 @@ impl ProcessingCache { .get(block_root) .map_or(false, |b| b.block_exists()) } - pub fn blocks_with_missing_components(&self, slot: Slot) -> Vec { + pub fn incomplete_processing_components(&self, slot: Slot) -> Vec { let mut roots_missing_components = vec![]; for (&block_root, info) in self.processing_cache.iter() { if info.slot == slot && !info.is_available() { 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 46d8d754fb1..d5c30987d2e 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -296,11 +296,11 @@ impl NetworkBeaconProcessor { } pub fn poll_delayed_lookups(&self, slot: Slot) { - let lookups = self + let block_roots = self .chain .data_availability_checker - .get_delayed_lookups(slot); - for block_root in lookups { + .incomplete_processing_components(slot); + for block_root in block_roots { if let Some(peer_ids) = self.delayed_lookup_peers.lock().pop(&block_root) { for peer_id in peer_ids { self.send_sync_message(SyncMessage::MissingGossipBlockComponents( From 8d85c8342eb103018d9a5c842ca8423a889d6547 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 21 Sep 2023 13:53:56 -0400 Subject: [PATCH 23/47] fix should delay lookup logic --- .../src/data_availability_checker.rs | 28 +++++++------------ 1 file changed, 10 insertions(+), 18 deletions(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index 4efb8fc0e13..655b49cdc31 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -337,7 +337,6 @@ impl DataAvailabilityChecker { .merge_block(commitments); } - /// Adds blob commitments to the processing cache. These commitments are unverified but caching /// them here is useful to avoid duplicate downloads of blobs, as well as understanding /// our block and blob download requirements. @@ -378,25 +377,18 @@ impl DataAvailabilityChecker { return false; } - let maximum_gossip_clock_disparity = self.spec.maximum_gossip_clock_disparity(); - let earliest_slot = self + let current_or_future_slot = self .slot_clock - .now_with_past_tolerance(maximum_gossip_clock_disparity); - let latest_slot = self + .now() + .map_or(false, |current_slot| current_slot <= slot); + + let delay_threshold_unmet = self .slot_clock - .now_with_future_tolerance(maximum_gossip_clock_disparity); - if let (Some(earliest_slot), Some(latest_slot)) = (earliest_slot, latest_slot) { - let msg_for_current_slot = slot >= earliest_slot && slot <= latest_slot; - let delay_threshold_unmet = self - .slot_clock - .millis_from_current_slot_start() - .map_or(false, |millis_into_slot| { - millis_into_slot < self.slot_clock.single_lookup_delay() - }); - msg_for_current_slot && delay_threshold_unmet - } else { - false - } + .millis_from_current_slot_start() + .map_or(false, |millis_into_slot| { + millis_into_slot < self.slot_clock.single_lookup_delay() + }); + current_or_future_slot && delay_threshold_unmet } /// The epoch at which we require a data availability check in block processing. From 2777b73261b7a36ae2d09e8c0579206b674024c2 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 21 Sep 2023 14:13:12 -0400 Subject: [PATCH 24/47] get rid of some wrapper methods --- .../src/data_availability_checker/child_components.rs | 8 -------- .../src/network_beacon_processor/gossip_methods.rs | 1 - beacon_node/network/src/sync/block_lookups/common.rs | 6 +++--- .../network/src/sync/block_lookups/single_block_lookup.rs | 4 ++-- 4 files changed, 5 insertions(+), 14 deletions(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker/child_components.rs b/beacon_node/beacon_chain/src/data_availability_checker/child_components.rs index f52f4ac62b4..323d05a9f65 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/child_components.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/child_components.rs @@ -42,12 +42,4 @@ impl ChildComponents { pub fn clear_blobs(&mut self) { self.downloaded_blobs = FixedBlobSidecarList::default(); } - - pub fn add_cached_child_block(&mut self, block: Arc>) { - self.merge_block(block) - } - - pub fn add_cached_child_blobs(&mut self, blobs: FixedBlobSidecarList) { - self.merge_blobs(blobs) - } } 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 8637212cccd..6f34e0f2e2c 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -1276,7 +1276,6 @@ impl NetworkBeaconProcessor { }; if let Err(e) = &result { - //TODO: drop from processing cache self.maybe_store_invalid_block( &invalid_block_storage, block_root, diff --git a/beacon_node/network/src/sync/block_lookups/common.rs b/beacon_node/network/src/sync/block_lookups/common.rs index 7f34d1836bf..2807e73babf 100644 --- a/beacon_node/network/src/sync/block_lookups/common.rs +++ b/beacon_node/network/src/sync/block_lookups/common.rs @@ -9,7 +9,7 @@ use crate::sync::block_lookups::{ use crate::sync::manager::{BlockProcessType, Id, SingleLookupReqId}; use crate::sync::network_context::SyncNetworkContext; use beacon_chain::block_verification_types::RpcBlock; -use beacon_chain::data_availability_checker::ChildComponents; +use beacon_chain::data_availability_checker::{AvailabilityView, ChildComponents}; use beacon_chain::{get_block_root, BeaconChainTypes}; use lighthouse_network::rpc::methods::BlobsByRootRequest; use lighthouse_network::rpc::BlocksByRootRequest; @@ -328,7 +328,7 @@ impl RequestState for BlockRequestState verified_response: Arc>, components: &mut ChildComponents, ) { - components.add_cached_child_block(verified_response); + components.merge_block(verified_response); } fn verified_to_reconstructed( @@ -434,7 +434,7 @@ impl RequestState for BlobRequestState, components: &mut ChildComponents, ) { - components.add_cached_child_blobs(verified_response); + components.merge_blobs(verified_response); } fn verified_to_reconstructed( 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 6d72294cbfd..445b0a06f17 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 @@ -177,9 +177,9 @@ impl SingleBlockLookup { downloaded_blobs, } = components; if let Some(block) = downloaded_block { - existing_components.add_cached_child_block(block); + existing_components.merge_block(block); } - existing_components.add_cached_child_blobs(downloaded_blobs); + existing_components.merge_blobs(downloaded_blobs); } else { self.child_components = Some(components); } From 8eecea5b2df3ceee97493bcae75747633919bc7f Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 21 Sep 2023 14:25:54 -0400 Subject: [PATCH 25/47] fix up single lookup changes --- .../network_beacon_processor/sync_methods.rs | 1 + .../network/src/sync/block_lookups/mod.rs | 53 ++++++++----------- .../src/sync/block_lookups/parent_lookup.rs | 6 +-- .../sync/block_lookups/single_block_lookup.rs | 46 ++++++++-------- 4 files changed, 48 insertions(+), 58 deletions(-) diff --git a/beacon_node/network/src/network_beacon_processor/sync_methods.rs b/beacon_node/network/src/network_beacon_processor/sync_methods.rs index d5c30987d2e..cdb4cb81854 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -302,6 +302,7 @@ impl NetworkBeaconProcessor { .incomplete_processing_components(slot); for block_root in block_roots { if let Some(peer_ids) = self.delayed_lookup_peers.lock().pop(&block_root) { + // Let the network layer de-dupe these. for peer_id in peer_ids { self.send_sync_message(SyncMessage::MissingGossipBlockComponents( peer_id, block_root, diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index 3c9b64516e1..db7bb48440b 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -127,14 +127,7 @@ impl BlockLookups { peer_source: PeerShouldHave, cx: &mut SyncNetworkContext, ) { - let lookup = self.new_current_lookup(block_root, None, &[peer_source], cx); - - // TODO: can get rid of some of this - 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.new_current_lookup(block_root, None, peer_source, cx) } /// Creates a lookup for the block with the given `block_root`, while caching other block @@ -150,15 +143,7 @@ impl BlockLookups { peer_source: PeerShouldHave, cx: &mut SyncNetworkContext, ) { - if !child_components.is_available() { - let lookup = - 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); - } - } + self.new_current_lookup(block_root, Some(child_components), peer_source, cx) } /// Attempts to trigger the request matching the given `block_root`. @@ -196,30 +181,30 @@ impl BlockLookups { &mut self, block_root: Hash256, child_components: Option>, - peers: &[PeerShouldHave], + peer: PeerShouldHave, cx: &mut SyncNetworkContext, - ) -> Option> { + ) { // Do not re-request a block that is already being requested if let Some((_, lookup)) = self .single_block_lookups .iter_mut() .find(|(_id, lookup)| lookup.is_for_block(block_root)) { - lookup.add_peers(peers); + lookup.add_peer(peer); if let Some(components) = child_components { lookup.add_child_components(components); } - return None; + return; } if let Some(parent_lookup) = self.parent_lookups.iter_mut().find(|parent_req| { parent_req.is_for_block(block_root) || parent_req.contains_block(&block_root) }) { - parent_lookup.add_peers(peers); + parent_lookup.add_peer(peer); // If the block was already downloaded, or is being downloaded in this moment, do not // request it. - return None; + return; } if self @@ -228,16 +213,24 @@ impl BlockLookups { .any(|(hashes, _last_parent_request)| hashes.contains(&block_root)) { // we are already processing this block, ignore it. - return None; + return; } - Some(SingleBlockLookup::new( + let msg = if child_components.is_some() { + "Searching for components of a block with unknown parent" + } else { + "Searching for block" + }; + + let lookup = SingleBlockLookup::new( block_root, child_components, - peers, + peer, self.da_checker.clone(), cx.next_id(), - )) + ); + lookup_creation_logging(msg, &lookup, peer, &self.log); + self.trigger_single_lookup(lookup, cx); } /// If a block is attempted to be processed but we do not know its parent, this function is @@ -265,7 +258,7 @@ impl BlockLookups { if let Some(parent_lookup) = self.parent_lookups.iter_mut().find(|parent_req| { parent_req.contains_block(&block_root) || parent_req.is_for_block(block_root) }) { - parent_lookup.add_peers(&[peer_source]); + parent_lookup.add_peer(peer_source); // we are already searching for this block, ignore it return; } @@ -1411,7 +1404,7 @@ fn lookup_creation_logging( debug!( log, "{}", msg; - "peer_id" => ?peer_source, + "peer_ids" => ?peer_source, "block" => ?block_root, "blob_indices" => ?blob_indices ); @@ -1419,7 +1412,7 @@ fn lookup_creation_logging( debug!( log, "{}", msg; - "peer_id" => ?peer_source, + "peer_ids" => ?peer_source, "block" => ?block_root, ); } 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 5671e2a062a..05d0f52d393 100644 --- a/beacon_node/network/src/sync/block_lookups/parent_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/parent_lookup.rs @@ -67,7 +67,7 @@ impl ParentLookup { let current_parent_request = SingleBlockLookup::new( parent_root, Some(<_>::default()), - &[peer_id], + peer_id, da_checker, cx.next_id(), ); @@ -210,8 +210,8 @@ impl ParentLookup { Ok(root_and_verified) } - pub fn add_peers(&mut self, peer_source: &[PeerShouldHave]) { - self.current_parent_request.add_peers(peer_source) + pub fn add_peer(&mut self, peer: PeerShouldHave) { + self.current_parent_request.add_peer(peer) } pub fn used_peers(&self) -> impl Iterator + '_ { 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 445b0a06f17..eeeb5f69afe 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 @@ -66,7 +66,7 @@ impl SingleBlockLookup { pub fn new( requested_block_root: Hash256, child_components: Option>, - peers: &[PeerShouldHave], + peers: PeerShouldHave, da_checker: Arc>, id: Id, ) -> Self { @@ -186,17 +186,15 @@ impl SingleBlockLookup { } /// 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_peer(&mut self, peer: PeerShouldHave) { + 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); } } } @@ -335,7 +333,7 @@ pub struct BlobRequestState { } impl BlobRequestState { - pub fn new(block_root: Hash256, peer_source: &[PeerShouldHave], is_deneb: bool) -> Self { + pub fn new(block_root: Hash256, peer_source: PeerShouldHave, is_deneb: bool) -> Self { let default_ids = MissingBlobs::new_without_block(block_root, is_deneb); Self { requested_ids: default_ids, @@ -354,7 +352,7 @@ pub struct BlockRequestState { } impl BlockRequestState { - pub fn new(block_root: Hash256, peers: &[PeerShouldHave]) -> Self { + pub fn new(block_root: Hash256, peers: PeerShouldHave) -> Self { Self { requested_block_root: block_root, state: SingleLookupRequestState::new(peers), @@ -407,17 +405,15 @@ pub struct SingleLookupRequestState { } impl SingleLookupRequestState { - pub fn new(peers: &[PeerShouldHave]) -> Self { + pub fn new(peer: PeerShouldHave) -> Self { let mut available_peers = HashSet::default(); let mut potential_peers = HashSet::default(); - for peer in peers { - match peer { - PeerShouldHave::BlockAndBlobs(peer_id) => { - available_peers.insert(*peer_id); - } - PeerShouldHave::Neither(peer_id) => { - potential_peers.insert(*peer_id); - } + match peer { + PeerShouldHave::BlockAndBlobs(peer_id) => { + available_peers.insert(peer_id); + } + PeerShouldHave::Neither(peer_id) => { + potential_peers.insert(peer_id); } } Self { @@ -616,7 +612,7 @@ mod tests { let mut sl = SingleBlockLookup::::new( block.canonical_root(), None, - &[peer_id], + peer_id, da_checker, 1, ); @@ -657,7 +653,7 @@ mod tests { let mut sl = SingleBlockLookup::::new( block.canonical_root(), None, - &[peer_id], + peer_id, da_checker, 1, ); From f2644abdc002e37833e848831d9d3ddf320a3a3d Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 21 Sep 2023 14:32:16 -0400 Subject: [PATCH 26/47] add a couple docs --- .../network/src/network_beacon_processor/sync_methods.rs | 2 ++ 1 file changed, 2 insertions(+) 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 cdb4cb81854..a3ca2e0f113 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -272,6 +272,7 @@ impl NetworkBeaconProcessor { Box::pin(process_fn) } + /// Attempt to process a list of blobs received from a direct RPC request. pub async fn process_rpc_blobs( self: Arc>, block_root: Hash256, @@ -295,6 +296,7 @@ impl NetworkBeaconProcessor { }); } + /// Poll the beacon chain for any delayed lookups that are now available. pub fn poll_delayed_lookups(&self, slot: Slot) { let block_roots = self .chain From 14ad2d2c200f0cd5ede0f17b3d0fd2e154ace449 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 21 Sep 2023 14:47:40 -0400 Subject: [PATCH 27/47] add single blob merge method and improve process_... docs --- beacon_node/beacon_chain/src/beacon_chain.rs | 28 +++++++---------- .../src/data_availability_checker.rs | 30 +++++++++++++++++-- .../availability_view.rs | 28 ++++++++++------- 3 files changed, 56 insertions(+), 30 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index eab6659a49b..a043775f0fe 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -2792,45 +2792,37 @@ impl BeaconChain { .map_err(BeaconChainError::TokioJoin)? } + /// Cache the blob in the processing cache, process it, then evict it from the cache if it was + /// imported or errors. pub async fn process_gossip_blob( self: &Arc, blob: GossipVerifiedBlob, ) -> Result> { - // TODO: fewer places where we initialize this fixed vec let block_root = blob.block_root(); - let mut commitments = KzgCommitmentOpts::::default(); - if let Some(commitment_opt) = commitments.get_mut(blob.as_blob().index as usize) { - *commitment_opt = Some(blob.as_blob().kzg_commitment); - } - self.data_availability_checker.notify_blob_commitments( - blob.as_blob().slot, - block_root, - commitments, - ); + self.data_availability_checker + .notify_gossip_blob(blob.as_blob().slot, block_root, &blob); let r = self.check_gossip_blob_availability_and_import(blob).await; self.remove_notified(&block_root, r) } + /// Cache the blobs in the processing cache, process it, then evict it from the cache if it was + /// imported or errors. pub async fn process_rpc_blobs( self: &Arc, slot: Slot, block_root: Hash256, blobs: FixedBlobSidecarList, ) -> Result> { - let mut commitments = KzgCommitmentOpts::::default(); - for blob in blobs.iter().flatten() { - if let Some(commitment) = commitments.get_mut(blob.index as usize) { - *commitment = Some(blob.kzg_commitment); - } - } self.data_availability_checker - .notify_blob_commitments(slot, block_root, commitments); + .notify_rpc_blobs(slot, block_root, &blobs); let r = self .check_rpc_blob_availability_and_import(slot, block_root, blobs) .await; self.remove_notified(&block_root, r) } + /// Remove any block components from the *processing cache* if we no longer require them. If the + /// block was imported full or erred, we no longer require them. fn remove_notified( &self, block_root: &Hash256, @@ -2844,6 +2836,8 @@ impl BeaconChain { r } + /// Wraps `process_block` in logic to cache the block's commitments in the processing cache + /// and evict if the block was imported or erred. pub async fn process_block_with_early_caching>( self: &Arc, block_root: Hash256, diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index 655b49cdc31..c3e492bb0d5 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -337,20 +337,44 @@ impl DataAvailabilityChecker { .merge_block(commitments); } + /// Add a single blob commitment to the processing cache. This commitment is unverified but caching + /// them here is useful to avoid duplicate downloads of blobs, as well as understanding + /// our block and blob download requirements. + pub fn notify_gossip_blob( + &self, + slot: Slot, + block_root: Hash256, + blob: &GossipVerifiedBlob, + ) { + let index = blob.as_blob().index; + let commitment = blob.as_blob().kzg_commitment; + self.processing_cache + .write() + .entry(block_root) + .or_insert_with(|| ProcessingComponents::new(slot)) + .merge_single_blob(index as usize, commitment); + } + /// Adds blob commitments to the processing cache. These commitments are unverified but caching /// them here is useful to avoid duplicate downloads of blobs, as well as understanding /// our block and blob download requirements. - pub fn notify_blob_commitments( + pub fn notify_rpc_blobs( &self, slot: Slot, block_root: Hash256, - blobs: KzgCommitmentOpts, + blobs: &FixedBlobSidecarList, ) { + let mut commitments = KzgCommitmentOpts::::default(); + for blob in blobs.iter().flatten() { + if let Some(commitment) = commitments.get_mut(blob.index as usize) { + *commitment = Some(blob.kzg_commitment); + } + } self.processing_cache .write() .entry(block_root) .or_insert_with(|| ProcessingComponents::new(slot)) - .merge_blobs(blobs); + .merge_blobs(commitments); } /// Clears the block and all blobs from the processing cache for a give root if they exist. diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index 2bffb18fcd4..d7dd8178e44 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -98,18 +98,26 @@ pub trait AvailabilityView { fn merge_blobs(&mut self, blobs: FixedVector, E::MaxBlobsPerBlock>) { for (index, blob) in blobs.to_vec().into_iter().enumerate() { let Some(blob) = blob else { continue }; - let commitment = *blob.get_commitment(); - - if let Some(cached_block) = self.get_cached_block() { - let block_commitment_opt = cached_block.get_commitments().get(index).copied(); - if let Some(block_commitment) = block_commitment_opt { - if block_commitment == commitment { - self.insert_blob_at_index(index, blob) - } + self.merge_single_blob(index, blob); + } + } + + /// Merges a single into the cache. + /// + /// Blobs are only inserted if: + /// 1. The blob entry at the index is empty and no block exists. + /// 2. The block exists and its commitment matches the blob's commitment. + fn merge_single_blob(&mut self, index: usize, blob: Self::BlobType) { + let commitment = *blob.get_commitment(); + if let Some(cached_block) = self.get_cached_block() { + let block_commitment_opt = cached_block.get_commitments().get(index).copied(); + if let Some(block_commitment) = block_commitment_opt { + if block_commitment == commitment { + self.insert_blob_at_index(index, blob) } - } else if !self.blob_exists(index) { - self.insert_blob_at_index(index, blob) } + } else if !self.blob_exists(index) { + self.insert_blob_at_index(index, blob) } } From ad1fabb2558a4f4436a080abfedf624eb9dab09c Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 21 Sep 2023 14:51:50 -0400 Subject: [PATCH 28/47] update some names --- beacon_node/beacon_chain/src/data_availability_checker.rs | 2 +- .../src/data_availability_checker/availability_view.rs | 6 +++--- .../network/src/sync/block_lookups/single_block_lookup.rs | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index c3e492bb0d5..d5e54c6b79c 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -139,7 +139,7 @@ impl DataAvailabilityChecker { } /// Get the processing info for a block. - pub fn get_processing_view( + pub fn get_processing_components( &self, block_root: Hash256, ) -> Option> { diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index d7dd8178e44..37b420cc77b 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -320,7 +320,7 @@ pub mod tests { FixedVector, ::MaxBlobsPerBlock>, ); - pub fn setup_processing_view( + pub fn setup_processing_components( block: SignedBeaconBlock, valid_blobs: FixedVector>, ::MaxBlobsPerBlock>, invalid_blobs: FixedVector>, ::MaxBlobsPerBlock>, @@ -539,11 +539,11 @@ pub mod tests { } generate_tests!( - processing_view_tests, + processing_components_tests, ProcessingComponents::, kzg_commitments, processing_blobs, - setup_processing_view + setup_processing_components ); generate_tests!( pending_components_tests, 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 eeeb5f69afe..d26b0915ac0 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 @@ -272,7 +272,7 @@ impl SingleBlockLookup { self.da_checker.get_missing_blob_ids(block_root, components) } else { let Some(processing_availability_view) = - self.da_checker.get_processing_view(block_root) + self.da_checker.get_processing_components(block_root) else { return MissingBlobs::new_without_block(block_root, self.da_checker.is_deneb()); }; From f1a1bc9322a3b0dc038fc1b6b99b9b0f3d91dcaa Mon Sep 17 00:00:00 2001 From: realbigsean Date: Thu, 21 Sep 2023 15:28:42 -0400 Subject: [PATCH 29/47] lints --- beacon_node/beacon_chain/src/beacon_chain.rs | 1 - .../src/data_availability_checker.rs | 6 ++---- .../availability_view.rs | 2 +- .../overflow_lru_cache.rs | 16 ++++++++-------- .../processing_cache.rs | 7 +++++-- .../network/src/sync/block_lookups/mod.rs | 4 +--- .../sync/block_lookups/single_block_lookup.rs | 10 ++++++---- 7 files changed, 23 insertions(+), 23 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index a043775f0fe..41a93068d1a 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -118,7 +118,6 @@ use store::{ use task_executor::{ShutdownReason, TaskExecutor}; use tokio_stream::Stream; use tree_hash::TreeHash; -use types::beacon_block_body::KzgCommitmentOpts; use types::beacon_state::CloneConfig; use types::blob_sidecar::{BlobSidecarList, FixedBlobSidecarList}; use types::sidecar::BlobItems; diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index d5e54c6b79c..62dc82535c5 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -9,8 +9,6 @@ pub use crate::data_availability_checker::child_components::ChildComponents; use crate::data_availability_checker::overflow_lru_cache::OverflowLRUCache; use crate::data_availability_checker::processing_cache::ProcessingCache; use crate::{BeaconChain, BeaconChainTypes, BeaconStore}; -use futures::StreamExt; -use itertools::Itertools; use kzg::Kzg; use kzg::{Error as KzgError, KzgCommitment}; use parking_lot::RwLock; @@ -614,8 +612,8 @@ impl MissingBlobs { } pub fn indices(&self) -> Vec { match self { - MissingBlobs::KnownMissing(v) => v.into_iter().map(|id| id.index).collect(), - MissingBlobs::PossibleMissing(v) => v.into_iter().map(|id| id.index).collect(), + MissingBlobs::KnownMissing(v) => v.iter().map(|id| id.index).collect(), + MissingBlobs::PossibleMissing(v) => v.iter().map(|id| id.index).collect(), MissingBlobs::BlobsNotRequired => vec![], } } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index 37b420cc77b..832c50b2d13 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -96,7 +96,7 @@ pub trait AvailabilityView { /// 1. The blob entry at the index is empty and no block exists. /// 2. The block exists and its commitment matches the blob's commitment. fn merge_blobs(&mut self, blobs: FixedVector, E::MaxBlobsPerBlock>) { - for (index, blob) in blobs.to_vec().into_iter().enumerate() { + for (index, blob) in blobs.iter().cloned().enumerate() { let Some(blob) = blob else { continue }; self.merge_single_blob(index, blob); } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs index 06182820928..a4f96b94857 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs @@ -1068,7 +1068,7 @@ mod test { .expect("kzg should verify"); kzg_verified_blobs.push(kzg_verified_blob); let availability = cache - .put_kzg_verified_blobs(root, kzg_verified_blobs.as_slice()) + .put_kzg_verified_blobs(root, kzg_verified_blobs.clone()) .expect("should put blob"); if blob_index == blobs_expected - 1 { assert!(matches!(availability, Availability::Available(_))); @@ -1096,7 +1096,7 @@ mod test { .expect("kzg should verify"); kzg_verified_blobs.push(kzg_verified_blob); let availability = cache - .put_kzg_verified_blobs(root, kzg_verified_blobs.as_slice()) + .put_kzg_verified_blobs(root, kzg_verified_blobs.clone()) .expect("should put blob"); assert_eq!( availability, @@ -1241,7 +1241,7 @@ mod test { .expect("kzg should verify"); kzg_verified_blobs.push(kzg_verified_blob); let availability = cache - .put_kzg_verified_blobs(roots[0], kzg_verified_blobs.as_slice()) + .put_kzg_verified_blobs(roots[0], kzg_verified_blobs.clone()) .expect("should put blob"); if blob_index == expected_blobs - 1 { assert!(matches!(availability, Availability::Available(_))); @@ -1348,7 +1348,7 @@ mod test { "should have pending blobs" ); let availability = cache - .put_kzg_verified_blobs(block_root, kzg_verified_blobs.as_slice()) + .put_kzg_verified_blobs(block_root, kzg_verified_blobs) .expect("should put blob"); assert!( matches!(availability, Availability::MissingComponents(_)), @@ -1357,7 +1357,7 @@ mod test { ); } else { let availability = cache - .put_kzg_verified_blobs(block_root, kzg_verified_blobs.as_slice()) + .put_kzg_verified_blobs(block_root, kzg_verified_blobs) .expect("should put blob"); let root = pending_block.block.as_block().canonical_root(); assert_eq!( @@ -1500,7 +1500,7 @@ mod test { "should have pending blobs" ); let availability = cache - .put_kzg_verified_blobs(block_root, kzg_verified_blobs.as_slice()) + .put_kzg_verified_blobs(block_root, kzg_verified_blobs) .expect("should put blob"); assert!( matches!(availability, Availability::MissingComponents(_)), @@ -1509,7 +1509,7 @@ mod test { ); } else { let availability = cache - .put_kzg_verified_blobs(block_root, kzg_verified_blobs.as_slice()) + .put_kzg_verified_blobs(block_root, kzg_verified_blobs) .expect("should put blob"); let root = pending_block.block.as_block().canonical_root(); assert_eq!( @@ -1601,7 +1601,7 @@ mod test { .expect("kzg should verify"); kzg_verified_blobs.push(kzg_verified_blob); let availability = recovered_cache - .put_kzg_verified_blobs(root, kzg_verified_blobs.as_slice()) + .put_kzg_verified_blobs(root, kzg_verified_blobs.clone()) .expect("should put blob"); if i == additional_blobs - 1 { assert!(matches!(availability, Availability::Available(_))) diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs index 6e3f4652259..5ccef967ce9 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -60,9 +60,12 @@ impl ProcessingComponents { processing_blobs: KzgCommitmentOpts::::default(), } } +} - #[cfg(test)] - pub fn default() -> Self { +// Not safe for use outside of tests as this always required a slot. +#[cfg(test)] +impl Default for ProcessingComponents { + fn default() -> Self { Self { slot: Slot::new(0), kzg_commitments: None, diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index db7bb48440b..ac157707019 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -13,9 +13,7 @@ use crate::sync::block_lookups::single_block_lookup::{ use crate::sync::manager::{Id, SingleLookupReqId}; use beacon_chain::block_verification_types::{AsBlock, RpcBlock}; pub use beacon_chain::data_availability_checker::ChildComponents; -use beacon_chain::data_availability_checker::{ - AvailabilityCheckError, AvailabilityView, DataAvailabilityChecker, -}; +use beacon_chain::data_availability_checker::{AvailabilityCheckError, DataAvailabilityChecker}; use beacon_chain::validator_monitor::timestamp_now; use beacon_chain::{AvailabilityProcessingStatus, BeaconChainTypes, BlockError}; pub use common::Current; 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 d26b0915ac0..e9aeb76c68e 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 @@ -603,8 +603,9 @@ mod tests { Duration::from_secs(spec.seconds_per_slot), ); let log = NullLoggerBuilder.build().expect("logger should build"); - let store = HotColdDB::open_ephemeral(StoreConfig::default(), ChainSpec::minimal(), log) - .expect("store"); + let store = + HotColdDB::open_ephemeral(StoreConfig::default(), ChainSpec::minimal(), log.clone()) + .expect("store"); let da_checker = Arc::new( DataAvailabilityChecker::new(slot_clock, None, store.into(), &log, spec) .expect("data availability checker"), @@ -642,8 +643,9 @@ mod tests { Duration::from_secs(spec.seconds_per_slot), ); let log = NullLoggerBuilder.build().expect("logger should build"); - let store = HotColdDB::open_ephemeral(StoreConfig::default(), ChainSpec::minimal(), log) - .expect("store"); + let store = + HotColdDB::open_ephemeral(StoreConfig::default(), ChainSpec::minimal(), log.clone()) + .expect("store"); let da_checker = Arc::new( DataAvailabilityChecker::new(slot_clock, None, store.into(), &log, spec) From 93843525ba3ca60c5db5858e2c08e42db194a3d3 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Mon, 25 Sep 2023 14:45:55 -0400 Subject: [PATCH 30/47] fix merge --- beacon_node/beacon_chain/src/test_utils.rs | 12 +++--------- .../src/network_beacon_processor/gossip_methods.rs | 4 ++-- beacon_node/network/src/sync/block_lookups/tests.rs | 1 - testing/ef_tests/src/cases/fork_choice.rs | 4 +--- 4 files changed, 6 insertions(+), 15 deletions(-) diff --git a/beacon_node/beacon_chain/src/test_utils.rs b/beacon_node/beacon_chain/src/test_utils.rs index 3439e34cab9..a40183c7906 100644 --- a/beacon_node/beacon_chain/src/test_utils.rs +++ b/beacon_node/beacon_chain/src/test_utils.rs @@ -2510,7 +2510,6 @@ pub enum NumBlobs { None, } -//TODO: port michael's changes to Rng from store updates pub fn generate_rand_block_and_blobs( fork_name: ForkName, num_blobs: NumBlobs, @@ -2524,17 +2523,12 @@ pub fn generate_rand_block_and_blobs( // get random number between 0 and Max Blobs let payload: &mut FullPayloadDeneb = &mut message.body.execution_payload; let num_blobs = match num_blobs { - NumBlobs::Random => { - let mut num_blobs = rand::random::() % E::max_blobs_per_block(); - if num_blobs == 0 { - num_blobs += 1; - } - num_blobs - } + NumBlobs::Random => 1 + rng.gen::() % E::max_blobs_per_block(), NumBlobs::None => 0, }; let (bundle, transactions) = - execution_layer::test_utils::generate_random_blobs::(num_blobs, kzg).unwrap(); + execution_layer::test_utils::generate_random_blobs::(num_blobs, kzg, rng) + .unwrap(); payload.execution_payload.transactions = <_>::default(); for tx in Vec::from(transactions) { 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 6f34e0f2e2c..4119371b87f 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -639,8 +639,8 @@ impl NetworkBeaconProcessor { self.log, "Successfully verified gossip blob"; "slot" => %slot, - "root" => %root, - "index" => %index + "root" => %root, + "index" => %index ); self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Accept); diff --git a/beacon_node/network/src/sync/block_lookups/tests.rs b/beacon_node/network/src/sync/block_lookups/tests.rs index 828f4b9d243..b94d3ea48eb 100644 --- a/beacon_node/network/src/sync/block_lookups/tests.rs +++ b/beacon_node/network/src/sync/block_lookups/tests.rs @@ -16,7 +16,6 @@ use beacon_chain::test_utils::{ use beacon_processor::WorkEvent; use lighthouse_network::rpc::RPCResponseErrorCode; use lighthouse_network::{NetworkGlobals, Request}; -use rand::Rng; use slot_clock::{ManualSlotClock, SlotClock, TestingSlotClock}; use store::MemoryStore; use tokio::sync::mpsc; diff --git a/testing/ef_tests/src/cases/fork_choice.rs b/testing/ef_tests/src/cases/fork_choice.rs index fea728535ba..db94106975e 100644 --- a/testing/ef_tests/src/cases/fork_choice.rs +++ b/testing/ef_tests/src/cases/fork_choice.rs @@ -449,9 +449,7 @@ impl Tester { let result = self.block_on_dangerous( self.harness .chain - .check_gossip_blob_availability_and_import( - GossipVerifiedBlob::__assumed_valid(signed_sidecar), - ), + .process_gossip_blob(GossipVerifiedBlob::__assumed_valid(signed_sidecar)), )?; if valid { assert!(result.is_ok()); From 32b92ff04c732a9f4117bf1af91f3dc24a2d9650 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Mon, 25 Sep 2023 14:47:54 -0400 Subject: [PATCH 31/47] remove blob indices from lookup creation log --- .../network/src/sync/block_lookups/mod.rs | 44 +++++-------------- 1 file changed, 11 insertions(+), 33 deletions(-) diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index ac157707019..f31dfc35da7 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -214,12 +214,6 @@ impl BlockLookups { return; } - let msg = if child_components.is_some() { - "Searching for components of a block with unknown parent" - } else { - "Searching for block" - }; - let lookup = SingleBlockLookup::new( block_root, child_components, @@ -227,7 +221,17 @@ impl BlockLookups { self.da_checker.clone(), cx.next_id(), ); - lookup_creation_logging(msg, &lookup, peer, &self.log); + let msg = if child_components.is_some() { + "Searching for components of a block with unknown parent" + } else { + "Searching for block" + }; + debug!( + log, + "{}", msg; + "peer_ids" => ?peer_source, + "block" => ?block_root, + ); self.trigger_single_lookup(lookup, cx); } @@ -1389,29 +1393,3 @@ impl BlockLookups { self.parent_lookups.drain(..).len() } } - -fn lookup_creation_logging( - msg: &str, - lookup: &SingleBlockLookup, - 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_ids" => ?peer_source, - "block" => ?block_root, - "blob_indices" => ?blob_indices - ); - } else { - debug!( - log, - "{}", msg; - "peer_ids" => ?peer_source, - "block" => ?block_root, - ); - } -} From 9fc5ca5295f611a219d0cca67ddbfd73f0584dc0 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Mon, 25 Sep 2023 14:56:14 -0400 Subject: [PATCH 32/47] remove blob indices from lookup creation log --- .../network/src/network_beacon_processor/sync_methods.rs | 3 +++ beacon_node/network/src/sync/block_lookups/mod.rs | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/beacon_node/network/src/network_beacon_processor/sync_methods.rs b/beacon_node/network/src/network_beacon_processor/sync_methods.rs index a3ca2e0f113..b5531e3ec4e 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -302,6 +302,9 @@ impl NetworkBeaconProcessor { .chain .data_availability_checker .incomplete_processing_components(slot); + if !block_roots.is_empty() { + debug!(self.log, "Found delayed lookups on poll"; "lookup_count" => block_roots.len()); + } for block_root in block_roots { if let Some(peer_ids) = self.delayed_lookup_peers.lock().pop(&block_root) { // Let the network layer de-dupe these. diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index f31dfc35da7..5099c2d4793 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -227,9 +227,9 @@ impl BlockLookups { "Searching for block" }; debug!( - log, + self.log, "{}", msg; - "peer_ids" => ?peer_source, + "peer_ids" => ?peer, "block" => ?block_root, ); self.trigger_single_lookup(lookup, cx); From bf80cab9e63e43f82a093e04f2687134ed28ce41 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Mon, 25 Sep 2023 14:58:36 -0400 Subject: [PATCH 33/47] delayed lookup logging improvement --- .../src/network_beacon_processor/sync_methods.rs | 6 ++++-- beacon_node/network/src/sync/block_lookups/mod.rs | 12 +++++++----- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/beacon_node/network/src/network_beacon_processor/sync_methods.rs b/beacon_node/network/src/network_beacon_processor/sync_methods.rs index b5531e3ec4e..8159586c13b 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -18,7 +18,7 @@ use beacon_processor::{ AsyncFn, BlockingFn, DuplicateCache, }; use lighthouse_network::PeerAction; -use slog::{debug, error, info, warn}; +use slog::{debug, error, info, trace, warn}; use slot_clock::SlotClock; use std::sync::Arc; use std::time::Duration; @@ -302,7 +302,9 @@ impl NetworkBeaconProcessor { .chain .data_availability_checker .incomplete_processing_components(slot); - if !block_roots.is_empty() { + if block_roots.is_empty() { + trace!(self.log, "No delayed lookups found on poll"); + } else { debug!(self.log, "Found delayed lookups on poll"; "lookup_count" => block_roots.len()); } for block_root in block_roots { diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index 5099c2d4793..c6fbffb867e 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -214,6 +214,12 @@ impl BlockLookups { return; } + let msg = if child_components.is_some() { + "Searching for components of a block with unknown parent" + } else { + "Searching for block" + }; + let lookup = SingleBlockLookup::new( block_root, child_components, @@ -221,11 +227,7 @@ impl BlockLookups { self.da_checker.clone(), cx.next_id(), ); - let msg = if child_components.is_some() { - "Searching for components of a block with unknown parent" - } else { - "Searching for block" - }; + debug!( self.log, "{}", msg; From c774a35dbaf5727741466625721efd22091ec76c Mon Sep 17 00:00:00 2001 From: realbigsean Date: Mon, 25 Sep 2023 16:00:33 -0400 Subject: [PATCH 34/47] check fork choice before doing any blob processing --- beacon_node/beacon_chain/src/beacon_chain.rs | 25 +++++++++++++++++-- .../network/src/sync/block_lookups/mod.rs | 2 +- 2 files changed, 24 insertions(+), 3 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 41a93068d1a..af8109d4701 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -477,8 +477,8 @@ pub struct BeaconChain { pub validator_monitor: RwLock>, /// The slot at which blocks are downloaded back to. pub genesis_backfill_slot: Slot, - // Provides a KZG verification and temporary storage for blocks and blobs as - // they are collected and combined. + /// Provides a KZG verification and temporary storage for blocks and blobs as + /// they are collected and combined. pub data_availability_checker: Arc>, /// The KZG trusted setup used by this chain. pub kzg: Option::Kzg>>>, @@ -2798,6 +2798,17 @@ impl BeaconChain { blob: GossipVerifiedBlob, ) -> Result> { let block_root = blob.block_root(); + + // If this block has already been imported to forkchoice it must have been available, so + // we don't need to process its blobs again. + if self + .canonical_head + .fork_choice_read_lock() + .contains_block(&block_root) + { + return Err(BlockError::BlockIsAlreadyKnown); + } + self.data_availability_checker .notify_gossip_blob(blob.as_blob().slot, block_root, &blob); let r = self.check_gossip_blob_availability_and_import(blob).await; @@ -2812,6 +2823,16 @@ impl BeaconChain { block_root: Hash256, blobs: FixedBlobSidecarList, ) -> Result> { + // If this block has already been imported to forkchoice it must have been available, so + // we don't need to process its blobs again. + if self + .canonical_head + .fork_choice_read_lock() + .contains_block(&block_root) + { + return Err(BlockError::BlockIsAlreadyKnown); + } + self.data_availability_checker .notify_rpc_blobs(slot, block_root, &blobs); let r = self diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index c6fbffb867e..8f17dccdc5d 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -217,7 +217,7 @@ impl BlockLookups { let msg = if child_components.is_some() { "Searching for components of a block with unknown parent" } else { - "Searching for block" + "Searching for block components" }; let lookup = SingleBlockLookup::new( From 816744ca8cbcd220a34db55d04c959ca936d66c9 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Mon, 25 Sep 2023 17:01:42 -0400 Subject: [PATCH 35/47] remove unused dep --- beacon_node/network/src/network_beacon_processor/tests.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/beacon_node/network/src/network_beacon_processor/tests.rs b/beacon_node/network/src/network_beacon_processor/tests.rs index 903d1ac6127..a074c7b425b 100644 --- a/beacon_node/network/src/network_beacon_processor/tests.rs +++ b/beacon_node/network/src/network_beacon_processor/tests.rs @@ -26,7 +26,6 @@ use lighthouse_network::{ use lru::LruCache; use parking_lot::Mutex; use slot_clock::SlotClock; -use std::collections::HashMap; use std::iter::Iterator; use std::sync::Arc; use std::time::Duration; From 6a5178a53dac41d7d04e036b6d0bc97543cfb888 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Wed, 27 Sep 2023 07:43:43 -0400 Subject: [PATCH 36/47] Update beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs Co-authored-by: Michael Sproul --- .../src/data_availability_checker/availability_view.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index 832c50b2d13..cf2efd41ff1 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -102,7 +102,7 @@ pub trait AvailabilityView { } } - /// Merges a single into the cache. + /// Merges a single blob into the cache. /// /// Blobs are only inserted if: /// 1. The blob entry at the index is empty and no block exists. From 303771d046e0a423763e57e26c86004db061867f Mon Sep 17 00:00:00 2001 From: realbigsean Date: Wed, 27 Sep 2023 07:43:55 -0400 Subject: [PATCH 37/47] Update beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs Co-authored-by: Michael Sproul --- .../src/data_availability_checker/availability_view.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index cf2efd41ff1..b3f604fbfb2 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -105,7 +105,7 @@ pub trait AvailabilityView { /// Merges a single blob into the cache. /// /// Blobs are only inserted if: - /// 1. The blob entry at the index is empty and no block exists. + /// 1. The blob entry at the index is empty and no block exists, or /// 2. The block exists and its commitment matches the blob's commitment. fn merge_single_blob(&mut self, index: usize, blob: Self::BlobType) { let commitment = *blob.get_commitment(); From 4a6caa3beb84ff80e4a7f42be2aa60ad7f3102d3 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Wed, 27 Sep 2023 07:44:19 -0400 Subject: [PATCH 38/47] Update beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs Co-authored-by: Michael Sproul --- .../src/data_availability_checker/availability_view.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index b3f604fbfb2..2d0babd28da 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -126,8 +126,7 @@ pub trait AvailabilityView { /// Blobs that don't match the new block's commitments are evicted. fn merge_block(&mut self, block: Self::BlockType) { self.insert_block(block); - let mut reinsert = FixedVector::default(); - std::mem::swap(self.get_cached_blobs_mut(), &mut reinsert); + let reinsert = std::mem::take(self.get_cached_blobs_mut()); self.merge_blobs(reinsert); } From 581ea31af4eac36bdf554a0cf5519a0ec33021db Mon Sep 17 00:00:00 2001 From: realbigsean Date: Wed, 27 Sep 2023 07:44:26 -0400 Subject: [PATCH 39/47] Update beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs Co-authored-by: Michael Sproul --- .../src/data_availability_checker/availability_view.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index 2d0babd28da..cda2b261233 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -241,7 +241,7 @@ impl GetCommitment for KzgVerifiedBlob { } } -// These implementations are required to implement `AvailabilityView` for `CachedChildComponents`. +// These implementations are required to implement `AvailabilityView` for `ChildComponents`. impl GetCommitments for Arc> { fn get_commitments(&self) -> KzgCommitments { self.message() From 5e3ee979de819d4b119b979fa832a69da88ac666 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Wed, 27 Sep 2023 07:44:34 -0400 Subject: [PATCH 40/47] Update beacon_node/network/src/sync/block_lookups/delayed_lookup.rs Co-authored-by: Michael Sproul --- beacon_node/network/src/sync/block_lookups/delayed_lookup.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs b/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs index 74d21fc4226..2cdb586524d 100644 --- a/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs @@ -57,7 +57,7 @@ pub fn spawn_delayed_lookup_service( let mut interval = interval_at(interval_start, slot_duration); loop { interval.tick().await; - let Some(slot) = beacon_chain.slot_clock.now_or_genesis() else { + let Some(slot) = beacon_chain.slot_clock.now_or_genesis() else { error!(log, "Skipping delayed lookup poll, unable to read slot clock"); continue }; From 61d56c5576a7a770b12e5c90092c13b57da989ba Mon Sep 17 00:00:00 2001 From: realbigsean Date: Fri, 29 Sep 2023 09:17:18 -0400 Subject: [PATCH 41/47] remove duplicate deps --- beacon_node/network/Cargo.toml | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/beacon_node/network/Cargo.toml b/beacon_node/network/Cargo.toml index 362445cc9f6..059a157476d 100644 --- a/beacon_node/network/Cargo.toml +++ b/beacon_node/network/Cargo.toml @@ -40,17 +40,7 @@ itertools = { workspace = true } num_cpus = { workspace = true } lru_cache = { workspace = true } if-addrs = "0.6.4" -strum = "0.24.0" -tokio-util = { version = "0.6.3", features = ["time"] } -derivative = "2.2.0" -delay_map = "0.3.0" -ethereum-types = { version = "0.14.1", optional = true } -operation_pool = { path = "../operation_pool" } -execution_layer = { path = "../execution_layer" } -beacon_processor = { path = "../beacon_processor" } -parking_lot = "0.12.0" -environment = { path = "../../lighthouse/environment" } -lru = "0.7.1" +lru = { workspace = true } strum = { workspace = true } tokio-util = { workspace = true } derivative = { workspace = true } From 188c9744ad06c77e8ee9eebe38a1b29d2f892472 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Mon, 2 Oct 2023 11:21:36 -0400 Subject: [PATCH 42/47] use gen range in random blobs geneartor --- Cargo.lock | 67 +++++++++++----------- beacon_node/beacon_chain/src/test_utils.rs | 4 +- 2 files changed, 34 insertions(+), 37 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b1086fadbc6..447b4de49d3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -197,30 +197,29 @@ dependencies = [ [[package]] name = "anstream" -version = "0.3.2" +version = "0.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ca84f3628370c59db74ee214b3263d58f9aadd9b4fe7e711fd87dc452b7f163" +checksum = "2ab91ebe16eb252986481c5b62f6098f3b698a45e34b5b98200cf20dd2484a44" dependencies = [ "anstyle", "anstyle-parse", "anstyle-query", "anstyle-wincon", "colorchoice", - "is-terminal", "utf8parse", ] [[package]] name = "anstyle" -version = "1.0.1" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3a30da5c5f2d5e72842e00bcb57657162cdabef0931f40e2deb9b4140440cecd" +checksum = "7079075b41f533b8c61d2a4d073c4676e1f8b249ff94a393b0595db304e0dd87" [[package]] name = "anstyle-parse" -version = "0.2.1" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "938874ff5980b03a87c5524b3ae5b59cf99b1d6bc836848df7bc5ada9643c333" +checksum = "317b9a89c1868f5ea6ff1d9539a69f45dffc21ce321ac1fd1160dfa48c8e2140" dependencies = [ "utf8parse", ] @@ -236,9 +235,9 @@ dependencies = [ [[package]] name = "anstyle-wincon" -version = "1.0.2" +version = "3.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c677ab05e09154296dd37acecd46420c17b9713e8366facafa8fc0885167cf4c" +checksum = "f0699d10d2f4d628a98ee7b57b289abbc98ff3bad977cb3152709d4bf2330628" dependencies = [ "anstyle", "windows-sys 0.48.0", @@ -572,7 +571,7 @@ name = "beacon-api-client" version = "0.1.0" source = "git+https://github.com/ralexstokes/beacon-api-client?rev=7f28993615fde52d563dd601a0511c34fe9b7c38#7f28993615fde52d563dd601a0511c34fe9b7c38" dependencies = [ - "clap 4.3.21", + "clap 4.4.6", "ethereum-consensus", "http", "itertools", @@ -1161,20 +1160,19 @@ dependencies = [ [[package]] name = "clap" -version = "4.3.21" +version = "4.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c27cdf28c0f604ba3f512b0c9a409f8de8513e4816705deb0498b627e7c3a3fd" +checksum = "d04704f56c2cde07f43e8e2c154b43f216dc5c92fc98ada720177362f953b956" dependencies = [ "clap_builder", "clap_derive", - "once_cell", ] [[package]] name = "clap_builder" -version = "4.3.21" +version = "4.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08a9f1ab5e9f01a9b81f202e8562eb9a10de70abf9eaeac1be465c28b75aa4aa" +checksum = "0e231faeaca65ebd1ea3c737966bf858971cd38c3849107aa3ea7de90a804e45" dependencies = [ "anstream", "anstyle", @@ -1184,9 +1182,9 @@ dependencies = [ [[package]] name = "clap_derive" -version = "4.3.12" +version = "4.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54a9bb5758fc5dfe728d1019941681eccaf0cf8a4189b692a0ee2f2ecf90a050" +checksum = "0862016ff20d69b84ef8247369fabf5c008a7417002411897d40ee1f4532b873" dependencies = [ "heck", "proc-macro2", @@ -1196,9 +1194,9 @@ dependencies = [ [[package]] name = "clap_lex" -version = "0.5.0" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2da6da31387c7e4ef160ffab6d5e7f00c42626fe39aea70a7b0f1773f7dd6c1b" +checksum = "cd7cc57abe963c6d3b9d8be5b06ba7c8957a930305ca90304f24ef040aa6f961" [[package]] name = "clap_utils" @@ -3472,6 +3470,15 @@ dependencies = [ "hmac 0.8.1", ] +[[package]] +name = "home" +version = "0.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5444c27eef6923071f7ebcc33e3444508466a76f7a2b93da00ed6e19f30c1ddb" +dependencies = [ + "windows-sys 0.48.0", +] + [[package]] name = "hostname" version = "0.3.1" @@ -3905,17 +3912,6 @@ version = "2.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "28b29a3cd74f0f4598934efe3aeba42bae0eb4680554128851ebbecb02af14e6" -[[package]] -name = "is-terminal" -version = "0.4.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb0889898416213fab133e1d33a0e5858a48177452750691bde3666d0fdbaf8b" -dependencies = [ - "hermit-abi 0.3.2", - "rustix 0.38.13", - "windows-sys 0.48.0", -] - [[package]] name = "itertools" version = "0.10.5" @@ -6102,9 +6098,9 @@ dependencies = [ [[package]] name = "prettyplease" -version = "0.2.12" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c64d9ba0963cdcea2e1b2230fbae2bab30eb25a174be395c41e764bfb65dd62" +checksum = "ae005bd773ab59b4725093fd7df83fd7892f7d8eafb48dbd7de6e024e4215f9d" dependencies = [ "proc-macro2", "syn 2.0.37", @@ -9076,13 +9072,14 @@ checksum = "14247bb57be4f377dfb94c72830b8ce8fc6beac03cf4bf7b9732eadd414123fc" [[package]] name = "which" -version = "4.4.0" +version = "4.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2441c784c52b289a054b7201fc93253e288f094e2f4be9058343127c4226a269" +checksum = "87ba24419a2078cd2b0f2ede2691b6c66d8e47836da3b6db8265ebad47afbfc7" dependencies = [ "either", - "libc", + "home", "once_cell", + "rustix 0.38.13", ] [[package]] diff --git a/beacon_node/beacon_chain/src/test_utils.rs b/beacon_node/beacon_chain/src/test_utils.rs index e16d293b055..0e334eee43a 100644 --- a/beacon_node/beacon_chain/src/test_utils.rs +++ b/beacon_node/beacon_chain/src/test_utils.rs @@ -2530,10 +2530,10 @@ pub fn generate_rand_block_and_blobs( let mut block = SignedBeaconBlock::from_block(inner, types::Signature::random_for_test(rng)); let mut blob_sidecars = vec![]; if let Ok(message) = block.message_deneb_mut() { - // get random number between 0 and Max Blobs + // Get either zero blobs or a random number of blobs between 1 and Max Blobs. let payload: &mut FullPayloadDeneb = &mut message.body.execution_payload; let num_blobs = match num_blobs { - NumBlobs::Random => 1 + rng.gen::() % E::max_blobs_per_block(), + NumBlobs::Random => rng.gen_range(1..=E::max_blobs_per_block()), NumBlobs::None => 0, }; let (bundle, transactions) = From 49b0c2bf146d8f3e6de1d83c861e060a34ba9e25 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Mon, 2 Oct 2023 11:27:03 -0400 Subject: [PATCH 43/47] rename processing cache fields --- .../availability_view.rs | 4 ++-- .../processing_cache.rs | 17 ++++++++--------- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index cda2b261233..d9c06cafeb9 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -184,8 +184,8 @@ impl_availability_view!( ProcessingComponents, KzgCommitments, KzgCommitment, - kzg_commitments, - processing_blobs + block_commitments, + blob_commitments ); impl_availability_view!( diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs index 5ccef967ce9..239668997b7 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -45,19 +45,18 @@ pub struct ProcessingComponents { /// Blobs required for a block can only be known if we have seen the block. So `Some` here /// means we've seen it, a `None` means we haven't. The `kzg_commitments` value helps us figure /// out whether incoming blobs actually match the block. - pub kzg_commitments: Option>, - /// On insertion, a collision at an index here when `required_blobs` is - /// `None` means we need to construct an entirely new `Data` entry. This is because we have - /// no way of knowing which blob is the correct one until we see the block. - pub processing_blobs: KzgCommitmentOpts, + pub block_commitments: Option>, + /// `KzgCommitments` for blobs are always known, even if we haven't seen the block. See + /// `AvailabilityView`'s trait definition for more details. + pub blob_commitments: KzgCommitmentOpts, } impl ProcessingComponents { pub fn new(slot: Slot) -> Self { Self { slot, - kzg_commitments: None, - processing_blobs: KzgCommitmentOpts::::default(), + block_commitments: None, + blob_commitments: KzgCommitmentOpts::::default(), } } } @@ -68,8 +67,8 @@ impl Default for ProcessingComponents { fn default() -> Self { Self { slot: Slot::new(0), - kzg_commitments: None, - processing_blobs: KzgCommitmentOpts::::default(), + block_commitments: None, + blob_commitments: KzgCommitmentOpts::::default(), } } } From 220fda1f2968df50ad90b5b6e1390c974853b2f0 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Mon, 2 Oct 2023 14:05:53 -0400 Subject: [PATCH 44/47] require block root in rpc block construction and check block root consistency --- beacon_node/beacon_chain/src/beacon_chain.rs | 6 +- .../beacon_chain/src/block_verification.rs | 14 ++- .../src/block_verification_types.rs | 98 +++++++++++++------ .../src/data_availability_checker.rs | 28 ++++-- .../availability_view.rs | 21 ++-- .../child_components.rs | 17 +++- .../overflow_lru_cache.rs | 21 +++- .../processing_cache.rs | 4 +- .../beacon_chain/src/early_attester_cache.rs | 2 +- .../beacon_chain/src/historical_blocks.rs | 5 +- beacon_node/beacon_chain/src/test_utils.rs | 11 ++- .../tests/attestation_production.rs | 4 +- .../beacon_chain/tests/block_verification.rs | 35 ++++--- beacon_node/beacon_chain/tests/store_tests.rs | 6 +- .../network_beacon_processor/sync_methods.rs | 2 +- .../network/src/sync/block_lookups/common.rs | 5 +- .../network/src/sync/block_lookups/mod.rs | 2 +- .../src/sync/block_lookups/parent_lookup.rs | 4 +- .../sync/block_lookups/single_block_lookup.rs | 9 +- .../network/src/sync/block_lookups/tests.rs | 11 ++- .../src/sync/block_sidecar_coupling.rs | 2 +- beacon_node/network/src/sync/manager.rs | 6 +- 22 files changed, 209 insertions(+), 104 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index fd5dd613575..14733a89ab4 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -11,7 +11,7 @@ use crate::blob_verification::{self, GossipBlobError, GossipVerifiedBlob}; use crate::block_times_cache::BlockTimesCache; use crate::block_verification::POS_PANDA_BANNER; use crate::block_verification::{ - check_block_is_finalized_checkpoint_or_descendant, check_block_relevancy, get_block_root, + check_block_is_finalized_checkpoint_or_descendant, check_block_relevancy, signature_verify_chain_segment, BlockError, ExecutionPendingBlock, GossipVerifiedBlock, IntoExecutionPendingBlock, }; @@ -2552,7 +2552,7 @@ impl BeaconChain { }); } - let block_root = get_block_root(block.as_block()); + let block_root = block.block_root(); if let Some((child_parent_root, child_slot)) = children.get(i) { // If this block has a child in this chain segment, ensure that its parent root matches @@ -3324,7 +3324,7 @@ impl BeaconChain { // If the write fails, revert fork choice to the version from disk, else we can // end up with blocks in fork choice that are missing from disk. // See https://github.com/sigp/lighthouse/issues/2028 - let (signed_block, blobs) = signed_block.deconstruct(); + let (_, signed_block, blobs) = signed_block.deconstruct(); let block = signed_block.message(); ops.extend( confirmed_state_roots diff --git a/beacon_node/beacon_chain/src/block_verification.rs b/beacon_node/beacon_chain/src/block_verification.rs index a534b515ac4..7f1a596ec3c 100644 --- a/beacon_node/beacon_chain/src/block_verification.rs +++ b/beacon_node/beacon_chain/src/block_verification.rs @@ -827,7 +827,7 @@ impl GossipVerifiedBlock { drop(fork_choice_read_lock); let block_epoch = block.slot().epoch(T::EthSpec::slots_per_epoch()); - let (parent_block, block) = verify_parent_block_is_known(chain, block)?; + let (parent_block, block) = verify_parent_block_is_known(block_root, chain, block)?; // Track the number of skip slots between the block and its parent. metrics::set_gauge( @@ -1085,7 +1085,10 @@ impl SignatureVerifiedBlock { if signature_verifier.verify().is_ok() { Ok(Self { - block: MaybeAvailableBlock::AvailabilityPending(block), + block: MaybeAvailableBlock::AvailabilityPending { + block_root: from.block_root, + block, + }, block_root: from.block_root, parent: Some(parent), consensus_context, @@ -1156,7 +1159,10 @@ impl IntoExecutionPendingBlock for Arc(block: &SignedBeaconBlock) -> Hash256 { /// fork choice. #[allow(clippy::type_complexity)] fn verify_parent_block_is_known( + block_root: Hash256, chain: &BeaconChain, block: Arc>, ) -> Result<(ProtoBlock, Arc>), BlockError> { @@ -1767,6 +1774,7 @@ fn verify_parent_block_is_known( Ok((proto_block, block)) } else { Err(BlockError::ParentUnknown(RpcBlock::new_without_blobs( + Some(block_root), block, ))) } diff --git a/beacon_node/beacon_chain/src/block_verification_types.rs b/beacon_node/beacon_chain/src/block_verification_types.rs index 46e4dba6140..7dae9d6cbe7 100644 --- a/beacon_node/beacon_chain/src/block_verification_types.rs +++ b/beacon_node/beacon_chain/src/block_verification_types.rs @@ -3,7 +3,7 @@ use crate::block_verification::BlockError; use crate::data_availability_checker::AvailabilityCheckError; pub use crate::data_availability_checker::{AvailableBlock, MaybeAvailableBlock}; use crate::eth1_finalization_cache::Eth1FinalizationData; -use crate::{GossipVerifiedBlock, PayloadVerificationOutcome}; +use crate::{get_block_root, GossipVerifiedBlock, PayloadVerificationOutcome}; use derivative::Derivative; use ssz_derive::{Decode, Encode}; use ssz_types::VariableList; @@ -35,9 +35,16 @@ use types::{ #[derive(Debug, Clone, Derivative)] #[derivative(Hash(bound = "E: EthSpec"))] pub struct RpcBlock { + block_root: Hash256, block: RpcBlockInner, } +impl RpcBlock { + pub fn block_root(&self) -> Hash256 { + self.block_root + } +} + /// Note: This variant is intentionally private because we want to safely construct the /// internal variants after applying consistency checks to ensure that the block and blobs /// are consistent with respect to each other. @@ -53,8 +60,14 @@ enum RpcBlockInner { impl RpcBlock { /// Constructs a `Block` variant. - pub fn new_without_blobs(block: Arc>) -> Self { + pub fn new_without_blobs( + block_root: Option, + block: Arc>, + ) -> Self { + let block_root = block_root.unwrap_or_else(|| get_block_root(&block)); + Self { + block_root, block: RpcBlockInner::Block(block), } } @@ -62,9 +75,12 @@ impl RpcBlock { /// Constructs a new `BlockAndBlobs` variant after making consistency /// checks between the provided blocks and blobs. pub fn new( + block_root: Option, block: Arc>, blobs: Option>, ) -> Result { + let block_root = block_root.unwrap_or_else(|| get_block_root(&block)); + if let (Some(blobs), Ok(block_commitments)) = ( blobs.as_ref(), block.message().body().blob_kzg_commitments(), @@ -73,6 +89,13 @@ impl RpcBlock { return Err(AvailabilityCheckError::MissingBlobs); } for (blob, &block_commitment) in blobs.iter().zip(block_commitments.iter()) { + let blob_block_root = blob.block_root; + if blob_block_root != block_root { + return Err(AvailabilityCheckError::InconsistentBlobBlockRoots { + block_root, + blob_block_root, + }); + } let blob_commitment = blob.kzg_commitment; if blob_commitment != block_commitment { return Err(AvailabilityCheckError::KzgCommitmentMismatch { @@ -86,10 +109,14 @@ impl RpcBlock { Some(blobs) => RpcBlockInner::BlockAndBlobs(block, blobs), None => RpcBlockInner::Block(block), }; - Ok(Self { block: inner }) + Ok(Self { + block_root, + block: inner, + }) } pub fn new_from_fixed( + block_root: Hash256, block: Arc>, blobs: FixedBlobSidecarList, ) -> Result { @@ -102,13 +129,20 @@ impl RpcBlock { } else { Some(VariableList::from(filtered)) }; - Self::new(block, blobs) + Self::new(Some(block_root), block, blobs) } - pub fn deconstruct(self) -> (Arc>, Option>) { + pub fn deconstruct( + self, + ) -> ( + Hash256, + Arc>, + Option>, + ) { + let block_root = self.block_root(); match self.block { - RpcBlockInner::Block(block) => (block, None), - RpcBlockInner::BlockAndBlobs(block, blobs) => (block, Some(blobs)), + RpcBlockInner::Block(block) => (block_root, block, None), + RpcBlockInner::BlockAndBlobs(block, blobs) => (block_root, block, Some(blobs)), } } pub fn n_blobs(&self) -> usize { @@ -119,18 +153,6 @@ impl RpcBlock { } } -impl From>> for RpcBlock { - fn from(value: Arc>) -> Self { - Self::new_without_blobs(value) - } -} - -impl From> for RpcBlock { - fn from(value: SignedBeaconBlock) -> Self { - Self::new_without_blobs(Arc::new(value)) - } -} - /// A block that has gone through all pre-deneb block processing checks including block processing /// and execution by an EL client. This block hasn't necessarily completed data availability checks. /// @@ -160,13 +182,14 @@ impl ExecutedBlock { payload_verification_outcome, )) } - MaybeAvailableBlock::AvailabilityPending(pending_block) => { - Self::AvailabilityPending(AvailabilityPendingExecutedBlock::new( - pending_block, - import_data, - payload_verification_outcome, - )) - } + MaybeAvailableBlock::AvailabilityPending { + block_root: _, + block: pending_block, + } => Self::AvailabilityPending(AvailabilityPendingExecutedBlock::new( + pending_block, + import_data, + payload_verification_outcome, + )), } } @@ -362,7 +385,7 @@ impl AsBlock for Arc> { } fn into_rpc_block(self) -> RpcBlock { - RpcBlock::new_without_blobs(self) + RpcBlock::new_without_blobs(None, self) } } @@ -388,13 +411,19 @@ impl AsBlock for MaybeAvailableBlock { fn as_block(&self) -> &SignedBeaconBlock { match &self { MaybeAvailableBlock::Available(block) => block.as_block(), - MaybeAvailableBlock::AvailabilityPending(block) => block, + MaybeAvailableBlock::AvailabilityPending { + block_root: _, + block, + } => block, } } fn block_cloned(&self) -> Arc> { match &self { MaybeAvailableBlock::Available(block) => block.block_cloned(), - MaybeAvailableBlock::AvailabilityPending(block) => block.clone(), + MaybeAvailableBlock::AvailabilityPending { + block_root: _, + block, + } => block.clone(), } } fn canonical_root(&self) -> Hash256 { @@ -404,7 +433,9 @@ impl AsBlock for MaybeAvailableBlock { fn into_rpc_block(self) -> RpcBlock { match self { MaybeAvailableBlock::Available(available_block) => available_block.into_rpc_block(), - MaybeAvailableBlock::AvailabilityPending(block) => RpcBlock::new_without_blobs(block), + MaybeAvailableBlock::AvailabilityPending { block_root, block } => { + RpcBlock::new_without_blobs(Some(block_root), block) + } } } } @@ -447,14 +478,17 @@ impl AsBlock for AvailableBlock { } fn into_rpc_block(self) -> RpcBlock { - let (block, blobs_opt) = self.deconstruct(); + let (block_root, block, blobs_opt) = self.deconstruct(); // Circumvent the constructor here, because an Available block will have already had // consistency checks performed. let inner = match blobs_opt { None => RpcBlockInner::Block(block), Some(blobs) => RpcBlockInner::BlockAndBlobs(block, blobs), }; - RpcBlock { block: inner } + RpcBlock { + block_root, + block: inner, + } } } diff --git a/beacon_node/beacon_chain/src/data_availability_checker.rs b/beacon_node/beacon_chain/src/data_availability_checker.rs index 86f400b0b44..be427ae9f69 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker.rs @@ -282,13 +282,14 @@ impl DataAvailabilityChecker { &self, block: RpcBlock, ) -> Result, AvailabilityCheckError> { - let (block, blobs) = block.deconstruct(); + let (block_root, block, blobs) = block.deconstruct(); match blobs { None => { if self.blobs_required_for_block(&block) { - Ok(MaybeAvailableBlock::AvailabilityPending(block)) + Ok(MaybeAvailableBlock::AvailabilityPending { block_root, block }) } else { Ok(MaybeAvailableBlock::Available(AvailableBlock { + block_root, block, blobs: None, })) @@ -306,6 +307,7 @@ impl DataAvailabilityChecker { None }; Ok(MaybeAvailableBlock::Available(AvailableBlock { + block_root, block, blobs: verified_blobs, })) @@ -539,6 +541,7 @@ async fn availability_cache_maintenance_service( /// A fully available block that is ready to be imported into fork choice. #[derive(Clone, Debug, PartialEq)] pub struct AvailableBlock { + block_root: Hash256, block: Arc>, blobs: Option>, } @@ -555,9 +558,19 @@ impl AvailableBlock { self.blobs.as_ref() } - pub fn deconstruct(self) -> (Arc>, Option>) { - let AvailableBlock { block, blobs } = self; - (block, blobs) + pub fn deconstruct( + self, + ) -> ( + Hash256, + Arc>, + Option>, + ) { + let AvailableBlock { + block_root, + block, + blobs, + } = self; + (block_root, block, blobs) } } @@ -568,7 +581,10 @@ pub enum MaybeAvailableBlock { /// post-4844 blocks, it contains a `SignedBeaconBlock` and a Blobs variant other than `Blobs::None`. Available(AvailableBlock), /// This variant is not fully available and requires blobs to become fully available. - AvailabilityPending(Arc>), + AvailabilityPending { + block_root: Hash256, + block: Arc>, + }, } #[derive(Debug, Clone)] diff --git a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs index d9c06cafeb9..eb1f23d48fb 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/availability_view.rs @@ -449,14 +449,15 @@ pub mod tests { ($module_name:ident, $type_name:ty, $block_field:ident, $blob_field:ident, $setup_fn:ident) => { mod $module_name { use super::*; + use types::Hash256; #[test] fn valid_block_invalid_blobs_valid_blobs() { let (block_commitments, blobs, random_blobs) = pre_setup(); let (block_commitments, blobs, random_blobs) = $setup_fn(block_commitments, blobs, random_blobs); - - let mut cache = <$type_name>::default(); + let block_root = Hash256::zero(); + let mut cache = <$type_name>::empty(block_root); cache.merge_block(block_commitments); cache.merge_blobs(random_blobs); cache.merge_blobs(blobs); @@ -469,8 +470,8 @@ pub mod tests { let (block_commitments, blobs, random_blobs) = pre_setup(); let (block_commitments, blobs, random_blobs) = $setup_fn(block_commitments, blobs, random_blobs); - - let mut cache = <$type_name>::default(); + let block_root = Hash256::zero(); + let mut cache = <$type_name>::empty(block_root); cache.merge_blobs(random_blobs); cache.merge_block(block_commitments); cache.merge_blobs(blobs); @@ -484,7 +485,8 @@ pub mod tests { let (block_commitments, blobs, random_blobs) = $setup_fn(block_commitments, blobs, random_blobs); - let mut cache = <$type_name>::default(); + let block_root = Hash256::zero(); + let mut cache = <$type_name>::empty(block_root); cache.merge_blobs(random_blobs); cache.merge_blobs(blobs); cache.merge_block(block_commitments); @@ -498,7 +500,8 @@ pub mod tests { let (block_commitments, blobs, random_blobs) = $setup_fn(block_commitments, blobs, random_blobs); - let mut cache = <$type_name>::default(); + let block_root = Hash256::zero(); + let mut cache = <$type_name>::empty(block_root); cache.merge_block(block_commitments); cache.merge_blobs(blobs); cache.merge_blobs(random_blobs); @@ -512,7 +515,8 @@ pub mod tests { let (block_commitments, blobs, random_blobs) = $setup_fn(block_commitments, blobs, random_blobs); - let mut cache = <$type_name>::default(); + let block_root = Hash256::zero(); + let mut cache = <$type_name>::empty(block_root); cache.merge_blobs(blobs); cache.merge_block(block_commitments); cache.merge_blobs(random_blobs); @@ -526,7 +530,8 @@ pub mod tests { let (block_commitments, blobs, random_blobs) = $setup_fn(block_commitments, blobs, random_blobs); - let mut cache = <$type_name>::default(); + let block_root = Hash256::zero(); + let mut cache = <$type_name>::empty(block_root); cache.merge_blobs(blobs); cache.merge_blobs(random_blobs); cache.merge_block(block_commitments); diff --git a/beacon_node/beacon_chain/src/data_availability_checker/child_components.rs b/beacon_node/beacon_chain/src/data_availability_checker/child_components.rs index 323d05a9f65..028bf9d67c8 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/child_components.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/child_components.rs @@ -1,5 +1,6 @@ use crate::block_verification_types::RpcBlock; use crate::data_availability_checker::AvailabilityView; +use bls::Hash256; use std::sync::Arc; use types::blob_sidecar::FixedBlobSidecarList; use types::{EthSpec, SignedBeaconBlock}; @@ -8,28 +9,36 @@ use types::{EthSpec, SignedBeaconBlock}; /// 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 ChildComponents { + pub block_root: Hash256, pub downloaded_block: Option>>, pub downloaded_blobs: FixedBlobSidecarList, } impl From> for ChildComponents { fn from(value: RpcBlock) -> Self { - let (block, blobs) = value.deconstruct(); + let (block_root, block, blobs) = value.deconstruct(); let fixed_blobs = blobs.map(|blobs| { FixedBlobSidecarList::from(blobs.into_iter().map(Some).collect::>()) }); - Self::new(Some(block), fixed_blobs) + Self::new(block_root, Some(block), fixed_blobs) } } impl ChildComponents { + pub fn empty(block_root: Hash256) -> Self { + Self { + block_root, + downloaded_block: None, + downloaded_blobs: <_>::default(), + } + } pub fn new( + block_root: Hash256, block: Option>>, blobs: Option>, ) -> Self { - let mut cache = Self::default(); + let mut cache = Self::empty(block_root); if let Some(block) = block { cache.merge_block(block); } diff --git a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs index a4f96b94857..8bf16c17323 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/overflow_lru_cache.rs @@ -49,18 +49,28 @@ use types::{BlobSidecar, Epoch, EthSpec, Hash256}; /// /// The blobs are all gossip and kzg verified. /// The block has completed all verifications except the availability check. -#[derive(Encode, Decode, Clone, Default)] +#[derive(Encode, Decode, Clone)] pub struct PendingComponents { + pub block_root: Hash256, pub verified_blobs: FixedVector>, T::MaxBlobsPerBlock>, pub executed_block: Option>, } impl PendingComponents { + pub fn empty(block_root: Hash256) -> Self { + Self { + block_root, + verified_blobs: FixedVector::default(), + executed_block: None, + } + } + /// Verifies an `SignedBeaconBlock` against a set of KZG verified blobs. /// This does not check whether a block *should* have blobs, these checks should have been /// completed when producing the `AvailabilityPendingBlock`. pub fn make_available(self) -> Result, AvailabilityCheckError> { let Self { + block_root, verified_blobs, executed_block, } = self; @@ -87,6 +97,7 @@ impl PendingComponents { } = executed_block; let available_block = AvailableBlock { + block_root, block, blobs: Some(verified_blobs), }; @@ -196,14 +207,14 @@ impl OverflowStore { match OverflowKey::from_ssz_bytes(&key_bytes)? { OverflowKey::Block(_) => { maybe_pending_components - .get_or_insert_with(PendingComponents::default) + .get_or_insert_with(|| PendingComponents::empty(block_root)) .executed_block = Some(AvailabilityPendingExecutedBlock::from_ssz_bytes( value_bytes.as_slice(), )?); } OverflowKey::Blob(_, index) => { *maybe_pending_components - .get_or_insert_with(PendingComponents::default) + .get_or_insert_with(|| PendingComponents::empty(block_root)) .verified_blobs .get_mut(index as usize) .ok_or(AvailabilityCheckError::BlobIndexInvalid(index as u64))? = @@ -409,7 +420,7 @@ impl OverflowLRUCache { // Grab existing entry or create a new entry. let mut pending_components = write_lock .pop_pending_components(block_root, &self.overflow_store)? - .unwrap_or_default(); + .unwrap_or_else(|| PendingComponents::empty(block_root)); // Merge in the blobs. pending_components.merge_blobs(fixed_blobs); @@ -438,7 +449,7 @@ impl OverflowLRUCache { // Grab existing entry or create a new entry. let mut pending_components = write_lock .pop_pending_components(block_root, &self.overflow_store)? - .unwrap_or_default(); + .unwrap_or_else(|| PendingComponents::empty(block_root)); // Merge in the block. pending_components.merge_block(executed_block); diff --git a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs index 239668997b7..969034c6570 100644 --- a/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs +++ b/beacon_node/beacon_chain/src/data_availability_checker/processing_cache.rs @@ -63,8 +63,8 @@ impl ProcessingComponents { // Not safe for use outside of tests as this always required a slot. #[cfg(test)] -impl Default for ProcessingComponents { - fn default() -> Self { +impl ProcessingComponents { + pub fn empty(_block_root: Hash256) -> Self { Self { slot: Slot::new(0), block_commitments: None, diff --git a/beacon_node/beacon_chain/src/early_attester_cache.rs b/beacon_node/beacon_chain/src/early_attester_cache.rs index 0aecbde1695..c58b045cf33 100644 --- a/beacon_node/beacon_chain/src/early_attester_cache.rs +++ b/beacon_node/beacon_chain/src/early_attester_cache.rs @@ -70,7 +70,7 @@ impl EarlyAttesterCache { }, }; - let (block, blobs) = block.deconstruct(); + let (_, block, blobs) = block.deconstruct(); let item = CacheItem { epoch, committee_lengths, diff --git a/beacon_node/beacon_chain/src/historical_blocks.rs b/beacon_node/beacon_chain/src/historical_blocks.rs index b40f6e72506..8f02355e926 100644 --- a/beacon_node/beacon_chain/src/historical_blocks.rs +++ b/beacon_node/beacon_chain/src/historical_blocks.rs @@ -106,10 +106,7 @@ impl BeaconChain { let mut signed_blocks = Vec::with_capacity(blocks_to_import.len()); for available_block in blocks_to_import.into_iter().rev() { - let (block, maybe_blobs) = available_block.deconstruct(); - - // Check chain integrity. - let block_root = block.canonical_root(); + let (block_root, block, maybe_blobs) = available_block.deconstruct(); if block_root != expected_block_root { return Err(HistoricalBlockError::MismatchedBlockRoot { diff --git a/beacon_node/beacon_chain/src/test_utils.rs b/beacon_node/beacon_chain/src/test_utils.rs index 0e334eee43a..4d2271a0a50 100644 --- a/beacon_node/beacon_chain/src/test_utils.rs +++ b/beacon_node/beacon_chain/src/test_utils.rs @@ -710,14 +710,14 @@ where let block = self.chain.head_beacon_block(); let block_root = block.canonical_root(); let blobs = self.chain.get_blobs(&block_root).unwrap(); - RpcBlock::new(block, Some(blobs)).unwrap() + RpcBlock::new(Some(block_root), block, Some(blobs)).unwrap() } pub fn get_full_block(&self, block_root: &Hash256) -> RpcBlock { let block = self.chain.get_blinded_block(block_root).unwrap().unwrap(); let full_block = self.chain.store.make_full_block(block_root, block).unwrap(); let blobs = self.chain.get_blobs(block_root).unwrap(); - RpcBlock::new(Arc::new(full_block), Some(blobs)).unwrap() + RpcBlock::new(Some(*block_root), Arc::new(full_block), Some(blobs)).unwrap() } pub fn get_all_validators(&self) -> Vec { @@ -1923,7 +1923,7 @@ where .chain .process_block( block_root, - RpcBlock::new(Arc::new(block), blobs_without_signatures).unwrap(), + RpcBlock::new(Some(block_root), Arc::new(block), blobs_without_signatures).unwrap(), NotifyExecutionLayer::Yes, || Ok(()), ) @@ -1948,11 +1948,12 @@ where .collect::>(), ) }); + let block_root = block.canonical_root(); let block_hash: SignedBeaconBlockHash = self .chain .process_block( - block.canonical_root(), - RpcBlock::new(Arc::new(block), blobs_without_signatures).unwrap(), + block_root, + RpcBlock::new(Some(block_root), Arc::new(block), blobs_without_signatures).unwrap(), NotifyExecutionLayer::Yes, || Ok(()), ) diff --git a/beacon_node/beacon_chain/tests/attestation_production.rs b/beacon_node/beacon_chain/tests/attestation_production.rs index 642ff329593..a8ad75304b4 100644 --- a/beacon_node/beacon_chain/tests/attestation_production.rs +++ b/beacon_node/beacon_chain/tests/attestation_production.rs @@ -134,7 +134,7 @@ async fn produces_attestations() { assert_eq!(data.target.root, target_root, "bad target root"); let rpc_block = - RpcBlock::::new(Arc::new(block.clone()), Some(blobs.clone())) + RpcBlock::::new(None, Arc::new(block.clone()), Some(blobs.clone())) .unwrap(); let beacon_chain::data_availability_checker::MaybeAvailableBlock::Available( available_block, @@ -213,7 +213,7 @@ async fn early_attester_cache_old_request() { .expect("should get blobs"); let rpc_block = - RpcBlock::::new(head.beacon_block.clone(), Some(head_blobs)).unwrap(); + RpcBlock::::new(None, head.beacon_block.clone(), Some(head_blobs)).unwrap(); let beacon_chain::data_availability_checker::MaybeAvailableBlock::Available(available_block) = harness .chain diff --git a/beacon_node/beacon_chain/tests/block_verification.rs b/beacon_node/beacon_chain/tests/block_verification.rs index 5490dc88584..e9f2501b47f 100644 --- a/beacon_node/beacon_chain/tests/block_verification.rs +++ b/beacon_node/beacon_chain/tests/block_verification.rs @@ -161,7 +161,7 @@ fn chain_segment_blocks( .iter() .zip(blobs.into_iter()) .map(|(snapshot, blobs)| { - RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap() + RpcBlock::new(None, snapshot.beacon_block.clone(), blobs.clone()).unwrap() }) .collect() } @@ -328,7 +328,10 @@ async fn chain_segment_non_linear_parent_roots() { let (mut block, signature) = blocks[3].as_block().clone().deconstruct(); *block.parent_root_mut() = Hash256::zero(); - blocks[3] = Arc::new(SignedBeaconBlock::from_block(block, signature)).into(); + blocks[3] = RpcBlock::new_without_blobs( + None, + Arc::new(SignedBeaconBlock::from_block(block, signature)), + ); assert!( matches!( @@ -362,7 +365,10 @@ async fn chain_segment_non_linear_slots() { .collect(); let (mut block, signature) = blocks[3].as_block().clone().deconstruct(); *block.slot_mut() = Slot::new(0); - blocks[3] = Arc::new(SignedBeaconBlock::from_block(block, signature)).into(); + blocks[3] = RpcBlock::new_without_blobs( + None, + Arc::new(SignedBeaconBlock::from_block(block, signature)), + ); assert!( matches!( @@ -386,7 +392,10 @@ async fn chain_segment_non_linear_slots() { .collect(); let (mut block, signature) = blocks[3].as_block().clone().deconstruct(); *block.slot_mut() = blocks[2].slot(); - blocks[3] = Arc::new(SignedBeaconBlock::from_block(block, signature)).into(); + blocks[3] = RpcBlock::new_without_blobs( + None, + Arc::new(SignedBeaconBlock::from_block(block, signature)), + ); assert!( matches!( @@ -413,7 +422,7 @@ async fn assert_invalid_signature( .iter() .zip(chain_segment_blobs.iter()) .map(|(snapshot, blobs)| { - RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap() + RpcBlock::new(None, snapshot.beacon_block.clone(), blobs.clone()).unwrap() }) .collect(); @@ -440,7 +449,7 @@ async fn assert_invalid_signature( .take(block_index) .zip(chain_segment_blobs.iter()) .map(|(snapshot, blobs)| { - RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap() + RpcBlock::new(None, snapshot.beacon_block.clone(), blobs.clone()).unwrap() }) .collect(); // We don't care if this fails, we just call this to ensure that all prior blocks have been @@ -456,6 +465,7 @@ async fn assert_invalid_signature( .process_block( snapshots[block_index].beacon_block.canonical_root(), RpcBlock::new( + None, snapshots[block_index].beacon_block.clone(), chain_segment_blobs[block_index].clone(), ) @@ -511,7 +521,7 @@ async fn invalid_signature_gossip_block() { .take(block_index) .zip(chain_segment_blobs.iter()) .map(|(snapshot, blobs)| { - RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap() + RpcBlock::new(None, snapshot.beacon_block.clone(), blobs.clone()).unwrap() }) .collect(); harness @@ -558,7 +568,7 @@ async fn invalid_signature_block_proposal() { .iter() .zip(chain_segment_blobs.iter()) .map(|(snapshot, blobs)| { - RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap() + RpcBlock::new(None, snapshot.beacon_block.clone(), blobs.clone()).unwrap() }) .collect::>(); // Ensure the block will be rejected if imported in a chain segment. @@ -771,7 +781,7 @@ async fn invalid_signature_deposit() { .iter() .zip(chain_segment_blobs.iter()) .map(|(snapshot, blobs)| { - RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap() + RpcBlock::new(None, snapshot.beacon_block.clone(), blobs.clone()).unwrap() }) .collect(); assert!( @@ -1359,7 +1369,10 @@ async fn add_base_block_to_altair_chain() { assert!(matches!( harness .chain - .process_chain_segment(vec![Arc::new(base_block).into()], NotifyExecutionLayer::Yes,) + .process_chain_segment( + vec![RpcBlock::new_without_blobs(None, Arc::new(base_block))], + NotifyExecutionLayer::Yes, + ) .await, ChainSegmentResult::Failed { imported_blocks: 0, @@ -1495,7 +1508,7 @@ async fn add_altair_block_to_base_chain() { harness .chain .process_chain_segment( - vec![Arc::new(altair_block).into()], + vec![RpcBlock::new_without_blobs(None, Arc::new(altair_block))], NotifyExecutionLayer::Yes ) .await, diff --git a/beacon_node/beacon_chain/tests/store_tests.rs b/beacon_node/beacon_chain/tests/store_tests.rs index 6b6206d5e02..90b680ca8f3 100644 --- a/beacon_node/beacon_chain/tests/store_tests.rs +++ b/beacon_node/beacon_chain/tests/store_tests.rs @@ -2217,7 +2217,7 @@ async fn weak_subjectivity_sync_test(slots: Vec, checkpoint_slot: Slot) { beacon_chain .process_block( full_block.canonical_root(), - RpcBlock::new(Arc::new(full_block), Some(blobs)).unwrap(), + RpcBlock::new(Some(block_root), Arc::new(full_block), Some(blobs)).unwrap(), NotifyExecutionLayer::Yes, || Ok(()), ) @@ -2277,7 +2277,9 @@ async fn weak_subjectivity_sync_test(slots: Vec, checkpoint_slot: Slot) { if let MaybeAvailableBlock::Available(block) = harness .chain .data_availability_checker - .check_rpc_block_availability(RpcBlock::new(Arc::new(full_block), Some(blobs)).unwrap()) + .check_rpc_block_availability( + RpcBlock::new(Some(block_root), Arc::new(full_block), Some(blobs)).unwrap(), + ) .expect("should check availability") { available_blocks.push(block); 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 8159586c13b..cb8a7f5097d 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -498,7 +498,7 @@ impl NetworkBeaconProcessor { .into_iter() .filter_map(|maybe_available| match maybe_available { MaybeAvailableBlock::Available(block) => Some(block), - MaybeAvailableBlock::AvailabilityPending(_) => None, + MaybeAvailableBlock::AvailabilityPending { .. } => None, }) .collect::>(), Err(e) => match e { diff --git a/beacon_node/network/src/sync/block_lookups/common.rs b/beacon_node/network/src/sync/block_lookups/common.rs index 2807e73babf..7f141edb5b1 100644 --- a/beacon_node/network/src/sync/block_lookups/common.rs +++ b/beacon_node/network/src/sync/block_lookups/common.rs @@ -227,6 +227,7 @@ pub trait RequestState { /// Convert a verified response to the type we send to the beacon processor. fn verified_to_reconstructed( + block_root: Hash256, verified: Self::VerifiedResponseType, ) -> Self::ReconstructedResponseType; @@ -332,9 +333,10 @@ impl RequestState for BlockRequestState } fn verified_to_reconstructed( + block_root: Hash256, block: Arc>, ) -> RpcBlock { - RpcBlock::new_without_blobs(block) + RpcBlock::new_without_blobs(Some(block_root), block) } fn send_reconstructed_for_processing( @@ -438,6 +440,7 @@ impl RequestState for BlobRequestState, ) -> FixedBlobSidecarList { blobs diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index 8f17dccdc5d..afc155d2d20 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -465,7 +465,7 @@ impl BlockLookups { id, self, block_root, - R::verified_to_reconstructed(verified_response), + R::verified_to_reconstructed(block_root, verified_response), seen_timestamp, cx, )?, 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 05d0f52d393..6e8da8a218a 100644 --- a/beacon_node/network/src/sync/block_lookups/parent_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/parent_lookup.rs @@ -5,7 +5,7 @@ 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::data_availability_checker::{ChildComponents, DataAvailabilityChecker}; use beacon_chain::BeaconChainTypes; use itertools::Itertools; use lighthouse_network::PeerId; @@ -66,7 +66,7 @@ impl ParentLookup { ) -> Self { let current_parent_request = SingleBlockLookup::new( parent_root, - Some(<_>::default()), + Some(ChildComponents::empty(block_root)), peer_id, da_checker, cx.next_id(), 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 e9aeb76c68e..878ae94aabd 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 @@ -96,7 +96,7 @@ impl SingleBlockLookup { self.block_request_state.requested_block_root = block_root; self.block_request_state.state.state = State::AwaitingDownload; self.blob_request_state.state.state = State::AwaitingDownload; - self.child_components = Some(ChildComponents::default()); + self.child_components = Some(ChildComponents::empty(block_root)); } /// Get all unique peers across block and blob requests. @@ -145,7 +145,11 @@ impl SingleBlockLookup { return CachedChild::DownloadIncomplete; } - match RpcBlock::new_from_fixed(block.clone(), components.downloaded_blobs.clone()) { + match RpcBlock::new_from_fixed( + self.block_request_state.requested_block_root, + block.clone(), + components.downloaded_blobs.clone(), + ) { Ok(rpc_block) => CachedChild::Ok(rpc_block), Err(e) => CachedChild::Err(e), } @@ -173,6 +177,7 @@ impl SingleBlockLookup { pub fn add_child_components(&mut self, components: ChildComponents) { if let Some(ref mut existing_components) = self.child_components { let ChildComponents { + block_root: _, downloaded_block, downloaded_blobs, } = components; diff --git a/beacon_node/network/src/sync/block_lookups/tests.rs b/beacon_node/network/src/sync/block_lookups/tests.rs index b94d3ea48eb..11ff97f9f4f 100644 --- a/beacon_node/network/src/sync/block_lookups/tests.rs +++ b/beacon_node/network/src/sync/block_lookups/tests.rs @@ -414,7 +414,7 @@ fn test_single_block_lookup_becomes_parent_request() { // parent request after processing. bl.single_block_component_processed::>( id.id, - BlockError::ParentUnknown(block.into()).into(), + BlockError::ParentUnknown(RpcBlock::new_without_blobs(None, block)).into(), &mut cx, ); assert_eq!(bl.single_block_lookups.len(), 1); @@ -923,7 +923,7 @@ fn test_parent_lookup_too_deep() { // the processing result bl.parent_block_processed( chain_hash, - BlockError::ParentUnknown(block.into()).into(), + BlockError::ParentUnknown(RpcBlock::new_without_blobs(None, block)).into(), &mut cx, ) } @@ -1133,7 +1133,7 @@ fn test_same_chain_race_condition() { } else { bl.parent_block_processed( chain_hash, - BlockError::ParentUnknown(block.into()).into(), + BlockError::ParentUnknown(RpcBlock::new_without_blobs(None, block)).into(), &mut cx, ) } @@ -1248,7 +1248,7 @@ mod deneb_only { block_root = child_root; bl.search_child_block( child_root, - ChildComponents::new(Some(child_block), None), + ChildComponents::new(child_root, Some(child_block), None), PeerShouldHave::Neither(peer_id), &mut cx, ); @@ -1286,7 +1286,7 @@ mod deneb_only { *blobs.index_mut(0) = Some(child_blob); bl.search_child_block( child_root, - ChildComponents::new(None, Some(blobs)), + ChildComponents::new(child_root, None, Some(blobs)), PeerShouldHave::Neither(peer_id), &mut cx, ); @@ -1509,6 +1509,7 @@ mod deneb_only { self.bl.parent_block_processed( self.block_root, BlockProcessingResult::Err(BlockError::ParentUnknown(RpcBlock::new_without_blobs( + Some(self.block_root), self.parent_block.clone().expect("parent block"), ))), &mut self.cx, diff --git a/beacon_node/network/src/sync/block_sidecar_coupling.rs b/beacon_node/network/src/sync/block_sidecar_coupling.rs index 32029777ee2..100e84a6bbc 100644 --- a/beacon_node/network/src/sync/block_sidecar_coupling.rs +++ b/beacon_node/network/src/sync/block_sidecar_coupling.rs @@ -66,7 +66,7 @@ impl BlocksAndBlobsRequestInfo { } } let blobs = VariableList::from(blobs_buffer.into_iter().flatten().collect::>()); - responses.push(RpcBlock::new(block, Some(blobs))?) + responses.push(RpcBlock::new(None, block, Some(blobs))?) } // if accumulated sidecars is not empty, throw an error. diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index cf90f9033eb..ab1053410da 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -655,7 +655,7 @@ impl SyncManager { block_root, parent_root, blob_slot, - ChildComponents::new(None, Some(blobs)), + ChildComponents::new(block_root, None, Some(blobs)), ); } SyncMessage::UnknownBlockHashFromAttestation(peer_id, block_hash) => { @@ -888,7 +888,7 @@ impl SyncManager { batch_id, &peer_id, id, - block.map(Into::into), + block.map(|b| RpcBlock::new_without_blobs(None, b)), ) { Ok(ProcessResult::SyncCompleted) => self.update_sync_state(), Ok(ProcessResult::Successful) => {} @@ -912,7 +912,7 @@ impl SyncManager { chain_id, batch_id, id, - block.map(Into::into), + block.map(|b| RpcBlock::new_without_blobs(None, b)), ); self.update_sync_state(); } From 75c50dbcef07eea7ea91fd4e91d38cdb9b74460c Mon Sep 17 00:00:00 2001 From: realbigsean Date: Mon, 2 Oct 2023 14:49:35 -0400 Subject: [PATCH 45/47] send peers as vec in single message --- .../gossip_methods.rs | 32 +++++++++-------- .../src/network_beacon_processor/mod.rs | 3 +- .../network_beacon_processor/sync_methods.rs | 10 +++--- .../network/src/sync/block_lookups/mod.rs | 14 ++++---- .../src/sync/block_lookups/parent_lookup.rs | 7 +++- .../sync/block_lookups/single_block_lookup.rs | 34 +++++++++++------- .../network/src/sync/block_lookups/tests.rs | 36 +++++++++++++------ beacon_node/network/src/sync/manager.rs | 35 ++++++++++++------ 8 files changed, 109 insertions(+), 62 deletions(-) diff --git a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs index 4119371b87f..fbcbf014133 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -4,6 +4,7 @@ use crate::{ service::NetworkMessage, sync::SyncMessage, }; +use std::collections::HashSet; use beacon_chain::blob_verification::{GossipBlobError, GossipVerifiedBlob}; use beacon_chain::block_verification_types::AsBlock; @@ -754,12 +755,7 @@ impl NetworkBeaconProcessor { } Ok(AvailabilityProcessingStatus::MissingComponents(_slot, block_root)) => { if delay_lookup { - let mut guard = self.delayed_lookup_peers.lock(); - if let Some(peers) = guard.get_mut(&block_root) { - peers.push(peer_id); - } else { - guard.push(block_root, vec![peer_id]); - } + self.cache_peer(peer_id, &block_root); trace!( self.log, "Processed blob, delaying lookup for other components"; @@ -776,7 +772,8 @@ impl NetworkBeaconProcessor { "block_root" => %block_root, ); self.send_sync_message(SyncMessage::MissingGossipBlockComponents( - peer_id, block_root, + vec![peer_id], + block_root, )); } } @@ -802,6 +799,18 @@ impl NetworkBeaconProcessor { } } + /// Cache the peer id for the given block root. + fn cache_peer(self: &Arc, peer_id: PeerId, block_root: &Hash256) { + let mut guard = self.delayed_lookup_peers.lock(); + if let Some(peers) = guard.get_mut(block_root) { + peers.insert(peer_id); + } else { + let mut peers = HashSet::new(); + peers.insert(peer_id); + guard.push(*block_root, peers); + } + } + /// Process the beacon block received from the gossip network and: /// /// - If it passes gossip propagation criteria, tell the network thread to forward it. @@ -1174,12 +1183,7 @@ impl NetworkBeaconProcessor { } Ok(AvailabilityProcessingStatus::MissingComponents(slot, block_root)) => { if delay_lookup { - let mut guard = self.delayed_lookup_peers.lock(); - if let Some(peers) = guard.get_mut(block_root) { - peers.push(peer_id); - } else { - guard.push(*block_root, vec![peer_id]); - } + self.cache_peer(peer_id, block_root); trace!( self.log, "Processed block, delaying lookup for other components"; @@ -1194,7 +1198,7 @@ impl NetworkBeaconProcessor { "block_root" => %block_root, ); self.send_sync_message(SyncMessage::MissingGossipBlockComponents( - peer_id, + vec![peer_id], *block_root, )); } diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index f5d4b8bfe49..800d6860110 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -22,6 +22,7 @@ use lru::LruCache; use parking_lot::Mutex; use slog::{debug, Logger}; use slot_clock::ManualSlotClock; +use std::collections::HashSet; use std::path::PathBuf; use std::sync::Arc; use std::time::Duration; @@ -63,7 +64,7 @@ pub struct NetworkBeaconProcessor { pub reprocess_tx: mpsc::Sender, pub network_globals: Arc>, pub invalid_block_storage: InvalidBlockStorage, - pub delayed_lookup_peers: Mutex>>, + pub delayed_lookup_peers: Mutex>>, pub executor: TaskExecutor, pub log: Logger, } 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 cb8a7f5097d..d2d589b35ad 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -309,12 +309,10 @@ impl NetworkBeaconProcessor { } for block_root in block_roots { if let Some(peer_ids) = self.delayed_lookup_peers.lock().pop(&block_root) { - // Let the network layer de-dupe these. - for peer_id in peer_ids { - self.send_sync_message(SyncMessage::MissingGossipBlockComponents( - peer_id, block_root, - )); - } + self.send_sync_message(SyncMessage::MissingGossipBlockComponents( + peer_ids.into_iter().collect(), + block_root, + )); } } } diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index afc155d2d20..fc21e9dba04 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -122,7 +122,7 @@ impl BlockLookups { pub fn search_block( &mut self, block_root: Hash256, - peer_source: PeerShouldHave, + peer_source: &[PeerShouldHave], cx: &mut SyncNetworkContext, ) { self.new_current_lookup(block_root, None, peer_source, cx) @@ -138,7 +138,7 @@ impl BlockLookups { &mut self, block_root: Hash256, child_components: ChildComponents, - peer_source: PeerShouldHave, + peer_source: &[PeerShouldHave], cx: &mut SyncNetworkContext, ) { self.new_current_lookup(block_root, Some(child_components), peer_source, cx) @@ -179,7 +179,7 @@ impl BlockLookups { &mut self, block_root: Hash256, child_components: Option>, - peer: PeerShouldHave, + peers: &[PeerShouldHave], cx: &mut SyncNetworkContext, ) { // Do not re-request a block that is already being requested @@ -188,7 +188,7 @@ impl BlockLookups { .iter_mut() .find(|(_id, lookup)| lookup.is_for_block(block_root)) { - lookup.add_peer(peer); + lookup.add_peers(peers); if let Some(components) = child_components { lookup.add_child_components(components); } @@ -198,7 +198,7 @@ impl BlockLookups { if let Some(parent_lookup) = self.parent_lookups.iter_mut().find(|parent_req| { parent_req.is_for_block(block_root) || parent_req.contains_block(&block_root) }) { - parent_lookup.add_peer(peer); + parent_lookup.add_peers(peers); // If the block was already downloaded, or is being downloaded in this moment, do not // request it. @@ -223,7 +223,7 @@ impl BlockLookups { let lookup = SingleBlockLookup::new( block_root, child_components, - peer, + peers, self.da_checker.clone(), cx.next_id(), ); @@ -231,7 +231,7 @@ impl BlockLookups { debug!( self.log, "{}", msg; - "peer_ids" => ?peer, + "peer_ids" => ?peers, "block" => ?block_root, ); self.trigger_single_lookup(lookup, cx); 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 6e8da8a218a..25434950bad 100644 --- a/beacon_node/network/src/sync/block_lookups/parent_lookup.rs +++ b/beacon_node/network/src/sync/block_lookups/parent_lookup.rs @@ -67,7 +67,7 @@ impl ParentLookup { let current_parent_request = SingleBlockLookup::new( parent_root, Some(ChildComponents::empty(block_root)), - peer_id, + &[peer_id], da_checker, cx.next_id(), ); @@ -214,6 +214,11 @@ impl ParentLookup { self.current_parent_request.add_peer(peer) } + /// Adds a list of peers to the parent request. + pub fn add_peers(&mut self, peers: &[PeerShouldHave]) { + self.current_parent_request.add_peers(peers) + } + pub fn used_peers(&self) -> impl Iterator + '_ { self.current_parent_request .block_request_state 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 878ae94aabd..e0f7d880949 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 @@ -66,7 +66,7 @@ impl SingleBlockLookup { pub fn new( requested_block_root: Hash256, child_components: Option>, - peers: PeerShouldHave, + peers: &[PeerShouldHave], da_checker: Arc>, id: Id, ) -> Self { @@ -204,6 +204,13 @@ impl SingleBlockLookup { } } + /// Add all given peers to both block and blob request states. + pub fn add_peers(&mut self, peers: &[PeerShouldHave]) { + for peer in peers { + self.add_peer(*peer); + } + } + /// Returns true if the block has already been downloaded. pub fn both_components_downloaded(&self) -> bool { self.block_request_state.state.component_downloaded @@ -338,7 +345,7 @@ pub struct BlobRequestState { } impl BlobRequestState { - pub fn new(block_root: Hash256, peer_source: PeerShouldHave, is_deneb: bool) -> Self { + pub fn new(block_root: Hash256, peer_source: &[PeerShouldHave], is_deneb: bool) -> Self { let default_ids = MissingBlobs::new_without_block(block_root, is_deneb); Self { requested_ids: default_ids, @@ -357,7 +364,7 @@ pub struct BlockRequestState { } impl BlockRequestState { - pub fn new(block_root: Hash256, peers: PeerShouldHave) -> Self { + pub fn new(block_root: Hash256, peers: &[PeerShouldHave]) -> Self { Self { requested_block_root: block_root, state: SingleLookupRequestState::new(peers), @@ -410,17 +417,20 @@ pub struct SingleLookupRequestState { } impl SingleLookupRequestState { - pub fn new(peer: PeerShouldHave) -> Self { + pub fn new(peers: &[PeerShouldHave]) -> Self { let mut available_peers = HashSet::default(); let mut potential_peers = HashSet::default(); - match peer { - PeerShouldHave::BlockAndBlobs(peer_id) => { - available_peers.insert(peer_id); - } - PeerShouldHave::Neither(peer_id) => { - potential_peers.insert(peer_id); + for peer in peers { + match peer { + PeerShouldHave::BlockAndBlobs(peer_id) => { + available_peers.insert(*peer_id); + } + PeerShouldHave::Neither(peer_id) => { + potential_peers.insert(*peer_id); + } } } + Self { state: State::AwaitingDownload, available_peers, @@ -618,7 +628,7 @@ mod tests { let mut sl = SingleBlockLookup::::new( block.canonical_root(), None, - peer_id, + &[peer_id], da_checker, 1, ); @@ -660,7 +670,7 @@ mod tests { let mut sl = SingleBlockLookup::::new( block.canonical_root(), None, - peer_id, + &[peer_id], da_checker, 1, ); diff --git a/beacon_node/network/src/sync/block_lookups/tests.rs b/beacon_node/network/src/sync/block_lookups/tests.rs index 11ff97f9f4f..59ac9c4338f 100644 --- a/beacon_node/network/src/sync/block_lookups/tests.rs +++ b/beacon_node/network/src/sync/block_lookups/tests.rs @@ -237,7 +237,11 @@ fn test_single_block_lookup_happy_path() { let peer_id = PeerId::random(); let block_root = block.canonical_root(); // Trigger the request - bl.search_block(block_root, PeerShouldHave::BlockAndBlobs(peer_id), &mut cx); + bl.search_block( + block_root, + &[PeerShouldHave::BlockAndBlobs(peer_id)], + &mut cx, + ); 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. @@ -285,7 +289,11 @@ fn test_single_block_lookup_empty_response() { let peer_id = PeerId::random(); // Trigger the request - bl.search_block(block_hash, PeerShouldHave::BlockAndBlobs(peer_id), &mut cx); + bl.search_block( + block_hash, + &[PeerShouldHave::BlockAndBlobs(peer_id)], + &mut cx, + ); 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. @@ -313,7 +321,11 @@ fn test_single_block_lookup_wrong_response() { let peer_id = PeerId::random(); // Trigger the request - bl.search_block(block_hash, PeerShouldHave::BlockAndBlobs(peer_id), &mut cx); + bl.search_block( + block_hash, + &[PeerShouldHave::BlockAndBlobs(peer_id)], + &mut cx, + ); 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. @@ -351,7 +363,11 @@ fn test_single_block_lookup_failure() { let peer_id = PeerId::random(); // Trigger the request - bl.search_block(block_hash, PeerShouldHave::BlockAndBlobs(peer_id), &mut cx); + bl.search_block( + block_hash, + &[PeerShouldHave::BlockAndBlobs(peer_id)], + &mut cx, + ); 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. @@ -385,7 +401,7 @@ fn test_single_block_lookup_becomes_parent_request() { // Trigger the request bl.search_block( block.canonical_root(), - PeerShouldHave::BlockAndBlobs(peer_id), + &[PeerShouldHave::BlockAndBlobs(peer_id)], &mut cx, ); let id = rig.expect_lookup_request(response_type); @@ -971,7 +987,7 @@ fn test_single_block_lookup_ignored_response() { // Trigger the request bl.search_block( block.canonical_root(), - PeerShouldHave::BlockAndBlobs(peer_id), + &[PeerShouldHave::BlockAndBlobs(peer_id)], &mut cx, ); let id = rig.expect_lookup_request(response_type); @@ -1224,7 +1240,7 @@ mod deneb_only { RequestTrigger::AttestationUnknownBlock => { bl.search_block( block_root, - PeerShouldHave::BlockAndBlobs(peer_id), + &[PeerShouldHave::BlockAndBlobs(peer_id)], &mut cx, ); let block_req_id = rig.expect_lookup_request(ResponseType::Block); @@ -1249,7 +1265,7 @@ mod deneb_only { bl.search_child_block( child_root, ChildComponents::new(child_root, Some(child_block), None), - PeerShouldHave::Neither(peer_id), + &[PeerShouldHave::Neither(peer_id)], &mut cx, ); @@ -1287,7 +1303,7 @@ mod deneb_only { bl.search_child_block( child_root, ChildComponents::new(child_root, None, Some(blobs)), - PeerShouldHave::Neither(peer_id), + &[PeerShouldHave::Neither(peer_id)], &mut cx, ); @@ -1305,7 +1321,7 @@ mod deneb_only { ) } RequestTrigger::GossipUnknownBlockOrBlob => { - bl.search_block(block_root, PeerShouldHave::Neither(peer_id), &mut cx); + bl.search_block(block_root, &[PeerShouldHave::Neither(peer_id)], &mut cx); 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) diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index ab1053410da..e84fc7e7b5f 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -144,7 +144,7 @@ pub enum SyncMessage { /// /// We will either attempt to find the block matching the unknown hash immediately or queue a lookup, /// which will then trigger the request when we receive `MissingGossipBlockComponentsDelayed`. - MissingGossipBlockComponents(PeerId, Hash256), + MissingGossipBlockComponents(Vec, Hash256), /// A peer has disconnected. Disconnect(PeerId), @@ -663,19 +663,29 @@ impl SyncManager { if self.synced_and_connected(&peer_id) { self.block_lookups.search_block( block_hash, - PeerShouldHave::BlockAndBlobs(peer_id), + &[PeerShouldHave::BlockAndBlobs(peer_id)], &mut self.network, ); } } SyncMessage::MissingGossipBlockComponents(peer_id, block_root) => { + let peers_guard = self.network_globals().peers.read(); + let connected_peers = peer_id + .into_iter() + .filter_map(|peer_id| { + if peers_guard.is_connected(&peer_id) { + Some(PeerShouldHave::Neither(peer_id)) + } else { + None + } + }) + .collect::>(); + drop(peers_guard); + // If we are not synced, ignore this block. - if self.synced_and_connected(&peer_id) { - self.block_lookups.search_block( - block_root, - PeerShouldHave::Neither(peer_id), - &mut self.network, - ) + if self.synced() && !connected_peers.is_empty() { + self.block_lookups + .search_block(block_root, &connected_peers, &mut self.network) } } SyncMessage::Disconnect(peer_id) => { @@ -759,7 +769,7 @@ impl SyncManager { self.block_lookups.search_child_block( block_root, child_components, - PeerShouldHave::Neither(peer_id), + &[PeerShouldHave::Neither(peer_id)], &mut self.network, ); } @@ -784,12 +794,15 @@ impl SyncManager { && self.network.is_execution_engine_online() } - fn synced_and_connected(&mut self, peer_id: &PeerId) -> bool { + fn synced(&mut self) -> bool { self.network_globals().sync_state.read().is_synced() - && self.network_globals().peers.read().is_connected(peer_id) && self.network.is_execution_engine_online() } + fn synced_and_connected(&mut self, peer_id: &PeerId) -> bool { + self.synced() && self.network_globals().peers.read().is_connected(peer_id) + } + fn handle_new_execution_engine_state(&mut self, engine_state: EngineState) { self.network.update_execution_engine_state(engine_state); From 1eec921f753e03407a92886ecf39460f67e01504 Mon Sep 17 00:00:00 2001 From: realbigsean Date: Mon, 2 Oct 2023 15:08:34 -0400 Subject: [PATCH 46/47] spawn delayed lookup service from network beacon processor --- .../src/network_beacon_processor/mod.rs | 67 +++++++++++++++++- beacon_node/network/src/router.rs | 1 + .../src/sync/block_lookups/delayed_lookup.rs | 70 ------------------- .../network/src/sync/block_lookups/mod.rs | 1 - beacon_node/network/src/sync/manager.rs | 6 +- 5 files changed, 67 insertions(+), 78 deletions(-) delete mode 100644 beacon_node/network/src/sync/block_lookups/delayed_lookup.rs diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 800d6860110..8094d4677c4 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -20,8 +20,8 @@ use lighthouse_network::{ }; use lru::LruCache; use parking_lot::Mutex; -use slog::{debug, Logger}; -use slot_clock::ManualSlotClock; +use slog::{crit, debug, error, trace, Logger}; +use slot_clock::{ManualSlotClock, SlotClock}; use std::collections::HashSet; use std::path::PathBuf; use std::sync::Arc; @@ -30,6 +30,7 @@ use store::MemoryStore; use task_executor::test_utils::TestRuntime; use task_executor::TaskExecutor; use tokio::sync::mpsc::{self, error::TrySendError}; +use tokio::time::{interval_at, Instant}; use types::*; pub use sync_methods::ChainSegmentProcessId; @@ -629,6 +630,68 @@ impl NetworkBeaconProcessor { "error" => %e) }); } + + /// This service is responsible for collecting lookup messages and sending them back to sync + /// for processing after a short delay. + /// + /// We want to delay lookups triggered from gossip for the following reasons: + /// + /// - We only want to make one request for components we are unlikely to see on gossip. This means + /// we don't have to repeatedly update our RPC request's state as we receive gossip components. + /// + /// - We are likely to receive blocks/blobs over gossip more quickly than we could via an RPC request. + /// + /// - Delaying a lookup means we are less likely to simultaneously download the same blocks/blobs + /// over gossip and RPC. + /// + /// - We would prefer to request peers based on whether we've seen them attest, because this gives + /// us an idea about whether they *should* have the block/blobs we're missing. This is because a + /// node should not attest to a block unless it has all the blobs for that block. This gives us a + /// stronger basis for peer scoring. + pub fn spawn_delayed_lookup_service(self: &Arc) { + let processor_clone = self.clone(); + let executor = self.executor.clone(); + let log = self.log.clone(); + let beacon_chain = self.chain.clone(); + executor.spawn( + async move { + let slot_duration = beacon_chain.slot_clock.slot_duration(); + let delay = beacon_chain.slot_clock.single_lookup_delay(); + let interval_start = match ( + beacon_chain.slot_clock.duration_to_next_slot(), + beacon_chain.slot_clock.seconds_from_current_slot_start(), + ) { + (Some(duration_to_next_slot), Some(seconds_from_current_slot_start)) => { + let duration_until_start = if seconds_from_current_slot_start > delay { + duration_to_next_slot + delay + } else { + delay - seconds_from_current_slot_start + }; + Instant::now() + duration_until_start + } + _ => { + crit!(log, + "Failed to read slot clock, delayed lookup service timing will be inaccurate.\ + This may degrade performance" + ); + Instant::now() + } + }; + + let mut interval = interval_at(interval_start, slot_duration); + loop { + interval.tick().await; + let Some(slot) = beacon_chain.slot_clock.now_or_genesis() else { + error!(log, "Skipping delayed lookup poll, unable to read slot clock"); + continue + }; + trace!(log, "Polling delayed lookups for slot: {slot}"); + processor_clone.poll_delayed_lookups(slot) + } + }, + "delayed_lookups", + ); + } } type TestBeaconChainType = diff --git a/beacon_node/network/src/router.rs b/beacon_node/network/src/router.rs index 179aef8a96c..7248147178e 100644 --- a/beacon_node/network/src/router.rs +++ b/beacon_node/network/src/router.rs @@ -118,6 +118,7 @@ impl Router { log: log.clone(), }; let network_beacon_processor = Arc::new(network_beacon_processor); + network_beacon_processor.spawn_delayed_lookup_service(); // spawn the sync thread crate::sync::manager::spawn( diff --git a/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs b/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs deleted file mode 100644 index 2cdb586524d..00000000000 --- a/beacon_node/network/src/sync/block_lookups/delayed_lookup.rs +++ /dev/null @@ -1,70 +0,0 @@ -use crate::network_beacon_processor::NetworkBeaconProcessor; -use beacon_chain::{BeaconChain, BeaconChainTypes}; -use slog::{crit, error, trace}; -use slot_clock::SlotClock; -use std::sync::Arc; -use tokio::time::interval_at; -use tokio::time::Instant; - -/// This service is responsible for collecting lookup messages and sending them back to sync -/// for processing after a short delay. -/// -/// We want to delay lookups triggered from gossip for the following reasons: -/// -/// - We only want to make one request for components we are unlikely to see on gossip. This means -/// we don't have to repeatedly update our RPC request's state as we receive gossip components. -/// -/// - We are likely to receive blocks/blobs over gossip more quickly than we could via an RPC request. -/// -/// - Delaying a lookup means we are less likely to simultaneously download the same blocks/blobs -/// over gossip and RPC. -/// -/// - We would prefer to request peers based on whether we've seen them attest, because this gives -/// us an idea about whether they *should* have the block/blobs we're missing. This is because a -/// node should not attest to a block unless it has all the blobs for that block. This gives us a -/// stronger basis for peer scoring. -pub fn spawn_delayed_lookup_service( - executor: &task_executor::TaskExecutor, - beacon_chain: Arc>, - beacon_processor: Arc>, - log: slog::Logger, -) { - executor.spawn( - async move { - let slot_duration = beacon_chain.slot_clock.slot_duration(); - let delay = beacon_chain.slot_clock.single_lookup_delay(); - let interval_start = match ( - beacon_chain.slot_clock.duration_to_next_slot(), - beacon_chain.slot_clock.seconds_from_current_slot_start(), - ) { - (Some(duration_to_next_slot), Some(seconds_from_current_slot_start)) => { - let duration_until_start = if seconds_from_current_slot_start > delay { - duration_to_next_slot + delay - } else { - delay - seconds_from_current_slot_start - }; - Instant::now() + duration_until_start - } - _ => { - crit!(log, - "Failed to read slot clock, delayed lookup service timing will be inaccurate.\ - This may degrade performance" - ); - Instant::now() - } - }; - - let mut interval = interval_at(interval_start, slot_duration); - loop { - interval.tick().await; - let Some(slot) = beacon_chain.slot_clock.now_or_genesis() else { - error!(log, "Skipping delayed lookup poll, unable to read slot clock"); - continue - }; - trace!(log, "Polling delayed lookups for slot: {slot}"); - beacon_processor.poll_delayed_lookups(slot) - } - }, - "delayed_lookups", - ); -} diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index fc21e9dba04..ffeebf3a3eb 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -37,7 +37,6 @@ use types::blob_sidecar::FixedBlobSidecarList; use types::Slot; pub mod common; -pub(crate) mod delayed_lookup; mod parent_lookup; mod single_block_lookup; #[cfg(test)] diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index e84fc7e7b5f..ccd765f9bc5 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -42,7 +42,6 @@ use crate::network_beacon_processor::{ChainSegmentProcessId, NetworkBeaconProces 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::{BlobRequestState, BlockRequestState}; use crate::sync::range_sync::ByRangeRequestType; use beacon_chain::block_verification_types::AsBlock; @@ -261,11 +260,8 @@ pub fn spawn( log: log.clone(), }; - let log_clone = log.clone(); - delayed_lookup::spawn_delayed_lookup_service(&executor, beacon_chain, beacon_processor, log); - // spawn the sync manager thread - debug!(log_clone, "Sync Manager started"); + debug!(log, "Sync Manager started"); executor.spawn(async move { Box::pin(sync_manager.main()).await }, "sync"); } From 2aba776d24ea09ab0272cba7dd0de633d129d0dd Mon Sep 17 00:00:00 2001 From: realbigsean Date: Mon, 2 Oct 2023 17:32:28 -0400 Subject: [PATCH 47/47] fix tests --- .../beacon_chain/tests/block_verification.rs | 62 ++++++++++++++----- .../src/network_beacon_processor/tests.rs | 11 ++-- 2 files changed, 54 insertions(+), 19 deletions(-) diff --git a/beacon_node/beacon_chain/tests/block_verification.rs b/beacon_node/beacon_chain/tests/block_verification.rs index e9f2501b47f..3ac39807146 100644 --- a/beacon_node/beacon_chain/tests/block_verification.rs +++ b/beacon_node/beacon_chain/tests/block_verification.rs @@ -204,17 +204,49 @@ fn update_proposal_signatures( } } -fn update_parent_roots(snapshots: &mut [BeaconSnapshot]) { +fn update_parent_roots( + snapshots: &mut [BeaconSnapshot], + blobs: &mut [Option>], +) { for i in 0..snapshots.len() { let root = snapshots[i].beacon_block.canonical_root(); - if let Some(child) = snapshots.get_mut(i + 1) { + if let (Some(child), Some(child_blobs)) = (snapshots.get_mut(i + 1), blobs.get_mut(i + 1)) { let (mut block, signature) = child.beacon_block.as_ref().clone().deconstruct(); *block.parent_root_mut() = root; - child.beacon_block = Arc::new(SignedBeaconBlock::from_block(block, signature)) + let new_child = Arc::new(SignedBeaconBlock::from_block(block, signature)); + let new_child_root = new_child.canonical_root(); + child.beacon_block = new_child; + if let Some(blobs) = child_blobs { + update_blob_roots(new_child_root, blobs); + } } } } +fn update_blob_roots(block_root: Hash256, blobs: &mut BlobSidecarList) { + for old_blob_sidecar in blobs.iter_mut() { + let index = old_blob_sidecar.index; + let slot = old_blob_sidecar.slot; + let block_parent_root = old_blob_sidecar.block_parent_root; + let proposer_index = old_blob_sidecar.proposer_index; + let blob = old_blob_sidecar.blob.clone(); + let kzg_commitment = old_blob_sidecar.kzg_commitment; + let kzg_proof = old_blob_sidecar.kzg_proof; + + let new_blob = Arc::new(BlobSidecar:: { + block_root, + index, + slot, + block_parent_root, + proposer_index, + blob, + kzg_commitment, + kzg_proof, + }); + *old_blob_sidecar = new_blob; + } +} + #[tokio::test] async fn chain_segment_full_segment() { let harness = get_harness(VALIDATOR_COUNT); @@ -588,7 +620,7 @@ async fn invalid_signature_block_proposal() { #[tokio::test] async fn invalid_signature_randao_reveal() { - let (chain_segment, chain_segment_blobs) = get_chain_segment().await; + let (chain_segment, mut chain_segment_blobs) = get_chain_segment().await; for &block_index in BLOCK_INDICES { let harness = get_invalid_sigs_harness(&chain_segment).await; let mut snapshots = chain_segment.clone(); @@ -600,7 +632,7 @@ async fn invalid_signature_randao_reveal() { *block.body_mut().randao_reveal_mut() = junk_signature(); snapshots[block_index].beacon_block = Arc::new(SignedBeaconBlock::from_block(block, signature)); - update_parent_roots(&mut snapshots); + update_parent_roots(&mut snapshots, &mut chain_segment_blobs); update_proposal_signatures(&mut snapshots, &harness); assert_invalid_signature( &chain_segment, @@ -616,7 +648,7 @@ async fn invalid_signature_randao_reveal() { #[tokio::test] async fn invalid_signature_proposer_slashing() { - let (chain_segment, chain_segment_blobs) = get_chain_segment().await; + let (chain_segment, mut chain_segment_blobs) = get_chain_segment().await; for &block_index in BLOCK_INDICES { let harness = get_invalid_sigs_harness(&chain_segment).await; let mut snapshots = chain_segment.clone(); @@ -642,7 +674,7 @@ async fn invalid_signature_proposer_slashing() { .expect("should update proposer slashing"); snapshots[block_index].beacon_block = Arc::new(SignedBeaconBlock::from_block(block, signature)); - update_parent_roots(&mut snapshots); + update_parent_roots(&mut snapshots, &mut chain_segment_blobs); update_proposal_signatures(&mut snapshots, &harness); assert_invalid_signature( &chain_segment, @@ -658,7 +690,7 @@ async fn invalid_signature_proposer_slashing() { #[tokio::test] async fn invalid_signature_attester_slashing() { - let (chain_segment, chain_segment_blobs) = get_chain_segment().await; + let (chain_segment, mut chain_segment_blobs) = get_chain_segment().await; for &block_index in BLOCK_INDICES { let harness = get_invalid_sigs_harness(&chain_segment).await; let mut snapshots = chain_segment.clone(); @@ -695,7 +727,7 @@ async fn invalid_signature_attester_slashing() { .expect("should update attester slashing"); snapshots[block_index].beacon_block = Arc::new(SignedBeaconBlock::from_block(block, signature)); - update_parent_roots(&mut snapshots); + update_parent_roots(&mut snapshots, &mut chain_segment_blobs); update_proposal_signatures(&mut snapshots, &harness); assert_invalid_signature( &chain_segment, @@ -711,7 +743,7 @@ async fn invalid_signature_attester_slashing() { #[tokio::test] async fn invalid_signature_attestation() { - let (chain_segment, chain_segment_blobs) = get_chain_segment().await; + let (chain_segment, mut chain_segment_blobs) = get_chain_segment().await; let mut checked_attestation = false; for &block_index in BLOCK_INDICES { @@ -726,7 +758,7 @@ async fn invalid_signature_attestation() { attestation.signature = junk_aggregate_signature(); snapshots[block_index].beacon_block = Arc::new(SignedBeaconBlock::from_block(block, signature)); - update_parent_roots(&mut snapshots); + update_parent_roots(&mut snapshots, &mut chain_segment_blobs); update_proposal_signatures(&mut snapshots, &harness); assert_invalid_signature( &chain_segment, @@ -749,7 +781,7 @@ async fn invalid_signature_attestation() { #[tokio::test] async fn invalid_signature_deposit() { - let (chain_segment, chain_segment_blobs) = get_chain_segment().await; + let (chain_segment, mut chain_segment_blobs) = get_chain_segment().await; for &block_index in BLOCK_INDICES { // Note: an invalid deposit signature is permitted! let harness = get_invalid_sigs_harness(&chain_segment).await; @@ -775,7 +807,7 @@ async fn invalid_signature_deposit() { .expect("should update deposit"); snapshots[block_index].beacon_block = Arc::new(SignedBeaconBlock::from_block(block, signature)); - update_parent_roots(&mut snapshots); + update_parent_roots(&mut snapshots, &mut chain_segment_blobs); update_proposal_signatures(&mut snapshots, &harness); let blocks: Vec> = snapshots .iter() @@ -800,7 +832,7 @@ async fn invalid_signature_deposit() { #[tokio::test] async fn invalid_signature_exit() { - let (chain_segment, chain_segment_blobs) = get_chain_segment().await; + let (chain_segment, mut chain_segment_blobs) = get_chain_segment().await; for &block_index in BLOCK_INDICES { let harness = get_invalid_sigs_harness(&chain_segment).await; let mut snapshots = chain_segment.clone(); @@ -823,7 +855,7 @@ async fn invalid_signature_exit() { .expect("should update deposit"); snapshots[block_index].beacon_block = Arc::new(SignedBeaconBlock::from_block(block, signature)); - update_parent_roots(&mut snapshots); + update_parent_roots(&mut snapshots, &mut chain_segment_blobs); update_proposal_signatures(&mut snapshots, &harness); assert_invalid_signature( &chain_segment, diff --git a/beacon_node/network/src/network_beacon_processor/tests.rs b/beacon_node/network/src/network_beacon_processor/tests.rs index 43cfb939164..0945aa74319 100644 --- a/beacon_node/network/src/network_beacon_processor/tests.rs +++ b/beacon_node/network/src/network_beacon_processor/tests.rs @@ -9,6 +9,7 @@ use crate::{ service::NetworkMessage, sync::{manager::BlockProcessType, SyncMessage}, }; +use beacon_chain::block_verification_types::RpcBlock; use beacon_chain::test_utils::{ test_spec, AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType, }; @@ -301,10 +302,11 @@ impl TestRig { } pub fn enqueue_rpc_block(&self) { + let block_root = self.next_block.canonical_root(); self.network_beacon_processor .send_rpc_beacon_block( - self.next_block.canonical_root(), - self.next_block.clone().into(), + block_root, + RpcBlock::new_without_blobs(Some(block_root), self.next_block.clone().into()), std::time::Duration::default(), BlockProcessType::ParentLookup { chain_hash: Hash256::random(), @@ -314,10 +316,11 @@ impl TestRig { } pub fn enqueue_single_lookup_rpc_block(&self) { + let block_root = self.next_block.canonical_root(); self.network_beacon_processor .send_rpc_beacon_block( - self.next_block.canonical_root(), - self.next_block.clone().into(), + block_root, + RpcBlock::new_without_blobs(Some(block_root), self.next_block.clone().into()), std::time::Duration::default(), BlockProcessType::SingleBlock { id: 1 }, )