Gossip conditions deneb (#4164)

* Add all gossip conditions

* Handle some gossip errors

* Update beacon_node/beacon_chain/src/blob_verification.rs

Co-authored-by: Divma <26765164+divagant-martian@users.noreply.github.com>

* Add an ObservedBlobSidecars cache

---------

Co-authored-by: Divma <26765164+divagant-martian@users.noreply.github.com>
This commit is contained in:
Pawan Dhananjay 2023-04-20 15:26:20 -07:00 committed by GitHub
parent e6b033aefd
commit 895bbd6c03
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 399 additions and 95 deletions

View File

@ -48,6 +48,7 @@ use crate::observed_aggregates::{
use crate::observed_attesters::{ use crate::observed_attesters::{
ObservedAggregators, ObservedAttesters, ObservedSyncAggregators, ObservedSyncContributors, ObservedAggregators, ObservedAttesters, ObservedSyncAggregators, ObservedSyncContributors,
}; };
use crate::observed_blob_sidecars::ObservedBlobSidecars;
use crate::observed_block_producers::ObservedBlockProducers; use crate::observed_block_producers::ObservedBlockProducers;
use crate::observed_operations::{ObservationOutcome, ObservedOperations}; use crate::observed_operations::{ObservationOutcome, ObservedOperations};
use crate::persisted_beacon_chain::{PersistedBeaconChain, DUMMY_CANONICAL_HEAD_BLOCK_ROOT}; use crate::persisted_beacon_chain::{PersistedBeaconChain, DUMMY_CANONICAL_HEAD_BLOCK_ROOT};
@ -401,6 +402,8 @@ pub struct BeaconChain<T: BeaconChainTypes> {
pub(crate) observed_sync_aggregators: RwLock<ObservedSyncAggregators<T::EthSpec>>, pub(crate) observed_sync_aggregators: RwLock<ObservedSyncAggregators<T::EthSpec>>,
/// Maintains a record of which validators have proposed blocks for each slot. /// Maintains a record of which validators have proposed blocks for each slot.
pub(crate) observed_block_producers: RwLock<ObservedBlockProducers<T::EthSpec>>, pub(crate) observed_block_producers: RwLock<ObservedBlockProducers<T::EthSpec>>,
/// Maintains a record of blob sidecars seen over the gossip network.
pub(crate) observed_blob_sidecars: RwLock<ObservedBlobSidecars<T::EthSpec>>,
/// Maintains a record of which validators have submitted voluntary exits. /// Maintains a record of which validators have submitted voluntary exits.
pub(crate) observed_voluntary_exits: Mutex<ObservedOperations<SignedVoluntaryExit, T::EthSpec>>, pub(crate) observed_voluntary_exits: Mutex<ObservedOperations<SignedVoluntaryExit, T::EthSpec>>,
/// Maintains a record of which validators we've seen proposer slashings for. /// Maintains a record of which validators we've seen proposer slashings for.

View File

@ -1,5 +1,6 @@
use derivative::Derivative; use derivative::Derivative;
use slot_clock::SlotClock; use slot_clock::SlotClock;
use state_processing::state_advance::partial_state_advance;
use std::sync::Arc; use std::sync::Arc;
use crate::beacon_chain::{ use crate::beacon_chain::{
@ -12,9 +13,11 @@ use crate::data_availability_checker::{
use crate::kzg_utils::{validate_blob, validate_blobs}; use crate::kzg_utils::{validate_blob, validate_blobs};
use crate::BeaconChainError; use crate::BeaconChainError;
use kzg::Kzg; use kzg::Kzg;
use std::borrow::Cow;
use types::{ use types::{
BeaconBlockRef, BeaconStateError, BlobSidecar, BlobSidecarList, Epoch, EthSpec, Hash256, BeaconBlockRef, BeaconState, BeaconStateError, BlobSidecar, BlobSidecarList, ChainSpec,
KzgCommitment, SignedBeaconBlock, SignedBeaconBlockHeader, SignedBlobSidecar, Slot, CloneConfig, Epoch, EthSpec, Hash256, KzgCommitment, RelativeEpoch, SignedBeaconBlock,
SignedBeaconBlockHeader, SignedBlobSidecar, Slot,
}; };
#[derive(Debug)] #[derive(Debug)]
@ -30,31 +33,6 @@ pub enum BlobError {
latest_permissible_slot: Slot, latest_permissible_slot: Slot,
}, },
/// The blob sidecar has a different slot than the block.
///
/// ## Peer scoring
///
/// Assuming the local clock is correct, the peer has sent an invalid message.
SlotMismatch {
blob_slot: Slot,
block_slot: Slot,
},
/// No kzg ccommitment associated with blob sidecar.
KzgCommitmentMissing,
/// No transactions in block
TransactionsMissing,
/// Blob transactions in the block do not correspond to the kzg commitments.
TransactionCommitmentMismatch,
TrustedSetupNotInitialized,
InvalidKzgProof,
KzgError(kzg::Error),
/// There was an error whilst processing the sync contribution. It is not known if it is valid or invalid. /// There was an error whilst processing the sync contribution. It is not known if it is valid or invalid.
/// ///
/// ## Peer scoring /// ## Peer scoring
@ -62,28 +40,20 @@ pub enum BlobError {
/// We were unable to process this sync committee message due to an internal error. It's unclear if the /// We were unable to process this sync committee message due to an internal error. It's unclear if the
/// sync committee message is valid. /// sync committee message is valid.
BeaconChainError(BeaconChainError), BeaconChainError(BeaconChainError),
/// No blobs for the specified block where we would expect blobs.
UnavailableBlobs,
/// Blobs provided for a pre-Deneb fork.
InconsistentFork,
/// The `blobs_sidecar.message.beacon_block_root` block is unknown. /// The `BlobSidecar` was gossiped over an incorrect subnet.
/// ///
/// ## Peer scoring /// ## Peer scoring
/// ///
/// The blob points to a block we have not yet imported. The blob cannot be imported /// The blob is invalid or the peer is faulty.
/// into fork choice yet InvalidSubnet { expected: u64, received: u64 },
UnknownHeadBlock {
beacon_block_root: Hash256,
},
/// The `BlobSidecar` was gossiped over an incorrect subnet.
InvalidSubnet {
expected: u64,
received: u64,
},
/// The sidecar corresponds to a slot older than the finalized head slot. /// The sidecar corresponds to a slot older than the finalized head slot.
///
/// ## Peer scoring
///
/// It's unclear if this blob is valid, but this blob is for a finalized slot and is
/// therefore useless to us.
PastFinalizedSlot { PastFinalizedSlot {
blob_slot: Slot, blob_slot: Slot,
finalized_slot: Slot, finalized_slot: Slot,
@ -91,21 +61,19 @@ pub enum BlobError {
/// The proposer index specified in the sidecar does not match the locally computed /// The proposer index specified in the sidecar does not match the locally computed
/// proposer index. /// proposer index.
ProposerIndexMismatch { ///
sidecar: usize, /// ## Peer scoring
local: usize, ///
}, /// The blob is invalid and the peer is faulty.
ProposerIndexMismatch { sidecar: usize, local: usize },
/// The proposal signature in invalid.
///
/// ## Peer scoring
///
/// The blob is invalid and the peer is faulty.
ProposerSignatureInvalid, ProposerSignatureInvalid,
/// A sidecar with same slot, beacon_block_root and proposer_index but different blob is received for
/// the same blob index.
RepeatSidecar {
proposer: usize,
slot: Slot,
blob_index: usize,
},
/// The proposal_index corresponding to blob.beacon_block_root is not known. /// The proposal_index corresponding to blob.beacon_block_root is not known.
/// ///
/// ## Peer scoring /// ## Peer scoring
@ -113,7 +81,34 @@ pub enum BlobError {
/// The block is invalid and the peer is faulty. /// The block is invalid and the peer is faulty.
UnknownValidator(u64), UnknownValidator(u64),
BlobCacheError(AvailabilityCheckError), /// The provided blob is not from a later slot than its parent.
///
/// ## Peer scoring
///
/// The blob is invalid and the peer is faulty.
BlobIsNotLaterThanParent { blob_slot: Slot, parent_slot: Slot },
/// The provided blob's parent block is unknown.
///
/// ## Peer scoring
///
/// We cannot process the blob without validating its parent, the peer isn't necessarily faulty.
BlobParentUnknown {
blob_root: Hash256,
blob_parent_root: Hash256,
},
/// A blob 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.
RepeatBlob {
proposer: u64,
slot: Slot,
index: u64,
},
} }
impl From<BeaconChainError> for BlobError { impl From<BeaconChainError> for BlobError {
@ -149,6 +144,8 @@ pub fn validate_blob_sidecar_for_gossip<T: BeaconChainTypes>(
let blob_slot = signed_blob_sidecar.message.slot; let blob_slot = signed_blob_sidecar.message.slot;
let blob_index = signed_blob_sidecar.message.index; let blob_index = signed_blob_sidecar.message.index;
let block_root = signed_blob_sidecar.message.block_root; let block_root = signed_blob_sidecar.message.block_root;
let block_parent_root = signed_blob_sidecar.message.block_parent_root;
let blob_proposer_index = signed_blob_sidecar.message.proposer_index;
// Verify that the blob_sidecar was received on the correct subnet. // Verify that the blob_sidecar was received on the correct subnet.
if blob_index != subnet { if blob_index != subnet {
@ -170,8 +167,6 @@ pub fn validate_blob_sidecar_for_gossip<T: BeaconChainTypes>(
}); });
} }
// TODO(pawan): Verify not from a past slot?
// Verify that the sidecar slot is greater than the latest finalized slot // Verify that the sidecar slot is greater than the latest finalized slot
let latest_finalized_slot = chain let latest_finalized_slot = chain
.head() .head()
@ -185,26 +180,93 @@ pub fn validate_blob_sidecar_for_gossip<T: BeaconChainTypes>(
}); });
} }
// TODO(pawan): should we verify locally that the parent root is correct // Verify that this is the first blob sidecar received for the (sidecar.block_root, sidecar.index) tuple
// or just use whatever the proposer gives us? if chain
.observed_blob_sidecars
.read()
.is_known(&signed_blob_sidecar.message)
.map_err(|e| BlobError::BeaconChainError(e.into()))?
{
return Err(BlobError::RepeatBlob {
proposer: blob_proposer_index,
slot: blob_slot,
index: blob_index,
});
}
// We have already verified that the blob is past finalization, so we can
// just check fork choice for the block's parent.
if let Some(parent_block) = chain
.canonical_head
.fork_choice_read_lock()
.get_block(&block_parent_root)
{
if parent_block.slot >= blob_slot {
return Err(BlobError::BlobIsNotLaterThanParent {
blob_slot,
parent_slot: parent_block.slot,
});
}
} else {
return Err(BlobError::BlobParentUnknown {
blob_root: block_root,
blob_parent_root: block_parent_root,
});
}
// Note: The spec checks the signature directly against `blob_sidecar.message.proposer_index`
// before checking that the provided proposer index is valid w.r.t the current shuffling.
//
// However, we check that the proposer_index matches against the shuffling first to avoid
// signature verification against an invalid proposer_index.
let proposer_shuffling_root = signed_blob_sidecar.message.block_parent_root; let proposer_shuffling_root = signed_blob_sidecar.message.block_parent_root;
let (proposer_index, fork) = match chain let proposer_opt = chain
.beacon_proposer_cache .beacon_proposer_cache
.lock() .lock()
.get_slot::<T::EthSpec>(proposer_shuffling_root, blob_slot) .get_slot::<T::EthSpec>(proposer_shuffling_root, blob_slot);
{
Some(proposer) => (proposer.index, proposer.fork), let (proposer_index, fork) = if let Some(proposer) = proposer_opt {
None => { (proposer.index, proposer.fork)
let state = &chain.canonical_head.cached_head().snapshot.beacon_state; } else {
// The cached head state is in the same epoch as the blob or the state has already been
// advanced to the blob's epoch
let snapshot = &chain.canonical_head.cached_head().snapshot;
if snapshot.beacon_state.current_epoch() == blob_slot.epoch(T::EthSpec::slots_per_epoch()) {
( (
state.get_beacon_proposer_index(blob_slot, &chain.spec)?, snapshot
state.fork(), .beacon_state
.get_beacon_proposer_index(blob_slot, &chain.spec)?,
snapshot.beacon_state.fork(),
) )
} }
// Need to advance the state to get the proposer index
else {
// The state produced is only valid for determining proposer/attester shuffling indices.
let mut cloned_state = snapshot.clone_with(CloneConfig::committee_caches_only());
let state = cheap_state_advance_to_obtain_committees(
&mut cloned_state.beacon_state,
None,
blob_slot,
&chain.spec,
)?;
let proposers = state.get_beacon_proposer_indices(&chain.spec)?;
let proposer_index = *proposers
.get(blob_slot.as_usize() % T::EthSpec::slots_per_epoch() as usize)
.ok_or_else(|| BeaconChainError::NoProposerForSlot(blob_slot))?;
// Prime the proposer shuffling cache with the newly-learned value.
chain.beacon_proposer_cache.lock().insert(
blob_slot.epoch(T::EthSpec::slots_per_epoch()),
proposer_shuffling_root,
proposers,
state.fork(),
)?;
(proposer_index, state.fork())
}
}; };
let blob_proposer_index = signed_blob_sidecar.message.proposer_index;
if proposer_index != blob_proposer_index as usize { if proposer_index != blob_proposer_index as usize {
return Err(BlobError::ProposerIndexMismatch { return Err(BlobError::ProposerIndexMismatch {
sidecar: blob_proposer_index as usize, sidecar: blob_proposer_index as usize,
@ -212,6 +274,7 @@ pub fn validate_blob_sidecar_for_gossip<T: BeaconChainTypes>(
}); });
} }
// Signature verification
let signature_is_valid = { let signature_is_valid = {
let pubkey_cache = chain let pubkey_cache = chain
.validator_pubkey_cache .validator_pubkey_cache
@ -236,25 +299,27 @@ pub fn validate_blob_sidecar_for_gossip<T: BeaconChainTypes>(
return Err(BlobError::ProposerSignatureInvalid); return Err(BlobError::ProposerSignatureInvalid);
} }
// TODO(pawan): kzg validations. // Now the signature is valid, store the proposal so we don't accept another blob sidecar
// with the same `BlobIdentifier`.
// TODO(pawan): Check if other blobs for the same proposer index and blob index have been // It's important to double-check that the proposer still hasn't been observed so we don't
// received and drop if required. // have a race-condition when verifying two blocks simultaneously.
//
// Verify if the corresponding block for this blob has been received. // Note: If this BlobSidecar goes on to fail full verification, we do not evict it from the seen_cache
// Note: this should be the last gossip check so that we can forward the blob // as alternate blob_sidecars for the same identifier can still be retrieved
// over the gossip network even if we haven't received the corresponding block yet // over rpc. Evicting them from this cache would allow faster propagation over gossip. So we allow
// as all other validations have passed. // retreieval of potentially valid blocks over rpc, but try to punish the proposer for signing
let block_opt = chain // invalid messages. Issue for more background
.canonical_head // https://github.com/ethereum/consensus-specs/issues/3261
.fork_choice_read_lock() if chain
.get_block(&block_root) .observed_blob_sidecars
.or_else(|| chain.early_attester_cache.get_proto_block(block_root)); // TODO(pawan): should we be checking this cache? .write()
.observe_sidecar(&signed_blob_sidecar.message)
// TODO(pawan): this may be redundant with the new `AvailabilityProcessingStatus::PendingBlock variant` .map_err(|e| BlobError::BeaconChainError(e.into()))?
if block_opt.is_none() { {
return Err(BlobError::UnknownHeadBlock { return Err(BlobError::RepeatBlob {
beacon_block_root: block_root, proposer: proposer_index as u64,
slot: blob_slot,
index: blob_index,
}); });
} }
@ -263,6 +328,57 @@ pub fn validate_blob_sidecar_for_gossip<T: BeaconChainTypes>(
}) })
} }
/// Performs a cheap (time-efficient) state advancement so the committees and proposer shuffling for
/// `slot` can be obtained from `state`.
///
/// The state advancement is "cheap" since it does not generate state roots. As a result, the
/// returned state might be holistically invalid but the committees/proposers will be correct (since
/// they do not rely upon state roots).
///
/// If the given `state` can already serve the `slot`, the committees will be built on the `state`
/// and `Cow::Borrowed(state)` will be returned. Otherwise, the state will be cloned, cheaply
/// advanced and then returned as a `Cow::Owned`. The end result is that the given `state` is never
/// mutated to be invalid (in fact, it is never changed beyond a simple committee cache build).
///
/// Note: This is a copy of the `block_verification::cheap_state_advance_to_obtain_committees` to return
/// a BlobError error type instead.
/// TODO(pawan): try to unify the 2 functions.
fn cheap_state_advance_to_obtain_committees<'a, E: EthSpec>(
state: &'a mut BeaconState<E>,
state_root_opt: Option<Hash256>,
blob_slot: Slot,
spec: &ChainSpec,
) -> Result<Cow<'a, BeaconState<E>>, BlobError> {
let block_epoch = blob_slot.epoch(E::slots_per_epoch());
if state.current_epoch() == block_epoch {
// Build both the current and previous epoch caches, as the previous epoch caches are
// useful for verifying attestations in blocks from the current epoch.
state.build_committee_cache(RelativeEpoch::Previous, spec)?;
state.build_committee_cache(RelativeEpoch::Current, spec)?;
Ok(Cow::Borrowed(state))
} else if state.slot() > blob_slot {
Err(BlobError::BlobIsNotLaterThanParent {
blob_slot,
parent_slot: state.slot(),
})
} else {
let mut state = state.clone_with(CloneConfig::committee_caches_only());
let target_slot = block_epoch.start_slot(E::slots_per_epoch());
// Advance the state into the same epoch as the block. Use the "partial" method since state
// roots are not important for proposer/attester shuffling.
partial_state_advance(&mut state, state_root_opt, target_slot, spec)
.map_err(|e| BlobError::BeaconChainError(BeaconChainError::from(e)))?;
state.build_committee_cache(RelativeEpoch::Previous, spec)?;
state.build_committee_cache(RelativeEpoch::Current, spec)?;
Ok(Cow::Owned(state))
}
}
/// Wrapper over a `BlobSidecar` for which we have completed kzg verification. /// Wrapper over a `BlobSidecar` for which we have completed kzg verification.
/// i.e. `verify_blob_kzg_proof(blob, commitment, proof) == true`. /// i.e. `verify_blob_kzg_proof(blob, commitment, proof) == true`.
#[derive(Debug, Derivative, Clone)] #[derive(Debug, Derivative, Clone)]

