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:
parent
e6b033aefd
commit
895bbd6c03
@ -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.
|
||||||
|
@ -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)]
|
||||||
|
@ -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
|
||||||
///
|
///
|
||||||
|
@ -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(),
|
||||||
|
@ -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| {
|
||||||
|
@ -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>>,
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
|
@ -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;
|
||||||
|
99
beacon_node/beacon_chain/src/observed_blob_sidecars.rs
Normal file
99
beacon_node/beacon_chain/src/observed_blob_sidecars.rs
Normal 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);
|
||||||
|
}
|
||||||
|
}
|
@ -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,
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user