diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 69a0b0c6229..7353fda794b 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -55,8 +55,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/blob_verification.rs b/beacon_node/beacon_chain/src/blob_verification.rs index ba875867a00..a15ebef897f 100644 --- a/beacon_node/beacon_chain/src/blob_verification.rs +++ b/beacon_node/beacon_chain/src/blob_verification.rs @@ -417,8 +417,8 @@ pub fn validate_blob_sidecar_for_gossip( // Verify that the blob_sidecar was received on the correct subnet. if blob_index != subnet { return Err(GossipBlobError::InvalidSubnet { - expected: blob_index, - received: subnet, + expected: subnet, + received: blob_index, }); } diff --git a/beacon_node/beacon_chain/src/block_verification.rs b/beacon_node/beacon_chain/src/block_verification.rs index 39760e860f4..dc989595f63 100644 --- a/beacon_node/beacon_chain/src/block_verification.rs +++ b/beacon_node/beacon_chain/src/block_verification.rs @@ -2075,7 +2075,7 @@ impl BlockBlobError for GossipBlobError { impl BlockBlobError for GossipDataColumnError { fn not_later_than_parent_error(data_column_slot: Slot, parent_slot: Slot) -> Self { - GossipDataColumnError::DataColumnIsNotLaterThanParent { + GossipDataColumnError::IsNotLaterThanParent { data_column_slot, parent_slot, } diff --git a/beacon_node/beacon_chain/src/builder.rs b/beacon_node/beacon_chain/src/builder.rs index 2d2b9c9a124..d5b408af526 100644 --- a/beacon_node/beacon_chain/src/builder.rs +++ b/beacon_node/beacon_chain/src/builder.rs @@ -936,6 +936,7 @@ where observed_sync_aggregators: <_>::default(), // TODO: allow for persisting and loading the pool from disk. observed_block_producers: <_>::default(), + observed_column_sidecars: <_>::default(), observed_blob_sidecars: <_>::default(), observed_slashable: <_>::default(), observed_voluntary_exits: <_>::default(), diff --git a/beacon_node/beacon_chain/src/data_column_verification.rs b/beacon_node/beacon_chain/src/data_column_verification.rs index b872896a4b0..fa43fa170eb 100644 --- a/beacon_node/beacon_chain/src/data_column_verification.rs +++ b/beacon_node/beacon_chain/src/data_column_verification.rs @@ -1,15 +1,23 @@ -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::kzg_utils::validate_data_column; use crate::{metrics, 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::iter; use std::sync::Arc; use types::data_column_sidecar::{ColumnIndex, DataColumnIdentifier}; use types::{ - BeaconStateError, DataColumnSidecar, EthSpec, Hash256, SignedBeaconBlockHeader, Slot, - VariableList, + BeaconStateError, DataColumnSidecar, DataColumnSubnetId, EthSpec, Hash256, + SignedBeaconBlockHeader, Slot, VariableList, }; /// An error occurred while validating a gossip data column. @@ -23,51 +31,108 @@ pub enum GossipDataColumnError { /// We were unable to process this data column due to an internal error. It's /// unclear if the data column is valid. BeaconChainError(BeaconChainError), - /// The proposal signature in invalid. /// /// ## Peer scoring /// /// The data column is invalid and the peer is faulty. ProposalSignatureInvalid, - /// The proposal_index corresponding to data column.beacon_block_root is not known. /// /// ## Peer scoring /// /// The data column is invalid and the peer is faulty. UnknownValidator(u64), - /// The provided data column is not from a later slot than its parent. /// /// ## Peer scoring /// /// The data column is invalid and the peer is faulty. - DataColumnIsNotLaterThanParent { + IsNotLaterThanParent { data_column_slot: Slot, parent_slot: Slot, }, - /// `Kzg` struct hasn't been initialized. This is an internal error. /// /// ## Peer scoring /// /// The peer isn't faulty, This is an internal error. KzgNotInitialized, - /// The kzg verification failed. /// /// ## Peer scoring /// /// The data column sidecar is invalid and the peer is faulty. - KzgError(kzg::Error), - - /// The provided data column's parent block is unknown. + 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 /// - /// We cannot process the data column without validating its parent, the peer isn't necessarily faulty. - DataColumnParentUnknown(Arc>), + /// 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(Arc>), + /// 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 { @@ -306,22 +371,254 @@ where pub fn validate_data_column_sidecar_for_gossip( data_column: Arc>, - _subnet: u64, + subnet: u64, chain: &BeaconChain, ) -> Result, GossipDataColumnError> { - // TODO(das): validate gossip rules - let block_root = data_column.block_root(); + let column_slot = data_column.slot(); + + verify_index_matches_subnet(&data_column, subnet)?; + 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)?; - // Kzg verification for gossip data column sidecar let kzg = chain .kzg - .as_ref() + .clone() .ok_or(GossipDataColumnError::KzgNotInitialized)?; - let kzg_verified_data_column = - KzgVerifiedDataColumn::new(data_column, kzg).map_err(GossipDataColumnError::KzgError)?; + 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, + 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> { + let _timer = metrics::start_timer(&metrics::DATA_COLUMN_SIDECAR_INCLUSION_PROOF_VERIFICATION); + 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(data_column.clone())); + }; + + // 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, +) -> Result<(), GossipDataColumnError> { + let expected_subnet: u64 = + DataColumnSubnetId::from_column_index::(data_column.index as usize).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 22a823fd324..ddca76ed6b8 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 ObservedBlobSidecarsError; use execution_layer::PayloadStatus; use fork_choice::ExecutionStatus; use futures::channel::mpsc::TrySendError; diff --git a/beacon_node/beacon_chain/src/lib.rs b/beacon_node/beacon_chain/src/lib.rs index 2fcde392fdc..b9b4afd3a4c 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; 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/metrics.rs b/beacon_node/beacon_chain/src/metrics.rs index b6eb6e0e4d7..49bf9e9db47 100644 --- a/beacon_node/beacon_chain/src/metrics.rs +++ b/beacon_node/beacon_chain/src/metrics.rs @@ -1061,6 +1061,10 @@ lazy_static! { "data_column_sidecar_computation_seconds", "Time taken to compute data column sidecar, including cells, proofs and inclusion proof" ); + pub static ref DATA_COLUMN_SIDECAR_INCLUSION_PROOF_VERIFICATION: Result = try_create_histogram( + "data_column_sidecar_inclusion_proof_verification_seconds", + "Time taken to verify data_column sidecar inclusion proof" + ); } // Fifth lazy-static block is used to account for macro recursion limit. diff --git a/beacon_node/beacon_chain/src/observed_blob_sidecars.rs b/beacon_node/beacon_chain/src/observed_data_sidecars.rs similarity index 78% rename from beacon_node/beacon_chain/src/observed_blob_sidecars.rs rename to beacon_node/beacon_chain/src/observed_data_sidecars.rs index 7d7f490ebb9..9485ef47692 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, 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 `ObservableSidecar` 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() -> 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() -> 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() -> usize { + E::number_of_columns() + } +} + +/// Maintains a cache of seen `ObservableSidecar`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,14 +70,14 @@ 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, + _phantom: PhantomData, } -impl Default for ObservedBlobSidecars { +impl Default for ObservedDataSidecars { /// Instantiates `Self` with `finalized_slot == 0`. fn default() -> Self { Self { @@ -45,49 +88,47 @@ impl Default for ObservedBlobSidecars { } } -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. +impl ObservedDataSidecars { + /// 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())); + 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() 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; @@ -125,7 +166,7 @@ mod tests { #[test] fn pruning() { - let mut cache = ObservedBlobSidecars::default(); + let mut cache = ObservedDataSidecars::>::default(); assert_eq!(cache.finalized_slot, 0, "finalized slot is zero"); assert_eq!(cache.items.len(), 0, "no slots should be present"); @@ -200,7 +241,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 +304,7 @@ mod tests { #[test] fn simple_observations() { - let mut cache = ObservedBlobSidecars::default(); + let mut cache = ObservedDataSidecars::>::default(); // Slot 0, index 0 let proposer_index_a = 420; @@ -423,7 +464,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 d439f142f31..8007ad25735 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -5,7 +5,7 @@ use crate::{ sync::SyncMessage, }; 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}, @@ -683,11 +683,82 @@ impl NetworkBeaconProcessor { .await } Err(err) => { - error!( - self.log, - "Internal error when verifying data column sidecar"; - "error" => ?err, - ) + match err { + GossipDataColumnError::ParentUnknown(column) => { + debug!( + self.log, + "Unknown parent hash for column"; + "action" => "requesting parent", + "block_root" => %column.block_root(), + "parent_root" => %column.block_parent_root(), + ); + self.send_sync_message(SyncMessage::UnknownParentDataColumn( + peer_id, column, + )); + } + 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 { .. } => { + // TODO(das): downgrade log to debug after interop + warn!( + 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 { .. } => { + // TODO(das): downgrade log to debug after interop + warn!( + 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, + ); + } + } } } } diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index f298dc1e4bb..44c9ab86b3b 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -20,7 +20,7 @@ use std::collections::hash_map::Entry; use std::sync::Arc; use std::time::Duration; use store::Hash256; -use types::{BlobSidecar, EthSpec, SignedBeaconBlock}; +use types::{BlobSidecar, DataColumnSidecar, EthSpec, SignedBeaconBlock}; pub mod common; pub mod parent_chain; @@ -34,6 +34,7 @@ pub const SINGLE_BLOCK_LOOKUP_MAX_ATTEMPTS: u8 = 4; pub enum BlockComponent { Block(DownloadResult>>), Blob(DownloadResult>>), + DataColumn(DownloadResult>>), } impl BlockComponent { @@ -41,12 +42,14 @@ impl BlockComponent { match self { BlockComponent::Block(block) => block.value.parent_root(), BlockComponent::Blob(blob) => blob.value.block_parent_root(), + BlockComponent::DataColumn(column) => column.value.block_parent_root(), } } fn get_type(&self) -> &'static str { match self { BlockComponent::Block(_) => "block", BlockComponent::Blob(_) => "blob", + BlockComponent::DataColumn(_) => "data_column", } } } 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 5d6b329312a..c33ad6fecf7 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 @@ -107,8 +107,8 @@ impl SingleBlockLookup { .block_request_state .state .insert_verified_response(block), - BlockComponent::Blob(_) => { - // For now ignore single blobs, as the blob request state assumes all blobs are + BlockComponent::Blob(_) | BlockComponent::DataColumn(_) => { + // For now ignore single blobs and columns, as the blob request state assumes all blobs are // attributed to the same peer = the peer serving the remaining blobs. Ignoring this // block component has a minor effect, causing the node to re-request this blob // once the parent chain is successfully resolved diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 8b115e89880..a81fe42cc83 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -145,6 +145,9 @@ pub enum SyncMessage { /// A blob with an unknown parent has been received. UnknownParentBlob(PeerId, Arc>), + /// A data column with an unknown parent has been received. + UnknownParentDataColumn(PeerId, Arc>), + /// A peer has sent an attestation that references a block that is unknown. This triggers the /// manager to attempt to find the block matching the unknown hash. UnknownBlockHashFromAttestation(PeerId, Hash256), @@ -680,6 +683,24 @@ impl SyncManager { }), ); } + SyncMessage::UnknownParentDataColumn(peer_id, data_column) => { + let data_column_slot = data_column.slot(); + let block_root = data_column.block_root(); + let parent_root = data_column.block_parent_root(); + debug!(self.log, "Received unknown parent data column message"; "block_root" => %block_root, "parent_root" => %parent_root); + self.handle_unknown_parent( + peer_id, + block_root, + parent_root, + data_column_slot, + BlockComponent::DataColumn(DownloadResult { + value: data_column, + block_root, + seen_timestamp: timestamp_now(), + peer_group: PeerGroup::from_single(peer_id), + }), + ); + } SyncMessage::UnknownBlockHashFromAttestation(peer_id, block_root) => { self.handle_unknown_block_root(peer_id, block_root); } diff --git a/beacon_node/network/src/sync/network_context/requests.rs b/beacon_node/network/src/sync/network_context/requests.rs index 2fc239b49b2..e2cdb37b1df 100644 --- a/beacon_node/network/src/sync/network_context/requests.rs +++ b/beacon_node/network/src/sync/network_context/requests.rs @@ -207,7 +207,7 @@ impl ActiveDataColumnsByRootRequest { "un-requested block root {block_root:?}" ))); } - if !data_column.verify_inclusion_proof().unwrap_or(false) { + if !data_column.verify_inclusion_proof() { return Err(RPCError::InvalidData("invalid inclusion proof".to_string())); } if !self.request.indices.contains(&data_column.index) { diff --git a/consensus/types/src/data_column_sidecar.rs b/consensus/types/src/data_column_sidecar.rs index 88094b946b2..62324fdef62 100644 --- a/consensus/types/src/data_column_sidecar.rs +++ b/consensus/types/src/data_column_sidecar.rs @@ -1,4 +1,4 @@ -use crate::beacon_block_body::KzgCommitments; +use crate::beacon_block_body::{KzgCommitments, BLOB_KZG_COMMITMENTS_INDEX}; use crate::test_utils::TestRandom; use crate::{ BeaconBlockHeader, EthSpec, Hash256, KzgProofs, SignedBeaconBlock, SignedBeaconBlockHeader, @@ -11,7 +11,7 @@ use derivative::Derivative; use kzg::Kzg; use kzg::{Blob as KzgBlob, Cell as KzgCell, Error as KzgError}; use kzg::{KzgCommitment, KzgProof}; -use merkle_proof::MerkleTreeError; +use merkle_proof::verify_merkle_proof; #[cfg(test)] use mockall_double::double; use safe_arith::ArithError; @@ -76,10 +76,25 @@ impl DataColumnSidecar { self.signed_block_header.message.tree_hash_root() } + pub fn block_parent_root(&self) -> Hash256 { + self.signed_block_header.message.parent_root + } + + pub fn block_proposer_index(&self) -> u64 { + self.signed_block_header.message.proposer_index + } + /// Verifies the kzg commitment inclusion merkle proof. - pub fn verify_inclusion_proof(&self) -> Result { - // TODO(das): implement - Ok(true) + pub fn verify_inclusion_proof(&self) -> bool { + let blob_kzg_commitments_root = self.kzg_commitments.tree_hash_root(); + + verify_merkle_proof( + blob_kzg_commitments_root, + &self.kzg_commitments_inclusion_proof, + E::kzg_commitments_inclusion_proof_depth(), + BLOB_KZG_COMMITMENTS_INDEX, + self.signed_block_header.message.body_root, + ) } pub fn build_sidecars(