View File

@ -243,7 +243,7 @@ pub enum BlockError<T: EthSpec> {
/// ///
/// The block is invalid and the peer is faulty. /// The block is invalid and the peer is faulty.
InvalidSignature, InvalidSignature,
/// The provided block is from an later slot than its parent. /// The provided block is not from a later slot than its parent.
/// ///
/// ## Peer scoring /// ## Peer scoring
/// ///

View File

@ -813,6 +813,8 @@ where
// TODO: allow for persisting and loading the pool from disk. // TODO: allow for persisting and loading the pool from disk.
observed_block_producers: <_>::default(), observed_block_producers: <_>::default(),
// TODO: allow for persisting and loading the pool from disk. // TODO: allow for persisting and loading the pool from disk.
observed_blob_sidecars: <_>::default(),
// TODO: allow for persisting and loading the pool from disk.
observed_voluntary_exits: <_>::default(), observed_voluntary_exits: <_>::default(),
observed_proposer_slashings: <_>::default(), observed_proposer_slashings: <_>::default(),
observed_attester_slashings: <_>::default(), observed_attester_slashings: <_>::default(),

View File

@ -952,6 +952,13 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.start_slot(T::EthSpec::slots_per_epoch()), .start_slot(T::EthSpec::slots_per_epoch()),
); );
self.observed_blob_sidecars.write().prune(
new_view
.finalized_checkpoint
.epoch
.start_slot(T::EthSpec::slots_per_epoch()),
);
self.snapshot_cache self.snapshot_cache
.try_write_for(BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT) .try_write_for(BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT)
.map(|mut snapshot_cache| { .map(|mut snapshot_cache| {

View File

@ -65,7 +65,6 @@ pub struct DataAvailabilityChecker<T: EthSpec, S: SlotClock> {
/// The blobs are all gossip and kzg verified. /// The blobs are all gossip and kzg verified.
/// The block has completed all verifications except the availability check. /// The block has completed all verifications except the availability check.
struct ReceivedComponents<T: EthSpec> { struct ReceivedComponents<T: EthSpec> {
/// We use a `BTreeMap` here to maintain the order of `BlobSidecar`s based on index.
verified_blobs: FixedVector<Option<KzgVerifiedBlob<T>>, T::MaxBlobsPerBlock>, verified_blobs: FixedVector<Option<KzgVerifiedBlob<T>>, T::MaxBlobsPerBlock>,
executed_block: Option<AvailabilityPendingExecutedBlock<T>>, executed_block: Option<AvailabilityPendingExecutedBlock<T>>,
} }

View File

@ -8,6 +8,7 @@ use crate::migrate::PruningError;
use crate::naive_aggregation_pool::Error as NaiveAggregationError; use crate::naive_aggregation_pool::Error as NaiveAggregationError;
use crate::observed_aggregates::Error as ObservedAttestationsError; use crate::observed_aggregates::Error as ObservedAttestationsError;
use crate::observed_attesters::Error as ObservedAttestersError; 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_block_producers::Error as ObservedBlockProducersError;
use execution_layer::PayloadStatus; use execution_layer::PayloadStatus;
use fork_choice::ExecutionStatus; use fork_choice::ExecutionStatus;
@ -101,6 +102,7 @@ pub enum BeaconChainError {
ObservedAttestationsError(ObservedAttestationsError), ObservedAttestationsError(ObservedAttestationsError),
ObservedAttestersError(ObservedAttestersError), ObservedAttestersError(ObservedAttestersError),
ObservedBlockProducersError(ObservedBlockProducersError), ObservedBlockProducersError(ObservedBlockProducersError),
ObservedBlobSidecarsError(ObservedBlobSidecarsError),
AttesterCacheError(AttesterCacheError), AttesterCacheError(AttesterCacheError),
PruningError(PruningError), PruningError(PruningError),
ArithError(ArithError), ArithError(ArithError),
@ -228,6 +230,7 @@ easy_from_to!(NaiveAggregationError, BeaconChainError);
easy_from_to!(ObservedAttestationsError, BeaconChainError); easy_from_to!(ObservedAttestationsError, BeaconChainError);
easy_from_to!(ObservedAttestersError, BeaconChainError); easy_from_to!(ObservedAttestersError, BeaconChainError);
easy_from_to!(ObservedBlockProducersError, BeaconChainError); easy_from_to!(ObservedBlockProducersError, BeaconChainError);
easy_from_to!(ObservedBlobSidecarsError, BeaconChainError);
easy_from_to!(AttesterCacheError, BeaconChainError); easy_from_to!(AttesterCacheError, BeaconChainError);
easy_from_to!(BlockSignatureVerifierError, BeaconChainError); easy_from_to!(BlockSignatureVerifierError, BeaconChainError);
easy_from_to!(PruningError, BeaconChainError); easy_from_to!(PruningError, BeaconChainError);

View File

@ -36,6 +36,7 @@ pub mod migrate;
mod naive_aggregation_pool; mod naive_aggregation_pool;
mod observed_aggregates; mod observed_aggregates;
mod observed_attesters; mod observed_attesters;
mod observed_blob_sidecars;
mod observed_block_producers; mod observed_block_producers;
pub mod observed_operations; pub mod observed_operations;
pub mod otb_verification_service; pub mod otb_verification_service;

View File

@ -0,0 +1,99 @@
//! Provides the `ObservedBlobSidecars` struct which allows for rejecting `BlobSidecar`s
//! that we have already seen over the gossip network.
//! Only `BlobSidecar`s that have completed proposer signature verification can be added
//! to this cache to reduce DoS risks.
use std::collections::{HashMap, HashSet};
use std::marker::PhantomData;
use std::sync::Arc;
use types::{BlobSidecar, EthSpec, Hash256, Slot};
#[derive(Debug, PartialEq)]
pub enum Error {
/// The slot of the provided `BlobSidecar` 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
/// here.
InvalidBlobIndex(u64),
}
/// Maintains a cache of seen `BlobSidecar`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
/// must call `Self::prune` manually.
///
/// 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<T: EthSpec> {
finalized_slot: Slot,
/// Stores all received blob indices for a given `(Root, Slot)` tuple.
items: HashMap<(Hash256, Slot), HashSet<u64>>,
_phantom: PhantomData<T>,
}
impl<E: EthSpec> Default for ObservedBlobSidecars<E> {
/// Instantiates `Self` with `finalized_slot == 0`.
fn default() -> Self {
Self {
finalized_slot: Slot::new(0),
items: HashMap::new(),
_phantom: PhantomData,
}
}
}
impl<T: EthSpec> ObservedBlobSidecars<T> {
/// Observe the `blob_sidecar` at (`blob_sidecar.block_root, blob_sidecar.slot`).
/// This will update `self` so future calls to it indicate that this `blob_sidecar` is known.
///
/// The supplied `blob_sidecar` **MUST** have completed proposer signature verification.
pub fn observe_sidecar(&mut self, blob_sidecar: &Arc<BlobSidecar<T>>) -> Result<bool, Error> {
self.sanitize_blob_sidecar(blob_sidecar)?;
let did_not_exist = self
.items
.entry((blob_sidecar.block_root, blob_sidecar.slot))
.or_insert_with(|| HashSet::with_capacity(T::max_blobs_per_block()))
.insert(blob_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 is_known(&self, blob_sidecar: &Arc<BlobSidecar<T>>) -> Result<bool, Error> {
self.sanitize_blob_sidecar(blob_sidecar)?;
let is_known = self
.items
.get(&(blob_sidecar.block_root, blob_sidecar.slot))
.map_or(false, |set| set.contains(&blob_sidecar.index));
Ok(is_known)
}
fn sanitize_blob_sidecar(&self, blob_sidecar: &Arc<BlobSidecar<T>>) -> Result<(), Error> {
if blob_sidecar.index >= T::max_blobs_per_block() as u64 {
return Err(Error::InvalidBlobIndex(blob_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,
finalized_slot,
});
}
Ok(())
}
/// Prune all values earlier than the given slot.
pub fn prune(&mut self, finalized_slot: Slot) {
if finalized_slot == 0 {
return;
}
self.finalized_slot = finalized_slot;
self.items.retain(|k, _| k.1 > finalized_slot);
}
}

View File

@ -1,6 +1,6 @@
use crate::{metrics, service::NetworkMessage, sync::SyncMessage}; use crate::{metrics, service::NetworkMessage, sync::SyncMessage};
use beacon_chain::blob_verification::{AsBlock, BlockWrapper, GossipVerifiedBlob}; use beacon_chain::blob_verification::{AsBlock, BlobError, BlockWrapper, GossipVerifiedBlob};
use beacon_chain::store::Error; use beacon_chain::store::Error;
use beacon_chain::{ use beacon_chain::{
attestation_verification::{self, Error as AttnError, VerifiedAttestation}, attestation_verification::{self, Error as AttnError, VerifiedAttestation},
@ -651,24 +651,98 @@ impl<T: BeaconChainTypes> Worker<T> {
#[allow(clippy::too_many_arguments)] #[allow(clippy::too_many_arguments)]
pub async fn process_gossip_blob( pub async fn process_gossip_blob(
self, self,
_message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
_peer_client: Client, _peer_client: Client,
blob_index: u64, blob_index: u64,
signed_blob: SignedBlobSidecar<T::EthSpec>, signed_blob: SignedBlobSidecar<T::EthSpec>,
_seen_duration: Duration, _seen_duration: Duration,
) { ) {
let slot = signed_blob.message.slot;
let root = signed_blob.message.block_root;
let index = signed_blob.message.index;
match self match self
.chain .chain
.verify_blob_sidecar_for_gossip(signed_blob, blob_index) .verify_blob_sidecar_for_gossip(signed_blob, blob_index)
{ {
Ok(gossip_verified_blob) => { Ok(gossip_verified_blob) => {
debug!(
self.log,
"Successfully verified gossip blob";
"slot" => %slot,
"root" => %root,
"index" => %index
);
self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Accept);
self.process_gossip_verified_blob(peer_id, gossip_verified_blob, _seen_duration) self.process_gossip_verified_blob(peer_id, gossip_verified_blob, _seen_duration)
.await .await
} }
Err(_) => { Err(err) => {
// TODO(pawan): handle all blob errors for peer scoring match err {
todo!() BlobError::BlobParentUnknown {
blob_root,
blob_parent_root,
} => {
debug!(
self.log,
"Unknown parent hash for blob";
"action" => "requesting parent",
"blob_root" => %blob_root,
"parent_root" => %blob_parent_root
);
// TODO: send blob to reprocessing queue and queue a sync request for the blob.
todo!();
}
BlobError::ProposerSignatureInvalid
| BlobError::UnknownValidator(_)
| BlobError::ProposerIndexMismatch { .. }
| BlobError::BlobIsNotLaterThanParent { .. }
| BlobError::InvalidSubnet { .. } => {
warn!(
self.log,
"Could not verify blob sidecar for gossip. Rejecting the blob sidecar";
"error" => ?err,
"slot" => %slot,
"root" => %root,
"index" => %index
);
// Prevent recurring behaviour by penalizing the peer slightly.
self.gossip_penalize_peer(
peer_id,
PeerAction::LowToleranceError,
"gossip_blob_low",
);
self.propagate_validation_result(
message_id,
peer_id,
MessageAcceptance::Reject,
);
}
BlobError::FutureSlot { .. }
| BlobError::BeaconChainError(_)
| BlobError::RepeatBlob { .. }
| BlobError::PastFinalizedSlot { .. } => {
warn!(
self.log,
"Could not verify blob sidecar for gossip. Ignoring the blob sidecar";
"error" => ?err,
"slot" => %slot,
"root" => %root,
"index" => %index
);
// Prevent recurring behaviour by penalizing the peer slightly.
self.gossip_penalize_peer(
peer_id,
PeerAction::HighToleranceError,
"gossip_blob_high",
);
self.propagate_validation_result(
message_id,
peer_id,
MessageAcceptance::Ignore,
);
}
}
} }
} }
} }