From 19b3ab39ee2d131abe3226ab58aeaaa73b7f99fb Mon Sep 17 00:00:00 2001 From: Jimmy Chen Date: Mon, 29 Jul 2024 15:52:10 +1000 Subject: [PATCH] Data column gossip validation and error handling (#6181) * Add gossip verification and error handling. * Merge branch 'unstable' into das-gossip-validation * Add inclusion proof verification and some renames for consistency --- beacon_node/beacon_chain/src/beacon_chain.rs | 6 +- beacon_node/beacon_chain/src/builder.rs | 4 +- .../src/data_column_verification.rs | 323 +++++++++++++++++- beacon_node/beacon_chain/src/errors.rs | 6 +- beacon_node/beacon_chain/src/lib.rs | 2 +- ..._sidecars.rs => observed_data_sidecars.rs} | 130 ++++--- .../gossip_methods.rs | 82 ++++- 7 files changed, 494 insertions(+), 59 deletions(-) rename beacon_node/beacon_chain/src/{observed_blob_sidecars.rs => observed_data_sidecars.rs} (77%) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 095429faa57..c6ed979d681 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -52,8 +52,8 @@ use crate::observed_aggregates::{ use crate::observed_attesters::{ ObservedAggregators, ObservedAttesters, ObservedSyncAggregators, ObservedSyncContributors, }; -use crate::observed_blob_sidecars::ObservedBlobSidecars; use crate::observed_block_producers::ObservedBlockProducers; +use crate::observed_data_sidecars::ObservedDataSidecars; use crate::observed_operations::{ObservationOutcome, ObservedOperations}; use crate::observed_slashable::ObservedSlashable; use crate::persisted_beacon_chain::{PersistedBeaconChain, DUMMY_CANONICAL_HEAD_BLOCK_ROOT}; @@ -415,7 +415,9 @@ pub struct BeaconChain { /// Maintains a record of which validators have proposed blocks for each slot. pub observed_block_producers: RwLock>, /// Maintains a record of blob sidecars seen over the gossip network. - pub observed_blob_sidecars: RwLock>, + pub observed_blob_sidecars: RwLock>>, + /// Maintains a record of column sidecars seen over the gossip network. + pub observed_column_sidecars: RwLock>>, /// Maintains a record of slashable message seen over the gossip network or RPC. pub observed_slashable: RwLock>, /// Maintains a record of which validators have submitted voluntary exits. diff --git a/beacon_node/beacon_chain/src/builder.rs b/beacon_node/beacon_chain/src/builder.rs index 14e61e12653..c86e35980ba 100644 --- a/beacon_node/beacon_chain/src/builder.rs +++ b/beacon_node/beacon_chain/src/builder.rs @@ -11,6 +11,7 @@ use crate::graffiti_calculator::{GraffitiCalculator, GraffitiOrigin}; use crate::head_tracker::HeadTracker; use crate::light_client_server_cache::LightClientServerCache; use crate::migrate::{BackgroundMigrator, MigratorConfig}; +use crate::observed_data_sidecars::ObservedDataSidecars; use crate::persisted_beacon_chain::PersistedBeaconChain; use crate::shuffling_cache::{BlockShufflingIds, ShufflingCache}; use crate::validator_monitor::{ValidatorMonitor, ValidatorMonitorConfig}; @@ -917,7 +918,8 @@ where observed_sync_aggregators: <_>::default(), // TODO: allow for persisting and loading the pool from disk. observed_block_producers: <_>::default(), - observed_blob_sidecars: <_>::default(), + observed_column_sidecars: RwLock::new(ObservedDataSidecars::new(self.spec.clone())), + observed_blob_sidecars: RwLock::new(ObservedDataSidecars::new(self.spec.clone())), observed_slashable: <_>::default(), observed_voluntary_exits: <_>::default(), observed_proposer_slashings: <_>::default(), diff --git a/beacon_node/beacon_chain/src/data_column_verification.rs b/beacon_node/beacon_chain/src/data_column_verification.rs index 2e88da8f6ad..53e83a80617 100644 --- a/beacon_node/beacon_chain/src/data_column_verification.rs +++ b/beacon_node/beacon_chain/src/data_column_verification.rs @@ -1,13 +1,21 @@ -use crate::block_verification::{process_block_slash_info, BlockSlashInfo}; +use crate::block_verification::{ + cheap_state_advance_to_obtain_committees, get_validator_pubkey_cache, process_block_slash_info, + BlockSlashInfo, +}; use crate::{BeaconChain, BeaconChainError, BeaconChainTypes}; use derivative::Derivative; +use fork_choice::ProtoBlock; use kzg::{Error as KzgError, Kzg}; +use proto_array::Block; +use slasher::test_utils::E; +use slog::debug; +use slot_clock::SlotClock; use ssz_derive::{Decode, Encode}; use std::sync::Arc; use types::data_column_sidecar::{ColumnIndex, DataColumnIdentifier}; use types::{ - BeaconStateError, DataColumnSidecar, EthSpec, Hash256, RuntimeVariableList, - SignedBeaconBlockHeader, Slot, + BeaconStateError, ChainSpec, DataColumnSidecar, DataColumnSubnetId, EthSpec, Hash256, + RuntimeVariableList, SignedBeaconBlockHeader, Slot, }; /// An error occurred while validating a gossip data column. @@ -54,6 +62,75 @@ pub enum GossipDataColumnError { /// /// The data column sidecar is invalid and the peer is faulty. InvalidKzgProof(kzg::Error), + /// The column was gossiped over an incorrect subnet. + /// + /// ## Peer scoring + /// + /// The column is invalid or the peer is faulty. + InvalidSubnetId { received: u64, expected: u64 }, + /// The column sidecar is from a slot that is later than the current slot (with respect to the + /// gossip clock disparity). + /// + /// ## Peer scoring + /// + /// Assuming the local clock is correct, the peer has sent an invalid message. + FutureSlot { + message_slot: Slot, + latest_permissible_slot: Slot, + }, + /// The sidecar corresponds to a slot older than the finalized head slot. + /// + /// ## Peer scoring + /// + /// It's unclear if this column is valid, but this column is for a finalized slot and is + /// therefore useless to us. + PastFinalizedSlot { + column_slot: Slot, + finalized_slot: Slot, + }, + /// The pubkey cache timed out. + /// + /// ## Peer scoring + /// + /// The column sidecar may be valid, this is an internal error. + PubkeyCacheTimeout, + /// The proposer index specified in the sidecar does not match the locally computed + /// proposer index. + /// + /// ## Peer scoring + /// + /// The column is invalid and the peer is faulty. + ProposerIndexMismatch { sidecar: usize, local: usize }, + /// The provided columns's parent block is unknown. + /// + /// ## Peer scoring + /// + /// We cannot process the columns without validating its parent, the peer isn't necessarily faulty. + ParentUnknown { parent_root: Hash256 }, + /// The column conflicts with finalization, no need to propagate. + /// + /// ## Peer scoring + /// + /// It's unclear if this column is valid, but it conflicts with finality and shouldn't be + /// imported. + NotFinalizedDescendant { block_parent_root: Hash256 }, + /// Invalid kzg commitment inclusion proof + /// + /// ## Peer scoring + /// + /// The column sidecar is invalid and the peer is faulty + InvalidInclusionProof, + /// A column has already been seen for the given `(sidecar.block_root, sidecar.index)` tuple + /// over gossip or no gossip sources. + /// + /// ## Peer scoring + /// + /// The peer isn't faulty, but we do not forward it over gossip. + PriorKnown { + proposer: u64, + slot: Slot, + index: ColumnIndex, + }, } impl From for GossipDataColumnError { @@ -183,18 +260,254 @@ where pub fn validate_data_column_sidecar_for_gossip( data_column: Arc>, - _subnet: u64, + subnet: u64, chain: &BeaconChain, ) -> Result, GossipDataColumnError> { - // TODO(das): implement gossip verification + let column_slot = data_column.slot(); + + verify_index_matches_subnet(&data_column, subnet, &chain.spec)?; + verify_sidecar_not_from_future_slot(chain, column_slot)?; + verify_slot_greater_than_latest_finalized_slot(chain, column_slot)?; + verify_is_first_sidecar(chain, &data_column)?; + verify_column_inclusion_proof(&data_column)?; + let parent_block = verify_parent_block_and_finalized_descendant(data_column.clone(), chain)?; + verify_slot_higher_than_parent(&parent_block, column_slot)?; + verify_proposer_and_signature(&data_column, &parent_block, chain)?; let kzg = chain .kzg .clone() .ok_or(GossipDataColumnError::KzgNotInitialized)?; let kzg_verified_data_column = verify_kzg_for_data_column(data_column.clone(), &kzg) .map_err(GossipDataColumnError::InvalidKzgProof)?; + + chain + .observed_slashable + .write() + .observe_slashable( + column_slot, + data_column.block_proposer_index(), + data_column.block_root(), + ) + .map_err(|e| GossipDataColumnError::BeaconChainError(e.into()))?; + Ok(GossipVerifiedDataColumn { block_root: data_column.block_root(), data_column: kzg_verified_data_column, }) } + +// Verify that this is the first column sidecar received for the tuple: +// (block_header.slot, block_header.proposer_index, column_sidecar.index) +fn verify_is_first_sidecar( + chain: &BeaconChain, + data_column: &DataColumnSidecar, +) -> Result<(), GossipDataColumnError> { + if chain + .observed_column_sidecars + .read() + .proposer_is_known(data_column) + .map_err(|e| GossipDataColumnError::BeaconChainError(e.into()))? + { + return Err(GossipDataColumnError::PriorKnown { + proposer: data_column.block_proposer_index(), + slot: data_column.slot(), + index: data_column.index, + }); + } + Ok(()) +} + +fn verify_column_inclusion_proof( + data_column: &DataColumnSidecar, +) -> Result<(), GossipDataColumnError> { + if !data_column.verify_inclusion_proof() { + return Err(GossipDataColumnError::InvalidInclusionProof); + } + Ok(()) +} + +fn verify_slot_higher_than_parent( + parent_block: &Block, + data_column_slot: Slot, +) -> Result<(), GossipDataColumnError> { + if parent_block.slot >= data_column_slot { + return Err(GossipDataColumnError::IsNotLaterThanParent { + data_column_slot, + parent_slot: parent_block.slot, + }); + } + Ok(()) +} + +fn verify_parent_block_and_finalized_descendant( + data_column: Arc>, + chain: &BeaconChain, +) -> Result { + let fork_choice = chain.canonical_head.fork_choice_read_lock(); + + // We have already verified that the column is past finalization, so we can + // just check fork choice for the block's parent. + let block_parent_root = data_column.block_parent_root(); + let Some(parent_block) = fork_choice.get_block(&block_parent_root) else { + return Err(GossipDataColumnError::ParentUnknown { + parent_root: block_parent_root, + }); + }; + + // Do not process a column that does not descend from the finalized root. + // We just loaded the parent_block, so we can be sure that it exists in fork choice. + if !fork_choice.is_finalized_checkpoint_or_descendant(block_parent_root) { + return Err(GossipDataColumnError::NotFinalizedDescendant { block_parent_root }); + } + + Ok(parent_block) +} + +fn verify_proposer_and_signature( + data_column: &DataColumnSidecar, + parent_block: &ProtoBlock, + chain: &BeaconChain, +) -> Result<(), GossipDataColumnError> { + let column_slot = data_column.slot(); + let column_epoch = column_slot.epoch(E::slots_per_epoch()); + let column_index = data_column.index; + let block_root = data_column.block_root(); + let block_parent_root = data_column.block_parent_root(); + + let proposer_shuffling_root = + if parent_block.slot.epoch(T::EthSpec::slots_per_epoch()) == column_epoch { + parent_block + .next_epoch_shuffling_id + .shuffling_decision_block + } else { + parent_block.root + }; + + let proposer_opt = chain + .beacon_proposer_cache + .lock() + .get_slot::(proposer_shuffling_root, column_slot); + + let (proposer_index, fork) = if let Some(proposer) = proposer_opt { + (proposer.index, proposer.fork) + } else { + debug!( + chain.log, + "Proposer shuffling cache miss for column verification"; + "block_root" => %block_root, + "index" => %column_index, + ); + let (parent_state_root, mut parent_state) = chain + .store + .get_advanced_hot_state(block_parent_root, column_slot, parent_block.state_root) + .map_err(|e| GossipDataColumnError::BeaconChainError(e.into()))? + .ok_or_else(|| { + BeaconChainError::DBInconsistent(format!( + "Missing state for parent block {block_parent_root:?}", + )) + })?; + + let state = cheap_state_advance_to_obtain_committees::<_, GossipDataColumnError>( + &mut parent_state, + Some(parent_state_root), + column_slot, + &chain.spec, + )?; + + let proposers = state.get_beacon_proposer_indices(&chain.spec)?; + let proposer_index = *proposers + .get(column_slot.as_usize() % T::EthSpec::slots_per_epoch() as usize) + .ok_or_else(|| BeaconChainError::NoProposerForSlot(column_slot))?; + + // Prime the proposer shuffling cache with the newly-learned value. + chain.beacon_proposer_cache.lock().insert( + column_epoch, + proposer_shuffling_root, + proposers, + state.fork(), + )?; + (proposer_index, state.fork()) + }; + + // Signature verify the signed block header. + let signature_is_valid = { + let pubkey_cache = get_validator_pubkey_cache(chain) + .map_err(|_| GossipDataColumnError::PubkeyCacheTimeout)?; + + let pubkey = pubkey_cache + .get(proposer_index) + .ok_or_else(|| GossipDataColumnError::UnknownValidator(proposer_index as u64))?; + let signed_block_header = &data_column.signed_block_header; + signed_block_header.verify_signature::( + pubkey, + &fork, + chain.genesis_validators_root, + &chain.spec, + ) + }; + + if !signature_is_valid { + return Err(GossipDataColumnError::ProposalSignatureInvalid); + } + + let column_proposer_index = data_column.block_proposer_index(); + if proposer_index != column_proposer_index as usize { + return Err(GossipDataColumnError::ProposerIndexMismatch { + sidecar: column_proposer_index as usize, + local: proposer_index, + }); + } + + Ok(()) +} + +fn verify_index_matches_subnet( + data_column: &DataColumnSidecar, + subnet: u64, + spec: &ChainSpec, +) -> Result<(), GossipDataColumnError> { + let expected_subnet: u64 = + DataColumnSubnetId::from_column_index::(data_column.index as usize, spec).into(); + if expected_subnet != subnet { + return Err(GossipDataColumnError::InvalidSubnetId { + received: subnet, + expected: expected_subnet, + }); + } + Ok(()) +} + +fn verify_slot_greater_than_latest_finalized_slot( + chain: &BeaconChain, + column_slot: Slot, +) -> Result<(), GossipDataColumnError> { + let latest_finalized_slot = chain + .head() + .finalized_checkpoint() + .epoch + .start_slot(T::EthSpec::slots_per_epoch()); + if column_slot <= latest_finalized_slot { + return Err(GossipDataColumnError::PastFinalizedSlot { + column_slot, + finalized_slot: latest_finalized_slot, + }); + } + Ok(()) +} + +fn verify_sidecar_not_from_future_slot( + chain: &BeaconChain, + column_slot: Slot, +) -> Result<(), GossipDataColumnError> { + let latest_permissible_slot = chain + .slot_clock + .now_with_future_tolerance(chain.spec.maximum_gossip_clock_disparity()) + .ok_or(BeaconChainError::UnableToReadSlot)?; + if column_slot > latest_permissible_slot { + return Err(GossipDataColumnError::FutureSlot { + message_slot: column_slot, + latest_permissible_slot, + }); + } + Ok(()) +} diff --git a/beacon_node/beacon_chain/src/errors.rs b/beacon_node/beacon_chain/src/errors.rs index 819de1f5c19..1e3d67f9d7a 100644 --- a/beacon_node/beacon_chain/src/errors.rs +++ b/beacon_node/beacon_chain/src/errors.rs @@ -9,8 +9,8 @@ use crate::migrate::PruningError; use crate::naive_aggregation_pool::Error as NaiveAggregationError; use crate::observed_aggregates::Error as ObservedAttestationsError; use crate::observed_attesters::Error as ObservedAttestersError; -use crate::observed_blob_sidecars::Error as ObservedBlobSidecarsError; use crate::observed_block_producers::Error as ObservedBlockProducersError; +use crate::observed_data_sidecars::Error as ObservedDataSidecarsError; use execution_layer::PayloadStatus; use fork_choice::ExecutionStatus; use futures::channel::mpsc::TrySendError; @@ -100,7 +100,7 @@ pub enum BeaconChainError { ObservedAttestationsError(ObservedAttestationsError), ObservedAttestersError(ObservedAttestersError), ObservedBlockProducersError(ObservedBlockProducersError), - ObservedBlobSidecarsError(ObservedBlobSidecarsError), + ObservedDataSidecarsError(ObservedDataSidecarsError), AttesterCacheError(AttesterCacheError), PruningError(PruningError), ArithError(ArithError), @@ -238,7 +238,7 @@ easy_from_to!(NaiveAggregationError, BeaconChainError); easy_from_to!(ObservedAttestationsError, BeaconChainError); easy_from_to!(ObservedAttestersError, BeaconChainError); easy_from_to!(ObservedBlockProducersError, BeaconChainError); -easy_from_to!(ObservedBlobSidecarsError, BeaconChainError); +easy_from_to!(ObservedDataSidecarsError, BeaconChainError); easy_from_to!(AttesterCacheError, BeaconChainError); easy_from_to!(BlockSignatureVerifierError, BeaconChainError); easy_from_to!(PruningError, BeaconChainError); diff --git a/beacon_node/beacon_chain/src/lib.rs b/beacon_node/beacon_chain/src/lib.rs index 5f3ccac4e4a..7bfb5b08beb 100644 --- a/beacon_node/beacon_chain/src/lib.rs +++ b/beacon_node/beacon_chain/src/lib.rs @@ -42,8 +42,8 @@ pub mod migrate; mod naive_aggregation_pool; pub mod observed_aggregates; mod observed_attesters; -mod observed_blob_sidecars; pub mod observed_block_producers; +mod observed_data_sidecars; pub mod observed_operations; mod observed_slashable; pub mod otb_verification_service; diff --git a/beacon_node/beacon_chain/src/observed_blob_sidecars.rs b/beacon_node/beacon_chain/src/observed_data_sidecars.rs similarity index 77% rename from beacon_node/beacon_chain/src/observed_blob_sidecars.rs rename to beacon_node/beacon_chain/src/observed_data_sidecars.rs index 7d7f490ebb9..601241dd8ad 100644 --- a/beacon_node/beacon_chain/src/observed_blob_sidecars.rs +++ b/beacon_node/beacon_chain/src/observed_data_sidecars.rs @@ -6,20 +6,63 @@ use crate::observed_block_producers::ProposalKey; use std::collections::{HashMap, HashSet}; use std::marker::PhantomData; -use types::{BlobSidecar, EthSpec, Slot}; +use types::{BlobSidecar, ChainSpec, DataColumnSidecar, EthSpec, Slot}; #[derive(Debug, PartialEq)] pub enum Error { - /// The slot of the provided `BlobSidecar` is prior to finalization and should not have been provided + /// The slot of the provided `ObservableDataSidecar` is prior to finalization and should not have been provided /// to this function. This is an internal error. - FinalizedBlob { slot: Slot, finalized_slot: Slot }, - /// The blob sidecar contains an invalid blob index, the blob sidecar is invalid. - /// Note: The invalid blob should have been caught and flagged as an error much before reaching + FinalizedDataSidecar { slot: Slot, finalized_slot: Slot }, + /// The data sidecar contains an invalid index, the data sidecar is invalid. + /// Note: The invalid data should have been caught and flagged as an error much before reaching /// here. - InvalidBlobIndex(u64), + InvalidDataIndex(u64), } -/// Maintains a cache of seen `BlobSidecar`s that are received over gossip +pub trait ObservableDataSidecar { + fn slot(&self) -> Slot; + fn block_proposer_index(&self) -> u64; + fn index(&self) -> u64; + fn max_num_of_items(spec: &ChainSpec) -> usize; +} + +impl ObservableDataSidecar for BlobSidecar { + fn slot(&self) -> Slot { + self.slot() + } + + fn block_proposer_index(&self) -> u64 { + self.block_proposer_index() + } + + fn index(&self) -> u64 { + self.index + } + + fn max_num_of_items(_spec: &ChainSpec) -> usize { + E::max_blobs_per_block() + } +} + +impl ObservableDataSidecar for DataColumnSidecar { + fn slot(&self) -> Slot { + self.slot() + } + + fn block_proposer_index(&self) -> u64 { + self.block_proposer_index() + } + + fn index(&self) -> u64 { + self.index + } + + fn max_num_of_items(spec: &ChainSpec) -> usize { + spec.number_of_columns + } +} + +/// Maintains a cache of seen `ObservableDataSidecar`s that are received over gossip /// and have been gossip verified. /// /// The cache supports pruning based upon the finalized epoch. It does not automatically prune, you @@ -27,67 +70,65 @@ pub enum Error { /// /// Note: To prevent DoS attacks, this cache must include only items that have received some DoS resistance /// like checking the proposer signature. -pub struct ObservedBlobSidecars { +pub struct ObservedDataSidecars { finalized_slot: Slot, - /// Stores all received blob indices for a given `(ValidatorIndex, Slot)` tuple. + /// Stores all received data indices for a given `(ValidatorIndex, Slot)` tuple. items: HashMap>, - _phantom: PhantomData, + spec: ChainSpec, + _phantom: PhantomData, } -impl Default for ObservedBlobSidecars { +impl ObservedDataSidecars { /// Instantiates `Self` with `finalized_slot == 0`. - fn default() -> Self { + pub fn new(spec: ChainSpec) -> Self { Self { finalized_slot: Slot::new(0), items: HashMap::new(), + spec, _phantom: PhantomData, } } -} -impl ObservedBlobSidecars { - /// Observe the `blob_sidecar` at (`blob_sidecar.block_proposer_index, blob_sidecar.slot`). - /// This will update `self` so future calls to it indicate that this `blob_sidecar` is known. + /// Observe the `data_sidecar` at (`data_sidecar.block_proposer_index, data_sidecar.slot`). + /// This will update `self` so future calls to it indicate that this `data_sidecar` is known. /// - /// The supplied `blob_sidecar` **MUST** have completed proposer signature verification. - pub fn observe_sidecar(&mut self, blob_sidecar: &BlobSidecar) -> Result { - self.sanitize_blob_sidecar(blob_sidecar)?; + /// The supplied `data_sidecar` **MUST** have completed proposer signature verification. + pub fn observe_sidecar(&mut self, data_sidecar: &T) -> Result { + self.sanitize_data_sidecar(data_sidecar)?; - let blob_indices = self + let data_indices = self .items .entry(ProposalKey { - slot: blob_sidecar.slot(), - proposer: blob_sidecar.block_proposer_index(), + slot: data_sidecar.slot(), + proposer: data_sidecar.block_proposer_index(), }) - .or_insert_with(|| HashSet::with_capacity(E::max_blobs_per_block())); - let did_not_exist = blob_indices.insert(blob_sidecar.index); + .or_insert_with(|| HashSet::with_capacity(T::max_num_of_items(&self.spec))); + let did_not_exist = data_indices.insert(data_sidecar.index()); Ok(!did_not_exist) } - /// Returns `true` if the `blob_sidecar` has already been observed in the cache within the prune window. - pub fn proposer_is_known(&self, blob_sidecar: &BlobSidecar) -> Result { - self.sanitize_blob_sidecar(blob_sidecar)?; + /// Returns `true` if the `data_sidecar` has already been observed in the cache within the prune window. + pub fn proposer_is_known(&self, data_sidecar: &T) -> Result { + self.sanitize_data_sidecar(data_sidecar)?; let is_known = self .items .get(&ProposalKey { - slot: blob_sidecar.slot(), - proposer: blob_sidecar.block_proposer_index(), + slot: data_sidecar.slot(), + proposer: data_sidecar.block_proposer_index(), }) - .map_or(false, |blob_indices| { - blob_indices.contains(&blob_sidecar.index) - }); + .map_or(false, |indices| indices.contains(&data_sidecar.index())); Ok(is_known) } - fn sanitize_blob_sidecar(&self, blob_sidecar: &BlobSidecar) -> Result<(), Error> { - if blob_sidecar.index >= E::max_blobs_per_block() as u64 { - return Err(Error::InvalidBlobIndex(blob_sidecar.index)); + fn sanitize_data_sidecar(&self, data_sidecar: &T) -> Result<(), Error> { + if data_sidecar.index() >= T::max_num_of_items(&self.spec) as u64 { + return Err(Error::InvalidDataIndex(data_sidecar.index())); } let finalized_slot = self.finalized_slot; - if finalized_slot > 0 && blob_sidecar.slot() <= finalized_slot { - return Err(Error::FinalizedBlob { - slot: blob_sidecar.slot(), + if finalized_slot > 0 && data_sidecar.slot() <= finalized_slot { + return Err(Error::FinalizedDataSidecar { + slot: data_sidecar.slot(), finalized_slot, }); } @@ -95,7 +136,7 @@ impl ObservedBlobSidecars { Ok(()) } - /// Prune `blob_sidecar` observations for slots less than or equal to the given slot. + /// Prune `data_sidecar` observations for slots less than or equal to the given slot. pub fn prune(&mut self, finalized_slot: Slot) { if finalized_slot == 0 { return; @@ -109,6 +150,7 @@ impl ObservedBlobSidecars { #[cfg(test)] mod tests { use super::*; + use crate::test_utils::test_spec; use bls::Hash256; use std::sync::Arc; use types::MainnetEthSpec; @@ -125,7 +167,8 @@ mod tests { #[test] fn pruning() { - let mut cache = ObservedBlobSidecars::default(); + let spec = test_spec::(); + let mut cache = ObservedDataSidecars::>::new(spec); assert_eq!(cache.finalized_slot, 0, "finalized slot is zero"); assert_eq!(cache.items.len(), 0, "no slots should be present"); @@ -200,7 +243,7 @@ mod tests { assert_eq!( cache.observe_sidecar(&block_b), - Err(Error::FinalizedBlob { + Err(Error::FinalizedDataSidecar { slot: E::slots_per_epoch().into(), finalized_slot: E::slots_per_epoch().into(), }), @@ -263,7 +306,8 @@ mod tests { #[test] fn simple_observations() { - let mut cache = ObservedBlobSidecars::default(); + let spec = test_spec::(); + let mut cache = ObservedDataSidecars::>::new(spec); // Slot 0, index 0 let proposer_index_a = 420; @@ -423,7 +467,7 @@ mod tests { let sidecar_d = get_blob_sidecar(0, proposer_index_a, invalid_index); assert_eq!( cache.observe_sidecar(&sidecar_d), - Err(Error::InvalidBlobIndex(invalid_index)), + Err(Error::InvalidDataIndex(invalid_index)), "cannot add an index > MaxBlobsPerBlock" ); } 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 781c447f811..4c5c34bfd83 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -6,7 +6,7 @@ use crate::{ }; use beacon_chain::blob_verification::{GossipBlobError, GossipVerifiedBlob}; use beacon_chain::block_verification_types::AsBlock; -use beacon_chain::data_column_verification::GossipVerifiedDataColumn; +use beacon_chain::data_column_verification::{GossipDataColumnError, GossipVerifiedDataColumn}; use beacon_chain::store::Error; use beacon_chain::{ attestation_verification::{self, Error as AttnError, VerifiedAttestation}, @@ -621,7 +621,7 @@ impl NetworkBeaconProcessor { ); match self .chain - .verify_data_column_sidecar_for_gossip(column_sidecar, *subnet_id) + .verify_data_column_sidecar_for_gossip(column_sidecar.clone(), *subnet_id) { Ok(gossip_verified_data_column) => { metrics::inc_counter( @@ -656,8 +656,82 @@ impl NetworkBeaconProcessor { ) .await } - Err(_) => { - // TODO(das) implement gossip error handling + Err(err) => { + match err { + GossipDataColumnError::ParentUnknown { parent_root } => { + debug!( + self.log, + "Unknown parent hash for column"; + "action" => "requesting parent", + "block_root" => %block_root, + "parent_root" => %parent_root, + ); + self.send_sync_message(SyncMessage::UnknownParentDataColumn( + peer_id, + column_sidecar, + )); + } + GossipDataColumnError::KzgNotInitialized + | GossipDataColumnError::PubkeyCacheTimeout + | GossipDataColumnError::BeaconChainError(_) => { + crit!( + self.log, + "Internal error when verifying column sidecar"; + "error" => ?err, + ) + } + GossipDataColumnError::ProposalSignatureInvalid + | GossipDataColumnError::UnknownValidator(_) + | GossipDataColumnError::ProposerIndexMismatch { .. } + | GossipDataColumnError::IsNotLaterThanParent { .. } + | GossipDataColumnError::InvalidSubnetId { .. } + | GossipDataColumnError::InvalidInclusionProof { .. } + | GossipDataColumnError::InvalidKzgProof { .. } + | GossipDataColumnError::NotFinalizedDescendant { .. } => { + debug!( + self.log, + "Could not verify column sidecar for gossip. Rejecting the column sidecar"; + "error" => ?err, + "slot" => %slot, + "block_root" => %block_root, + "index" => %index, + ); + // Prevent recurring behaviour by penalizing the peer slightly. + self.gossip_penalize_peer( + peer_id, + PeerAction::LowToleranceError, + "gossip_data_column_low", + ); + self.propagate_validation_result( + message_id, + peer_id, + MessageAcceptance::Reject, + ); + } + GossipDataColumnError::FutureSlot { .. } + | GossipDataColumnError::PriorKnown { .. } + | GossipDataColumnError::PastFinalizedSlot { .. } => { + debug!( + self.log, + "Could not verify column sidecar for gossip. Ignoring the column sidecar"; + "error" => ?err, + "slot" => %slot, + "block_root" => %block_root, + "index" => %index, + ); + // Prevent recurring behaviour by penalizing the peer slightly. + self.gossip_penalize_peer( + peer_id, + PeerAction::HighToleranceError, + "gossip_data_column_high", + ); + self.propagate_validation_result( + message_id, + peer_id, + MessageAcceptance::Ignore, + ); + } + } } } }