Rework block processing (#4092)

* introduce availability pending block

* add intoavailableblock trait

* small fixes

* add 'gossip blob cache' and start to clean up processing and transition types

* shard memory blob cache

* Initial commit

* Fix after rebase

* Add gossip verification conditions

* cache cleanup

* general chaos

* extended chaos

* cargo fmt

* more progress

* more progress

* tons of changes, just tryna compile

* everything, everywhere, all at once

* Reprocess an ExecutedBlock on unavailable blobs

* Add sus gossip verification for blobs

* Merge stuff

* Remove reprocessing cache stuff

* lint

* Add a wrapper to allow construction of only valid `AvailableBlock`s

* rename blob arc list to blob list

* merge cleanuo

* Revert "merge cleanuo"

This reverts commit 5e98326878c77528d0c4668c5a4db4a4b0fbaeaa.

* Revert "Revert "merge cleanuo""

This reverts commit 3a4009443a5812b3028abe855079307436dc5419.

* fix rpc methods

* move beacon block and blob to eth2/types

* rename gossip blob cache to data availability checker

* lots of changes

* fix some compilation issues

* fix compilation issues

* fix compilation issues

* fix compilation issues

* fix compilation issues

* fix compilation issues

* cargo fmt

* use a common data structure for block import types

* fix availability check on proposal import

* refactor the blob cache and split the block wrapper into two types

* add type conversion for signed block and block wrapper

* fix beacon chain tests and do some renaming, add some comments

* Partial processing (#4)

* move beacon block and blob to eth2/types

* rename gossip blob cache to data availability checker

* lots of changes

* fix some compilation issues

* fix compilation issues

* fix compilation issues

* fix compilation issues

* fix compilation issues

* fix compilation issues

* cargo fmt

* use a common data structure for block import types

* fix availability check on proposal import

* refactor the blob cache and split the block wrapper into two types

* add type conversion for signed block and block wrapper

* fix beacon chain tests and do some renaming, add some comments

* cargo update (#6)

---------

Co-authored-by: realbigsean <sean@sigmaprime.io>
Co-authored-by: realbigsean <seananderson33@gmail.com>
This commit is contained in:
Pawan Dhananjay 2023-03-25 03:00:41 +05:30 committed by GitHub
parent 25a2d8f078
commit b276af98b7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
46 changed files with 2167 additions and 1487 deletions

1032
Cargo.lock generated

File diff suppressed because it is too large Load Diff

View File

@ -8,15 +8,19 @@ use crate::beacon_block_streamer::{BeaconBlockStreamer, CheckEarlyAttesterCache}
use crate::beacon_proposer_cache::compute_proposer_duties_from_head;
use crate::beacon_proposer_cache::BeaconProposerCache;
use crate::blob_cache::BlobCache;
use crate::blob_verification::{AsBlock, AvailableBlock, BlockWrapper};
use crate::blob_verification::{self, AsBlock, BlobError, BlockWrapper, GossipVerifiedBlob};
use crate::block_times_cache::BlockTimesCache;
use crate::block_verification::POS_PANDA_BANNER;
use crate::block_verification::{
check_block_is_finalized_checkpoint_or_descendant, check_block_relevancy, get_block_root,
signature_verify_chain_segment, BlockError, ExecutionPendingBlock, GossipVerifiedBlock,
IntoExecutionPendingBlock, PayloadVerificationOutcome, POS_PANDA_BANNER,
signature_verify_chain_segment, AvailableExecutedBlock, BlockError, BlockImportData,
ExecutedBlock, ExecutionPendingBlock, GossipVerifiedBlock, IntoExecutionPendingBlock,
};
pub use crate::canonical_head::{CanonicalHead, CanonicalHeadRwLock};
use crate::chain_config::ChainConfig;
use crate::data_availability_checker::{
Availability, AvailabilityCheckError, AvailableBlock, DataAvailabilityChecker,
};
use crate::early_attester_cache::EarlyAttesterCache;
use crate::errors::{BeaconChainError as Error, BlockProductionError};
use crate::eth1_chain::{Eth1Chain, Eth1ChainBackend};
@ -109,8 +113,9 @@ use store::{
use task_executor::{ShutdownReason, TaskExecutor};
use tokio_stream::Stream;
use tree_hash::TreeHash;
use types::beacon_block_body::KzgCommitments;
use types::beacon_state::CloneConfig;
use types::blobs_sidecar::KzgCommitments;
use types::blob_sidecar::{BlobIdentifier, BlobSidecarList, Blobs};
use types::consts::eip4844::MIN_EPOCHS_FOR_BLOBS_SIDECARS_REQUESTS;
use types::consts::merge::INTERVALS_PER_SLOT;
use types::*;
@ -183,6 +188,36 @@ pub enum WhenSlotSkipped {
Prev,
}
#[derive(Debug, PartialEq)]
pub enum AvailabilityProcessingStatus {
PendingBlobs(Vec<BlobIdentifier>),
PendingBlock(Hash256),
Imported(Hash256),
}
//TODO(sean) using this in tests for now
impl TryInto<SignedBeaconBlockHash> for AvailabilityProcessingStatus {
type Error = ();
fn try_into(self) -> Result<SignedBeaconBlockHash, Self::Error> {
match self {
AvailabilityProcessingStatus::Imported(hash) => Ok(hash.into()),
_ => Err(()),
}
}
}
impl TryInto<Hash256> for AvailabilityProcessingStatus {
type Error = ();
fn try_into(self) -> Result<Hash256, Self::Error> {
match self {
AvailabilityProcessingStatus::Imported(hash) => Ok(hash),
_ => Err(()),
}
}
}
/// The result of a chain segment processing.
pub enum ChainSegmentResult<T: EthSpec> {
/// Processing this chain segment finished successfully.
@ -433,8 +468,9 @@ pub struct BeaconChain<T: BeaconChainTypes> {
pub slasher: Option<Arc<Slasher<T::EthSpec>>>,
/// Provides monitoring of a set of explicitly defined validators.
pub validator_monitor: RwLock<ValidatorMonitor<T::EthSpec>>,
pub blob_cache: BlobCache<T::EthSpec>,
pub kzg: Option<Arc<kzg::Kzg>>,
pub proposal_blob_cache: BlobCache<T::EthSpec>,
pub data_availability_checker: DataAvailabilityChecker<T::EthSpec, T::SlotClock>,
pub kzg: Option<Arc<Kzg>>,
}
type BeaconBlockAndState<T, Payload> = (BeaconBlock<T, Payload>, BeaconState<T>);
@ -991,19 +1027,9 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
&self,
block_root: &Hash256,
) -> Result<Option<BlobSidecarList<T::EthSpec>>, Error> {
// If there is no data availability boundary, the Eip4844 fork is disabled.
if let Some(finalized_data_availability_boundary) =
self.finalized_data_availability_boundary()
{
self.early_attester_cache
.get_blobs(*block_root)
.map_or_else(
|| self.get_blobs(block_root, finalized_data_availability_boundary),
|blobs| Ok(Some(blobs)),
)
} else {
Ok(None)
}
self.early_attester_cache
.get_blobs(*block_root)
.map_or_else(|| self.get_blobs(block_root), |blobs| Ok(Some(blobs)))
}
/// Returns the block at the given root, if any.
@ -1086,35 +1112,8 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
pub fn get_blobs(
&self,
block_root: &Hash256,
data_availability_boundary: Epoch,
) -> Result<Option<BlobSidecarList<T::EthSpec>>, Error> {
match self.store.get_blobs(block_root)? {
Some(blob_sidecar_list) => Ok(Some(blob_sidecar_list)),
None => {
// Check for the corresponding block to understand whether we *should* have blobs.
self.get_blinded_block(block_root)?
.map(|block| {
// If there are no KZG commitments in the block, we know the sidecar should
// be empty.
let expected_kzg_commitments =
match block.message().body().blob_kzg_commitments() {
Ok(kzg_commitments) => kzg_commitments,
Err(_) => return Err(Error::NoKzgCommitmentsFieldOnBlock),
};
if expected_kzg_commitments.is_empty() {
// TODO (mark): verify this
Ok(BlobSidecarList::empty())
} else if data_availability_boundary <= block.epoch() {
// We should have blobs for all blocks younger than the boundary.
Err(Error::BlobsUnavailable)
} else {
// We shouldn't have blobs for blocks older than the boundary.
Err(Error::BlobsOlderThanDataAvailabilityBoundary(block.epoch()))
}
})
.transpose()
}
}
Ok(self.store.get_blobs(block_root)?)
}
pub fn get_blinded_block(
@ -1936,6 +1935,15 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
})
}
pub fn verify_blob_sidecar_for_gossip(
self: &Arc<Self>,
blob_sidecar: SignedBlobSidecar<T::EthSpec>,
subnet_id: u64,
) -> Result<GossipVerifiedBlob<T::EthSpec>, BlobError> // TODO(pawan): make a GossipVerifedBlob type
{
blob_verification::validate_blob_sidecar_for_gossip(blob_sidecar, subnet_id, self)
}
/// Accepts some 'LightClientOptimisticUpdate' from the network and attempts to verify it
pub fn verify_optimistic_update_for_gossip(
self: &Arc<Self>,
@ -2686,13 +2694,29 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.map_err(BeaconChainError::TokioJoin)?
}
pub async fn process_blob(
self: &Arc<Self>,
blob: GossipVerifiedBlob<T::EthSpec>,
count_unrealized: CountUnrealized,
) -> Result<AvailabilityProcessingStatus, BlockError<T::EthSpec>> {
self.check_availability_and_maybe_import(
|chain| chain.data_availability_checker.put_gossip_blob(blob),
count_unrealized,
)
.await
}
/// Returns `Ok(block_root)` if the given `unverified_block` was successfully verified and
/// imported into the chain.
///
/// For post deneb blocks, this returns a `BlockError::AvailabilityPending` error
/// if the corresponding blobs are not in the required caches.
///
/// Items that implement `IntoExecutionPendingBlock` include:
///
/// - `SignedBeaconBlock`
/// - `GossipVerifiedBlock`
/// - `BlockWrapper`
///
/// ## Errors
///
@ -2704,105 +2728,67 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
unverified_block: B,
count_unrealized: CountUnrealized,
notify_execution_layer: NotifyExecutionLayer,
) -> Result<Hash256, BlockError<T::EthSpec>> {
) -> Result<AvailabilityProcessingStatus, BlockError<T::EthSpec>> {
// Start the Prometheus timer.
let _full_timer = metrics::start_timer(&metrics::BLOCK_PROCESSING_TIMES);
// Increment the Prometheus counter for block processing requests.
metrics::inc_counter(&metrics::BLOCK_PROCESSING_REQUESTS);
let slot = unverified_block.block().slot();
// A small closure to group the verification and import errors.
let chain = self.clone();
let import_block = async move {
let execution_pending = unverified_block.into_execution_pending_block(
block_root,
&chain,
notify_execution_layer,
)?;
chain
.import_execution_pending_block(execution_pending, count_unrealized)
let execution_pending = unverified_block.into_execution_pending_block(
block_root,
&chain,
notify_execution_layer,
)?;
let executed_block = self
.clone()
.into_executed_block(execution_pending)
.await
.map_err(|e| self.handle_block_error(e))?;
match executed_block {
ExecutedBlock::Available(block) => {
self.import_available_block(Box::new(block), count_unrealized)
.await
}
ExecutedBlock::AvailabilityPending(block) => {
self.check_availability_and_maybe_import(
|chain| {
chain
.data_availability_checker
.put_pending_executed_block(block)
},
count_unrealized,
)
.await
};
// Verify and import the block.
match import_block.await {
// The block was successfully verified and imported. Yay.
Ok(block_root) => {
trace!(
self.log,
"Beacon block imported";
"block_root" => ?block_root,
"block_slot" => slot,
);
// Increment the Prometheus counter for block processing successes.
metrics::inc_counter(&metrics::BLOCK_PROCESSING_SUCCESSES);
Ok(block_root)
}
Err(e @ BlockError::BeaconChainError(BeaconChainError::TokioJoin(_))) => {
debug!(
self.log,
"Beacon block processing cancelled";
"error" => ?e,
);
Err(e)
}
// There was an error whilst attempting to verify and import the block. The block might
// be partially verified or partially imported.
Err(BlockError::BeaconChainError(e)) => {
crit!(
self.log,
"Beacon block processing error";
"error" => ?e,
);
Err(BlockError::BeaconChainError(e))
}
// The block failed verification.
Err(other) => {
trace!(
self.log,
"Beacon block rejected";
"reason" => other.to_string(),
);
Err(other)
}
}
}
/// Accepts a fully-verified block and imports it into the chain without performing any
/// additional verification.
/// Accepts a fully-verified block and awaits on it's payload verification handle to
/// get a fully `ExecutedBlock`
///
/// An error is returned if the block was unable to be imported. It may be partially imported
/// (i.e., this function is not atomic).
async fn import_execution_pending_block(
/// An error is returned if the verification handle couldn't be awaited.
async fn into_executed_block(
self: Arc<Self>,
execution_pending_block: ExecutionPendingBlock<T>,
count_unrealized: CountUnrealized,
) -> Result<Hash256, BlockError<T::EthSpec>> {
) -> Result<ExecutedBlock<T::EthSpec>, BlockError<T::EthSpec>> {
let ExecutionPendingBlock {
block,
block_root,
state,
parent_block,
confirmed_state_roots,
import_data,
payload_verification_handle,
parent_eth1_finalization_data,
consensus_context,
} = execution_pending_block;
let PayloadVerificationOutcome {
payload_verification_status,
is_valid_merge_transition_block,
} = payload_verification_handle
let payload_verification_outcome = payload_verification_handle
.await
.map_err(BeaconChainError::TokioJoin)?
.ok_or(BeaconChainError::RuntimeShutdown)??;
// Log the PoS pandas if a merge transition just occurred.
if is_valid_merge_transition_block {
if payload_verification_outcome.is_valid_merge_transition_block {
info!(self.log, "{}", POS_PANDA_BANNER);
info!(
self.log,
@ -2830,9 +2816,91 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.into_root()
);
}
Ok(ExecutedBlock::new(
block,
import_data,
payload_verification_outcome,
))
}
fn handle_block_error(&self, e: BlockError<T::EthSpec>) -> BlockError<T::EthSpec> {
match e {
e @ BlockError::BeaconChainError(BeaconChainError::TokioJoin(_)) => {
debug!(
self.log,
"Beacon block processing cancelled";
"error" => ?e,
);
e
}
BlockError::BeaconChainError(e) => {
crit!(
self.log,
"Beacon block processing error";
"error" => ?e,
);
BlockError::BeaconChainError(e)
}
other => {
trace!(
self.log,
"Beacon block rejected";
"reason" => other.to_string(),
);
other
}
}
}
/// Accepts a fully-verified, available block and imports it into the chain without performing any
/// additional verification.
///
/// An error is returned if the block was unable to be imported. It may be partially imported
/// (i.e., this function is not atomic).
pub async fn check_availability_and_maybe_import(
self: &Arc<Self>,
cache_fn: impl FnOnce(Arc<Self>) -> Result<Availability<T::EthSpec>, AvailabilityCheckError>,
count_unrealized: CountUnrealized,
) -> Result<AvailabilityProcessingStatus, BlockError<T::EthSpec>> {
let availability = cache_fn(self.clone())?;
match availability {
Availability::Available(block) => {
self.import_available_block(block, count_unrealized).await
}
Availability::PendingBlock(block_root) => {
Ok(AvailabilityProcessingStatus::PendingBlock(block_root))
}
Availability::PendingBlobs(blob_ids) => {
Ok(AvailabilityProcessingStatus::PendingBlobs(blob_ids))
}
}
}
async fn import_available_block(
self: &Arc<Self>,
block: Box<AvailableExecutedBlock<T::EthSpec>>,
count_unrealized: CountUnrealized,
) -> Result<AvailabilityProcessingStatus, BlockError<T::EthSpec>> {
let AvailableExecutedBlock {
block,
import_data,
payload_verification_outcome,
} = *block;
let BlockImportData {
block_root,
state,
parent_block,
parent_eth1_finalization_data,
confirmed_state_roots,
consensus_context,
} = import_data;
let slot = block.slot();
// import
let chain = self.clone();
let block_hash = self
let result = self
.spawn_blocking_handle(
move || {
chain.import_block(
@ -2840,7 +2908,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
block_root,
state,
confirmed_state_roots,
payload_verification_status,
payload_verification_outcome.payload_verification_status,
count_unrealized,
parent_block,
parent_eth1_finalization_data,
@ -2849,12 +2917,32 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
},
"payload_verification_handle",
)
.await??;
.await
.map_err(|e| {
let b = BlockError::from(e);
self.handle_block_error(b)
})?;
Ok(block_hash)
match result {
// The block was successfully verified and imported. Yay.
Ok(block_root) => {
trace!(
self.log,
"Beacon block imported";
"block_root" => ?block_root,
"block_slot" => slot,
);
// Increment the Prometheus counter for block processing successes.
metrics::inc_counter(&metrics::BLOCK_PROCESSING_SUCCESSES);
Ok(AvailabilityProcessingStatus::Imported(block_root))
}
Err(e) => Err(self.handle_block_error(e)),
}
}
/// Accepts a fully-verified block and imports it into the chain without performing any
/// Accepts a fully-verified and available block and imports it into the chain without performing any
/// additional verification.
///
/// An error is returned if the block was unable to be imported. It may be partially imported
@ -3038,27 +3126,17 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
ops.push(StoreOp::PutBlock(block_root, signed_block.clone()));
ops.push(StoreOp::PutState(block.state_root(), &state));
// Only consider blobs if the eip4844 fork is enabled.
if let Some(data_availability_boundary) = self.data_availability_boundary() {
let block_epoch = block.slot().epoch(T::EthSpec::slots_per_epoch());
let margin_epochs = self.store.get_config().blob_prune_margin_epochs;
let import_boundary = data_availability_boundary - margin_epochs;
// Only store blobs at the data availability boundary, minus any configured epochs
// margin, or younger (of higher epoch number).
if block_epoch >= import_boundary {
if let Some(blobs) = blobs {
if !blobs.is_empty() {
//FIXME(sean) using this for debugging for now
info!(
self.log, "Writing blobs to store";
"block_root" => ?block_root
);
ops.push(StoreOp::PutBlobs(block_root, blobs));
}
}
if let Some(blobs) = blobs {
if !blobs.is_empty() {
//FIXME(sean) using this for debugging for now
info!(
self.log, "Writing blobs to store";
"block_root" => ?block_root
);
ops.push(StoreOp::PutBlobs(block_root, blobs));
}
}
let txn_lock = self.store.hot_db.begin_rw_transaction();
if let Err(e) = self.store.do_atomically_with_block_and_blobs_cache(ops) {
@ -4782,12 +4860,11 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.as_ref()
.ok_or(BlockProductionError::TrustedSetupNotInitialized)?;
let beacon_block_root = block.canonical_root();
let expected_kzg_commitments: &KzgCommitments<T::EthSpec> =
block.body().blob_kzg_commitments().map_err(|_| {
BlockProductionError::InvalidBlockVariant(
"EIP4844 block does not contain kzg commitments".to_string(),
)
})?;
let expected_kzg_commitments = block.body().blob_kzg_commitments().map_err(|_| {
BlockProductionError::InvalidBlockVariant(
"EIP4844 block does not contain kzg commitments".to_string(),
)
})?;
if expected_kzg_commitments.len() != blobs.len() {
return Err(BlockProductionError::MissingKzgCommitment(format!(
@ -4836,7 +4913,8 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.collect::<Result<Vec<_>, BlockProductionError>>()?,
);
self.blob_cache.put(beacon_block_root, blob_sidecars);
self.proposal_blob_cache
.put(beacon_block_root, blob_sidecars);
}
metrics::inc_counter(&metrics::BLOCK_PRODUCTION_SUCCESSES);
@ -4854,8 +4932,8 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
fn compute_blob_kzg_proofs(
kzg: &Arc<Kzg>,
blobs: &Blobs<<T as BeaconChainTypes>::EthSpec>,
expected_kzg_commitments: &KzgCommitments<<T as BeaconChainTypes>::EthSpec>,
blobs: &Blobs<T::EthSpec>,
expected_kzg_commitments: &KzgCommitments<T::EthSpec>,
slot: Slot,
) -> Result<Vec<KzgProof>, BlockProductionError> {
blobs
@ -6119,18 +6197,10 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
})
}
/// The epoch that is a data availability boundary, or the latest finalized epoch.
/// `None` if the `Eip4844` fork is disabled.
pub fn finalized_data_availability_boundary(&self) -> Option<Epoch> {
self.data_availability_boundary().map(|boundary| {
std::cmp::max(
boundary,
self.canonical_head
.cached_head()
.finalized_checkpoint()
.epoch,
)
})
/// Returns true if the given epoch lies within the da boundary and false otherwise.
pub fn block_needs_da_check(&self, block_epoch: Epoch) -> bool {
self.data_availability_boundary()
.map_or(false, |da_epoch| block_epoch >= da_epoch)
}
/// Returns `true` if we are at or past the `Eip4844` fork. This will always return `false` if

View File

@ -2,16 +2,20 @@ use derivative::Derivative;
use slot_clock::SlotClock;
use std::sync::Arc;
use crate::beacon_chain::{BeaconChain, BeaconChainTypes, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
use crate::BeaconChainError;
use state_processing::per_block_processing::eip4844::eip4844::verify_kzg_commitments_against_transactions;
use types::signed_beacon_block::BlobReconstructionError;
use types::{
BeaconBlockRef, BeaconStateError, BlobsSidecar, EthSpec, Hash256, KzgCommitment,
SignedBeaconBlock, SignedBeaconBlockAndBlobsSidecar, SignedBeaconBlockHeader, Slot,
Transactions,
use crate::beacon_chain::{
BeaconChain, BeaconChainTypes, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT,
};
use crate::data_availability_checker::{
AvailabilityCheckError, AvailabilityPendingBlock, AvailableBlock,
};
use crate::kzg_utils::{validate_blob, validate_blobs};
use crate::BeaconChainError;
use kzg::Kzg;
use types::{
BeaconBlockRef, BeaconStateError, BlobSidecar, BlobSidecarList, Epoch, EthSpec, Hash256,
KzgCommitment, SignedBeaconBlock, SignedBeaconBlockHeader, SignedBlobSidecar, Slot,
};
use types::{Epoch, ExecPayload};
#[derive(Debug)]
pub enum BlobError {
@ -62,15 +66,54 @@ pub enum BlobError {
UnavailableBlobs,
/// Blobs provided for a pre-Eip4844 fork.
InconsistentFork,
}
impl From<BlobReconstructionError> for BlobError {
fn from(e: BlobReconstructionError) -> Self {
match e {
BlobReconstructionError::UnavailableBlobs => BlobError::UnavailableBlobs,
BlobReconstructionError::InconsistentFork => BlobError::InconsistentFork,
}
}
/// The `blobs_sidecar.message.beacon_block_root` block is unknown.
///
/// ## Peer scoring
///
/// The blob points to a block we have not yet imported. The blob cannot be imported
/// into fork choice yet
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.
PastFinalizedSlot {
blob_slot: Slot,
finalized_slot: Slot,
},
/// The proposer index specified in the sidecar does not match the locally computed
/// proposer index.
ProposerIndexMismatch {
sidecar: usize,
local: usize,
},
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.
///
/// ## Peer scoring
///
/// The block is invalid and the peer is faulty.
UnknownValidator(u64),
BlobCacheError(AvailabilityCheckError),
}
impl From<BeaconChainError> for BlobError {
@ -85,302 +128,220 @@ impl From<BeaconStateError> for BlobError {
}
}
pub fn validate_blob_for_gossip<T: BeaconChainTypes>(
block_wrapper: BlockWrapper<T::EthSpec>,
block_root: Hash256,
chain: &BeaconChain<T>,
) -> Result<AvailableBlock<T::EthSpec>, BlobError> {
if let BlockWrapper::BlockAndBlob(ref block, ref blobs_sidecar) = block_wrapper {
let blob_slot = blobs_sidecar.beacon_block_slot;
// Do not gossip or process blobs from future or past slots.
let latest_permissible_slot = chain
.slot_clock
.now_with_future_tolerance(MAXIMUM_GOSSIP_CLOCK_DISPARITY)
.ok_or(BeaconChainError::UnableToReadSlot)?;
if blob_slot > latest_permissible_slot {
return Err(BlobError::FutureSlot {
message_slot: latest_permissible_slot,
latest_permissible_slot: blob_slot,
});
}
if blob_slot != block.slot() {
return Err(BlobError::SlotMismatch {
blob_slot,
block_slot: block.slot(),
});
}
}
block_wrapper.into_available_block(block_root, chain)
/// A wrapper around a `BlobSidecar` that indicates it has been approved for re-gossiping on
/// the p2p network.
#[derive(Debug)]
pub struct GossipVerifiedBlob<T: EthSpec> {
blob: Arc<BlobSidecar<T>>,
}
fn verify_data_availability<T: BeaconChainTypes>(
_blob_sidecar: &BlobsSidecar<T::EthSpec>,
kzg_commitments: &[KzgCommitment],
transactions: &Transactions<T::EthSpec>,
_block_slot: Slot,
_block_root: Hash256,
impl<T: EthSpec> GossipVerifiedBlob<T> {
pub fn block_root(&self) -> Hash256 {
self.blob.block_root
}
}
pub fn validate_blob_sidecar_for_gossip<T: BeaconChainTypes>(
signed_blob_sidecar: SignedBlobSidecar<T::EthSpec>,
subnet: u64,
chain: &BeaconChain<T>,
) -> Result<(), BlobError> {
if verify_kzg_commitments_against_transactions::<T::EthSpec>(transactions, kzg_commitments)
.is_err()
) -> Result<GossipVerifiedBlob<T::EthSpec>, BlobError> {
let blob_slot = signed_blob_sidecar.message.slot;
let blob_index = signed_blob_sidecar.message.index;
let block_root = signed_blob_sidecar.message.block_root;
// Verify that the blob_sidecar was received on the correct subnet.
if blob_index != subnet {
return Err(BlobError::InvalidSubnet {
expected: blob_index,
received: subnet,
});
}
// Verify that the sidecar is not from a future slot.
let latest_permissible_slot = chain
.slot_clock
.now_with_future_tolerance(MAXIMUM_GOSSIP_CLOCK_DISPARITY)
.ok_or(BeaconChainError::UnableToReadSlot)?;
if blob_slot > latest_permissible_slot {
return Err(BlobError::FutureSlot {
message_slot: blob_slot,
latest_permissible_slot,
});
}
// TODO(pawan): Verify not from a past slot?
// Verify that the sidecar slot is greater than the latest finalized slot
let latest_finalized_slot = chain
.head()
.finalized_checkpoint()
.epoch
.start_slot(T::EthSpec::slots_per_epoch());
if blob_slot <= latest_finalized_slot {
return Err(BlobError::PastFinalizedSlot {
blob_slot,
finalized_slot: latest_finalized_slot,
});
}
// TODO(pawan): should we verify locally that the parent root is correct
// or just use whatever the proposer gives us?
let proposer_shuffling_root = signed_blob_sidecar.message.block_parent_root;
let (proposer_index, fork) = match chain
.beacon_proposer_cache
.lock()
.get_slot::<T::EthSpec>(proposer_shuffling_root, blob_slot)
{
return Err(BlobError::TransactionCommitmentMismatch);
}
// Validatate that the kzg proof is valid against the commitments and blobs
let _kzg = chain
.kzg
.as_ref()
.ok_or(BlobError::TrustedSetupNotInitialized)?;
todo!("use `kzg_utils::validate_blobs` once the function is updated")
// if !kzg_utils::validate_blobs_sidecar(
// kzg,
// block_slot,
// block_root,
// kzg_commitments,
// blob_sidecar,
// )
// .map_err(BlobError::KzgError)?
// {
// return Err(BlobError::InvalidKzgProof);
// }
// Ok(())
}
/// A wrapper over a [`SignedBeaconBlock`] or a [`SignedBeaconBlockAndBlobsSidecar`]. This makes no
/// claims about data availability and should not be used in consensus. This struct is useful in
/// networking when we want to send blocks around without consensus checks.
#[derive(Clone, Debug, Derivative)]
#[derivative(PartialEq, Hash(bound = "E: EthSpec"))]
pub enum BlockWrapper<E: EthSpec> {
Block(Arc<SignedBeaconBlock<E>>),
BlockAndBlob(Arc<SignedBeaconBlock<E>>, Arc<BlobsSidecar<E>>),
}
impl<E: EthSpec> BlockWrapper<E> {
pub fn new(
block: Arc<SignedBeaconBlock<E>>,
blobs_sidecar: Option<Arc<BlobsSidecar<E>>>,
) -> Self {
if let Some(blobs_sidecar) = blobs_sidecar {
BlockWrapper::BlockAndBlob(block, blobs_sidecar)
} else {
BlockWrapper::Block(block)
Some(proposer) => (proposer.index, proposer.fork),
None => {
let state = &chain.canonical_head.cached_head().snapshot.beacon_state;
(
state.get_beacon_proposer_index(blob_slot, &chain.spec)?,
state.fork(),
)
}
};
let blob_proposer_index = signed_blob_sidecar.message.proposer_index;
if proposer_index != blob_proposer_index as usize {
return Err(BlobError::ProposerIndexMismatch {
sidecar: blob_proposer_index as usize,
local: proposer_index,
});
}
let signature_is_valid = {
let pubkey_cache = chain
.validator_pubkey_cache
.try_read_for(VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT)
.ok_or(BeaconChainError::ValidatorPubkeyCacheLockTimeout)
.map_err(BlobError::BeaconChainError)?;
let pubkey = pubkey_cache
.get(proposer_index)
.ok_or_else(|| BlobError::UnknownValidator(proposer_index as u64))?;
signed_blob_sidecar.verify_signature(
None,
pubkey,
&fork,
chain.genesis_validators_root,
&chain.spec,
)
};
if !signature_is_valid {
return Err(BlobError::ProposerSignatureInvalid);
}
// TODO(pawan): kzg validations.
// TODO(pawan): Check if other blobs for the same proposer index and blob index have been
// received and drop if required.
// Verify if the corresponding block for this blob has been received.
// Note: this should be the last gossip check so that we can forward the blob
// over the gossip network even if we haven't received the corresponding block yet
// as all other validations have passed.
let block_opt = chain
.canonical_head
.fork_choice_read_lock()
.get_block(&block_root)
.or_else(|| chain.early_attester_cache.get_proto_block(block_root)); // TODO(pawan): should we be checking this cache?
// TODO(pawan): this may be redundant with the new `AvailabilityProcessingStatus::PendingBlock variant`
if block_opt.is_none() {
return Err(BlobError::UnknownHeadBlock {
beacon_block_root: block_root,
});
}
Ok(GossipVerifiedBlob {
blob: signed_blob_sidecar.message,
})
}
#[derive(Debug, Clone)]
pub struct KzgVerifiedBlob<T: EthSpec> {
blob: Arc<BlobSidecar<T>>,
}
impl<T: EthSpec> KzgVerifiedBlob<T> {
pub fn to_blob(self) -> Arc<BlobSidecar<T>> {
self.blob
}
pub fn as_blob(&self) -> &BlobSidecar<T> {
&self.blob
}
pub fn clone_blob(&self) -> Arc<BlobSidecar<T>> {
self.blob.clone()
}
pub fn kzg_commitment(&self) -> KzgCommitment {
self.blob.kzg_commitment
}
pub fn block_root(&self) -> Hash256 {
self.blob.block_root
}
}
impl<E: EthSpec> From<SignedBeaconBlock<E>> for BlockWrapper<E> {
fn from(block: SignedBeaconBlock<E>) -> Self {
BlockWrapper::Block(Arc::new(block))
pub fn verify_kzg_for_blob<T: EthSpec>(
blob: GossipVerifiedBlob<T>,
kzg: &Kzg,
) -> Result<KzgVerifiedBlob<T>, AvailabilityCheckError> {
//TODO(sean) remove clone
if validate_blob::<T>(
kzg,
blob.blob.blob.clone(),
blob.blob.kzg_commitment,
blob.blob.kzg_proof,
)
.map_err(AvailabilityCheckError::Kzg)?
{
Ok(KzgVerifiedBlob { blob: blob.blob })
} else {
Err(AvailabilityCheckError::KzgVerificationFailed)
}
}
impl<E: EthSpec> From<SignedBeaconBlockAndBlobsSidecar<E>> for BlockWrapper<E> {
fn from(block: SignedBeaconBlockAndBlobsSidecar<E>) -> Self {
let SignedBeaconBlockAndBlobsSidecar {
beacon_block,
blobs_sidecar,
} = block;
BlockWrapper::BlockAndBlob(beacon_block, blobs_sidecar)
pub fn verify_kzg_for_blob_list<T: EthSpec>(
blob_list: BlobSidecarList<T>,
kzg: &Kzg,
) -> Result<KzgVerifiedBlobList<T>, AvailabilityCheckError> {
let (blobs, (commitments, proofs)): (Vec<_>, (Vec<_>, Vec<_>)) = blob_list
.clone()
.into_iter()
//TODO(sean) remove clone
.map(|blob| (blob.blob.clone(), (blob.kzg_commitment, blob.kzg_proof)))
.unzip();
if validate_blobs::<T>(
kzg,
commitments.as_slice(),
blobs.as_slice(),
proofs.as_slice(),
)
.map_err(AvailabilityCheckError::Kzg)?
{
Ok(blob_list
.into_iter()
.map(|blob| KzgVerifiedBlob { blob })
.collect())
} else {
Err(AvailabilityCheckError::KzgVerificationFailed)
}
}
impl<E: EthSpec> From<Arc<SignedBeaconBlock<E>>> for BlockWrapper<E> {
fn from(block: Arc<SignedBeaconBlock<E>>) -> Self {
BlockWrapper::Block(block)
}
}
pub type KzgVerifiedBlobList<T> = Vec<KzgVerifiedBlob<T>>;
#[derive(Copy, Clone)]
pub enum DataAvailabilityCheckRequired {
Yes,
No,
}
pub trait IntoAvailableBlock<T: BeaconChainTypes> {
fn into_available_block(
self,
block_root: Hash256,
chain: &BeaconChain<T>,
) -> Result<AvailableBlock<T::EthSpec>, BlobError>;
}
impl<T: BeaconChainTypes> IntoAvailableBlock<T> for BlockWrapper<T::EthSpec> {
fn into_available_block(
self,
block_root: Hash256,
chain: &BeaconChain<T>,
) -> Result<AvailableBlock<T::EthSpec>, BlobError> {
let data_availability_boundary = chain.data_availability_boundary();
let da_check_required =
data_availability_boundary.map_or(DataAvailabilityCheckRequired::No, |boundary| {
if self.slot().epoch(T::EthSpec::slots_per_epoch()) >= boundary {
DataAvailabilityCheckRequired::Yes
} else {
DataAvailabilityCheckRequired::No
}
});
match self {
BlockWrapper::Block(block) => AvailableBlock::new(block, block_root, da_check_required),
BlockWrapper::BlockAndBlob(block, blobs_sidecar) => {
if matches!(da_check_required, DataAvailabilityCheckRequired::Yes) {
let kzg_commitments = block
.message()
.body()
.blob_kzg_commitments()
.map_err(|_| BlobError::KzgCommitmentMissing)?;
let transactions = block
.message()
.body()
.execution_payload_eip4844()
.map(|payload| payload.transactions())
.map_err(|_| BlobError::TransactionsMissing)?
.ok_or(BlobError::TransactionsMissing)?;
verify_data_availability(
&blobs_sidecar,
kzg_commitments,
transactions,
block.slot(),
block_root,
chain,
)?;
}
AvailableBlock::new_with_blobs(block, blobs_sidecar, da_check_required)
}
}
}
}
/// A wrapper over a [`SignedBeaconBlock`] or a [`SignedBeaconBlockAndBlobsSidecar`]. An
/// `AvailableBlock` has passed any required data availability checks and should be used in
/// consensus. This newtype wraps `AvailableBlockInner` to ensure data availability checks
/// cannot be circumvented on construction.
#[derive(Clone, Debug, Derivative)]
#[derivative(PartialEq, Hash(bound = "E: EthSpec"))]
pub struct AvailableBlock<E: EthSpec>(AvailableBlockInner<E>);
/// A wrapper over a [`SignedBeaconBlock`] or a [`SignedBeaconBlockAndBlobsSidecar`].
#[derive(Clone, Debug, Derivative)]
#[derivative(PartialEq, Hash(bound = "E: EthSpec"))]
enum AvailableBlockInner<E: EthSpec> {
Block(Arc<SignedBeaconBlock<E>>),
BlockAndBlob(SignedBeaconBlockAndBlobsSidecar<E>),
}
impl<E: EthSpec> AvailableBlock<E> {
pub fn new(
beacon_block: Arc<SignedBeaconBlock<E>>,
block_root: Hash256,
da_check_required: DataAvailabilityCheckRequired,
) -> Result<Self, BlobError> {
match beacon_block.as_ref() {
// No data availability check required prior to Eip4844.
SignedBeaconBlock::Base(_)
| SignedBeaconBlock::Altair(_)
| SignedBeaconBlock::Capella(_)
| SignedBeaconBlock::Merge(_) => {
Ok(AvailableBlock(AvailableBlockInner::Block(beacon_block)))
}
SignedBeaconBlock::Eip4844(_) => {
match da_check_required {
DataAvailabilityCheckRequired::Yes => {
// Attempt to reconstruct empty blobs here.
let blobs_sidecar = beacon_block
.reconstruct_empty_blobs(Some(block_root))
.map(Arc::new)?;
Ok(AvailableBlock(AvailableBlockInner::BlockAndBlob(
SignedBeaconBlockAndBlobsSidecar {
beacon_block,
blobs_sidecar,
},
)))
}
DataAvailabilityCheckRequired::No => {
Ok(AvailableBlock(AvailableBlockInner::Block(beacon_block)))
}
}
}
}
}
/// This function is private because an `AvailableBlock` should be
/// constructed via the `into_available_block` method.
fn new_with_blobs(
beacon_block: Arc<SignedBeaconBlock<E>>,
blobs_sidecar: Arc<BlobsSidecar<E>>,
da_check_required: DataAvailabilityCheckRequired,
) -> Result<Self, BlobError> {
match beacon_block.as_ref() {
// This method shouldn't be called with a pre-Eip4844 block.
SignedBeaconBlock::Base(_)
| SignedBeaconBlock::Altair(_)
| SignedBeaconBlock::Capella(_)
| SignedBeaconBlock::Merge(_) => Err(BlobError::InconsistentFork),
SignedBeaconBlock::Eip4844(_) => {
match da_check_required {
DataAvailabilityCheckRequired::Yes => Ok(AvailableBlock(
AvailableBlockInner::BlockAndBlob(SignedBeaconBlockAndBlobsSidecar {
beacon_block,
blobs_sidecar,
}),
)),
DataAvailabilityCheckRequired::No => {
// Blobs were not verified so we drop them, we'll instead just pass around
// an available `Eip4844` block without blobs.
Ok(AvailableBlock(AvailableBlockInner::Block(beacon_block)))
}
}
}
}
}
pub fn blobs(&self) -> Option<Arc<BlobsSidecar<E>>> {
match &self.0 {
AvailableBlockInner::Block(_) => None,
AvailableBlockInner::BlockAndBlob(block_sidecar_pair) => {
Some(block_sidecar_pair.blobs_sidecar.clone())
}
}
}
pub fn deconstruct(self) -> (Arc<SignedBeaconBlock<E>>, Option<Arc<BlobsSidecar<E>>>) {
match self.0 {
AvailableBlockInner::Block(block) => (block, None),
AvailableBlockInner::BlockAndBlob(block_sidecar_pair) => {
let SignedBeaconBlockAndBlobsSidecar {
beacon_block,
blobs_sidecar,
} = block_sidecar_pair;
(beacon_block, Some(blobs_sidecar))
}
}
}
}
pub trait IntoBlockWrapper<E: EthSpec>: AsBlock<E> {
fn into_block_wrapper(self) -> BlockWrapper<E>;
}
impl<E: EthSpec> IntoBlockWrapper<E> for BlockWrapper<E> {
fn into_block_wrapper(self) -> BlockWrapper<E> {
self
}
}
impl<E: EthSpec> IntoBlockWrapper<E> for AvailableBlock<E> {
fn into_block_wrapper(self) -> BlockWrapper<E> {
let (block, blobs) = self.deconstruct();
if let Some(blobs) = blobs {
BlockWrapper::BlockAndBlob(block, blobs)
} else {
BlockWrapper::Block(block)
}
}
#[derive(Debug, Clone)]
pub enum MaybeAvailableBlock<E: EthSpec> {
/// This variant is fully available.
/// i.e. for pre-eip4844 blocks, it contains a (`SignedBeaconBlock`, `Blobs::None`) and for
/// post-4844 blocks, it contains a `SignedBeaconBlock` and a Blobs variant other than `Blobs::None`.
Available(AvailableBlock<E>),
/// This variant is not fully available and requires blobs to become fully available.
AvailabilityPending(AvailabilityPendingBlock<E>),
}
pub trait AsBlock<E: EthSpec> {
@ -393,193 +354,149 @@ pub trait AsBlock<E: EthSpec> {
fn as_block(&self) -> &SignedBeaconBlock<E>;
fn block_cloned(&self) -> Arc<SignedBeaconBlock<E>>;
fn canonical_root(&self) -> Hash256;
fn into_block_wrapper(self) -> BlockWrapper<E>;
}
impl<E: EthSpec> AsBlock<E> for MaybeAvailableBlock<E> {
fn slot(&self) -> Slot {
self.as_block().slot()
}
fn epoch(&self) -> Epoch {
self.as_block().epoch()
}
fn parent_root(&self) -> Hash256 {
self.as_block().parent_root()
}
fn state_root(&self) -> Hash256 {
self.as_block().state_root()
}
fn signed_block_header(&self) -> SignedBeaconBlockHeader {
self.as_block().signed_block_header()
}
fn message(&self) -> BeaconBlockRef<E> {
self.as_block().message()
}
fn as_block(&self) -> &SignedBeaconBlock<E> {
match &self {
MaybeAvailableBlock::Available(block) => block.as_block(),
MaybeAvailableBlock::AvailabilityPending(block) => block.as_block(),
}
}
fn block_cloned(&self) -> Arc<SignedBeaconBlock<E>> {
match &self {
MaybeAvailableBlock::Available(block) => block.block_cloned(),
MaybeAvailableBlock::AvailabilityPending(block) => block.block_cloned(),
}
}
fn canonical_root(&self) -> Hash256 {
self.as_block().canonical_root()
}
fn into_block_wrapper(self) -> BlockWrapper<E> {
match self {
MaybeAvailableBlock::Available(available_block) => available_block.into_block_wrapper(),
MaybeAvailableBlock::AvailabilityPending(pending_block) => {
BlockWrapper::Block(pending_block.to_block())
}
}
}
}
impl<E: EthSpec> AsBlock<E> for &MaybeAvailableBlock<E> {
fn slot(&self) -> Slot {
self.as_block().slot()
}
fn epoch(&self) -> Epoch {
self.as_block().epoch()
}
fn parent_root(&self) -> Hash256 {
self.as_block().parent_root()
}
fn state_root(&self) -> Hash256 {
self.as_block().state_root()
}
fn signed_block_header(&self) -> SignedBeaconBlockHeader {
self.as_block().signed_block_header()
}
fn message(&self) -> BeaconBlockRef<E> {
self.as_block().message()
}
fn as_block(&self) -> &SignedBeaconBlock<E> {
match &self {
MaybeAvailableBlock::Available(block) => block.as_block(),
MaybeAvailableBlock::AvailabilityPending(block) => block.as_block(),
}
}
fn block_cloned(&self) -> Arc<SignedBeaconBlock<E>> {
match &self {
MaybeAvailableBlock::Available(block) => block.block_cloned(),
MaybeAvailableBlock::AvailabilityPending(block) => block.block_cloned(),
}
}
fn canonical_root(&self) -> Hash256 {
self.as_block().canonical_root()
}
fn into_block_wrapper(self) -> BlockWrapper<E> {
self.clone().into_block_wrapper()
}
}
#[derive(Debug, Clone, Derivative)]
#[derivative(Hash(bound = "E: EthSpec"))]
pub enum BlockWrapper<E: EthSpec> {
Block(Arc<SignedBeaconBlock<E>>),
BlockAndBlobs(Arc<SignedBeaconBlock<E>>, BlobSidecarList<E>),
}
impl<E: EthSpec> AsBlock<E> for BlockWrapper<E> {
fn slot(&self) -> Slot {
match self {
BlockWrapper::Block(block) => block.slot(),
BlockWrapper::BlockAndBlob(block, _) => block.slot(),
}
self.as_block().slot()
}
fn epoch(&self) -> Epoch {
match self {
BlockWrapper::Block(block) => block.epoch(),
BlockWrapper::BlockAndBlob(block, _) => block.epoch(),
}
self.as_block().epoch()
}
fn parent_root(&self) -> Hash256 {
match self {
BlockWrapper::Block(block) => block.parent_root(),
BlockWrapper::BlockAndBlob(block, _) => block.parent_root(),
}
self.as_block().parent_root()
}
fn state_root(&self) -> Hash256 {
match self {
BlockWrapper::Block(block) => block.state_root(),
BlockWrapper::BlockAndBlob(block, _) => block.state_root(),
}
self.as_block().state_root()
}
fn signed_block_header(&self) -> SignedBeaconBlockHeader {
match &self {
BlockWrapper::Block(block) => block.signed_block_header(),
BlockWrapper::BlockAndBlob(block, _) => block.signed_block_header(),
}
self.as_block().signed_block_header()
}
fn message(&self) -> BeaconBlockRef<E> {
match &self {
BlockWrapper::Block(block) => block.message(),
BlockWrapper::BlockAndBlob(block, _) => block.message(),
}
self.as_block().message()
}
fn as_block(&self) -> &SignedBeaconBlock<E> {
match &self {
BlockWrapper::Block(block) => block,
BlockWrapper::BlockAndBlob(block, _) => block,
BlockWrapper::BlockAndBlobs(block, _) => block,
}
}
fn block_cloned(&self) -> Arc<SignedBeaconBlock<E>> {
match &self {
BlockWrapper::Block(block) => block.clone(),
BlockWrapper::BlockAndBlob(block, _) => block.clone(),
BlockWrapper::BlockAndBlobs(block, _) => block.clone(),
}
}
fn canonical_root(&self) -> Hash256 {
match &self {
BlockWrapper::Block(block) => block.canonical_root(),
BlockWrapper::BlockAndBlob(block, _) => block.canonical_root(),
}
self.as_block().canonical_root()
}
fn into_block_wrapper(self) -> BlockWrapper<E> {
self
}
}
impl<E: EthSpec> AsBlock<E> for &BlockWrapper<E> {
fn slot(&self) -> Slot {
match self {
BlockWrapper::Block(block) => block.slot(),
BlockWrapper::BlockAndBlob(block, _) => block.slot(),
}
}
fn epoch(&self) -> Epoch {
match self {
BlockWrapper::Block(block) => block.epoch(),
BlockWrapper::BlockAndBlob(block, _) => block.epoch(),
}
}
fn parent_root(&self) -> Hash256 {
match self {
BlockWrapper::Block(block) => block.parent_root(),
BlockWrapper::BlockAndBlob(block, _) => block.parent_root(),
}
}
fn state_root(&self) -> Hash256 {
match self {
BlockWrapper::Block(block) => block.state_root(),
BlockWrapper::BlockAndBlob(block, _) => block.state_root(),
}
}
fn signed_block_header(&self) -> SignedBeaconBlockHeader {
match &self {
BlockWrapper::Block(block) => block.signed_block_header(),
BlockWrapper::BlockAndBlob(block, _) => block.signed_block_header(),
}
}
fn message(&self) -> BeaconBlockRef<E> {
match &self {
BlockWrapper::Block(block) => block.message(),
BlockWrapper::BlockAndBlob(block, _) => block.message(),
}
}
fn as_block(&self) -> &SignedBeaconBlock<E> {
match &self {
BlockWrapper::Block(block) => block,
BlockWrapper::BlockAndBlob(block, _) => block,
}
}
fn block_cloned(&self) -> Arc<SignedBeaconBlock<E>> {
match &self {
BlockWrapper::Block(block) => block.clone(),
BlockWrapper::BlockAndBlob(block, _) => block.clone(),
}
}
fn canonical_root(&self) -> Hash256 {
match &self {
BlockWrapper::Block(block) => block.canonical_root(),
BlockWrapper::BlockAndBlob(block, _) => block.canonical_root(),
}
impl<E: EthSpec> From<Arc<SignedBeaconBlock<E>>> for BlockWrapper<E> {
fn from(value: Arc<SignedBeaconBlock<E>>) -> Self {
Self::Block(value)
}
}
impl<E: EthSpec> AsBlock<E> for AvailableBlock<E> {
fn slot(&self) -> Slot {
match &self.0 {
AvailableBlockInner::Block(block) => block.slot(),
AvailableBlockInner::BlockAndBlob(block_sidecar_pair) => {
block_sidecar_pair.beacon_block.slot()
}
}
}
fn epoch(&self) -> Epoch {
match &self.0 {
AvailableBlockInner::Block(block) => block.epoch(),
AvailableBlockInner::BlockAndBlob(block_sidecar_pair) => {
block_sidecar_pair.beacon_block.epoch()
}
}
}
fn parent_root(&self) -> Hash256 {
match &self.0 {
AvailableBlockInner::Block(block) => block.parent_root(),
AvailableBlockInner::BlockAndBlob(block_sidecar_pair) => {
block_sidecar_pair.beacon_block.parent_root()
}
}
}
fn state_root(&self) -> Hash256 {
match &self.0 {
AvailableBlockInner::Block(block) => block.state_root(),
AvailableBlockInner::BlockAndBlob(block_sidecar_pair) => {
block_sidecar_pair.beacon_block.state_root()
}
}
}
fn signed_block_header(&self) -> SignedBeaconBlockHeader {
match &self.0 {
AvailableBlockInner::Block(block) => block.signed_block_header(),
AvailableBlockInner::BlockAndBlob(block_sidecar_pair) => {
block_sidecar_pair.beacon_block.signed_block_header()
}
}
}
fn message(&self) -> BeaconBlockRef<E> {
match &self.0 {
AvailableBlockInner::Block(block) => block.message(),
AvailableBlockInner::BlockAndBlob(block_sidecar_pair) => {
block_sidecar_pair.beacon_block.message()
}
}
}
fn as_block(&self) -> &SignedBeaconBlock<E> {
match &self.0 {
AvailableBlockInner::Block(block) => block,
AvailableBlockInner::BlockAndBlob(block_sidecar_pair) => {
&block_sidecar_pair.beacon_block
}
}
}
fn block_cloned(&self) -> Arc<SignedBeaconBlock<E>> {
match &self.0 {
AvailableBlockInner::Block(block) => block.clone(),
AvailableBlockInner::BlockAndBlob(block_sidecar_pair) => {
block_sidecar_pair.beacon_block.clone()
}
}
}
fn canonical_root(&self) -> Hash256 {
match &self.0 {
AvailableBlockInner::Block(block) => block.canonical_root(),
AvailableBlockInner::BlockAndBlob(block_sidecar_pair) => {
block_sidecar_pair.beacon_block.canonical_root()
}
}
impl<E: EthSpec> From<SignedBeaconBlock<E>> for BlockWrapper<E> {
fn from(value: SignedBeaconBlock<E>) -> Self {
Self::Block(Arc::new(value))
}
}

View File

@ -24,9 +24,6 @@
//! ▼
//! SignedBeaconBlock
//! |
//! ▼
//! AvailableBlock
//! |
//! |---------------
//! | |
//! | ▼
@ -51,9 +48,9 @@
// returned alongside.
#![allow(clippy::result_large_err)]
use crate::blob_verification::{
validate_blob_for_gossip, AsBlock, AvailableBlock, BlobError, BlockWrapper, IntoAvailableBlock,
IntoBlockWrapper,
use crate::blob_verification::{AsBlock, BlobError, BlockWrapper, MaybeAvailableBlock};
use crate::data_availability_checker::{
AvailabilityCheckError, AvailabilityPendingBlock, AvailableBlock,
};
use crate::eth1_finalization_cache::Eth1FinalizationData;
use crate::execution_payload::{
@ -96,7 +93,6 @@ use std::time::Duration;
use store::{Error as DBError, HotStateSummary, KeyValueStore, StoreOp};
use task_executor::JoinHandle;
use tree_hash::TreeHash;
use types::signed_beacon_block::BlobReconstructionError;
use types::ExecPayload;
use types::{
BeaconBlockRef, BeaconState, BeaconStateError, BlindedPayload, ChainSpec, CloneConfig, Epoch,
@ -314,6 +310,7 @@ pub enum BlockError<T: EthSpec> {
parent_root: Hash256,
},
BlobValidation(BlobError),
AvailabilityCheck(AvailabilityCheckError),
}
impl<T: EthSpec> From<BlobError> for BlockError<T> {
@ -322,6 +319,12 @@ impl<T: EthSpec> From<BlobError> for BlockError<T> {
}
}
impl<T: EthSpec> From<AvailabilityCheckError> for BlockError<T> {
fn from(e: AvailabilityCheckError) -> Self {
Self::AvailabilityCheck(e)
}
}
/// Returned when block validation failed due to some issue verifying
/// the execution payload.
#[derive(Debug)]
@ -494,13 +497,8 @@ impl<T: EthSpec> From<ArithError> for BlockError<T> {
}
}
impl<T: EthSpec> From<BlobReconstructionError> for BlockError<T> {
fn from(e: BlobReconstructionError) -> Self {
BlockError::BlobValidation(BlobError::from(e))
}
}
/// Stores information about verifying a payload against an execution engine.
#[derive(Clone)]
pub struct PayloadVerificationOutcome {
pub payload_verification_status: PayloadVerificationStatus,
pub is_valid_merge_transition_block: bool,
@ -605,14 +603,14 @@ pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
signature_verifier.include_all_signatures(block.as_block(), &mut consensus_context)?;
//FIXME(sean) batch kzg verification
let available_block = block.clone().into_available_block(*block_root, chain)?;
consensus_context = consensus_context.set_kzg_commitments_consistent(true);
let maybe_available_block = chain
.data_availability_checker
.check_availability(block.clone())?;
// Save the block and its consensus context. The context will have had its proposer index
// and attesting indices filled in, which can be used to accelerate later block processing.
signature_verified_blocks.push(SignatureVerifiedBlock {
block: available_block,
block: maybe_available_block,
block_root: *block_root,
parent: None,
consensus_context,
@ -637,7 +635,7 @@ pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
#[derive(Derivative)]
#[derivative(Debug(bound = "T: BeaconChainTypes"))]
pub struct GossipVerifiedBlock<T: BeaconChainTypes> {
pub block: AvailableBlock<T::EthSpec>,
pub block: MaybeAvailableBlock<T::EthSpec>,
pub block_root: Hash256,
parent: Option<PreProcessingSnapshot<T::EthSpec>>,
consensus_context: ConsensusContext<T::EthSpec>,
@ -646,7 +644,7 @@ pub struct GossipVerifiedBlock<T: BeaconChainTypes> {
/// A wrapper around a `SignedBeaconBlock` that indicates that all signatures (except the deposit
/// signatures) have been verified.
pub struct SignatureVerifiedBlock<T: BeaconChainTypes> {
block: AvailableBlock<T::EthSpec>,
block: MaybeAvailableBlock<T::EthSpec>,
block_root: Hash256,
parent: Option<PreProcessingSnapshot<T::EthSpec>>,
consensus_context: ConsensusContext<T::EthSpec>,
@ -669,14 +667,103 @@ type PayloadVerificationHandle<E> =
/// due to finality or some other event. A `ExecutionPendingBlock` should be imported into the
/// `BeaconChain` immediately after it is instantiated.
pub struct ExecutionPendingBlock<T: BeaconChainTypes> {
pub block: AvailableBlock<T::EthSpec>,
pub block: MaybeAvailableBlock<T::EthSpec>,
pub import_data: BlockImportData<T::EthSpec>,
pub payload_verification_handle: PayloadVerificationHandle<T::EthSpec>,
}
pub enum ExecutedBlock<E: EthSpec> {
Available(AvailableExecutedBlock<E>),
AvailabilityPending(AvailabilityPendingExecutedBlock<E>),
}
impl<E: EthSpec> ExecutedBlock<E> {
pub fn as_block(&self) -> &SignedBeaconBlock<E> {
match self {
Self::Available(available) => available.block.block(),
Self::AvailabilityPending(pending) => pending.block.as_block(),
}
}
}
impl<E: EthSpec> std::fmt::Debug for ExecutedBlock<E> {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(f, "{:?}", self.as_block())
}
}
impl<E: EthSpec> ExecutedBlock<E> {
pub fn new(
block: MaybeAvailableBlock<E>,
import_data: BlockImportData<E>,
payload_verification_outcome: PayloadVerificationOutcome,
) -> Self {
match block {
MaybeAvailableBlock::Available(available_block) => {
Self::Available(AvailableExecutedBlock::new(
available_block,
import_data,
payload_verification_outcome,
))
}
MaybeAvailableBlock::AvailabilityPending(pending_block) => {
Self::AvailabilityPending(AvailabilityPendingExecutedBlock::new(
pending_block,
import_data,
payload_verification_outcome,
))
}
}
}
}
pub struct AvailableExecutedBlock<E: EthSpec> {
pub block: AvailableBlock<E>,
pub import_data: BlockImportData<E>,
pub payload_verification_outcome: PayloadVerificationOutcome,
}
impl<E: EthSpec> AvailableExecutedBlock<E> {
pub fn new(
block: AvailableBlock<E>,
import_data: BlockImportData<E>,
payload_verification_outcome: PayloadVerificationOutcome,
) -> Self {
Self {
block,
import_data,
payload_verification_outcome,
}
}
}
pub struct AvailabilityPendingExecutedBlock<E: EthSpec> {
pub block: AvailabilityPendingBlock<E>,
pub import_data: BlockImportData<E>,
pub payload_verification_outcome: PayloadVerificationOutcome,
}
impl<E: EthSpec> AvailabilityPendingExecutedBlock<E> {
pub fn new(
block: AvailabilityPendingBlock<E>,
import_data: BlockImportData<E>,
payload_verification_outcome: PayloadVerificationOutcome,
) -> Self {
Self {
block,
import_data,
payload_verification_outcome,
}
}
}
pub struct BlockImportData<E: EthSpec> {
pub block_root: Hash256,
pub state: BeaconState<T::EthSpec>,
pub parent_block: SignedBeaconBlock<T::EthSpec, BlindedPayload<T::EthSpec>>,
pub state: BeaconState<E>,
pub parent_block: SignedBeaconBlock<E, BlindedPayload<E>>,
pub parent_eth1_finalization_data: Eth1FinalizationData,
pub confirmed_state_roots: Vec<Hash256>,
pub consensus_context: ConsensusContext<T::EthSpec>,
pub payload_verification_handle: PayloadVerificationHandle<T::EthSpec>,
pub consensus_context: ConsensusContext<E>,
}
/// Implemented on types that can be converted into a `ExecutionPendingBlock`.
@ -720,19 +807,20 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
block: BlockWrapper<T::EthSpec>,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError<T::EthSpec>> {
let maybe_available = chain.data_availability_checker.check_availability(block)?;
// If the block is valid for gossip we don't supply it to the slasher here because
// we assume it will be transformed into a fully verified block. We *do* need to supply
// it to the slasher if an error occurs, because that's the end of this block's journey,
// and it could be a repeat proposal (a likely cause for slashing!).
let header = block.signed_block_header();
Self::new_without_slasher_checks(block, chain).map_err(|e| {
let header = maybe_available.signed_block_header();
Self::new_without_slasher_checks(maybe_available, chain).map_err(|e| {
process_block_slash_info(chain, BlockSlashInfo::from_early_error(header, e))
})
}
/// As for new, but doesn't pass the block to the slasher.
fn new_without_slasher_checks(
block: BlockWrapper<T::EthSpec>,
block: MaybeAvailableBlock<T::EthSpec>,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError<T::EthSpec>> {
// Ensure the block is the correct structure for the fork at `block.slot()`.
@ -929,16 +1017,14 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
// Validate the block's execution_payload (if any).
validate_execution_payload_for_gossip(&parent_block, block.message(), chain)?;
let available_block = validate_blob_for_gossip(block, block_root, chain)?;
// Having checked the proposer index and the block root we can cache them.
let consensus_context = ConsensusContext::new(available_block.slot())
let consensus_context = ConsensusContext::new(block.slot())
.set_current_block_root(block_root)
.set_proposer_index(available_block.as_block().message().proposer_index())
.set_proposer_index(block.as_block().message().proposer_index())
.set_kzg_commitments_consistent(true);
Ok(Self {
block: available_block,
block,
block_root,
parent,
consensus_context,
@ -978,10 +1064,11 @@ impl<T: BeaconChainTypes> SignatureVerifiedBlock<T> {
///
/// Returns an error if the block is invalid, or if the block was unable to be verified.
pub fn new(
block: AvailableBlock<T::EthSpec>,
block: BlockWrapper<T::EthSpec>,
block_root: Hash256,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError<T::EthSpec>> {
let block = chain.data_availability_checker.check_availability(block)?;
// Ensure the block is the correct structure for the fork at `block.slot()`.
block
.as_block()
@ -1028,7 +1115,7 @@ impl<T: BeaconChainTypes> SignatureVerifiedBlock<T> {
/// As for `new` above but producing `BlockSlashInfo`.
pub fn check_slashable(
block: AvailableBlock<T::EthSpec>,
block: BlockWrapper<T::EthSpec>,
block_root: Hash256,
chain: &BeaconChain<T>,
) -> Result<Self, BlockSlashInfo<BlockError<T::EthSpec>>> {
@ -1137,12 +1224,7 @@ impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for Arc<SignedBeaconBlock
let block_root = check_block_relevancy(&self, block_root, chain)
.map_err(|e| BlockSlashInfo::SignatureNotChecked(self.signed_block_header(), e))?;
let header = self.signed_block_header();
let available_block = BlockWrapper::from(self)
.into_available_block(block_root, chain)
.map_err(|e| BlockSlashInfo::from_early_error(header, BlockError::BlobValidation(e)))?;
SignatureVerifiedBlock::check_slashable(available_block, block_root, chain)?
SignatureVerifiedBlock::check_slashable(self.into(), block_root, chain)?
.into_execution_pending_block_slashable(block_root, chain, notify_execution_layer)
}
@ -1151,7 +1233,7 @@ impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for Arc<SignedBeaconBlock
}
}
impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for AvailableBlock<T::EthSpec> {
impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for BlockWrapper<T::EthSpec> {
/// Verifies the `SignedBeaconBlock` by first transforming it into a `SignatureVerifiedBlock`
/// and then using that implementation of `IntoExecutionPendingBlock` to complete verification.
fn into_execution_pending_block_slashable(
@ -1182,7 +1264,7 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
///
/// Returns an error if the block is invalid, or if the block was unable to be verified.
pub fn from_signature_verified_components(
block: AvailableBlock<T::EthSpec>,
block: MaybeAvailableBlock<T::EthSpec>,
block_root: Hash256,
parent: PreProcessingSnapshot<T::EthSpec>,
mut consensus_context: ConsensusContext<T::EthSpec>,
@ -1562,12 +1644,14 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
Ok(Self {
block,
block_root,
state,
parent_block: parent.beacon_block,
parent_eth1_finalization_data,
confirmed_state_roots,
consensus_context,
import_data: BlockImportData {
block_root,
state,
parent_block: parent.beacon_block,
parent_eth1_finalization_data,
confirmed_state_roots,
consensus_context,
},
payload_verification_handle,
})
}
@ -1648,7 +1732,7 @@ fn check_block_against_finalized_slot<T: BeaconChainTypes>(
/// Taking a lock on the `chain.canonical_head.fork_choice` might cause a deadlock here.
pub fn check_block_is_finalized_checkpoint_or_descendant<
T: BeaconChainTypes,
B: IntoBlockWrapper<T::EthSpec>,
B: AsBlock<T::EthSpec>,
>(
chain: &BeaconChain<T>,
fork_choice: &BeaconForkChoice<T>,
@ -1746,8 +1830,8 @@ pub fn get_block_root<E: EthSpec>(block: &SignedBeaconBlock<E>) -> Hash256 {
#[allow(clippy::type_complexity)]
fn verify_parent_block_is_known<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
block: BlockWrapper<T::EthSpec>,
) -> Result<(ProtoBlock, BlockWrapper<T::EthSpec>), BlockError<T::EthSpec>> {
block: MaybeAvailableBlock<T::EthSpec>,
) -> Result<(ProtoBlock, MaybeAvailableBlock<T::EthSpec>), BlockError<T::EthSpec>> {
if let Some(proto_block) = chain
.canonical_head
.fork_choice_read_lock()
@ -1755,7 +1839,7 @@ fn verify_parent_block_is_known<T: BeaconChainTypes>(
{
Ok((proto_block, block))
} else {
Err(BlockError::ParentUnknown(block))
Err(BlockError::ParentUnknown(block.into_block_wrapper()))
}
}
@ -1764,7 +1848,7 @@ fn verify_parent_block_is_known<T: BeaconChainTypes>(
/// Returns `Err(BlockError::ParentUnknown)` if the parent is not found, or if an error occurs
/// whilst attempting the operation.
#[allow(clippy::type_complexity)]
fn load_parent<T: BeaconChainTypes, B: IntoBlockWrapper<T::EthSpec>>(
fn load_parent<T: BeaconChainTypes, B: AsBlock<T::EthSpec>>(
block_root: Hash256,
block: B,
chain: &BeaconChain<T>,

View File

@ -1,5 +1,6 @@
use crate::beacon_chain::{CanonicalHead, BEACON_CHAIN_DB_KEY, ETH1_CACHE_DB_KEY, OP_POOL_DB_KEY};
use crate::blob_cache::BlobCache;
use crate::data_availability_checker::DataAvailabilityChecker;
use crate::eth1_chain::{CachingEth1Backend, SszEth1};
use crate::eth1_finalization_cache::Eth1FinalizationCache;
use crate::fork_choice_signal::ForkChoiceSignalTx;
@ -641,7 +642,8 @@ where
let kzg = if let Some(trusted_setup) = self.trusted_setup {
let kzg = Kzg::new_from_trusted_setup(trusted_setup)
.map_err(|e| format!("Failed to load trusted setup: {:?}", e))?;
Some(Arc::new(kzg))
let kzg_arc = Arc::new(kzg);
Some(kzg_arc)
} else {
None
};
@ -781,14 +783,14 @@ where
let shuffling_cache_size = self.chain_config.shuffling_cache_size;
let beacon_chain = BeaconChain {
spec: self.spec,
spec: self.spec.clone(),
config: self.chain_config,
store,
task_executor: self
.task_executor
.ok_or("Cannot build without task executor")?,
store_migrator,
slot_clock,
slot_clock: slot_clock.clone(),
op_pool: self.op_pool.ok_or("Cannot build without op pool")?,
// TODO: allow for persisting and loading the pool from disk.
naive_aggregation_pool: <_>::default(),
@ -847,7 +849,13 @@ where
graffiti: self.graffiti,
slasher: self.slasher.clone(),
validator_monitor: RwLock::new(validator_monitor),
blob_cache: BlobCache::default(),
//TODO(sean) should we move kzg solely to the da checker?
data_availability_checker: DataAvailabilityChecker::new(
slot_clock,
kzg.clone(),
self.spec,
),
proposal_blob_cache: BlobCache::default(),
kzg,
};

View File

@ -0,0 +1,516 @@
use crate::blob_verification::{
verify_kzg_for_blob, verify_kzg_for_blob_list, AsBlock, BlockWrapper, GossipVerifiedBlob,
KzgVerifiedBlob, KzgVerifiedBlobList, MaybeAvailableBlock,
};
use crate::block_verification::{AvailabilityPendingExecutedBlock, AvailableExecutedBlock};
use kzg::Error as KzgError;
use kzg::Kzg;
use parking_lot::{Mutex, RwLock};
use slot_clock::SlotClock;
use ssz_types::{Error, VariableList};
use state_processing::per_block_processing::eip4844::eip4844::verify_kzg_commitments_against_transactions;
use std::collections::hash_map::Entry;
use std::collections::HashMap;
use std::sync::Arc;
use types::beacon_block_body::KzgCommitments;
use types::blob_sidecar::{BlobIdentifier, BlobSidecar};
use types::consts::eip4844::MIN_EPOCHS_FOR_BLOBS_SIDECARS_REQUESTS;
use types::{
BeaconBlockRef, BlobSidecarList, ChainSpec, Epoch, EthSpec, ExecPayload, FullPayload, Hash256,
SignedBeaconBlock, SignedBeaconBlockHeader, Slot,
};
#[derive(Debug)]
pub enum AvailabilityCheckError {
DuplicateBlob(Hash256),
Kzg(KzgError),
KzgVerificationFailed,
KzgNotInitialized,
SszTypes(ssz_types::Error),
MissingBlobs,
NumBlobsMismatch {
num_kzg_commitments: usize,
num_blobs: usize,
},
TxKzgCommitmentMismatch,
KzgCommitmentMismatch {
blob_index: u64,
},
Pending,
IncorrectFork,
}
impl From<ssz_types::Error> for AvailabilityCheckError {
fn from(value: Error) -> Self {
Self::SszTypes(value)
}
}
/// This cache contains
/// - blobs that have been gossip verified
/// - commitments for blocks that have been gossip verified, but the commitments themselves
/// have not been verified against blobs
/// - blocks that have been fully verified and only require a data availability check
pub struct DataAvailabilityChecker<T: EthSpec, S: SlotClock> {
rpc_blob_cache: RwLock<HashMap<BlobIdentifier, Arc<BlobSidecar<T>>>>,
gossip_blob_cache: Mutex<HashMap<Hash256, GossipBlobCache<T>>>,
slot_clock: S,
kzg: Option<Arc<Kzg>>,
spec: ChainSpec,
}
struct GossipBlobCache<T: EthSpec> {
verified_blobs: Vec<KzgVerifiedBlob<T>>,
executed_block: Option<AvailabilityPendingExecutedBlock<T>>,
}
pub enum Availability<T: EthSpec> {
PendingBlobs(Vec<BlobIdentifier>),
PendingBlock(Hash256),
Available(Box<AvailableExecutedBlock<T>>),
}
impl<T: EthSpec, S: SlotClock> DataAvailabilityChecker<T, S> {
pub fn new(slot_clock: S, kzg: Option<Arc<Kzg>>, spec: ChainSpec) -> Self {
Self {
rpc_blob_cache: <_>::default(),
gossip_blob_cache: <_>::default(),
slot_clock,
kzg,
spec,
}
}
/// This first validate the KZG commitments included in the blob sidecar.
/// Check if we've cached other blobs for this block. If it completes a set and we also
/// have a block cached, return the Availability variant triggering block import.
/// Otherwise cache the blob sidecar.
///
/// This should only accept gossip verified blobs, so we should not have to worry about dupes.
pub fn put_gossip_blob(
&self,
verified_blob: GossipVerifiedBlob<T>,
) -> Result<Availability<T>, AvailabilityCheckError> {
let block_root = verified_blob.block_root();
let kzg_verified_blob = if let Some(kzg) = self.kzg.as_ref() {
verify_kzg_for_blob(verified_blob, kzg)?
} else {
return Err(AvailabilityCheckError::KzgNotInitialized);
};
//TODO(sean) can we just use a referece to the blob here?
let blob = kzg_verified_blob.clone_blob();
// check if we have a block
// check if the complete set matches the block
// verify, otherwise cache
let mut blob_cache = self.gossip_blob_cache.lock();
// Gossip cache.
let availability = match blob_cache.entry(blob.block_root) {
Entry::Occupied(mut occupied_entry) => {
// All blobs reaching this cache should be gossip verified and gossip verification
// should filter duplicates, as well as validate indices.
let cache = occupied_entry.get_mut();
cache
.verified_blobs
.insert(blob.index as usize, kzg_verified_blob);
if let Some(executed_block) = cache.executed_block.take() {
self.check_block_availability_or_cache(cache, executed_block)?
} else {
Availability::PendingBlock(block_root)
}
}
Entry::Vacant(vacant_entry) => {
let block_root = kzg_verified_blob.block_root();
vacant_entry.insert(GossipBlobCache {
verified_blobs: vec![kzg_verified_blob],
executed_block: None,
});
Availability::PendingBlock(block_root)
}
};
drop(blob_cache);
// RPC cache.
self.rpc_blob_cache.write().insert(blob.id(), blob.clone());
Ok(availability)
}
/// Check if we have all the blobs for a block. If we do, return the Availability variant that
/// triggers import of the block.
pub fn put_pending_executed_block(
&self,
executed_block: AvailabilityPendingExecutedBlock<T>,
) -> Result<Availability<T>, AvailabilityCheckError> {
let mut guard = self.gossip_blob_cache.lock();
let entry = guard.entry(executed_block.import_data.block_root);
let availability = match entry {
Entry::Occupied(mut occupied_entry) => {
let cache: &mut GossipBlobCache<T> = occupied_entry.get_mut();
self.check_block_availability_or_cache(cache, executed_block)?
}
Entry::Vacant(vacant_entry) => {
let kzg_commitments_len = executed_block.block.kzg_commitments()?.len();
let mut blob_ids = Vec::with_capacity(kzg_commitments_len);
for i in 0..kzg_commitments_len {
blob_ids.push(BlobIdentifier {
block_root: executed_block.import_data.block_root,
index: i as u64,
});
}
vacant_entry.insert(GossipBlobCache {
verified_blobs: vec![],
executed_block: Some(executed_block),
});
Availability::PendingBlobs(blob_ids)
}
};
Ok(availability)
}
fn check_block_availability_or_cache(
&self,
cache: &mut GossipBlobCache<T>,
executed_block: AvailabilityPendingExecutedBlock<T>,
) -> Result<Availability<T>, AvailabilityCheckError> {
let AvailabilityPendingExecutedBlock {
block,
import_data,
payload_verification_outcome,
} = executed_block;
let kzg_commitments_len = block.kzg_commitments()?.len();
let verified_commitments_len = cache.verified_blobs.len();
if kzg_commitments_len == verified_commitments_len {
//TODO(sean) can we remove this clone
let blobs = cache.verified_blobs.clone();
let available_block = self.make_available(block, blobs)?;
Ok(Availability::Available(Box::new(
AvailableExecutedBlock::new(
available_block,
import_data,
payload_verification_outcome,
),
)))
} else {
let mut missing_blobs = Vec::with_capacity(kzg_commitments_len);
for i in 0..kzg_commitments_len {
if cache.verified_blobs.get(i).is_none() {
missing_blobs.push(BlobIdentifier {
block_root: import_data.block_root,
index: i as u64,
})
}
}
let _ = cache
.executed_block
.insert(AvailabilityPendingExecutedBlock::new(
block,
import_data,
payload_verification_outcome,
));
Ok(Availability::PendingBlobs(missing_blobs))
}
}
/// Checks if a block is available, returns a MaybeAvailableBlock enum that may include the fully
/// available block.
pub fn check_availability(
&self,
block: BlockWrapper<T>,
) -> Result<MaybeAvailableBlock<T>, AvailabilityCheckError> {
match block {
BlockWrapper::Block(block) => self.check_availability_without_blobs(block),
BlockWrapper::BlockAndBlobs(block, blob_list) => {
let kzg = self
.kzg
.as_ref()
.ok_or(AvailabilityCheckError::KzgNotInitialized)?;
let verified_blobs = verify_kzg_for_blob_list(blob_list, kzg)?;
Ok(MaybeAvailableBlock::Available(
self.check_availability_with_blobs(block, verified_blobs)?,
))
}
}
}
/// Checks if a block is available, returning an error if the block is not immediately available.
/// Does not access the gossip cache.
pub fn try_check_availability(
&self,
block: BlockWrapper<T>,
) -> Result<AvailableBlock<T>, AvailabilityCheckError> {
match block {
BlockWrapper::Block(block) => {
let blob_requirements = self.get_blob_requirements(&block)?;
let blobs = match blob_requirements {
BlobRequirements::EmptyBlobs => VerifiedBlobs::EmptyBlobs,
BlobRequirements::NotRequired => VerifiedBlobs::NotRequired,
BlobRequirements::PreEip4844 => VerifiedBlobs::PreEip4844,
BlobRequirements::Required => return Err(AvailabilityCheckError::MissingBlobs),
};
Ok(AvailableBlock { block, blobs })
}
BlockWrapper::BlockAndBlobs(_, _) => Err(AvailabilityCheckError::Pending),
}
}
/// Verifies a block against a set of KZG verified blobs. Returns an AvailableBlock if block's
/// commitments are consistent with the provided verified blob commitments.
pub fn check_availability_with_blobs(
&self,
block: Arc<SignedBeaconBlock<T>>,
blobs: KzgVerifiedBlobList<T>,
) -> Result<AvailableBlock<T>, AvailabilityCheckError> {
match self.check_availability_without_blobs(block)? {
MaybeAvailableBlock::Available(block) => Ok(block),
MaybeAvailableBlock::AvailabilityPending(pending_block) => {
self.make_available(pending_block, blobs)
}
}
}
/// Verifies a block as much as possible, returning a MaybeAvailableBlock enum that may include
/// an AvailableBlock if no blobs are required. Otherwise this will return an AvailabilityPendingBlock.
pub fn check_availability_without_blobs(
&self,
block: Arc<SignedBeaconBlock<T>>,
) -> Result<MaybeAvailableBlock<T>, AvailabilityCheckError> {
let blob_requirements = self.get_blob_requirements(&block)?;
let blobs = match blob_requirements {
BlobRequirements::EmptyBlobs => VerifiedBlobs::EmptyBlobs,
BlobRequirements::NotRequired => VerifiedBlobs::NotRequired,
BlobRequirements::PreEip4844 => VerifiedBlobs::PreEip4844,
BlobRequirements::Required => {
return Ok(MaybeAvailableBlock::AvailabilityPending(
AvailabilityPendingBlock { block },
))
}
};
Ok(MaybeAvailableBlock::Available(AvailableBlock {
block,
blobs,
}))
}
/// Verifies an AvailabilityPendingBlock against a set of KZG verified blobs.
/// This does not check whether a block *should* have blobs, these checks should must have been
/// completed when producing the AvailabilityPendingBlock.
pub fn make_available(
&self,
block: AvailabilityPendingBlock<T>,
blobs: KzgVerifiedBlobList<T>,
) -> Result<AvailableBlock<T>, AvailabilityCheckError> {
let block_kzg_commitments = block.kzg_commitments()?;
if blobs.len() != block_kzg_commitments.len() {
return Err(AvailabilityCheckError::NumBlobsMismatch {
num_kzg_commitments: block_kzg_commitments.len(),
num_blobs: blobs.len(),
});
}
for (block_commitment, blob) in block_kzg_commitments.iter().zip(blobs.iter()) {
if *block_commitment != blob.kzg_commitment() {
return Err(AvailabilityCheckError::KzgCommitmentMismatch {
blob_index: blob.as_blob().index,
});
}
}
let blobs = VariableList::new(blobs.into_iter().map(|blob| blob.to_blob()).collect())?;
Ok(AvailableBlock {
block: block.block,
blobs: VerifiedBlobs::Available(blobs),
})
}
/// Determines the blob requirements for a block. Answers the question: "Does this block require
/// blobs?".
fn get_blob_requirements(
&self,
block: &Arc<SignedBeaconBlock<T, FullPayload<T>>>,
) -> Result<BlobRequirements, AvailabilityCheckError> {
let verified_blobs = if let (Ok(block_kzg_commitments), Ok(payload)) = (
block.message().body().blob_kzg_commitments(),
block.message().body().execution_payload(),
) {
if let Some(transactions) = payload.transactions() {
let verified = verify_kzg_commitments_against_transactions::<T>(
transactions,
block_kzg_commitments,
)
.map_err(|_| AvailabilityCheckError::TxKzgCommitmentMismatch)?;
if !verified {
return Err(AvailabilityCheckError::TxKzgCommitmentMismatch);
}
}
if self.da_check_required(block.epoch()) {
if block_kzg_commitments.is_empty() {
BlobRequirements::EmptyBlobs
} else {
BlobRequirements::Required
}
} else {
BlobRequirements::NotRequired
}
} else {
BlobRequirements::PreEip4844
};
Ok(verified_blobs)
}
/// The epoch at which we require a data availability check in block processing.
/// `None` if the `Eip4844` fork is disabled.
pub fn data_availability_boundary(&self) -> Option<Epoch> {
self.spec.eip4844_fork_epoch.and_then(|fork_epoch| {
self.slot_clock
.now()
.map(|slot| slot.epoch(T::slots_per_epoch()))
.map(|current_epoch| {
std::cmp::max(
fork_epoch,
current_epoch.saturating_sub(*MIN_EPOCHS_FOR_BLOBS_SIDECARS_REQUESTS),
)
})
})
}
/// Returns true if the given epoch lies within the da boundary and false otherwise.
pub fn da_check_required(&self, block_epoch: Epoch) -> bool {
self.data_availability_boundary()
.map_or(false, |da_epoch| block_epoch >= da_epoch)
}
}
pub enum BlobRequirements {
Required,
/// This block is from outside the data availability boundary so doesn't require
/// a data availability check.
NotRequired,
/// The block's `kzg_commitments` field is empty so it does not contain any blobs.
EmptyBlobs,
/// This is a block prior to the 4844 fork, so doesn't require any blobs
PreEip4844,
}
#[derive(Clone, Debug, PartialEq)]
pub struct AvailabilityPendingBlock<E: EthSpec> {
block: Arc<SignedBeaconBlock<E>>,
}
impl<E: EthSpec> AvailabilityPendingBlock<E> {
pub fn to_block(self) -> Arc<SignedBeaconBlock<E>> {
self.block
}
pub fn as_block(&self) -> &SignedBeaconBlock<E> {
&self.block
}
pub fn block_cloned(&self) -> Arc<SignedBeaconBlock<E>> {
self.block.clone()
}
pub fn kzg_commitments(&self) -> Result<&KzgCommitments<E>, AvailabilityCheckError> {
self.block
.message()
.body()
.blob_kzg_commitments()
.map_err(|_| AvailabilityCheckError::IncorrectFork)
}
}
#[derive(Clone, Debug, PartialEq)]
pub struct AvailableBlock<E: EthSpec> {
block: Arc<SignedBeaconBlock<E>>,
blobs: VerifiedBlobs<E>,
}
impl<E: EthSpec> AvailableBlock<E> {
pub fn block(&self) -> &SignedBeaconBlock<E> {
&self.block
}
pub fn deconstruct(self) -> (Arc<SignedBeaconBlock<E>>, Option<BlobSidecarList<E>>) {
match self.blobs {
VerifiedBlobs::EmptyBlobs | VerifiedBlobs::NotRequired | VerifiedBlobs::PreEip4844 => {
(self.block, None)
}
VerifiedBlobs::Available(blobs) => (self.block, Some(blobs)),
}
}
}
#[derive(Clone, Debug, PartialEq)]
pub enum VerifiedBlobs<E: EthSpec> {
/// These blobs are available.
Available(BlobSidecarList<E>),
/// This block is from outside the data availability boundary so doesn't require
/// a data availability check.
NotRequired,
/// The block's `kzg_commitments` field is empty so it does not contain any blobs.
EmptyBlobs,
/// This is a block prior to the 4844 fork, so doesn't require any blobs
PreEip4844,
}
impl<E: EthSpec> AsBlock<E> for AvailableBlock<E> {
fn slot(&self) -> Slot {
self.block.slot()
}
fn epoch(&self) -> Epoch {
self.block.epoch()
}
fn parent_root(&self) -> Hash256 {
self.block.parent_root()
}
fn state_root(&self) -> Hash256 {
self.block.state_root()
}
fn signed_block_header(&self) -> SignedBeaconBlockHeader {
self.block.signed_block_header()
}
fn message(&self) -> BeaconBlockRef<E> {
self.block.message()
}
fn as_block(&self) -> &SignedBeaconBlock<E> {
&self.block
}
fn block_cloned(&self) -> Arc<SignedBeaconBlock<E>> {
self.block.clone()
}
fn canonical_root(&self) -> Hash256 {
self.block.canonical_root()
}
fn into_block_wrapper(self) -> BlockWrapper<E> {
let (block, blobs_opt) = self.deconstruct();
if let Some(blobs) = blobs_opt {
BlockWrapper::BlockAndBlobs(block, blobs)
} else {
BlockWrapper::Block(block)
}
}
}

View File

@ -1,4 +1,4 @@
use crate::blob_verification::AvailableBlock;
use crate::data_availability_checker::AvailableBlock;
use crate::{
attester_cache::{CommitteeLengths, Error},
metrics,
@ -6,6 +6,7 @@ use crate::{
use parking_lot::RwLock;
use proto_array::Block as ProtoBlock;
use std::sync::Arc;
use types::blob_sidecar::BlobSidecarList;
use types::*;
pub struct CacheItem<E: EthSpec> {
@ -21,6 +22,7 @@ pub struct CacheItem<E: EthSpec> {
* Values used to make the block available.
*/
block: Arc<SignedBeaconBlock<E>>,
//TODO(sean) remove this and just use the da checker?'
blobs: Option<BlobSidecarList<E>>,
proto_block: ProtoBlock,
}

View File

@ -16,6 +16,7 @@ pub mod builder;
pub mod canonical_head;
pub mod capella_readiness;
pub mod chain_config;
pub mod data_availability_checker;
mod early_attester_cache;
mod errors;
pub mod eth1_chain;
@ -54,9 +55,10 @@ pub mod validator_monitor;
pub mod validator_pubkey_cache;
pub use self::beacon_chain::{
AttestationProcessingOutcome, BeaconChain, BeaconChainTypes, BeaconStore, ChainSegmentResult,
CountUnrealized, ForkChoiceError, OverrideForkchoiceUpdate, ProduceBlockVerification,
StateSkipConfig, WhenSlotSkipped, INVALID_FINALIZED_MERGE_TRANSITION_BLOCK_SHUTDOWN_REASON,
AttestationProcessingOutcome, AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes,
BeaconStore, ChainSegmentResult, CountUnrealized, ForkChoiceError, OverrideForkchoiceUpdate,
ProduceBlockVerification, StateSkipConfig, WhenSlotSkipped,
INVALID_FINALIZED_MERGE_TRANSITION_BLOCK_SHUTDOWN_REASON,
INVALID_JUSTIFIED_PAYLOAD_SHUTDOWN_REASON, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
};
pub use self::beacon_snapshot::BeaconSnapshot;
@ -66,7 +68,7 @@ pub use self::historical_blocks::HistoricalBlockError;
pub use attestation_verification::Error as AttestationError;
pub use beacon_fork_choice_store::{BeaconForkChoiceStore, Error as ForkChoiceStoreError};
pub use block_verification::{
get_block_root, BlockError, ExecutionPayloadError, GossipVerifiedBlock,
get_block_root, BlockError, ExecutedBlock, ExecutionPayloadError, GossipVerifiedBlock,
};
pub use canonical_head::{CachedHead, CanonicalHead, CanonicalHeadRwLock};
pub use eth1_chain::{Eth1Chain, Eth1ChainBackend};

View File

@ -1680,7 +1680,8 @@ where
NotifyExecutionLayer::Yes,
)
.await?
.into();
.try_into()
.unwrap();
self.chain.recompute_head_at_current_slot().await;
Ok(block_hash)
}
@ -1698,7 +1699,8 @@ where
NotifyExecutionLayer::Yes,
)
.await?
.into();
.try_into()
.unwrap();
self.chain.recompute_head_at_current_slot().await;
Ok(block_hash)
}

View File

@ -1,9 +1,7 @@
#![cfg(not(debug_assertions))]
use beacon_chain::{
blob_verification::{BlockWrapper, IntoAvailableBlock},
test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy},
};
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy};
use beacon_chain::{StateSkipConfig, WhenSlotSkipped};
use lazy_static::lazy_static;
use std::sync::Arc;
@ -135,6 +133,10 @@ async fn produces_attestations() {
assert_eq!(data.target.root, target_root, "bad target root");
let block_wrapper: BlockWrapper<MainnetEthSpec> = Arc::new(block.clone()).into();
let available_block = chain
.data_availability_checker
.try_check_availability(block_wrapper)
.unwrap();
let early_attestation = {
let proto_block = chain
@ -146,9 +148,7 @@ async fn produces_attestations() {
.early_attester_cache
.add_head_block(
block_root,
block_wrapper
.into_available_block(block_root, chain)
.expect("should wrap into available block"),
available_block,
proto_block,
&state,
&chain.spec,
@ -199,18 +199,19 @@ async fn early_attester_cache_old_request() {
.get_block(&head.beacon_block_root)
.unwrap();
let block: BlockWrapper<MainnetEthSpec> = head.beacon_block.clone().into();
let block_wrapper: BlockWrapper<MainnetEthSpec> = head.beacon_block.clone().into();
let available_block = harness
.chain
.data_availability_checker
.try_check_availability(block_wrapper)
.unwrap();
let chain = &harness.chain;
harness
.chain
.early_attester_cache
.add_head_block(
head.beacon_block_root,
block
.clone()
.into_available_block(head.beacon_block_root, &chain)
.expect("should wrap into available block"),
available_block,
head_proto_block,
&head.beacon_state,
&harness.chain.spec,

View File

@ -1,7 +1,8 @@
#![cfg(not(debug_assertions))]
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::{
blob_verification::{AsBlock, BlockWrapper},
blob_verification::AsBlock,
test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType},
};
use beacon_chain::{BeaconSnapshot, BlockError, ChainSegmentResult, NotifyExecutionLayer};

View File

@ -702,6 +702,8 @@ async fn invalidates_all_descendants() {
NotifyExecutionLayer::Yes,
)
.await
.unwrap()
.try_into()
.unwrap();
rig.recompute_head().await;
@ -799,6 +801,8 @@ async fn switches_heads() {
NotifyExecutionLayer::Yes,
)
.await
.unwrap()
.try_into()
.unwrap();
rig.recompute_head().await;

View File

@ -681,19 +681,20 @@ async fn run_skip_slot_test(skip_slots: u64) {
Slot::new(0)
);
assert_eq!(
harness_b
.chain
.process_block(
harness_a.chain.head_snapshot().beacon_block_root,
harness_a.chain.head_snapshot().beacon_block.clone(),
CountUnrealized::True,
NotifyExecutionLayer::Yes,
)
.await
.unwrap(),
harness_a.chain.head_snapshot().beacon_block_root
);
let status = harness_b
.chain
.process_block(
harness_a.chain.head_snapshot().beacon_block_root,
harness_a.chain.head_snapshot().beacon_block.clone(),
CountUnrealized::True,
NotifyExecutionLayer::Yes,
)
.await
.unwrap();
let root: Hash256 = status.try_into().unwrap();
assert_eq!(root, harness_a.chain.head_snapshot().beacon_block_root);
harness_b.chain.recompute_head_at_current_slot().await;

View File

@ -2,9 +2,10 @@ use super::*;
use serde::{Deserialize, Serialize};
use strum::EnumString;
use superstruct::superstruct;
use types::blobs_sidecar::KzgCommitments;
use types::beacon_block_body::KzgCommitments;
use types::blob_sidecar::Blobs;
use types::{
Blobs, EthSpec, ExecutionBlockHash, ExecutionPayload, ExecutionPayloadCapella,
EthSpec, ExecutionBlockHash, ExecutionPayload, ExecutionPayloadCapella,
ExecutionPayloadEip4844, ExecutionPayloadMerge, FixedVector, Transactions, Unsigned,
VariableList, Withdrawal,
};

View File

@ -41,15 +41,16 @@ use tokio::{
};
use tokio_stream::wrappers::WatchStream;
use tree_hash::TreeHash;
use types::beacon_block_body::KzgCommitments;
use types::blob_sidecar::Blobs;
use types::consts::eip4844::BLOB_TX_TYPE;
use types::transaction::{AccessTuple, BlobTransaction, EcdsaSignature, SignedBlobTransaction};
use types::Withdrawals;
use types::{AbstractExecPayload, BeaconStateError, ExecPayload, VersionedHash};
use types::{
blobs_sidecar::{Blobs, KzgCommitments},
BlindedPayload, BlockType, ChainSpec, Epoch, ExecutionBlockHash, ExecutionPayload,
ExecutionPayloadCapella, ExecutionPayloadEip4844, ExecutionPayloadMerge, ForkName,
};
use types::{AbstractExecPayload, BeaconStateError, ExecPayload, VersionedHash};
use types::{
ProposerPreparationData, PublicKeyBytes, Signature, SignedBeaconBlock, Slot, Transaction,
Uint256,

View File

@ -218,10 +218,7 @@ impl BlockId {
chain: &BeaconChain<T>,
) -> Result<BlobSidecarList<T::EthSpec>, warp::Rejection> {
let root = self.root(chain)?.0;
let Some(data_availability_boundary) = chain.data_availability_boundary() else {
return Err(warp_utils::reject::custom_not_found("Deneb fork disabled".into()));
};
match chain.get_blobs(&root, data_availability_boundary) {
match chain.get_blobs(&root) {
Ok(Some(blob_sidecar_list)) => Ok(blob_sidecar_list),
Ok(None) => Err(warp_utils::reject::custom_not_found(format!(
"No blobs with block root {} found in the store",

View File

@ -1,7 +1,7 @@
use beacon_chain::{BeaconChain, BeaconChainTypes, BlockProductionError};
use eth2::types::BlockContents;
use eth2::types::{BeaconBlockAndBlobSidecars, BlockContents};
use std::sync::Arc;
use types::{AbstractExecPayload, BeaconBlock, BeaconBlockAndBlobSidecars, ForkName};
use types::{AbstractExecPayload, BeaconBlock, ForkName};
type Error = warp::reject::Rejection;
@ -16,7 +16,7 @@ pub fn build_block_contents<T: BeaconChainTypes, Payload: AbstractExecPayload<T:
}
ForkName::Eip4844 => {
let block_root = &block.canonical_root();
if let Some(blob_sidecars) = chain.blob_cache.pop(block_root) {
if let Some(blob_sidecars) = chain.proposal_blob_cache.pop(block_root) {
let block_and_blobs = BeaconBlockAndBlobSidecars {
block,
blob_sidecars,

View File

@ -1,10 +1,10 @@
use crate::metrics;
use beacon_chain::blob_verification::{AsBlock, BlockWrapper, IntoAvailableBlock};
use beacon_chain::blob_verification::{AsBlock, BlockWrapper};
use beacon_chain::validator_monitor::{get_block_delay_ms, timestamp_now};
use beacon_chain::{
BeaconChain, BeaconChainTypes, BlockError, CountUnrealized, NotifyExecutionLayer,
};
use eth2::types::SignedBlockContents;
use beacon_chain::{AvailabilityProcessingStatus, NotifyExecutionLayer};
use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError, CountUnrealized};
use eth2::types::{SignedBlockContents, VariableList};
use execution_layer::ProvenancedPayload;
use lighthouse_network::PubsubMessage;
use network::NetworkMessage;
@ -70,54 +70,51 @@ pub async fn publish_block<T: BeaconChainTypes>(
}
SignedBeaconBlock::Eip4844(_) => {
crate::publish_pubsub_message(network_tx, PubsubMessage::BeaconBlock(block.clone()))?;
if let Some(blobs) = maybe_blobs {
for (blob_index, blob) in blobs.into_iter().enumerate() {
if let Some(signed_blobs) = maybe_blobs {
for (blob_index, blob) in signed_blobs.clone().into_iter().enumerate() {
crate::publish_pubsub_message(
network_tx,
PubsubMessage::BlobSidecar(Box::new((blob_index as u64, blob))),
)?;
}
let blobs_vec = signed_blobs.into_iter().map(|blob| blob.message).collect();
let blobs = VariableList::new(blobs_vec).map_err(|e| {
warp_utils::reject::custom_server_error(format!("Invalid blobs length: {e:?}"))
})?;
BlockWrapper::BlockAndBlobs(block, blobs)
} else {
block.into()
}
block.into()
}
};
let available_block = match wrapped_block.into_available_block(block_root, &chain) {
Ok(available_block) => available_block,
Err(e) => {
let msg = format!("{:?}", e);
error!(
log,
"Invalid block provided to HTTP API";
"reason" => &msg
);
return Err(warp_utils::reject::broadcast_without_import(msg));
}
};
// Determine the delay after the start of the slot, register it with metrics.
let block_clone = wrapped_block.block_cloned();
let slot = block_clone.message().slot();
let proposer_index = block_clone.message().proposer_index();
match chain
.process_block(
block_root,
available_block.clone(),
wrapped_block,
CountUnrealized::True,
NotifyExecutionLayer::Yes,
)
.await
{
Ok(root) => {
Ok(AvailabilityProcessingStatus::Imported(root)) => {
info!(
log,
"Valid block from HTTP API";
"block_delay" => ?delay,
"root" => format!("{}", root),
"proposer_index" => available_block.message().proposer_index(),
"slot" => available_block.slot(),
"proposer_index" => proposer_index,
"slot" =>slot,
);
// Notify the validator monitor.
chain.validator_monitor.read().register_api_block(
seen_timestamp,
available_block.message(),
block_clone.message(),
root,
&chain.slot_clock,
);
@ -133,7 +130,7 @@ pub async fn publish_block<T: BeaconChainTypes>(
late_block_logging(
&chain,
seen_timestamp,
available_block.message(),
block_clone.message(),
root,
"local",
&log,
@ -142,12 +139,30 @@ pub async fn publish_block<T: BeaconChainTypes>(
Ok(())
}
Ok(AvailabilityProcessingStatus::PendingBlock(block_root)) => {
let msg = format!("Missing block with root {:?}", block_root);
error!(
log,
"Invalid block provided to HTTP API";
"reason" => &msg
);
Err(warp_utils::reject::broadcast_without_import(msg))
}
Ok(AvailabilityProcessingStatus::PendingBlobs(blob_ids)) => {
let msg = format!("Missing blobs {:?}", blob_ids);
error!(
log,
"Invalid block provided to HTTP API";
"reason" => &msg
);
Err(warp_utils::reject::broadcast_without_import(msg))
}
Err(BlockError::BlockIsAlreadyKnown) => {
info!(
log,
"Block from HTTP API already known";
"block" => ?block_root,
"slot" => available_block.slot(),
"slot" => slot,
);
Ok(())
}

View File

@ -20,10 +20,9 @@ use tokio_util::{
codec::Framed,
compat::{Compat, FuturesAsyncReadCompatExt},
};
use types::BlobsSidecar;
use types::{
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockCapella, BeaconBlockMerge,
EmptyBlock, EthSpec, ForkContext, ForkName, Hash256, MainnetEthSpec, Signature,
BlobSidecar, EmptyBlock, EthSpec, ForkContext, ForkName, Hash256, MainnetEthSpec, Signature,
SignedBeaconBlock,
};
@ -115,12 +114,12 @@ lazy_static! {
.as_ssz_bytes()
.len();
pub static ref BLOBS_SIDECAR_MIN: usize = BlobsSidecar::<MainnetEthSpec>::empty().as_ssz_bytes().len();
pub static ref BLOBS_SIDECAR_MAX: usize = BlobsSidecar::<MainnetEthSpec>::max_size();
pub static ref BLOB_SIDECAR_MIN: usize = BlobSidecar::<MainnetEthSpec>::empty().as_ssz_bytes().len();
pub static ref BLOB_SIDECAR_MAX: usize = BlobSidecar::<MainnetEthSpec>::max_size();
//FIXME(sean) these are underestimates
pub static ref SIGNED_BLOCK_AND_BLOBS_MIN: usize = *BLOBS_SIDECAR_MIN + *SIGNED_BEACON_BLOCK_BASE_MIN;
pub static ref SIGNED_BLOCK_AND_BLOBS_MAX: usize =*BLOBS_SIDECAR_MAX + *SIGNED_BEACON_BLOCK_EIP4844_MAX;
pub static ref SIGNED_BLOCK_AND_BLOBS_MIN: usize = *BLOB_SIDECAR_MIN + *SIGNED_BEACON_BLOCK_BASE_MIN;
pub static ref SIGNED_BLOCK_AND_BLOBS_MAX: usize =*BLOB_SIDECAR_MAX + *SIGNED_BEACON_BLOCK_EIP4844_MAX;
}
/// The maximum bytes that can be sent across the RPC pre-merge.
@ -385,7 +384,7 @@ impl ProtocolId {
Protocol::Goodbye => RpcLimits::new(0, 0), // Goodbye request has no response
Protocol::BlocksByRange => rpc_block_limits_by_fork(fork_context.current_fork()),
Protocol::BlocksByRoot => rpc_block_limits_by_fork(fork_context.current_fork()),
Protocol::BlobsByRange => RpcLimits::new(*BLOBS_SIDECAR_MIN, *BLOBS_SIDECAR_MAX),
Protocol::BlobsByRange => RpcLimits::new(*BLOB_SIDECAR_MIN, *BLOB_SIDECAR_MAX),
Protocol::BlobsByRoot => {
// TODO: wrong too
RpcLimits::new(*SIGNED_BLOCK_AND_BLOBS_MIN, *SIGNED_BLOCK_AND_BLOBS_MAX)

View File

@ -41,7 +41,7 @@ num_cpus = "1.13.0"
lru_cache = { path = "../../common/lru_cache" }
if-addrs = "0.6.4"
strum = "0.24.0"
tokio-util = { version = "0.6.3", features = ["time"] }
tokio-util = { version = "0.7.7", features = ["time"] }
derivative = "2.2.0"
delay_map = "0.3.0"
ethereum-types = { version = "0.14.1", optional = true }

View File

@ -449,7 +449,7 @@ impl<T: BeaconChainTypes> WorkEvent<T> {
peer_id: PeerId,
peer_client: Client,
blob_index: u64,
signed_blob: Arc<SignedBlobSidecar<T::EthSpec>>,
signed_blob: SignedBlobSidecar<T::EthSpec>,
seen_timestamp: Duration,
) -> Self {
Self {
@ -459,7 +459,7 @@ impl<T: BeaconChainTypes> WorkEvent<T> {
peer_id,
peer_client,
blob_index,
signed_blob,
signed_blob: Box::new(signed_blob),
seen_timestamp,
},
}
@ -729,7 +729,7 @@ impl<T: BeaconChainTypes> WorkEvent<T> {
impl<T: BeaconChainTypes> std::convert::From<ReadyWork<T>> for WorkEvent<T> {
fn from(ready_work: ReadyWork<T>) -> Self {
match ready_work {
ReadyWork::Block(QueuedGossipBlock {
ReadyWork::GossipBlock(QueuedGossipBlock {
peer_id,
block,
seen_timestamp,
@ -864,7 +864,7 @@ pub enum Work<T: BeaconChainTypes> {
peer_id: PeerId,
peer_client: Client,
blob_index: u64,
signed_blob: Arc<SignedBlobSidecar<T::EthSpec>>,
signed_blob: Box<SignedBlobSidecar<T::EthSpec>>,
seen_timestamp: Duration,
},
DelayedImportBlock {
@ -1759,7 +1759,7 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
peer_id,
peer_client,
blob_index,
signed_blob,
*signed_blob,
seen_timestamp,
)
.await

View File

@ -13,14 +13,15 @@
use super::MAX_SCHEDULED_WORK_QUEUE_LEN;
use crate::metrics;
use crate::sync::manager::BlockProcessType;
use beacon_chain::blob_verification::{AsBlock, BlockWrapper};
use beacon_chain::blob_verification::AsBlock;
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
use fnv::FnvHashMap;
use futures::task::Poll;
use futures::{Stream, StreamExt};
use lighthouse_network::{MessageId, PeerId};
use logging::TimeLatch;
use slog::{crit, debug, error, trace, warn, Logger};
use slog::{debug, error, trace, warn, Logger};
use slot_clock::SlotClock;
use std::collections::{HashMap, HashSet};
use std::pin::Pin;
@ -28,7 +29,6 @@ use std::task::Context;
use std::time::Duration;
use task_executor::TaskExecutor;
use tokio::sync::mpsc::{self, Receiver, Sender};
use tokio::time::error::Error as TimeError;
use tokio_util::time::delay_queue::{DelayQueue, Key as DelayKey};
use types::{
Attestation, EthSpec, Hash256, LightClientOptimisticUpdate, SignedAggregateAndProof, SubnetId,
@ -87,7 +87,7 @@ pub enum ReprocessQueueMessage<T: BeaconChainTypes> {
/// Events sent by the scheduler once they are ready for re-processing.
pub enum ReadyWork<T: BeaconChainTypes> {
Block(QueuedGossipBlock<T>),
GossipBlock(QueuedGossipBlock<T>),
RpcBlock(QueuedRpcBlock<T::EthSpec>),
Unaggregate(QueuedUnaggregate<T::EthSpec>),
Aggregate(QueuedAggregate<T::EthSpec>),
@ -154,8 +154,6 @@ enum InboundEvent<T: BeaconChainTypes> {
ReadyAttestation(QueuedAttestationId),
/// A light client update that is ready for re-processing.
ReadyLightClientUpdate(QueuedLightClientUpdateId),
/// A `DelayQueue` returned an error.
DelayQueueError(TimeError, &'static str),
/// A message sent to the `ReprocessQueue`
Msg(ReprocessQueueMessage<T>),
}
@ -233,54 +231,42 @@ impl<T: BeaconChainTypes> Stream for ReprocessQueue<T> {
// The sequential nature of blockchains means it is generally better to try and import all
// existing blocks before new ones.
match self.gossip_block_delay_queue.poll_expired(cx) {
Poll::Ready(Some(Ok(queued_block))) => {
Poll::Ready(Some(queued_block)) => {
return Poll::Ready(Some(InboundEvent::ReadyGossipBlock(
queued_block.into_inner(),
)));
}
Poll::Ready(Some(Err(e))) => {
return Poll::Ready(Some(InboundEvent::DelayQueueError(e, "gossip_block_queue")));
}
// `Poll::Ready(None)` means that there are no more entries in the delay queue and we
// will continue to get this result until something else is added into the queue.
Poll::Ready(None) | Poll::Pending => (),
}
match self.rpc_block_delay_queue.poll_expired(cx) {
Poll::Ready(Some(Ok(queued_block))) => {
Poll::Ready(Some(queued_block)) => {
return Poll::Ready(Some(InboundEvent::ReadyRpcBlock(queued_block.into_inner())));
}
Poll::Ready(Some(Err(e))) => {
return Poll::Ready(Some(InboundEvent::DelayQueueError(e, "rpc_block_queue")));
}
// `Poll::Ready(None)` means that there are no more entries in the delay queue and we
// will continue to get this result until something else is added into the queue.
Poll::Ready(None) | Poll::Pending => (),
}
match self.attestations_delay_queue.poll_expired(cx) {
Poll::Ready(Some(Ok(attestation_id))) => {
Poll::Ready(Some(attestation_id)) => {
return Poll::Ready(Some(InboundEvent::ReadyAttestation(
attestation_id.into_inner(),
)));
}
Poll::Ready(Some(Err(e))) => {
return Poll::Ready(Some(InboundEvent::DelayQueueError(e, "attestations_queue")));
}
// `Poll::Ready(None)` means that there are no more entries in the delay queue and we
// will continue to get this result until something else is added into the queue.
Poll::Ready(None) | Poll::Pending => (),
}
match self.lc_updates_delay_queue.poll_expired(cx) {
Poll::Ready(Some(Ok(lc_id))) => {
Poll::Ready(Some(lc_id)) => {
return Poll::Ready(Some(InboundEvent::ReadyLightClientUpdate(
lc_id.into_inner(),
)));
}
Poll::Ready(Some(Err(e))) => {
return Poll::Ready(Some(InboundEvent::DelayQueueError(e, "lc_updates_queue")));
}
// `Poll::Ready(None)` means that there are no more entries in the delay queue and we
// will continue to get this result until something else is added into the queue.
Poll::Ready(None) | Poll::Pending => (),
@ -400,7 +386,7 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
if block_slot <= now
&& self
.ready_work_tx
.try_send(ReadyWork::Block(early_block))
.try_send(ReadyWork::GossipBlock(early_block))
.is_err()
{
error!(
@ -694,7 +680,7 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
if self
.ready_work_tx
.try_send(ReadyWork::Block(ready_block))
.try_send(ReadyWork::GossipBlock(ready_block))
.is_err()
{
error!(
@ -703,14 +689,7 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
);
}
}
InboundEvent::DelayQueueError(e, queue_name) => {
crit!(
log,
"Failed to poll queue";
"queue" => queue_name,
"e" => ?e
)
}
InboundEvent::ReadyAttestation(queued_id) => {
metrics::inc_counter(
&metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_EXPIRED_ATTESTATIONS,

View File

@ -1,6 +1,6 @@
use crate::{metrics, service::NetworkMessage, sync::SyncMessage};
use beacon_chain::blob_verification::{AsBlock, BlockWrapper};
use beacon_chain::blob_verification::{AsBlock, BlockWrapper, GossipVerifiedBlob};
use beacon_chain::store::Error;
use beacon_chain::{
attestation_verification::{self, Error as AttnError, VerifiedAttestation},
@ -9,15 +9,14 @@ use beacon_chain::{
observed_operations::ObservationOutcome,
sync_committee_verification::{self, Error as SyncCommitteeError},
validator_monitor::get_block_delay_ms,
BeaconChainError, BeaconChainTypes, BlockError, CountUnrealized, ForkChoiceError,
GossipVerifiedBlock, NotifyExecutionLayer,
AvailabilityProcessingStatus, BeaconChainError, BeaconChainTypes, BlockError, CountUnrealized,
ForkChoiceError, GossipVerifiedBlock, NotifyExecutionLayer,
};
use lighthouse_network::{Client, MessageAcceptance, MessageId, PeerAction, PeerId, ReportSource};
use operation_pool::ReceivedPreCapella;
use slog::{crit, debug, error, info, trace, warn};
use slot_clock::SlotClock;
use ssz::Encode;
use std::sync::Arc;
use std::time::{Duration, SystemTime, UNIX_EPOCH};
use store::hot_cold_store::HotColdDBError;
use tokio::sync::mpsc;
@ -654,19 +653,57 @@ impl<T: BeaconChainTypes> Worker<T> {
self,
_message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
_peer_client: Client,
blob_index: u64,
signed_blob: Arc<SignedBlobSidecar<T::EthSpec>>,
signed_blob: SignedBlobSidecar<T::EthSpec>,
_seen_duration: Duration,
) {
// TODO: gossip verification
crit!(self.log, "UNIMPLEMENTED gossip blob verification";
"peer_id" => %peer_id,
"client" => %peer_client,
"blob_topic" => blob_index,
"blob_index" => signed_blob.message.index,
"blob_slot" => signed_blob.message.slot
);
match self
.chain
.verify_blob_sidecar_for_gossip(signed_blob, blob_index)
{
Ok(gossip_verified_blob) => {
self.process_gossip_verified_blob(peer_id, gossip_verified_blob, _seen_duration)
.await
}
Err(_) => {
// TODO(pawan): handle all blob errors for peer scoring
todo!()
}
}
}
pub async fn process_gossip_verified_blob(
self,
peer_id: PeerId,
verified_blob: GossipVerifiedBlob<T::EthSpec>,
// This value is not used presently, but it might come in handy for debugging.
_seen_duration: Duration,
) {
// TODO
match self
.chain
.process_blob(verified_blob, CountUnrealized::True)
.await
{
Ok(AvailabilityProcessingStatus::Imported(_hash)) => {
todo!()
// add to metrics
// logging
}
Ok(AvailabilityProcessingStatus::PendingBlobs(pending_blobs)) => self
.send_sync_message(SyncMessage::UnknownBlobHash {
peer_id,
pending_blobs,
}),
Ok(AvailabilityProcessingStatus::PendingBlock(block_hash)) => {
self.send_sync_message(SyncMessage::UnknownBlockHash(peer_id, block_hash));
}
Err(_err) => {
// handle errors
todo!()
}
}
}
/// Process the beacon block received from the gossip network and:
@ -802,6 +839,9 @@ impl<T: BeaconChainTypes> Worker<T> {
verified_block
}
Err(BlockError::AvailabilityCheck(_err)) => {
todo!()
}
Err(BlockError::ParentUnknown(block)) => {
debug!(
self.log,
@ -984,7 +1024,7 @@ impl<T: BeaconChainTypes> Worker<T> {
)
.await
{
Ok(block_root) => {
Ok(AvailabilityProcessingStatus::Imported(block_root)) => {
metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL);
if reprocess_tx
@ -1011,6 +1051,24 @@ impl<T: BeaconChainTypes> Worker<T> {
self.chain.recompute_head_at_current_slot().await;
}
Ok(AvailabilityProcessingStatus::PendingBlock(block_root)) => {
// This error variant doesn't make any sense in this context
crit!(
self.log,
"Internal error. Cannot get AvailabilityProcessingStatus::PendingBlock on processing block";
"block_root" => %block_root
);
}
Ok(AvailabilityProcessingStatus::PendingBlobs(pending_blobs)) => {
// make rpc request for blob
self.send_sync_message(SyncMessage::UnknownBlobHash {
peer_id,
pending_blobs,
});
}
Err(BlockError::AvailabilityCheck(_)) => {
todo!()
}
Err(BlockError::ParentUnknown(block)) => {
// Inform the sync manager to find parents for this block
// This should not occur. It should be checked by `should_forward_block`

View File

@ -1,5 +1,3 @@
use std::sync::Arc;
use crate::beacon_processor::{worker::FUTURE_SLOT_TOLERANCE, SendOnDrop};
use crate::service::NetworkMessage;
use crate::status::ToStatusMessage;
@ -15,7 +13,6 @@ use lighthouse_network::{PeerId, PeerRequestId, ReportSource, Response, SyncInfo
use slog::{debug, error, warn};
use slot_clock::SlotClock;
use std::collections::{hash_map::Entry, HashMap};
use std::sync::Arc;
use task_executor::TaskExecutor;
use tokio_stream::StreamExt;
use types::blob_sidecar::BlobIdentifier;
@ -840,7 +837,7 @@ impl<T: BeaconChainTypes> Worker<T> {
let mut send_response = true;
for root in block_roots {
match self.chain.get_blobs(&root, data_availability_boundary) {
match self.chain.get_blobs(&root) {
Ok(Some(blob_sidecar_list)) => {
for blob_sidecar in blob_sidecar_list.iter() {
blobs_sent += 1;

View File

@ -7,8 +7,9 @@ use crate::beacon_processor::DuplicateCache;
use crate::metrics;
use crate::sync::manager::{BlockProcessType, SyncMessage};
use crate::sync::{BatchProcessResult, ChainId};
use beacon_chain::blob_verification::{AsBlock, BlockWrapper, IntoAvailableBlock};
use beacon_chain::CountUnrealized;
use beacon_chain::blob_verification::AsBlock;
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::{AvailabilityProcessingStatus, CountUnrealized};
use beacon_chain::{
BeaconChainError, BeaconChainTypes, BlockError, ChainSegmentResult, HistoricalBlockError,
NotifyExecutionLayer,
@ -86,28 +87,22 @@ impl<T: BeaconChainTypes> Worker<T> {
};
let slot = block.slot();
let parent_root = block.message().parent_root();
let available_block = block
.into_available_block(block_root, &self.chain)
.map_err(BlockError::BlobValidation);
let result = match available_block {
Ok(block) => {
self.chain
.process_block(
block_root,
block,
CountUnrealized::True,
NotifyExecutionLayer::Yes,
)
.await
}
Err(e) => Err(e),
};
let result = self
.chain
.process_block(
block_root,
block,
CountUnrealized::True,
NotifyExecutionLayer::Yes,
)
.await;
metrics::inc_counter(&metrics::BEACON_PROCESSOR_RPC_BLOCK_IMPORTED_TOTAL);
// RPC block imported, regardless of process type
if let &Ok(hash) = &result {
//TODO(sean) handle pending availability variants
if let &Ok(AvailabilityProcessingStatus::Imported(hash)) = &result {
info!(self.log, "New RPC block received"; "slot" => slot, "hash" => %hash);
// Trigger processing for work referencing this block.

View File

@ -280,7 +280,6 @@ impl<T: BeaconChainTypes> Router<T> {
PubsubMessage::BlobSidecar(data) => {
let (blob_index, signed_blob) = *data;
let peer_client = self.network_globals.client(&peer_id);
let signed_blob = Arc::new(signed_blob);
self.send_beacon_processor_work(BeaconWorkEvent::gossip_signed_blob_sidecar(
message_id,
peer_id,

View File

@ -2,7 +2,8 @@ use std::collections::hash_map::Entry;
use std::collections::HashMap;
use std::time::Duration;
use beacon_chain::blob_verification::{AsBlock, BlockWrapper};
use beacon_chain::blob_verification::AsBlock;
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::{BeaconChainTypes, BlockError};
use fnv::FnvHashMap;
use lighthouse_network::rpc::{RPCError, RPCResponseErrorCode};

View File

@ -1,5 +1,6 @@
use super::RootBlockTuple;
use beacon_chain::blob_verification::{AsBlock, BlockWrapper};
use beacon_chain::blob_verification::AsBlock;
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::BeaconChainTypes;
use lighthouse_network::PeerId;
use store::Hash256;

View File

@ -1,5 +1,6 @@
use super::RootBlockTuple;
use beacon_chain::blob_verification::{AsBlock, BlockWrapper};
use beacon_chain::blob_verification::AsBlock;
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::get_block_root;
use lighthouse_network::{rpc::BlocksByRootRequest, PeerId};
use rand::seq::IteratorRandom;

View File

@ -42,7 +42,8 @@ use crate::beacon_processor::{ChainSegmentProcessId, WorkEvent as BeaconWorkEven
use crate::service::NetworkMessage;
use crate::status::ToStatusMessage;
use crate::sync::range_sync::ByRangeRequestType;
use beacon_chain::blob_verification::{AsBlock, BlockWrapper};
use beacon_chain::blob_verification::AsBlock;
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError, EngineState};
use futures::StreamExt;
use lighthouse_network::rpc::methods::MAX_REQUEST_BLOCKS;
@ -56,6 +57,7 @@ use std::ops::Sub;
use std::sync::Arc;
use std::time::Duration;
use tokio::sync::mpsc;
use types::blob_sidecar::BlobIdentifier;
use types::{BlobSidecar, EthSpec, Hash256, SignedBeaconBlock, Slot};
/// The number of slots ahead of us that is allowed before requesting a long-range (batch) Sync
@ -119,6 +121,13 @@ pub enum SyncMessage<T: EthSpec> {
/// manager to attempt to find the block matching the unknown hash.
UnknownBlockHash(PeerId, Hash256),
/// A peer has sent us a block that we haven't received all the blobs for. This triggers
/// the manager to attempt to find the pending blobs for the given block root.
UnknownBlobHash {
peer_id: PeerId,
pending_blobs: Vec<BlobIdentifier>,
},
/// A peer has disconnected.
Disconnect(PeerId),
@ -598,6 +607,9 @@ impl<T: BeaconChainTypes> SyncManager<T> {
.search_block(block_hash, peer_id, &mut self.network);
}
}
SyncMessage::UnknownBlobHash { .. } => {
unimplemented!()
}
SyncMessage::Disconnect(peer_id) => {
self.peer_disconnect(&peer_id);
}

View File

@ -38,6 +38,7 @@ use std::marker::PhantomData;
use std::path::{Path, PathBuf};
use std::sync::Arc;
use std::time::Duration;
use types::blob_sidecar::BlobSidecarList;
use types::consts::eip4844::MIN_EPOCHS_FOR_BLOBS_SIDECARS_REQUESTS;
use types::*;
@ -547,7 +548,7 @@ impl<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> HotColdDB<E, Hot, Cold>
/// Check if the blobs sidecar for a block exists on disk.
pub fn blobs_sidecar_exists(&self, block_root: &Hash256) -> Result<bool, Error> {
self.get_item::<BlobsSidecar<E>>(block_root)
self.get_item::<BlobSidecarList<E>>(block_root)
.map(|blobs| blobs.is_some())
}

View File

@ -1,7 +1,7 @@
use crate::{DBColumn, Error, StoreItem};
use ssz::{Decode, Encode};
use types::{
BlobsSidecar, EthSpec, ExecutionPayload, ExecutionPayloadCapella, ExecutionPayloadEip4844,
BlobSidecarList, EthSpec, ExecutionPayload, ExecutionPayloadCapella, ExecutionPayloadEip4844,
ExecutionPayloadMerge,
};
@ -25,7 +25,7 @@ macro_rules! impl_store_item {
impl_store_item!(ExecutionPayloadMerge);
impl_store_item!(ExecutionPayloadCapella);
impl_store_item!(ExecutionPayloadEip4844);
impl_store_item!(BlobsSidecar);
impl_store_item!(BlobSidecarList);
/// This fork-agnostic implementation should be only used for writing.
///

View File

@ -43,6 +43,7 @@ pub use metrics::scrape_for_metrics;
use parking_lot::MutexGuard;
use std::sync::Arc;
use strum::{EnumString, IntoStaticStr};
use types::blob_sidecar::BlobSidecarList;
pub use types::*;
pub type ColumnIter<'a> = Box<dyn Iterator<Item = Result<(Hash256, Vec<u8>), Error>> + 'a>;

View File

@ -1396,3 +1396,32 @@ pub struct SignedBeaconBlockAndBlobSidecars<T: EthSpec, Payload: AbstractExecPay
pub signed_block: SignedBeaconBlock<T, Payload>,
pub signed_blob_sidecars: SignedBlobSidecarList<T>,
}
#[derive(Debug, Clone, Serialize, Deserialize, Encode)]
#[serde(bound = "T: EthSpec, Payload: AbstractExecPayload<T>")]
pub struct BeaconBlockAndBlobSidecars<T: EthSpec, Payload: AbstractExecPayload<T>> {
pub block: BeaconBlock<T, Payload>,
pub blob_sidecars: BlobSidecarList<T>,
}
impl<T: EthSpec, Payload: AbstractExecPayload<T>> ForkVersionDeserialize
for BeaconBlockAndBlobSidecars<T, Payload>
{
fn deserialize_by_fork<'de, D: serde::Deserializer<'de>>(
value: serde_json::value::Value,
fork_name: ForkName,
) -> Result<Self, D::Error> {
#[derive(Deserialize)]
#[serde(bound = "T: EthSpec")]
struct Helper<T: EthSpec> {
block: serde_json::Value,
blob_sidecars: BlobSidecarList<T>,
}
let helper: Helper<T> = serde_json::from_value(value).map_err(serde::de::Error::custom)?;
Ok(Self {
block: BeaconBlock::deserialize_by_fork::<'de, D>(helper.block, fork_name)?,
blob_sidecars: helper.blob_sidecars,
})
}
}

View File

@ -7,7 +7,7 @@ use types::{
ChainSpec, Epoch, EthSpec, Hash256, IndexedAttestation, SignedBeaconBlock, Slot,
};
#[derive(Debug)]
#[derive(Debug, Clone)]
pub struct ConsensusContext<T: EthSpec> {
/// Slot to act as an identifier/safeguard
slot: Slot,

View File

@ -1,37 +0,0 @@
use crate::{
AbstractExecPayload, BeaconBlock, BlobSidecarList, EthSpec, ForkName, ForkVersionDeserialize,
};
use derivative::Derivative;
use serde_derive::{Deserialize, Serialize};
use ssz_derive::Encode;
use tree_hash_derive::TreeHash;
#[derive(Debug, Clone, Serialize, Deserialize, Encode, TreeHash, Derivative)]
#[derivative(PartialEq, Hash(bound = "T: EthSpec"))]
#[serde(bound = "T: EthSpec, Payload: AbstractExecPayload<T>")]
pub struct BeaconBlockAndBlobSidecars<T: EthSpec, Payload: AbstractExecPayload<T>> {
pub block: BeaconBlock<T, Payload>,
pub blob_sidecars: BlobSidecarList<T>,
}
impl<T: EthSpec, Payload: AbstractExecPayload<T>> ForkVersionDeserialize
for BeaconBlockAndBlobSidecars<T, Payload>
{
fn deserialize_by_fork<'de, D: serde::Deserializer<'de>>(
value: serde_json::value::Value,
fork_name: ForkName,
) -> Result<Self, D::Error> {
#[derive(Deserialize)]
#[serde(bound = "T: EthSpec")]
struct Helper<T: EthSpec> {
block: serde_json::Value,
blob_sidecars: BlobSidecarList<T>,
}
let helper: Helper<T> = serde_json::from_value(value).map_err(serde::de::Error::custom)?;
Ok(Self {
block: BeaconBlock::deserialize_by_fork::<'de, D>(helper.block, fork_name)?,
blob_sidecars: helper.blob_sidecars,
})
}
}

View File

@ -1,5 +1,5 @@
use crate::test_utils::TestRandom;
use crate::*;
use crate::{blobs_sidecar::KzgCommitments, test_utils::TestRandom};
use derivative::Derivative;
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
@ -9,6 +9,8 @@ use superstruct::superstruct;
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
pub type KzgCommitments<T> = VariableList<KzgCommitment, <T as EthSpec>::MaxBlobsPerBlock>;
/// The body of a `BeaconChain` block, containing operations.
///
/// This *superstruct* abstracts over the hard-fork.

View File

@ -11,7 +11,7 @@ use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
/// Container of the data that identifies an individual blob.
#[derive(Encode, Decode, Clone, Debug, PartialEq)]
#[derive(Encode, Decode, Clone, Debug, PartialEq, Eq, Hash)]
pub struct BlobIdentifier {
pub block_root: Hash256,
pub index: u64,
@ -35,7 +35,6 @@ pub struct BlobIdentifier {
#[derivative(PartialEq, Hash(bound = "T: EthSpec"))]
pub struct BlobSidecar<T: EthSpec> {
pub block_root: Hash256,
// TODO: fix the type, should fit in u8 as well
#[serde(with = "eth2_serde_utils::quoted_u64")]
pub index: u64,
pub slot: Slot,
@ -49,10 +48,18 @@ pub struct BlobSidecar<T: EthSpec> {
}
pub type BlobSidecarList<T> = VariableList<Arc<BlobSidecar<T>>, <T as EthSpec>::MaxBlobsPerBlock>;
pub type Blobs<T> = VariableList<Blob<T>, <T as EthSpec>::MaxExtraDataBytes>;
impl<T: EthSpec> SignedRoot for BlobSidecar<T> {}
impl<T: EthSpec> BlobSidecar<T> {
pub fn id(&self) -> BlobIdentifier {
BlobIdentifier {
block_root: self.block_root,
index: self.index,
}
}
pub fn empty() -> Self {
Self::default()
}

View File

@ -1,62 +0,0 @@
use crate::test_utils::TestRandom;
use crate::{Blob, EthSpec, Hash256, KzgCommitment, SignedRoot, Slot};
use derivative::Derivative;
use kzg::KzgProof;
use serde_derive::{Deserialize, Serialize};
use ssz::Encode;
use ssz_derive::{Decode, Encode};
use ssz_types::VariableList;
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
pub type KzgCommitments<T> = VariableList<KzgCommitment, <T as EthSpec>::MaxBlobsPerBlock>;
pub type Blobs<T> = VariableList<Blob<T>, <T as EthSpec>::MaxBlobsPerBlock>;
#[derive(
Debug,
Clone,
Serialize,
Deserialize,
Encode,
Decode,
TreeHash,
Default,
TestRandom,
Derivative,
arbitrary::Arbitrary,
)]
#[serde(bound = "T: EthSpec")]
#[arbitrary(bound = "T: EthSpec")]
#[derivative(PartialEq, Hash(bound = "T: EthSpec"))]
pub struct BlobsSidecar<T: EthSpec> {
pub beacon_block_root: Hash256,
pub beacon_block_slot: Slot,
#[serde(with = "ssz_types::serde_utils::list_of_hex_fixed_vec")]
pub blobs: Blobs<T>,
pub kzg_aggregated_proof: KzgProof,
}
impl<T: EthSpec> SignedRoot for BlobsSidecar<T> {}
impl<T: EthSpec> BlobsSidecar<T> {
pub fn empty() -> Self {
Self::default()
}
pub fn empty_from_parts(beacon_block_root: Hash256, beacon_block_slot: Slot) -> Self {
Self {
beacon_block_root,
beacon_block_slot,
blobs: VariableList::empty(),
kzg_aggregated_proof: KzgProof::empty(),
}
}
#[allow(clippy::integer_arithmetic)]
pub fn max_size() -> usize {
// Fixed part
Self::empty().as_ssz_bytes().len()
// Max size of variable length `blobs` field
+ (T::max_blobs_per_block() * <Blob<T> as Encode>::ssz_fixed_len())
}
}

View File

@ -97,11 +97,8 @@ pub mod slot_data;
#[cfg(feature = "sqlite")]
pub mod sqlite;
pub mod beacon_block_and_blob_sidecars;
pub mod blob_sidecar;
pub mod blobs_sidecar;
pub mod signed_blob;
pub mod signed_block_and_blobs;
pub mod transaction;
use ethereum_types::{H160, H256};
@ -115,7 +112,6 @@ pub use crate::beacon_block::{
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockCapella, BeaconBlockEip4844,
BeaconBlockMerge, BeaconBlockRef, BeaconBlockRefMut, BlindedBeaconBlock, EmptyBlock,
};
pub use crate::beacon_block_and_blob_sidecars::BeaconBlockAndBlobSidecars;
pub use crate::beacon_block_body::{
BeaconBlockBody, BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyCapella,
BeaconBlockBodyEip4844, BeaconBlockBodyMerge, BeaconBlockBodyRef, BeaconBlockBodyRefMut,
@ -124,7 +120,6 @@ pub use crate::beacon_block_header::BeaconBlockHeader;
pub use crate::beacon_committee::{BeaconCommittee, OwnedBeaconCommittee};
pub use crate::beacon_state::{BeaconTreeHashCache, Error as BeaconStateError, *};
pub use crate::blob_sidecar::{BlobSidecar, BlobSidecarList};
pub use crate::blobs_sidecar::{Blobs, BlobsSidecar};
pub use crate::bls_to_execution_change::BlsToExecutionChange;
pub use crate::chain_spec::{ChainSpec, Config, Domain};
pub use crate::checkpoint::Checkpoint;
@ -183,9 +178,6 @@ pub use crate::signed_beacon_block::{
};
pub use crate::signed_beacon_block_header::SignedBeaconBlockHeader;
pub use crate::signed_blob::*;
pub use crate::signed_block_and_blobs::{
SignedBeaconBlockAndBlobsSidecar, SignedBeaconBlockAndBlobsSidecarDecode,
};
pub use crate::signed_bls_to_execution_change::SignedBlsToExecutionChange;
pub use crate::signed_contribution_and_proof::SignedContributionAndProof;
pub use crate::signed_voluntary_exit::SignedVoluntaryExit;

View File

@ -35,14 +35,6 @@ impl From<SignedBeaconBlockHash> for Hash256 {
}
}
#[derive(Debug)]
pub enum BlobReconstructionError {
/// No blobs for the specified block where we would expect blobs.
UnavailableBlobs,
/// Blobs provided for a pre-Eip4844 fork.
InconsistentFork,
}
/// A `BeaconBlock` and a signature from its proposer.
#[superstruct(
variants(Base, Altair, Merge, Capella, Eip4844),
@ -250,28 +242,6 @@ impl<E: EthSpec, Payload: AbstractExecPayload<E>> SignedBeaconBlock<E, Payload>
pub fn canonical_root(&self) -> Hash256 {
self.message().tree_hash_root()
}
/// Reconstructs an empty `BlobsSidecar`, using the given block root if provided, else calculates it.
/// If this block has kzg commitments, an error will be returned. If this block is from prior to the
/// Eip4844 fork, this will error.
pub fn reconstruct_empty_blobs(
&self,
block_root_opt: Option<Hash256>,
) -> Result<BlobsSidecar<E>, BlobReconstructionError> {
let kzg_commitments = self
.message()
.body()
.blob_kzg_commitments()
.map_err(|_| BlobReconstructionError::InconsistentFork)?;
if kzg_commitments.is_empty() {
Ok(BlobsSidecar::empty_from_parts(
block_root_opt.unwrap_or_else(|| self.canonical_root()),
self.slot(),
))
} else {
Err(BlobReconstructionError::UnavailableBlobs)
}
}
}
// We can convert pre-Bellatrix blocks without payloads into blocks with payloads.

View File

@ -1,10 +1,15 @@
use crate::{test_utils::TestRandom, BlobSidecar, EthSpec, Signature};
use crate::{
test_utils::TestRandom, BlobSidecar, ChainSpec, Domain, EthSpec, Fork, Hash256, Signature,
SignedRoot, SigningData,
};
use bls::PublicKey;
use derivative::Derivative;
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use ssz_types::VariableList;
use std::sync::Arc;
use test_random_derive::TestRandom;
use tree_hash::TreeHash;
use tree_hash_derive::TreeHash;
#[derive(
@ -30,3 +35,37 @@ pub struct SignedBlobSidecar<T: EthSpec> {
pub type SignedBlobSidecarList<T> =
VariableList<SignedBlobSidecar<T>, <T as EthSpec>::MaxBlobsPerBlock>;
impl<T: EthSpec> SignedBlobSidecar<T> {
/// Verify `self.signature`.
///
/// If the root of `block.message` is already known it can be passed in via `object_root_opt`.
/// Otherwise, it will be computed locally.
pub fn verify_signature(
&self,
object_root_opt: Option<Hash256>,
pubkey: &PublicKey,
fork: &Fork,
genesis_validators_root: Hash256,
spec: &ChainSpec,
) -> bool {
let domain = spec.get_domain(
self.message.slot.epoch(T::slots_per_epoch()),
Domain::BlobSidecar,
fork,
genesis_validators_root,
);
let message = if let Some(object_root) = object_root_opt {
SigningData {
object_root,
domain,
}
.tree_hash_root()
} else {
self.message.signing_root(domain)
};
self.signature.verify(pubkey, message)
}
}

View File

@ -1,35 +0,0 @@
use crate::{BlobsSidecar, EthSpec, SignedBeaconBlock, SignedBeaconBlockEip4844};
use derivative::Derivative;
use serde_derive::{Deserialize, Serialize};
use ssz::{Decode, DecodeError};
use ssz_derive::{Decode, Encode};
use std::sync::Arc;
use tree_hash_derive::TreeHash;
#[derive(Debug, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, PartialEq)]
#[serde(bound = "T: EthSpec")]
pub struct SignedBeaconBlockAndBlobsSidecarDecode<T: EthSpec> {
pub beacon_block: SignedBeaconBlockEip4844<T>,
pub blobs_sidecar: BlobsSidecar<T>,
}
// TODO: will be removed once we decouple blobs in Gossip
#[derive(Debug, Clone, Serialize, Deserialize, Encode, TreeHash, Derivative)]
#[derivative(PartialEq, Hash(bound = "T: EthSpec"))]
pub struct SignedBeaconBlockAndBlobsSidecar<T: EthSpec> {
pub beacon_block: Arc<SignedBeaconBlock<T>>,
pub blobs_sidecar: Arc<BlobsSidecar<T>>,
}
impl<T: EthSpec> SignedBeaconBlockAndBlobsSidecar<T> {
pub fn from_ssz_bytes(bytes: &[u8]) -> Result<Self, DecodeError> {
let SignedBeaconBlockAndBlobsSidecarDecode {
beacon_block,
blobs_sidecar,
} = SignedBeaconBlockAndBlobsSidecarDecode::from_ssz_bytes(bytes)?;
Ok(SignedBeaconBlockAndBlobsSidecar {
beacon_block: Arc::new(SignedBeaconBlock::Eip4844(beacon_block)),
blobs_sidecar: Arc::new(blobs_sidecar),
})
}
}

View File

@ -63,7 +63,7 @@ pub fn run_parse_ssz<T: EthSpec>(matches: &ArgMatches) -> Result<(), String> {
"state_merge" => decode_and_print::<BeaconStateMerge<T>>(&bytes, format)?,
"state_capella" => decode_and_print::<BeaconStateCapella<T>>(&bytes, format)?,
"state_eip4844" => decode_and_print::<BeaconStateEip4844<T>>(&bytes, format)?,
"blobs_sidecar" => decode_and_print::<BlobsSidecar<T>>(&bytes, format)?,
"blob_sidecar" => decode_and_print::<BlobSidecar<T>>(&bytes, format)?,
other => return Err(format!("Unknown type: {}", other)),
};

View File

@ -50,7 +50,7 @@ type_name_generic!(BeaconBlockBodyCapella, "BeaconBlockBody");
type_name_generic!(BeaconBlockBodyEip4844, "BeaconBlockBody");
type_name!(BeaconBlockHeader);
type_name_generic!(BeaconState);
type_name_generic!(BlobsSidecar);
type_name_generic!(BlobSidecar);
type_name!(Checkpoint);
type_name_generic!(ContributionAndProof);
type_name!(Deposit);
@ -88,9 +88,5 @@ type_name!(Validator);
type_name!(VoluntaryExit);
type_name!(Withdrawal);
type_name!(BlsToExecutionChange, "BLSToExecutionChange");
type_name_generic!(
SignedBeaconBlockAndBlobsSidecarDecode,
"SignedBeaconBlockAndBlobsSidecar"
);
type_name!(SignedBlsToExecutionChange, "SignedBLSToExecutionChange");
type_name!(HistoricalSummary);

View File

@ -216,7 +216,6 @@ macro_rules! ssz_static_test_no_run {
mod ssz_static {
use ef_tests::{Handler, SszStaticHandler, SszStaticTHCHandler, SszStaticWithSpecHandler};
use types::historical_summary::HistoricalSummary;
use types::signed_block_and_blobs::SignedBeaconBlockAndBlobsSidecarDecode;
use types::*;
ssz_static_test!(aggregate_and_proof, AggregateAndProof<_>);
@ -378,12 +377,6 @@ mod ssz_static {
SszStaticHandler::<BlobsSidecar<MainnetEthSpec>, MainnetEthSpec>::eip4844_only().run();
}
#[test]
fn signed_blobs_sidecar() {
SszStaticHandler::<SignedBeaconBlockAndBlobsSidecarDecode<MinimalEthSpec>, MinimalEthSpec>::eip4844_only().run();
SszStaticHandler::<SignedBeaconBlockAndBlobsSidecarDecode<MainnetEthSpec>, MainnetEthSpec>::eip4844_only().run();
}
#[test]
fn historical_summary() {
SszStaticHandler::<HistoricalSummary, MinimalEthSpec>::capella_and_later().run();