Refactor deneb block processing (#4511)

* Revert "fix merge"

This reverts commit 405e95b0ce.

* refactor deneb block processing

* cargo fmt

* fix ci
This commit is contained in:
realbigsean 2023-07-25 10:51:10 -04:00 committed by GitHub
parent 3735450749
commit 33dd13c798
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
33 changed files with 931 additions and 952 deletions

View File

@ -8,13 +8,16 @@ 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::{self, AsBlock, BlobError, BlockWrapper, GossipVerifiedBlob};
use crate::blob_verification::{self, BlobError, 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, AvailableExecutedBlock, BlockError, BlockImportData,
ExecutedBlock, ExecutionPendingBlock, GossipVerifiedBlock, IntoExecutionPendingBlock,
signature_verify_chain_segment, BlockError, ExecutionPendingBlock, GossipVerifiedBlock,
IntoExecutionPendingBlock,
};
use crate::block_verification_types::{
AsBlock, AvailableExecutedBlock, BlockImportData, ExecutedBlock, RpcBlock,
};
pub use crate::canonical_head::{CanonicalHead, CanonicalHeadRwLock};
use crate::chain_config::ChainConfig;
@ -122,7 +125,7 @@ use types::*;
pub type ForkChoiceError = fork_choice::Error<crate::ForkChoiceStoreError>;
/// Alias to appease clippy.
type HashBlockTuple<E> = (Hash256, BlockWrapper<E>);
type HashBlockTuple<E> = (Hash256, RpcBlock<E>);
/// The time-out before failure during an operation to take a read/write RwLock on the block
/// processing cache.
@ -2521,7 +2524,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// This method is potentially long-running and should not run on the core executor.
pub fn filter_chain_segment(
self: &Arc<Self>,
chain_segment: Vec<BlockWrapper<T::EthSpec>>,
chain_segment: Vec<RpcBlock<T::EthSpec>>,
) -> Result<Vec<HashBlockTuple<T::EthSpec>>, ChainSegmentResult<T::EthSpec>> {
// This function will never import any blocks.
let imported_blocks = 0;
@ -2627,7 +2630,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// `Self::process_block`.
pub async fn process_chain_segment(
self: &Arc<Self>,
chain_segment: Vec<BlockWrapper<T::EthSpec>>,
chain_segment: Vec<RpcBlock<T::EthSpec>>,
notify_execution_layer: NotifyExecutionLayer,
) -> ChainSegmentResult<T::EthSpec> {
let mut imported_blocks = 0;
@ -2804,7 +2807,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
///
/// - `SignedBeaconBlock`
/// - `GossipVerifiedBlock`
/// - `BlockWrapper`
/// - `RpcBlock`
///
/// ## Errors
///

View File

@ -7,22 +7,18 @@ use crate::beacon_chain::{
BeaconChain, BeaconChainTypes, BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT,
MAXIMUM_GOSSIP_CLOCK_DISPARITY, VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT,
};
use crate::data_availability_checker::{
AvailabilityCheckError, AvailabilityPendingBlock, AvailableBlock,
};
use crate::data_availability_checker::AvailabilityCheckError;
use crate::kzg_utils::{validate_blob, validate_blobs};
use crate::BeaconChainError;
use eth2::types::BlockContentsTuple;
use kzg::Kzg;
use slog::{debug, warn};
use ssz_derive::{Decode, Encode};
use ssz_types::{FixedVector, VariableList};
use ssz_types::VariableList;
use std::borrow::Cow;
use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList};
use types::blob_sidecar::BlobIdentifier;
use types::{
BeaconBlockRef, BeaconState, BeaconStateError, BlobSidecar, BlobSidecarList, ChainSpec,
CloneConfig, Epoch, EthSpec, FullPayload, Hash256, KzgCommitment, RelativeEpoch,
SignedBeaconBlock, SignedBeaconBlockHeader, SignedBlobSidecar, Slot,
BeaconState, BeaconStateError, BlobSidecar, BlobSidecarList, ChainSpec, CloneConfig, EthSpec,
Hash256, KzgCommitment, RelativeEpoch, SignedBlobSidecar, Slot,
};
#[derive(Debug)]
@ -519,13 +515,12 @@ pub fn verify_kzg_for_blob<T: EthSpec>(
/// Note: This function should be preferred over calling `verify_kzg_for_blob`
/// in a loop since this function kzg verifies a list of blobs more efficiently.
pub fn verify_kzg_for_blob_list<T: EthSpec>(
blob_list: Vec<Arc<BlobSidecar<T>>>,
blob_list: &BlobSidecarList<T>,
kzg: &Kzg<T::Kzg>,
) -> Result<KzgVerifiedBlobList<T>, AvailabilityCheckError> {
) -> Result<(), AvailabilityCheckError> {
let _timer = crate::metrics::start_timer(&crate::metrics::KZG_VERIFICATION_BATCH_TIMES);
let (blobs, (commitments, proofs)): (Vec<_>, (Vec<_>, Vec<_>)) = blob_list
.clone()
.into_iter()
.iter()
.map(|blob| (blob.blob.clone(), (blob.kzg_commitment, blob.kzg_proof)))
.unzip();
if validate_blobs::<T>(
@ -536,225 +531,8 @@ pub fn verify_kzg_for_blob_list<T: EthSpec>(
)
.map_err(AvailabilityCheckError::Kzg)?
{
Ok(blob_list
.into_iter()
.map(|blob| KzgVerifiedBlob { blob })
.collect())
Ok(())
} else {
Err(AvailabilityCheckError::KzgVerificationFailed)
}
}
pub type KzgVerifiedBlobList<T> = Vec<KzgVerifiedBlob<T>>;
#[derive(Debug, Clone)]
pub enum MaybeAvailableBlock<E: EthSpec> {
/// This variant is fully available.
/// i.e. for pre-deneb 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>),
}
/// Trait for common block operations.
pub trait AsBlock<E: EthSpec> {
fn slot(&self) -> Slot;
fn epoch(&self) -> Epoch;
fn parent_root(&self) -> Hash256;
fn state_root(&self) -> Hash256;
fn signed_block_header(&self) -> SignedBeaconBlockHeader;
fn message(&self) -> BeaconBlockRef<E>;
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>>, FixedBlobSidecarList<E>),
}
impl<E: EthSpec> BlockWrapper<E> {
pub fn new(block: Arc<SignedBeaconBlock<E>>, blobs: Option<BlobSidecarList<E>>) -> Self {
match blobs {
Some(blobs) => {
let blobs = FixedVector::from(blobs.into_iter().map(Some).collect::<Vec<_>>());
BlockWrapper::BlockAndBlobs(block, blobs)
}
None => BlockWrapper::Block(block),
}
}
pub fn deconstruct(self) -> (Arc<SignedBeaconBlock<E>>, Option<FixedBlobSidecarList<E>>) {
match self {
BlockWrapper::Block(block) => (block, None),
BlockWrapper::BlockAndBlobs(block, blobs) => (block, Some(blobs)),
}
}
}
impl<E: EthSpec> AsBlock<E> for BlockWrapper<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 {
BlockWrapper::Block(block) => block,
BlockWrapper::BlockAndBlobs(block, _) => block,
}
}
fn block_cloned(&self) -> Arc<SignedBeaconBlock<E>> {
match &self {
BlockWrapper::Block(block) => block.clone(),
BlockWrapper::BlockAndBlobs(block, _) => block.clone(),
}
}
fn canonical_root(&self) -> Hash256 {
self.as_block().canonical_root()
}
fn into_block_wrapper(self) -> BlockWrapper<E> {
self
}
}
impl<E: EthSpec> BlockWrapper<E> {
pub fn n_blobs(&self) -> usize {
match self {
BlockWrapper::Block(_) => 0,
BlockWrapper::BlockAndBlobs(_, blobs) => blobs.len(),
}
}
}
impl<E: EthSpec> From<Arc<SignedBeaconBlock<E>>> for BlockWrapper<E> {
fn from(value: Arc<SignedBeaconBlock<E>>) -> Self {
Self::Block(value)
}
}
impl<E: EthSpec> From<SignedBeaconBlock<E>> for BlockWrapper<E> {
fn from(value: SignedBeaconBlock<E>) -> Self {
Self::Block(Arc::new(value))
}
}
impl<E: EthSpec> From<BlockContentsTuple<E, FullPayload<E>>> for BlockWrapper<E> {
fn from(value: BlockContentsTuple<E, FullPayload<E>>) -> Self {
match value.1 {
Some(variable_list) => {
let mut blobs = Vec::with_capacity(E::max_blobs_per_block());
for blob in variable_list {
if blob.message.index < E::max_blobs_per_block() as u64 {
blobs.insert(blob.message.index as usize, Some(blob.message));
}
}
Self::BlockAndBlobs(Arc::new(value.0), FixedVector::from(blobs))
}
None => Self::Block(Arc::new(value.0)),
}
}
}

View File

@ -48,13 +48,11 @@
// returned alongside.
#![allow(clippy::result_large_err)]
use crate::blob_verification::{
AsBlock, BlobError, BlockWrapper, GossipVerifiedBlob, GossipVerifiedBlobList,
MaybeAvailableBlock,
};
use crate::data_availability_checker::{
AvailabilityCheckError, AvailabilityPendingBlock, AvailableBlock,
use crate::blob_verification::{BlobError, GossipVerifiedBlob};
use crate::block_verification_types::{
AsBlock, BlockImportData, GossipVerifiedBlockContents, RpcBlock,
};
use crate::data_availability_checker::{AvailabilityCheckError, MaybeAvailableBlock};
use crate::eth1_finalization_cache::Eth1FinalizationData;
use crate::execution_payload::{
is_optimistic_candidate_block, validate_execution_payload_for_gossip, validate_merge_block,
@ -99,13 +97,11 @@ use std::time::Duration;
use store::{Error as DBError, HotStateSummary, KeyValueStore, SignedBlobSidecarList, StoreOp};
use task_executor::JoinHandle;
use tree_hash::TreeHash;
use types::blob_sidecar::BlobIdentifier;
use types::ExecPayload;
use types::{ssz_tagged_beacon_state, ssz_tagged_signed_beacon_block};
use types::{
BeaconBlockRef, BeaconState, BeaconStateError, BlindedPayload, ChainSpec, CloneConfig, Epoch,
EthSpec, ExecutionBlockHash, Hash256, InconsistentFork, PublicKey, PublicKeyBytes,
RelativeEpoch, SignedBeaconBlock, SignedBeaconBlockHeader, Slot,
BeaconBlockRef, BeaconState, BeaconStateError, ChainSpec, CloneConfig, Epoch, EthSpec,
ExecutionBlockHash, Hash256, InconsistentFork, PublicKey, PublicKeyBytes, RelativeEpoch,
SignedBeaconBlock, SignedBeaconBlockHeader, Slot,
};
pub const POS_PANDA_BANNER: &str = r#"
@ -153,7 +149,7 @@ pub enum BlockError<T: EthSpec> {
///
/// It's unclear if this block is valid, but it cannot be processed without already knowing
/// its parent.
ParentUnknown(BlockWrapper<T>),
ParentUnknown(RpcBlock<T>),
/// The block slot is greater than the present slot.
///
/// ## Peer scoring
@ -558,7 +554,7 @@ fn process_block_slash_info<T: BeaconChainTypes>(
/// The given `chain_segment` must contain only blocks from the same epoch, otherwise an error
/// will be returned.
pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
mut chain_segment: Vec<(Hash256, BlockWrapper<T::EthSpec>)>,
mut chain_segment: Vec<(Hash256, RpcBlock<T::EthSpec>)>,
chain: &BeaconChain<T>,
) -> Result<Vec<SignatureVerifiedBlock<T>>, BlockError<T::EthSpec>> {
if chain_segment.is_empty() {
@ -595,7 +591,7 @@ pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
let maybe_available_block = chain
.data_availability_checker
.check_availability(block.clone())?;
.check_rpc_block_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.
@ -625,19 +621,12 @@ pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
#[derive(Derivative)]
#[derivative(Debug(bound = "T: BeaconChainTypes"))]
pub struct GossipVerifiedBlock<T: BeaconChainTypes> {
pub block: MaybeAvailableBlock<T::EthSpec>,
pub block: Arc<SignedBeaconBlock<T::EthSpec>>,
pub block_root: Hash256,
parent: Option<PreProcessingSnapshot<T::EthSpec>>,
consensus_context: ConsensusContext<T::EthSpec>,
}
impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
/// Useful for publishing after gossip verification.
pub fn into_block_wrapper(self) -> BlockWrapper<T::EthSpec> {
self.block.into_block_wrapper()
}
}
/// A wrapper around a `SignedBeaconBlock` that indicates that all signatures (except the deposit
/// signatures) have been verified.
pub struct SignatureVerifiedBlock<T: BeaconChainTypes> {
@ -669,147 +658,6 @@ pub struct ExecutionPendingBlock<T: BeaconChainTypes> {
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,
))
}
}
}
}
#[derive(Debug, PartialEq)]
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 fn get_all_blob_ids(&self) -> Vec<BlobIdentifier> {
let num_blobs_expected = self
.block
.message()
.body()
.blob_kzg_commitments()
.map_or(0, |commitments| commitments.len());
let mut blob_ids = Vec::with_capacity(num_blobs_expected);
for i in 0..num_blobs_expected {
blob_ids.push(BlobIdentifier {
block_root: self.import_data.block_root,
index: i as u64,
});
}
blob_ids
}
}
#[derive(Encode, Decode, Clone)]
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 fn num_blobs_expected(&self) -> usize {
self.block
.kzg_commitments()
.map_or(0, |commitments| commitments.len())
}
pub fn get_all_blob_ids(&self) -> Vec<BlobIdentifier> {
let block_root = self.import_data.block_root;
self.block
.get_filtered_blob_ids(Some(block_root), |_, _| true)
}
pub fn get_filtered_blob_ids(
&self,
filter: impl Fn(usize, Hash256) -> bool,
) -> Vec<BlobIdentifier> {
self.block
.get_filtered_blob_ids(Some(self.import_data.block_root), filter)
}
}
#[derive(Debug, PartialEq, Encode, Decode, Clone)]
// TODO (mark): investigate using an Arc<state> / Arc<parent_block>
// here to make this cheaper to clone
pub struct BlockImportData<E: EthSpec> {
pub block_root: Hash256,
#[ssz(with = "ssz_tagged_beacon_state")]
pub state: BeaconState<E>,
#[ssz(with = "ssz_tagged_signed_beacon_block")]
pub parent_block: SignedBeaconBlock<E, BlindedPayload<E>>,
pub parent_eth1_finalization_data: Eth1FinalizationData,
pub confirmed_state_roots: Vec<Hash256>,
pub consensus_context: ConsensusContext<E>,
}
pub type GossipVerifiedBlockContents<T> =
(GossipVerifiedBlock<T>, Option<GossipVerifiedBlobList<T>>);
pub trait IntoGossipVerifiedBlockContents<T: BeaconChainTypes>: Sized {
fn into_gossip_verified_block(
self,
@ -911,27 +759,23 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
block: Arc<SignedBeaconBlock<T::EthSpec>>,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError<T::EthSpec>> {
let maybe_available = chain
.data_availability_checker
.check_availability(block.into())?;
// 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 = maybe_available.signed_block_header();
Self::new_without_slasher_checks(maybe_available, chain).map_err(|e| {
let header = block.signed_block_header();
Self::new_without_slasher_checks(block, 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: MaybeAvailableBlock<T::EthSpec>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError<T::EthSpec>> {
// Ensure the block is the correct structure for the fork at `block.slot()`.
block
.as_block()
.fork_name(&chain.spec)
.map_err(BlockError::InconsistentFork)?;
@ -947,7 +791,7 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
});
}
let block_root = get_block_root(block.as_block());
let block_root = get_block_root(&block);
// Disallow blocks that conflict with the anchor (weak subjectivity checkpoint), if any.
check_block_against_anchor_slot(block.message(), chain)?;
@ -1067,7 +911,7 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
let pubkey = pubkey_cache
.get(block.message().proposer_index() as usize)
.ok_or_else(|| BlockError::UnknownValidator(block.message().proposer_index()))?;
block.as_block().verify_signature(
block.verify_signature(
Some(block_root),
pubkey,
&fork,
@ -1111,8 +955,7 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
// Having checked the proposer index and the block root we can cache them.
let consensus_context = ConsensusContext::new(block.slot())
.set_current_block_root(block_root)
.set_proposer_index(block.as_block().message().proposer_index())
.set_kzg_commitments_consistent(true);
.set_proposer_index(block.as_block().message().proposer_index());
Ok(Self {
block,
@ -1155,11 +998,10 @@ 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: BlockWrapper<T::EthSpec>,
block: MaybeAvailableBlock<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()
@ -1182,10 +1024,8 @@ impl<T: BeaconChainTypes> SignatureVerifiedBlock<T> {
let mut signature_verifier = get_signature_verifier(&state, &pubkey_cache, &chain.spec);
let mut consensus_context = ConsensusContext::new(block.slot())
.set_current_block_root(block_root)
// An `AvailabileBlock is passed in here, so we know this check has been run.`
.set_kzg_commitments_consistent(true);
let mut consensus_context =
ConsensusContext::new(block.slot()).set_current_block_root(block_root);
signature_verifier.include_all_signatures(block.as_block(), &mut consensus_context)?;
@ -1203,7 +1043,7 @@ impl<T: BeaconChainTypes> SignatureVerifiedBlock<T> {
/// As for `new` above but producing `BlockSlashInfo`.
pub fn check_slashable(
block: BlockWrapper<T::EthSpec>,
block: MaybeAvailableBlock<T::EthSpec>,
block_root: Hash256,
chain: &BeaconChain<T>,
) -> Result<Self, BlockSlashInfo<BlockError<T::EthSpec>>> {
@ -1238,11 +1078,11 @@ impl<T: BeaconChainTypes> SignatureVerifiedBlock<T> {
// signature.
let mut consensus_context = from.consensus_context;
signature_verifier
.include_all_signatures_except_proposal(block.as_block(), &mut consensus_context)?;
.include_all_signatures_except_proposal(block.as_ref(), &mut consensus_context)?;
if signature_verifier.verify().is_ok() {
Ok(Self {
block,
block: MaybeAvailableBlock::AvailabilityPending(block),
block_root: from.block_root,
parent: Some(parent),
consensus_context,
@ -1299,6 +1139,7 @@ impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for SignatureVerifiedBloc
}
}
//TODO(sean) can this be deleted
impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for Arc<SignedBeaconBlock<T::EthSpec>> {
/// Verifies the `SignedBeaconBlock` by first transforming it into a `SignatureVerifiedBlock`
/// and then using that implementation of `IntoExecutionPendingBlock` to complete verification.
@ -1311,8 +1152,16 @@ impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for Arc<SignedBeaconBlock
// Perform an early check to prevent wasting time on irrelevant blocks.
let block_root = check_block_relevancy(&self, block_root, chain)
.map_err(|e| BlockSlashInfo::SignatureNotChecked(self.signed_block_header(), e))?;
SignatureVerifiedBlock::check_slashable(self.into(), block_root, chain)?
let maybe_available = chain
.data_availability_checker
.check_rpc_block_availability(RpcBlock::new_without_blobs(self.clone()))
.map_err(|e| {
BlockSlashInfo::SignatureNotChecked(
self.signed_block_header(),
BlockError::AvailabilityCheck(e),
)
})?;
SignatureVerifiedBlock::check_slashable(maybe_available, block_root, chain)?
.into_execution_pending_block_slashable(block_root, chain, notify_execution_layer)
}
@ -1321,7 +1170,7 @@ impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for Arc<SignedBeaconBlock
}
}
impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for BlockWrapper<T::EthSpec> {
impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for RpcBlock<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(
@ -1333,8 +1182,16 @@ impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for BlockWrapper<T::EthSp
// Perform an early check to prevent wasting time on irrelevant blocks.
let block_root = check_block_relevancy(self.as_block(), block_root, chain)
.map_err(|e| BlockSlashInfo::SignatureNotChecked(self.signed_block_header(), e))?;
SignatureVerifiedBlock::check_slashable(self, block_root, chain)?
let maybe_available = chain
.data_availability_checker
.check_rpc_block_availability(self.clone())
.map_err(|e| {
BlockSlashInfo::SignatureNotChecked(
self.signed_block_header(),
BlockError::AvailabilityCheck(e),
)
})?;
SignatureVerifiedBlock::check_slashable(maybe_available, block_root, chain)?
.into_execution_pending_block_slashable(block_root, chain, notify_execution_layer)
}
@ -1388,7 +1245,7 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
// because it will revert finalization. Note that the finalized block is stored in fork
// choice, so we will not reject any child of the finalized block (this is relevant during
// genesis).
return Err(BlockError::ParentUnknown(block.into_block_wrapper()));
return Err(BlockError::ParentUnknown(block.into_rpc_block()));
}
/*
@ -1826,7 +1683,7 @@ pub fn check_block_is_finalized_checkpoint_or_descendant<
block_parent_root: block.parent_root(),
})
} else {
Err(BlockError::ParentUnknown(block.into_block_wrapper()))
Err(BlockError::ParentUnknown(block.into_rpc_block()))
}
}
}
@ -1898,8 +1755,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: MaybeAvailableBlock<T::EthSpec>,
) -> Result<(ProtoBlock, MaybeAvailableBlock<T::EthSpec>), BlockError<T::EthSpec>> {
block: Arc<SignedBeaconBlock<T::EthSpec>>,
) -> Result<(ProtoBlock, Arc<SignedBeaconBlock<T::EthSpec>>), BlockError<T::EthSpec>> {
if let Some(proto_block) = chain
.canonical_head
.fork_choice_read_lock()
@ -1907,7 +1764,9 @@ fn verify_parent_block_is_known<T: BeaconChainTypes>(
{
Ok((proto_block, block))
} else {
Err(BlockError::ParentUnknown(block.into_block_wrapper()))
Err(BlockError::ParentUnknown(RpcBlock::new_without_blobs(
block,
)))
}
}
@ -1938,7 +1797,7 @@ fn load_parent<T: BeaconChainTypes, B: AsBlock<T::EthSpec>>(
.fork_choice_read_lock()
.contains_block(&block.parent_root())
{
return Err(BlockError::ParentUnknown(block.into_block_wrapper()));
return Err(BlockError::ParentUnknown(block.into_rpc_block()));
}
let block_delay = chain

View File

@ -0,0 +1,438 @@
use crate::blob_verification::GossipVerifiedBlobList;
use crate::data_availability_checker::AvailabilityCheckError;
pub use crate::data_availability_checker::{AvailableBlock, MaybeAvailableBlock};
use crate::eth1_finalization_cache::Eth1FinalizationData;
use crate::{data_availability_checker, GossipVerifiedBlock, PayloadVerificationOutcome};
use derivative::Derivative;
use ssz_derive::{Decode, Encode};
use state_processing::ConsensusContext;
use std::sync::Arc;
use types::{
blob_sidecar::BlobIdentifier, ssz_tagged_beacon_state, ssz_tagged_signed_beacon_block,
ssz_tagged_signed_beacon_block_arc,
};
use types::{
BeaconBlockRef, BeaconState, BlindedPayload, BlobSidecarList, Epoch, EthSpec, Hash256,
SignedBeaconBlock, SignedBeaconBlockHeader, Slot,
};
/// A block that has been received over RPC. It has 2 internal variants:
///
/// 1. `BlockAndBlobs`: A fully available post deneb block with all the blobs available. This variant
/// is only constructed after making consistency checks between blocks and blobs.
/// Hence, it is fully self contained w.r.t verification. i.e. this block has all the required
/// data to get verfied and imported into fork choice.
///
/// 2. `Block`: This can be a fully available pre-deneb block **or** a post-deneb block that may or may
/// not require blobs to be considered fully available.
///
/// Note: We make a distinction over blocks received over gossip because
/// in a post-deneb world, the blobs corresponding to a given block that are received
/// over rpc do not contain the proposer signature for dos resistance.
#[derive(Debug, Clone, Derivative)]
#[derivative(Hash(bound = "E: EthSpec"))]
pub struct RpcBlock<E: EthSpec> {
block: RpcBlockInner<E>,
}
/// Note: This variant is intentionally private because we want to safely construct the
/// internal variants after applying consistency checks to ensure that the block and blobs
/// are consistent with respect to each other.
#[derive(Debug, Clone, Derivative)]
#[derivative(Hash(bound = "E: EthSpec"))]
enum RpcBlockInner<E: EthSpec> {
/// Single block lookup response. This should potentially hit the data availability cache.
Block(Arc<SignedBeaconBlock<E>>),
/// This variant is used with parent lookups and by-range responses. It should have all blobs
/// ordered, all block roots matching, and the correct number of blobs for this block.
BlockAndBlobs(Arc<SignedBeaconBlock<E>>, BlobSidecarList<E>),
}
impl<E: EthSpec> RpcBlock<E> {
/// Constructs a `Block` variant.
pub fn new_without_blobs(block: Arc<SignedBeaconBlock<E>>) -> Self {
Self {
block: RpcBlockInner::Block(block),
}
}
/// Constructs a new `BlockAndBlobs` variant after making consistency
/// checks between the provided blocks and blobs.
pub fn new(
block: Arc<SignedBeaconBlock<E>>,
blobs: Option<BlobSidecarList<E>>,
) -> Result<Self, AvailabilityCheckError> {
if let Some(blobs) = blobs.as_ref() {
data_availability_checker::consistency_checks(&block, blobs)?;
}
let inner = match blobs {
Some(blobs) => RpcBlockInner::BlockAndBlobs(block, blobs),
None => RpcBlockInner::Block(block),
};
Ok(Self { block: inner })
}
pub fn deconstruct(self) -> (Arc<SignedBeaconBlock<E>>, Option<BlobSidecarList<E>>) {
match self.block {
RpcBlockInner::Block(block) => (block, None),
RpcBlockInner::BlockAndBlobs(block, blobs) => (block, Some(blobs)),
}
}
pub fn n_blobs(&self) -> usize {
match &self.block {
RpcBlockInner::Block(_) => 0,
RpcBlockInner::BlockAndBlobs(_, blobs) => blobs.len(),
}
}
}
impl<E: EthSpec> From<Arc<SignedBeaconBlock<E>>> for RpcBlock<E> {
fn from(value: Arc<SignedBeaconBlock<E>>) -> Self {
Self::new_without_blobs(value)
}
}
impl<E: EthSpec> From<SignedBeaconBlock<E>> for RpcBlock<E> {
fn from(value: SignedBeaconBlock<E>) -> Self {
Self::new_without_blobs(Arc::new(value))
}
}
/// A block that has gone through all pre-deneb block processing checks including block processing
/// and execution by an EL client. This block hasn't completed data availability checks.
///
///
/// It contains 2 variants:
/// 1. `Available`: This block has been executed and also contains all data to consider it a
/// fully available block. i.e. for post-deneb, this implies that this contains all the
/// required blobs.
/// 2. `AvailabilityPending`: This block hasn't received all required blobs to consider it a
/// fully available block.
pub enum ExecutedBlock<E: EthSpec> {
Available(AvailableExecutedBlock<E>),
AvailabilityPending(AvailabilityPendingExecutedBlock<E>),
}
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 fn as_block(&self) -> &SignedBeaconBlock<E> {
match self {
Self::Available(available) => available.block.block(),
Self::AvailabilityPending(pending) => &pending.block,
}
}
}
/// A block that has completed all pre-deneb block processing checks including verification
/// by an EL client **and** has all requisite blob data to be imported into fork choice.
#[derive(PartialEq)]
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 fn get_all_blob_ids(&self) -> Vec<BlobIdentifier> {
let num_blobs_expected = self
.block
.message()
.body()
.blob_kzg_commitments()
.map_or(0, |commitments| commitments.len());
let mut blob_ids = Vec::with_capacity(num_blobs_expected);
for i in 0..num_blobs_expected {
blob_ids.push(BlobIdentifier {
block_root: self.import_data.block_root,
index: i as u64,
});
}
blob_ids
}
}
/// A block that has completed all pre-deneb block processing checks, verification
/// by an EL client but does not have all requisite blob data to get imported into
/// fork choice.
#[derive(Encode, Decode, Clone)]
pub struct AvailabilityPendingExecutedBlock<E: EthSpec> {
#[ssz(with = "ssz_tagged_signed_beacon_block_arc")]
pub block: Arc<SignedBeaconBlock<E>>,
pub import_data: BlockImportData<E>,
pub payload_verification_outcome: PayloadVerificationOutcome,
}
impl<E: EthSpec> AvailabilityPendingExecutedBlock<E> {
pub fn new(
block: Arc<SignedBeaconBlock<E>>,
import_data: BlockImportData<E>,
payload_verification_outcome: PayloadVerificationOutcome,
) -> Self {
Self {
block,
import_data,
payload_verification_outcome,
}
}
pub fn num_blobs_expected(&self) -> usize {
self.block
.message()
.body()
.blob_kzg_commitments()
.map_or(0, |commitments| commitments.len())
}
pub fn get_all_blob_ids(&self) -> Vec<BlobIdentifier> {
let block_root = self.import_data.block_root;
self.block
.get_filtered_blob_ids(Some(block_root), |_, _| true)
}
pub fn get_filtered_blob_ids(
&self,
filter: impl Fn(usize, Hash256) -> bool,
) -> Vec<BlobIdentifier> {
self.block
.get_filtered_blob_ids(Some(self.import_data.block_root), filter)
}
}
#[derive(Debug, PartialEq, Encode, Decode, Clone)]
// TODO (mark): investigate using an Arc<state> / Arc<parent_block>
// here to make this cheaper to clone
pub struct BlockImportData<E: EthSpec> {
pub block_root: Hash256,
#[ssz(with = "ssz_tagged_beacon_state")]
pub state: BeaconState<E>,
#[ssz(with = "ssz_tagged_signed_beacon_block")]
pub parent_block: SignedBeaconBlock<E, BlindedPayload<E>>,
pub parent_eth1_finalization_data: Eth1FinalizationData,
pub confirmed_state_roots: Vec<Hash256>,
pub consensus_context: ConsensusContext<E>,
}
pub type GossipVerifiedBlockContents<T> =
(GossipVerifiedBlock<T>, Option<GossipVerifiedBlobList<T>>);
/// Trait for common block operations.
pub trait AsBlock<E: EthSpec> {
fn slot(&self) -> Slot;
fn epoch(&self) -> Epoch;
fn parent_root(&self) -> Hash256;
fn state_root(&self) -> Hash256;
fn signed_block_header(&self) -> SignedBeaconBlockHeader;
fn message(&self) -> BeaconBlockRef<E>;
fn as_block(&self) -> &SignedBeaconBlock<E>;
fn block_cloned(&self) -> Arc<SignedBeaconBlock<E>>;
fn canonical_root(&self) -> Hash256;
fn into_rpc_block(self) -> RpcBlock<E>;
}
impl<E: EthSpec> AsBlock<E> for Arc<SignedBeaconBlock<E>> {
fn slot(&self) -> Slot {
SignedBeaconBlock::slot(self)
}
fn epoch(&self) -> Epoch {
SignedBeaconBlock::epoch(self)
}
fn parent_root(&self) -> Hash256 {
SignedBeaconBlock::parent_root(self)
}
fn state_root(&self) -> Hash256 {
SignedBeaconBlock::state_root(self)
}
fn signed_block_header(&self) -> SignedBeaconBlockHeader {
SignedBeaconBlock::signed_block_header(self)
}
fn message(&self) -> BeaconBlockRef<E> {
SignedBeaconBlock::message(self)
}
fn as_block(&self) -> &SignedBeaconBlock<E> {
self
}
fn block_cloned(&self) -> Arc<SignedBeaconBlock<E>> {
Arc::<SignedBeaconBlock<E>>::clone(self)
}
fn canonical_root(&self) -> Hash256 {
SignedBeaconBlock::canonical_root(self)
}
fn into_rpc_block(self) -> RpcBlock<E> {
RpcBlock::new_without_blobs(self)
}
}
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,
}
}
fn block_cloned(&self) -> Arc<SignedBeaconBlock<E>> {
match &self {
MaybeAvailableBlock::Available(block) => block.block_cloned(),
MaybeAvailableBlock::AvailabilityPending(block) => block.clone(),
}
}
fn canonical_root(&self) -> Hash256 {
self.as_block().canonical_root()
}
fn into_rpc_block(self) -> RpcBlock<E> {
match self {
MaybeAvailableBlock::Available(available_block) => available_block.into_rpc_block(),
MaybeAvailableBlock::AvailabilityPending(block) => RpcBlock::new_without_blobs(block),
}
}
}
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>> {
AvailableBlock::block_cloned(self)
}
fn canonical_root(&self) -> Hash256 {
self.block().canonical_root()
}
fn into_rpc_block(self) -> RpcBlock<E> {
let (block, blobs_opt) = self.deconstruct();
// Circumvent the constructor here, because an Available block will have already had
// consistency checks performed.
let inner = match blobs_opt {
None => RpcBlockInner::Block(block),
Some(blobs) => RpcBlockInner::BlockAndBlobs(block, blobs),
};
RpcBlock { block: inner }
}
}
impl<E: EthSpec> AsBlock<E> for RpcBlock<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.block {
RpcBlockInner::Block(block) => block,
RpcBlockInner::BlockAndBlobs(block, _) => block,
}
}
fn block_cloned(&self) -> Arc<SignedBeaconBlock<E>> {
match &self.block {
RpcBlockInner::Block(block) => block.clone(),
RpcBlockInner::BlockAndBlobs(block, _) => block.clone(),
}
}
fn canonical_root(&self) -> Hash256 {
self.as_block().canonical_root()
}
fn into_rpc_block(self) -> RpcBlock<E> {
self
}
}

View File

@ -1,28 +1,25 @@
use crate::blob_verification::{
verify_kzg_for_blob, verify_kzg_for_blob_list, AsBlock, BlockWrapper, GossipVerifiedBlob,
KzgVerifiedBlob, KzgVerifiedBlobList, MaybeAvailableBlock,
verify_kzg_for_blob, verify_kzg_for_blob_list, GossipVerifiedBlob, KzgVerifiedBlob,
};
use crate::block_verification_types::{
AvailabilityPendingExecutedBlock, AvailableExecutedBlock, RpcBlock,
};
use crate::block_verification::{AvailabilityPendingExecutedBlock, AvailableExecutedBlock};
use crate::data_availability_checker::overflow_lru_cache::OverflowLRUCache;
use crate::{BeaconChain, BeaconChainTypes, BeaconStore};
use kzg::Error as KzgError;
use kzg::Kzg;
use slog::{debug, error};
use slot_clock::SlotClock;
use ssz_types::{Error, FixedVector, VariableList};
use ssz_types::{Error, VariableList};
use std::collections::HashSet;
use std::fmt;
use std::fmt::Debug;
use std::sync::Arc;
use strum::IntoStaticStr;
use task_executor::TaskExecutor;
use types::beacon_block_body::KzgCommitments;
use types::blob_sidecar::{BlobIdentifier, BlobSidecar, FixedBlobSidecarList};
use types::consts::deneb::MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS;
use types::ssz_tagged_signed_beacon_block;
use types::{
BeaconBlockRef, BlobSidecarList, ChainSpec, Epoch, EthSpec, FullPayload, Hash256,
SignedBeaconBlock, SignedBeaconBlockHeader, Slot,
};
use types::{BlobSidecarList, ChainSpec, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot};
mod overflow_lru_cache;
@ -50,12 +47,20 @@ pub enum AvailabilityCheckError {
},
IncorrectFork,
BlobIndexInvalid(u64),
UnorderedBlobs {
blob_index: u64,
expected_index: u64,
},
StoreError(store::Error),
DecodeError(ssz::DecodeError),
BlockBlobRootMismatch {
block_root: Hash256,
blob_block_root: Hash256,
},
BlockBlobSlotMismatch {
block_slot: Slot,
blob_slot: Slot,
},
}
impl From<ssz_types::Error> for AvailabilityCheckError {
@ -92,12 +97,23 @@ pub struct DataAvailabilityChecker<T: BeaconChainTypes> {
///
/// Indicates if the block is fully `Available` or if we need blobs or blocks
/// to "complete" the requirements for an `AvailableBlock`.
#[derive(Debug, PartialEq)]
#[derive(PartialEq)]
pub enum Availability<T: EthSpec> {
MissingComponents(Hash256),
Available(Box<AvailableExecutedBlock<T>>),
}
impl<T: EthSpec> Debug for Availability<T> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
Self::MissingComponents(block_root) => {
write!(f, "MissingComponents({})", block_root)
}
Self::Available(block) => write!(f, "Available({:?})", block.import_data.block_root),
}
}
}
impl<T: EthSpec> Availability<T> {
/// Returns all the blob identifiers associated with an `AvailableBlock`.
/// Returns `None` if avaiability hasn't been fully satisfied yet.
@ -230,86 +246,51 @@ impl<T: BeaconChainTypes> DataAvailabilityChecker<T> {
/// Checks if a block is available, returns a `MaybeAvailableBlock` that may include the fully
/// available block.
pub fn check_availability(
pub fn check_rpc_block_availability(
&self,
block: BlockWrapper<T::EthSpec>,
block: RpcBlock<T::EthSpec>,
) -> Result<MaybeAvailableBlock<T::EthSpec>, 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 filtered_blobs = blob_list.iter().flatten().cloned().collect();
let verified_blobs = verify_kzg_for_blob_list(filtered_blobs, kzg)?;
Ok(MaybeAvailableBlock::Available(
self.check_availability_with_blobs(block, verified_blobs)?,
))
let (block, blobs) = block.deconstruct();
match blobs {
None => {
if self.blobs_required_for_block(&block) {
Ok(MaybeAvailableBlock::AvailabilityPending(block))
} else {
Ok(MaybeAvailableBlock::Available(AvailableBlock {
block,
blobs: None,
}))
}
}
Some(blob_list) => {
let verified_blobs = if self.blobs_required_for_block(&block) {
let kzg = self
.kzg
.as_ref()
.ok_or(AvailabilityCheckError::KzgNotInitialized)?;
verify_kzg_for_blob_list(&blob_list, kzg)?;
Some(blob_list)
} else {
None
};
Ok(MaybeAvailableBlock::Available(AvailableBlock {
block,
blobs: verified_blobs,
}))
}
}
}
/// 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::EthSpec>>,
blobs: KzgVerifiedBlobList<T::EthSpec>,
) -> Result<AvailableBlock<T::EthSpec>, AvailabilityCheckError> {
match self.check_availability_without_blobs(block)? {
MaybeAvailableBlock::Available(block) => Ok(block),
MaybeAvailableBlock::AvailabilityPending(pending_block) => {
pending_block.make_available(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::EthSpec>>,
) -> Result<MaybeAvailableBlock<T::EthSpec>, AvailabilityCheckError> {
let blob_requirements = self.get_blob_requirements(&block)?;
let blobs = match blob_requirements {
BlobRequirements::EmptyBlobs => VerifiedBlobs::EmptyBlobs,
BlobRequirements::NotRequired => VerifiedBlobs::NotRequired,
BlobRequirements::PreDeneb => VerifiedBlobs::PreDeneb,
BlobRequirements::Required => {
return Ok(MaybeAvailableBlock::AvailabilityPending(
AvailabilityPendingBlock { block },
))
}
};
Ok(MaybeAvailableBlock::Available(AvailableBlock {
block,
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::EthSpec, FullPayload<T::EthSpec>>>,
) -> Result<BlobRequirements, AvailabilityCheckError> {
let verified_blobs =
if let Ok(block_kzg_commitments) = block.message().body().blob_kzg_commitments() {
if self.da_check_required(block.epoch()) {
if block_kzg_commitments.is_empty() {
BlobRequirements::EmptyBlobs
} else {
BlobRequirements::Required
}
} else {
BlobRequirements::NotRequired
}
} else {
BlobRequirements::PreDeneb
};
Ok(verified_blobs)
fn blobs_required_for_block(&self, block: &SignedBeaconBlock<T::EthSpec>) -> bool {
let block_within_da_period = self.da_check_required(block.epoch());
let block_has_kzg_commitments = block
.message()
.body()
.blob_kzg_commitments()
.map_or(false, |commitments| !commitments.is_empty());
block_within_da_period && block_has_kzg_commitments
}
/// The epoch at which we require a data availability check in block processing.
@ -340,6 +321,87 @@ impl<T: BeaconChainTypes> DataAvailabilityChecker<T> {
}
}
/// Verifies an `SignedBeaconBlock` against a set of KZG verified blobs.
/// This does not check whether a block *should* have blobs, these checks should have been
/// completed when producing the `AvailabilityPendingBlock`.
pub fn make_available<T: EthSpec>(
block: Arc<SignedBeaconBlock<T>>,
blobs: Vec<KzgVerifiedBlob<T>>,
) -> Result<AvailableBlock<T>, AvailabilityCheckError> {
let blobs = VariableList::new(blobs.into_iter().map(|blob| blob.to_blob()).collect())?;
consistency_checks(&block, &blobs)?;
Ok(AvailableBlock {
block,
blobs: Some(blobs),
})
}
/// Makes the following checks to ensure that the list of blobs correspond block:
///
/// * Check that a block is post-deneb
/// * Checks that the number of blobs is equal to the length of kzg commitments in the list
/// * Checks that the index, slot, root and kzg_commitment in the block match the blobs in the correct order
///
/// Returns `Ok(())` if all consistency checks pass and an error otherwise.
pub fn consistency_checks<T: EthSpec>(
block: &SignedBeaconBlock<T>,
blobs: &[Arc<BlobSidecar<T>>],
) -> Result<(), AvailabilityCheckError> {
let Ok(block_kzg_commitments) = block
.message()
.body()
.blob_kzg_commitments() else {
return Ok(())
};
if blobs.len() != block_kzg_commitments.len() {
return Err(AvailabilityCheckError::NumBlobsMismatch {
num_kzg_commitments: block_kzg_commitments.len(),
num_blobs: blobs.len(),
});
}
if block_kzg_commitments.is_empty() {
return Ok(());
}
let block_root = blobs
.first()
.map(|blob| blob.block_root)
.unwrap_or(block.canonical_root());
for (index, (block_commitment, blob)) in
block_kzg_commitments.iter().zip(blobs.iter()).enumerate()
{
let index = index as u64;
if index != blob.index {
return Err(AvailabilityCheckError::UnorderedBlobs {
blob_index: blob.index,
expected_index: index,
});
}
if block_root != blob.block_root {
return Err(AvailabilityCheckError::BlockBlobRootMismatch {
block_root,
blob_block_root: blob.block_root,
});
}
if block.slot() != blob.slot {
return Err(AvailabilityCheckError::BlockBlobSlotMismatch {
block_slot: block.slot(),
blob_slot: blob.slot,
});
}
if *block_commitment != blob.kzg_commitment {
return Err(AvailabilityCheckError::KzgCommitmentMismatch {
blob_index: blob.index,
});
}
}
Ok(())
}
pub fn start_availability_cache_maintenance_service<T: BeaconChainTypes>(
executor: TaskExecutor,
chain: Arc<BeaconChain<T>>,
@ -425,244 +487,37 @@ async fn availability_cache_maintenance_service<T: BeaconChainTypes>(
}
}
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
PreDeneb,
}
/// A wrapper over a `SignedBeaconBlock` where we have not verified availability of
/// corresponding `BlobSidecar`s and hence, is not ready for import into fork choice.
///
/// Note: This wrapper does not necessarily correspond to a pre-deneb block as a pre-deneb
/// block that is ready for import will be of type `AvailableBlock` with its `blobs` field
/// set to `VerifiedBlobs::PreDeneb`.
#[derive(Clone, Debug, PartialEq)]
pub struct AvailabilityPendingBlock<E: EthSpec> {
block: Arc<SignedBeaconBlock<E>>,
}
impl<E: EthSpec> AvailabilityPendingBlock<E> {
pub fn slot(&self) -> Slot {
self.block.slot()
}
pub fn num_blobs_expected(&self) -> usize {
self.block.num_expected_blobs()
}
pub fn get_all_blob_ids(&self, block_root: Option<Hash256>) -> Vec<BlobIdentifier> {
self.block.get_expected_blob_ids(block_root)
}
pub fn get_filtered_blob_ids(
&self,
block_root: Option<Hash256>,
filter: impl Fn(usize, Hash256) -> bool,
) -> Vec<BlobIdentifier> {
self.block.get_filtered_blob_ids(block_root, filter)
}
}
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)
}
/// Verifies an AvailabilityPendingBlock against a set of KZG verified blobs.
/// This does not check whether a block *should* have blobs, these checks should have been
/// completed when producing the `AvailabilityPendingBlock`.
pub fn make_available(
self,
blobs: Vec<KzgVerifiedBlob<E>>,
) -> Result<AvailableBlock<E>, AvailabilityCheckError> {
let block_kzg_commitments = self.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: self.block,
blobs: VerifiedBlobs::Available(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
PreDeneb,
}
impl<E: EthSpec> VerifiedBlobs<E> {
pub fn to_blobs(self) -> Option<BlobSidecarList<E>> {
match self {
Self::Available(blobs) => Some(blobs),
Self::NotRequired => None,
Self::EmptyBlobs => None,
Self::PreDeneb => None,
}
}
}
/// A fully available block that is ready to be imported into fork choice.
#[derive(Clone, Debug, PartialEq)]
pub struct AvailableBlock<E: EthSpec> {
block: Arc<SignedBeaconBlock<E>>,
blobs: VerifiedBlobs<E>,
blobs: Option<BlobSidecarList<E>>,
}
impl<E: EthSpec> AvailableBlock<E> {
pub fn block(&self) -> &SignedBeaconBlock<E> {
&self.block
}
pub fn da_check_required(&self) -> bool {
match self.blobs {
VerifiedBlobs::PreDeneb | VerifiedBlobs::NotRequired => false,
VerifiedBlobs::EmptyBlobs | VerifiedBlobs::Available(_) => true,
}
}
pub fn deconstruct(self) -> (Arc<SignedBeaconBlock<E>>, Option<BlobSidecarList<E>>) {
match self.blobs {
VerifiedBlobs::EmptyBlobs | VerifiedBlobs::NotRequired | VerifiedBlobs::PreDeneb => {
(self.block, None)
}
VerifiedBlobs::Available(blobs) => (self.block, Some(blobs)),
}
}
pub fn blobs(&self) -> Option<&BlobSidecarList<E>> {
match &self.blobs {
VerifiedBlobs::Available(blobs) => Some(blobs),
_ => None,
}
}
}
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>> {
pub fn block_cloned(&self) -> Arc<SignedBeaconBlock<E>> {
self.block.clone()
}
fn canonical_root(&self) -> Hash256 {
self.block.canonical_root()
pub fn blobs(&self) -> Option<&BlobSidecarList<E>> {
self.blobs.as_ref()
}
fn into_block_wrapper(self) -> BlockWrapper<E> {
let (block, blobs_opt) = self.deconstruct();
if let Some(blobs) = blobs_opt {
let blobs_vec = blobs.iter().cloned().map(Option::Some).collect::<Vec<_>>();
BlockWrapper::BlockAndBlobs(block, FixedVector::from(blobs_vec))
} else {
BlockWrapper::Block(block)
}
pub fn deconstruct(self) -> (Arc<SignedBeaconBlock<E>>, Option<BlobSidecarList<E>>) {
let AvailableBlock { block, blobs } = self;
(block, blobs)
}
}
// The standard implementation of Encode for SignedBeaconBlock
// requires us to use ssz(enum_behaviour = "transparent"). This
// prevents us from implementing Decode. We need to use a
// custom Encode and Decode in this wrapper object that essentially
// encodes it as if it were ssz(enum_behaviour = "union")
impl<E: EthSpec> ssz::Encode for AvailabilityPendingBlock<E> {
fn is_ssz_fixed_len() -> bool {
ssz_tagged_signed_beacon_block::encode::is_ssz_fixed_len()
}
fn ssz_append(&self, buf: &mut Vec<u8>) {
ssz_tagged_signed_beacon_block::encode::ssz_append(self.block.as_ref(), buf);
}
fn ssz_bytes_len(&self) -> usize {
ssz_tagged_signed_beacon_block::encode::ssz_bytes_len(self.block.as_ref())
}
}
impl<E: EthSpec> ssz::Decode for AvailabilityPendingBlock<E> {
fn is_ssz_fixed_len() -> bool {
ssz_tagged_signed_beacon_block::decode::is_ssz_fixed_len()
}
fn from_ssz_bytes(bytes: &[u8]) -> Result<Self, ssz::DecodeError> {
Ok(Self {
block: Arc::new(ssz_tagged_signed_beacon_block::decode::from_ssz_bytes(
bytes,
)?),
})
}
}
#[cfg(test)]
mod test {
#[test]
fn check_encode_decode_availability_pending_block() {
// todo.. (difficult to create default beacon blocks to test)
}
#[derive(Debug, Clone)]
pub enum MaybeAvailableBlock<E: EthSpec> {
/// This variant is fully available.
/// i.e. for pre-deneb 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(Arc<SignedBeaconBlock<E>>),
}

View File

@ -29,8 +29,10 @@
use crate::beacon_chain::BeaconStore;
use crate::blob_verification::KzgVerifiedBlob;
use crate::block_verification::{AvailabilityPendingExecutedBlock, AvailableExecutedBlock};
use crate::data_availability_checker::{Availability, AvailabilityCheckError};
use crate::block_verification_types::{
AsBlock, AvailabilityPendingExecutedBlock, AvailableExecutedBlock,
};
use crate::data_availability_checker::{make_available, Availability, AvailabilityCheckError};
use crate::store::{DBColumn, KeyValueStore};
use crate::BeaconChainTypes;
use lru::LruCache;
@ -102,7 +104,7 @@ impl<T: EthSpec> PendingComponents<T> {
pub fn epoch(&self) -> Option<Epoch> {
self.executed_block
.as_ref()
.map(|pending_block| pending_block.block.as_block().epoch())
.map(|pending_block| pending_block.block.epoch())
.or_else(|| {
for maybe_blob in self.verified_blobs.iter() {
if maybe_blob.is_some() {
@ -119,7 +121,7 @@ impl<T: EthSpec> PendingComponents<T> {
let block_opt = self
.executed_block
.as_ref()
.map(|block| block.block.block.clone());
.map(|block| block.block.clone());
let blobs = self
.verified_blobs
.iter()
@ -538,7 +540,7 @@ impl<T: BeaconChainTypes> OverflowLRUCache<T> {
import_data,
payload_verification_outcome,
} = executed_block;
let available_block = block.make_available(vec![])?;
let available_block = make_available(block, vec![])?;
return Ok(Availability::Available(Box::new(
AvailableExecutedBlock::new(
available_block,
@ -588,7 +590,7 @@ impl<T: BeaconChainTypes> OverflowLRUCache<T> {
return Ok(Availability::MissingComponents(import_data.block_root))
};
let available_block = block.make_available(verified_blobs)?;
let available_block = make_available(block, verified_blobs)?;
Ok(Availability::Available(Box::new(
AvailableExecutedBlock::new(
available_block,
@ -758,7 +760,6 @@ impl<T: BeaconChainTypes> OverflowLRUCache<T> {
value_bytes.as_slice(),
)?
.block
.as_block()
.epoch()
}
OverflowKey::Blob(_, _) => {
@ -853,8 +854,8 @@ mod test {
blob_verification::{
validate_blob_sidecar_for_gossip, verify_kzg_for_blob, GossipVerifiedBlob,
},
block_verification::{BlockImportData, PayloadVerificationOutcome},
data_availability_checker::AvailabilityPendingBlock,
block_verification::PayloadVerificationOutcome,
block_verification_types::BlockImportData,
eth1_finalization_cache::Eth1FinalizationData,
test_utils::{BaseHarnessType, BeaconChainHarness, DiskHarnessType},
};
@ -1129,10 +1130,6 @@ mod test {
};
let slot = block.slot();
let apb: AvailabilityPendingBlock<E> = AvailabilityPendingBlock {
block: Arc::new(block),
};
let consensus_context = ConsensusContext::<E>::new(slot);
let import_data: BlockImportData<E> = BlockImportData {
block_root,
@ -1149,7 +1146,7 @@ mod test {
};
let availability_pending_block = AvailabilityPendingExecutedBlock {
block: apb,
block: Arc::new(block),
import_data,
payload_verification_outcome,
};
@ -1301,7 +1298,7 @@ mod test {
// we need blocks with blobs
continue;
}
let root = pending_block.block.block.canonical_root();
let root = pending_block.block.canonical_root();
pending_blocks.push_back(pending_block);
pending_blobs.push_back(blobs);
roots.push_back(root);
@ -1462,7 +1459,7 @@ mod test {
// we need blocks with blobs
continue;
}
let root = pending_block.block.as_block().canonical_root();
let root = pending_block.block.canonical_root();
let epoch = pending_block
.block
.as_block()

View File

@ -12,6 +12,7 @@ pub mod blob_verification;
pub mod block_reward;
mod block_times_cache;
mod block_verification;
pub mod block_verification_types;
pub mod builder;
pub mod canonical_head;
pub mod capella_readiness;
@ -69,10 +70,12 @@ 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, AvailabilityPendingExecutedBlock, BlockError, ExecutedBlock,
ExecutionPayloadError, ExecutionPendingBlock, GossipVerifiedBlock, IntoExecutionPendingBlock,
IntoGossipVerifiedBlockContents, PayloadVerificationOutcome, PayloadVerificationStatus,
get_block_root, BlockError, ExecutionPayloadError, ExecutionPendingBlock, GossipVerifiedBlock,
IntoExecutionPendingBlock, IntoGossipVerifiedBlockContents, PayloadVerificationOutcome,
PayloadVerificationStatus,
};
pub use block_verification_types::AvailabilityPendingExecutedBlock;
pub use block_verification_types::ExecutedBlock;
pub use canonical_head::{CachedHead, CanonicalHead, CanonicalHeadRwLock};
pub use eth1_chain::{Eth1Chain, Eth1ChainBackend};
pub use events::ServerSentEventHandler;

View File

@ -1,4 +1,4 @@
use crate::blob_verification::{AsBlock, BlockWrapper};
use crate::block_verification_types::{AsBlock, RpcBlock};
use crate::observed_operations::ObservationOutcome;
pub use crate::persisted_beacon_chain::PersistedBeaconChain;
pub use crate::{
@ -694,18 +694,18 @@ where
.execution_block_generator()
}
pub fn get_head_block(&self) -> BlockWrapper<E> {
pub fn get_head_block(&self) -> RpcBlock<E> {
let block = self.chain.head_beacon_block();
let block_root = block.canonical_root();
let blobs = self.chain.get_blobs(&block_root).unwrap();
BlockWrapper::new(block, blobs)
RpcBlock::new(block, blobs).unwrap()
}
pub fn get_full_block(&self, block_root: &Hash256) -> BlockWrapper<E> {
pub fn get_full_block(&self, block_root: &Hash256) -> RpcBlock<E> {
let block = self.chain.get_blinded_block(block_root).unwrap().unwrap();
let full_block = self.chain.store.make_full_block(block_root, block).unwrap();
let blobs = self.chain.get_blobs(block_root).unwrap();
BlockWrapper::new(Arc::new(full_block), blobs)
RpcBlock::new(Arc::new(full_block), blobs).unwrap()
}
pub fn get_all_validators(&self) -> Vec<usize> {
@ -1873,18 +1873,31 @@ where
(deposits, state)
}
pub async fn process_block<B: Into<BlockWrapper<E>>>(
pub async fn process_block(
&self,
slot: Slot,
block_root: Hash256,
block: B,
block_contents: BlockContentsTuple<E, FullPayload<E>>,
) -> Result<SignedBeaconBlockHash, BlockError<E>> {
self.set_current_slot(slot);
let (block, blobs) = block_contents;
// Note: we are just dropping signatures here and skipping signature verification.
let blobs_without_signatures = blobs.as_ref().map(|blobs| {
VariableList::from(
blobs
.into_iter()
.map(|blob| blob.message.clone())
.collect::<Vec<_>>(),
)
});
let block_hash: SignedBeaconBlockHash = self
.chain
.process_block(block_root, block.into(), NotifyExecutionLayer::Yes, || {
Ok(())
})
.process_block(
block_root,
RpcBlock::new(Arc::new(block), blobs_without_signatures).unwrap(),
NotifyExecutionLayer::Yes,
|| Ok(()),
)
.await?
.try_into()
.unwrap();
@ -1892,16 +1905,25 @@ where
Ok(block_hash)
}
pub async fn process_block_result<B: Into<BlockWrapper<E>>>(
pub async fn process_block_result(
&self,
block: B,
block_contents: BlockContentsTuple<E, FullPayload<E>>,
) -> Result<SignedBeaconBlockHash, BlockError<E>> {
let wrapped_block = block.into();
let (block, blobs) = block_contents;
// Note: we are just dropping signatures here and skipping signature verification.
let blobs_without_signatures = blobs.as_ref().map(|blobs| {
VariableList::from(
blobs
.into_iter()
.map(|blob| blob.message.clone())
.collect::<Vec<_>>(),
)
});
let block_hash: SignedBeaconBlockHash = self
.chain
.process_block(
wrapped_block.canonical_root(),
wrapped_block,
block.canonical_root(),
RpcBlock::new(Arc::new(block), blobs_without_signatures).unwrap(),
NotifyExecutionLayer::Yes,
|| Ok(()),
)
@ -1976,11 +1998,16 @@ where
BlockError<E>,
> {
self.set_current_slot(slot);
let (block, new_state) = self.make_block(state, slot).await;
let (block_contents, new_state) = self.make_block(state, slot).await;
let block_hash = self
.process_block(slot, block.0.canonical_root(), block.clone())
.process_block(
slot,
block_contents.0.canonical_root(),
block_contents.clone(),
)
.await?;
Ok((block_hash, block, new_state))
Ok((block_hash, block_contents, new_state))
}
pub fn attest_block(

View File

@ -1,6 +1,6 @@
#![cfg(not(debug_assertions))]
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::block_verification_types::RpcBlock;
use beacon_chain::test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy};
use beacon_chain::{StateSkipConfig, WhenSlotSkipped};
use lazy_static::lazy_static;
@ -133,11 +133,11 @@ async fn produces_attestations() {
assert_eq!(data.target.epoch, state.current_epoch(), "bad target epoch");
assert_eq!(data.target.root, target_root, "bad target root");
let block_wrapper =
BlockWrapper::<MainnetEthSpec>::new(Arc::new(block.clone()), blobs.clone());
let beacon_chain::blob_verification::MaybeAvailableBlock::Available(available_block) = chain
let rpc_block =
RpcBlock::<MainnetEthSpec>::new(Arc::new(block.clone()), blobs.clone()).unwrap();
let beacon_chain::data_availability_checker::MaybeAvailableBlock::Available(available_block) = chain
.data_availability_checker
.check_availability(block_wrapper)
.check_rpc_block_availability(rpc_block)
.unwrap()
else {
panic!("block should be available")
@ -209,10 +209,10 @@ async fn early_attester_cache_old_request() {
.get_blobs(&head.beacon_block_root)
.expect("should get blobs");
let block_wrapper = BlockWrapper::<MainnetEthSpec>::new(head.beacon_block.clone(), head_blobs);
let beacon_chain::blob_verification::MaybeAvailableBlock::Available(available_block) = harness.chain
let rpc_block = RpcBlock::<MainnetEthSpec>::new(head.beacon_block.clone(), head_blobs).unwrap();
let beacon_chain::data_availability_checker::MaybeAvailableBlock::Available(available_block) = harness.chain
.data_availability_checker
.check_availability(block_wrapper)
.check_rpc_block_availability(rpc_block)
.unwrap()
else {
panic!("block should be available")

View File

@ -1,12 +1,10 @@
#![cfg(not(debug_assertions))]
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::block_verification_types::{AsBlock, ExecutedBlock, RpcBlock};
use beacon_chain::test_utils::BlobSignatureKey;
use beacon_chain::{
blob_verification::AsBlock,
test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType},
AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes, ExecutedBlock,
ExecutionPendingBlock,
AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes, ExecutionPendingBlock,
};
use beacon_chain::{
BeaconSnapshot, BlockError, ChainSegmentResult, IntoExecutionPendingBlock, NotifyExecutionLayer,
@ -156,11 +154,13 @@ fn get_harness(validator_count: usize) -> BeaconChainHarness<EphemeralHarnessTyp
fn chain_segment_blocks(
chain_segment: &[BeaconSnapshot<E>],
blobs: &[Option<BlobSidecarList<E>>],
) -> Vec<BlockWrapper<E>> {
) -> Vec<RpcBlock<E>> {
chain_segment
.iter()
.zip(blobs.into_iter())
.map(|(snapshot, blobs)| BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone()))
.map(|(snapshot, blobs)| {
RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()
})
.collect()
}
@ -217,7 +217,7 @@ fn update_parent_roots(snapshots: &mut [BeaconSnapshot<E>]) {
async fn chain_segment_full_segment() {
let harness = get_harness(VALIDATOR_COUNT);
let (chain_segment, chain_segment_blobs) = get_chain_segment().await;
let blocks: Vec<BlockWrapper<E>> = chain_segment_blocks(&chain_segment, &chain_segment_blobs)
let blocks: Vec<RpcBlock<E>> = chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter()
.map(|block| block.into())
.collect();
@ -256,11 +256,10 @@ async fn chain_segment_varying_chunk_size() {
for chunk_size in &[1, 2, 3, 5, 31, 32, 33, 42] {
let harness = get_harness(VALIDATOR_COUNT);
let (chain_segment, chain_segment_blobs) = get_chain_segment().await;
let blocks: Vec<BlockWrapper<E>> =
chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter()
.map(|block| block.into())
.collect();
let blocks: Vec<RpcBlock<E>> = chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter()
.map(|block| block.into())
.collect();
harness
.chain
@ -299,11 +298,10 @@ async fn chain_segment_non_linear_parent_roots() {
/*
* Test with a block removed.
*/
let mut blocks: Vec<BlockWrapper<E>> =
chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter()
.map(|block| block.into())
.collect();
let mut blocks: Vec<RpcBlock<E>> = chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter()
.map(|block| block.into())
.collect();
blocks.remove(2);
assert!(
@ -321,11 +319,10 @@ async fn chain_segment_non_linear_parent_roots() {
/*
* Test with a modified parent root.
*/
let mut blocks: Vec<BlockWrapper<E>> =
chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter()
.map(|block| block.into())
.collect();
let mut blocks: Vec<RpcBlock<E>> = chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter()
.map(|block| block.into())
.collect();
let (mut block, signature) = blocks[3].as_block().clone().deconstruct();
*block.parent_root_mut() = Hash256::zero();
@ -357,11 +354,10 @@ async fn chain_segment_non_linear_slots() {
* Test where a child is lower than the parent.
*/
let mut blocks: Vec<BlockWrapper<E>> =
chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter()
.map(|block| block.into())
.collect();
let mut blocks: Vec<RpcBlock<E>> = chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter()
.map(|block| block.into())
.collect();
let (mut block, signature) = blocks[3].as_block().clone().deconstruct();
*block.slot_mut() = Slot::new(0);
blocks[3] = Arc::new(SignedBeaconBlock::from_block(block, signature)).into();
@ -382,11 +378,10 @@ async fn chain_segment_non_linear_slots() {
* Test where a child is equal to the parent.
*/
let mut blocks: Vec<BlockWrapper<E>> =
chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter()
.map(|block| block.into())
.collect();
let mut blocks: Vec<RpcBlock<E>> = chain_segment_blocks(&chain_segment, &chain_segment_blobs)
.into_iter()
.map(|block| block.into())
.collect();
let (mut block, signature) = blocks[3].as_block().clone().deconstruct();
*block.slot_mut() = blocks[2].slot();
blocks[3] = Arc::new(SignedBeaconBlock::from_block(block, signature)).into();
@ -412,10 +407,12 @@ async fn assert_invalid_signature(
snapshots: &[BeaconSnapshot<E>],
item: &str,
) {
let blocks: Vec<BlockWrapper<E>> = snapshots
let blocks: Vec<RpcBlock<E>> = snapshots
.iter()
.zip(chain_segment_blobs.iter())
.map(|(snapshot, blobs)| BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone()))
.map(|(snapshot, blobs)| {
RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()
})
.collect();
// Ensure the block will be rejected if imported in a chain segment.
@ -440,7 +437,9 @@ async fn assert_invalid_signature(
.iter()
.take(block_index)
.zip(chain_segment_blobs.iter())
.map(|(snapshot, blobs)| BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone()))
.map(|(snapshot, blobs)| {
RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()
})
.collect();
// We don't care if this fails, we just call this to ensure that all prior blocks have been
// imported prior to this test.
@ -454,10 +453,11 @@ async fn assert_invalid_signature(
.chain
.process_block(
snapshots[block_index].beacon_block.canonical_root(),
BlockWrapper::new(
RpcBlock::new(
snapshots[block_index].beacon_block.clone(),
chain_segment_blobs[block_index].clone(),
),
)
.unwrap(),
NotifyExecutionLayer::Yes,
|| Ok(()),
)
@ -509,7 +509,7 @@ async fn invalid_signature_gossip_block() {
.take(block_index)
.zip(chain_segment_blobs.iter())
.map(|(snapshot, blobs)| {
BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone())
RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()
})
.collect();
harness
@ -552,11 +552,11 @@ async fn invalid_signature_block_proposal() {
block.clone(),
junk_signature(),
));
let blocks: Vec<BlockWrapper<E>> = snapshots
let blocks: Vec<RpcBlock<E>> = snapshots
.iter()
.zip(chain_segment_blobs.iter())
.map(|(snapshot, blobs)| {
BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone())
RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()
})
.collect::<Vec<_>>();
// Ensure the block will be rejected if imported in a chain segment.
@ -765,11 +765,11 @@ async fn invalid_signature_deposit() {
Arc::new(SignedBeaconBlock::from_block(block, signature));
update_parent_roots(&mut snapshots);
update_proposal_signatures(&mut snapshots, &harness);
let blocks: Vec<BlockWrapper<E>> = snapshots
let blocks: Vec<RpcBlock<E>> = snapshots
.iter()
.zip(chain_segment_blobs.iter())
.map(|(snapshot, blobs)| {
BlockWrapper::new(snapshot.beacon_block.clone(), blobs.clone())
RpcBlock::new(snapshot.beacon_block.clone(), blobs.clone()).unwrap()
})
.collect();
assert!(

View File

@ -221,7 +221,7 @@ impl InvalidPayloadRig {
let head = self.harness.chain.head_snapshot();
let state = head.beacon_state.clone_with_only_committee_caches();
let slot = slot_override.unwrap_or(state.slot() + 1);
let ((block, _), post_state) = self.harness.make_block(state, slot).await;
let ((block, blobs), post_state) = self.harness.make_block(state, slot).await;
let block_root = block.canonical_root();
let set_new_payload = |payload: Payload| match payload {
@ -285,7 +285,7 @@ impl InvalidPayloadRig {
}
let root = self
.harness
.process_block(slot, block.canonical_root(), block.clone())
.process_block(slot, block.canonical_root(), (block.clone(), blobs.clone()))
.await
.unwrap();
@ -326,7 +326,7 @@ impl InvalidPayloadRig {
match self
.harness
.process_block(slot, block.canonical_root(), block)
.process_block(slot, block.canonical_root(), (block, blobs))
.await
{
Err(error) if evaluate_error(&error) => (),

View File

@ -1,7 +1,7 @@
#![cfg(not(debug_assertions))]
use beacon_chain::attestation_verification::Error as AttnError;
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::block_verification_types::RpcBlock;
use beacon_chain::builder::BeaconChainBuilder;
use beacon_chain::schema_change::migrate_schema;
use beacon_chain::test_utils::{
@ -10,7 +10,7 @@ use beacon_chain::test_utils::{
};
use beacon_chain::validator_monitor::DEFAULT_INDIVIDUAL_TRACKING_THRESHOLD;
use beacon_chain::{
blob_verification::MaybeAvailableBlock, historical_blocks::HistoricalBlockError,
data_availability_checker::MaybeAvailableBlock, historical_blocks::HistoricalBlockError,
migrate::MigratorConfig, BeaconChain, BeaconChainError, BeaconChainTypes, BeaconSnapshot,
ChainConfig, NotifyExecutionLayer, ServerSentEventHandler, WhenSlotSkipped,
};
@ -2039,7 +2039,10 @@ async fn garbage_collect_temp_states_from_failed_block() {
// The block should be rejected, but should store a bunch of temporary states.
harness.set_current_slot(block_slot);
harness.process_block_result(block).await.unwrap_err();
harness
.process_block_result((block, None))
.await
.unwrap_err();
assert_eq!(
store.iter_temporary_state_roots().count(),
@ -2176,7 +2179,7 @@ async fn weak_subjectivity_sync() {
beacon_chain
.process_block(
full_block.canonical_root(),
BlockWrapper::new(Arc::new(full_block), blobs),
RpcBlock::new(Arc::new(full_block), blobs).unwrap(),
NotifyExecutionLayer::Yes,
|| Ok(()),
)
@ -2236,7 +2239,7 @@ async fn weak_subjectivity_sync() {
if let MaybeAvailableBlock::Available(block) = harness
.chain
.data_availability_checker
.check_availability(BlockWrapper::new(Arc::new(full_block), blobs))
.check_rpc_block_availability(RpcBlock::new(Arc::new(full_block), blobs).unwrap())
.expect("should check availability")
{
available_blocks.push(block);
@ -2456,14 +2459,14 @@ async fn revert_minority_fork_on_resume() {
harness1.process_attestations(attestations.clone());
harness2.process_attestations(attestations);
let ((block, _), new_state) = harness1.make_block(state, slot).await;
let ((block, blobs), new_state) = harness1.make_block(state, slot).await;
harness1
.process_block(slot, block.canonical_root(), block.clone())
.process_block(slot, block.canonical_root(), (block.clone(), blobs.clone()))
.await
.unwrap();
harness2
.process_block(slot, block.canonical_root(), block.clone())
.process_block(slot, block.canonical_root(), (block.clone(), blobs.clone()))
.await
.unwrap();
@ -2497,17 +2500,17 @@ async fn revert_minority_fork_on_resume() {
harness2.process_attestations(attestations);
// Minority chain block (no attesters).
let ((block1, _), new_state1) = harness1.make_block(state1, slot).await;
let ((block1, blobs1), new_state1) = harness1.make_block(state1, slot).await;
harness1
.process_block(slot, block1.canonical_root(), block1)
.process_block(slot, block1.canonical_root(), (block1, blobs1))
.await
.unwrap();
state1 = new_state1;
// Majority chain block (all attesters).
let ((block2, _), new_state2) = harness2.make_block(state2, slot).await;
let ((block2, blobs2), new_state2) = harness2.make_block(state2, slot).await;
harness2
.process_block(slot, block2.canonical_root(), block2.clone())
.process_block(slot, block2.canonical_root(), (block2.clone(), blobs2))
.await
.unwrap();
@ -2560,7 +2563,7 @@ async fn revert_minority_fork_on_resume() {
let initial_split_slot = resumed_harness.chain.store.get_split_slot();
for block in &majority_blocks {
resumed_harness
.process_block_result(block.clone())
.process_block_result((block.clone(), None))
.await
.unwrap();

View File

@ -1,6 +1,6 @@
use crate::metrics;
use beacon_chain::blob_verification::AsBlock;
use beacon_chain::block_verification_types::AsBlock;
use beacon_chain::validator_monitor::{get_block_delay_ms, timestamp_now};
use beacon_chain::{
AvailabilityProcessingStatus, BeaconChain, BeaconChainError, BeaconChainTypes, BlockError,

View File

@ -636,7 +636,7 @@ pub async fn proposer_boost_re_org_test(
// Applying block C should cause it to become head regardless (re-org or continuation).
let block_root_c = harness
.process_block_result(block_c.clone())
.process_block_result((block_c.clone(), None))
.await
.unwrap()
.into();

View File

@ -124,7 +124,7 @@ async fn el_error_on_new_payload() {
// Attempt to process the block, which should error.
harness.advance_slot();
assert!(matches!(
harness.process_block_result(block.clone()).await,
harness.process_block_result((block.clone(), None)).await,
Err(BlockError::ExecutionPayloadError(_))
));
@ -143,7 +143,7 @@ async fn el_error_on_new_payload() {
validation_error: None,
},
);
harness.process_block_result(block).await.unwrap();
harness.process_block_result((block, None)).await.unwrap();
let api_response = tester.client.get_node_syncing().await.unwrap().data;
assert_eq!(api_response.el_offline, Some(false));

View File

@ -5,8 +5,8 @@ use crate::{
sync::SyncMessage,
};
use beacon_chain::blob_verification::AsBlock;
use beacon_chain::blob_verification::{BlobError, GossipVerifiedBlob};
use beacon_chain::block_verification_types::AsBlock;
use beacon_chain::store::Error;
use beacon_chain::{
attestation_verification::{self, Error as AttnError, VerifiedAttestation},

View File

@ -2,7 +2,7 @@ use crate::{
service::NetworkMessage,
sync::{manager::BlockProcessType, SyncMessage},
};
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::block_verification_types::RpcBlock;
use beacon_chain::{
builder::Witness, eth1_chain::CachingEth1Backend, test_utils::BeaconChainHarness, BeaconChain,
};
@ -409,7 +409,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
pub fn send_rpc_beacon_block(
self: &Arc<Self>,
block_root: Hash256,
block: BlockWrapper<T::EthSpec>,
block: RpcBlock<T::EthSpec>,
seen_timestamp: Duration,
process_type: BlockProcessType,
) -> Result<(), Error<T::EthSpec>> {
@ -450,7 +450,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
pub fn send_chain_segment(
self: &Arc<Self>,
process_id: ChainSegmentProcessId,
blocks: Vec<BlockWrapper<T::EthSpec>>,
blocks: Vec<RpcBlock<T::EthSpec>>,
) -> Result<(), Error<T::EthSpec>> {
let is_backfill = matches!(&process_id, ChainSegmentProcessId::BackSyncBatchId { .. });
let processor = self.clone();

View File

@ -6,8 +6,9 @@ use crate::sync::{
manager::{BlockProcessType, SyncMessage},
ChainId,
};
use beacon_chain::blob_verification::{AsBlock, BlockWrapper, MaybeAvailableBlock};
use beacon_chain::block_verification_types::{AsBlock, RpcBlock};
use beacon_chain::data_availability_checker::AvailabilityCheckError;
use beacon_chain::data_availability_checker::MaybeAvailableBlock;
use beacon_chain::{
observed_block_producers::Error as ObserveError, validator_monitor::get_block_delay_ms,
AvailabilityProcessingStatus, BeaconChainError, BeaconChainTypes, BlockError,
@ -54,7 +55,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
pub fn generate_rpc_beacon_block_process_fn(
self: Arc<Self>,
block_root: Hash256,
block: BlockWrapper<T::EthSpec>,
block: RpcBlock<T::EthSpec>,
seen_timestamp: Duration,
process_type: BlockProcessType,
) -> AsyncFn {
@ -78,7 +79,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
pub fn generate_rpc_beacon_block_fns(
self: Arc<Self>,
block_root: Hash256,
block: BlockWrapper<T::EthSpec>,
block: RpcBlock<T::EthSpec>,
seen_timestamp: Duration,
process_type: BlockProcessType,
) -> (AsyncFn, BlockingFn) {
@ -106,7 +107,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
pub async fn process_rpc_block(
self: Arc<NetworkBeaconProcessor<T>>,
block_root: Hash256,
block: BlockWrapper<T::EthSpec>,
block: RpcBlock<T::EthSpec>,
seen_timestamp: Duration,
process_type: BlockProcessType,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage>,
@ -315,7 +316,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
pub async fn process_chain_segment(
&self,
sync_type: ChainSegmentProcessId,
downloaded_blocks: Vec<BlockWrapper<T::EthSpec>>,
downloaded_blocks: Vec<RpcBlock<T::EthSpec>>,
notify_execution_layer: NotifyExecutionLayer,
) {
let result = match sync_type {
@ -440,7 +441,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
/// Helper function to process blocks batches which only consumes the chain and blocks to process.
async fn process_blocks<'a>(
&self,
downloaded_blocks: impl Iterator<Item = &'a BlockWrapper<T::EthSpec>>,
downloaded_blocks: impl Iterator<Item = &'a RpcBlock<T::EthSpec>>,
notify_execution_layer: NotifyExecutionLayer,
) -> (usize, Result<(), ChainSegmentFailed>) {
let blocks: Vec<_> = downloaded_blocks.cloned().collect();
@ -473,7 +474,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
/// Helper function to process backfill block batches which only consumes the chain and blocks to process.
fn process_backfill_blocks(
&self,
downloaded_blocks: Vec<BlockWrapper<T::EthSpec>>,
downloaded_blocks: Vec<RpcBlock<T::EthSpec>>,
) -> (usize, Result<(), ChainSegmentFailed>) {
let total_blocks = downloaded_blocks.len();
let available_blocks = match downloaded_blocks
@ -481,7 +482,7 @@ impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
.map(|block| {
self.chain
.data_availability_checker
.check_availability(block)
.check_rpc_block_availability(block)
})
.collect::<Result<Vec<_>, _>>()
{

View File

@ -14,7 +14,7 @@ use crate::sync::network_context::SyncNetworkContext;
use crate::sync::range_sync::{
BatchConfig, BatchId, BatchInfo, BatchOperationOutcome, BatchProcessingResult, BatchState,
};
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::block_verification_types::RpcBlock;
use beacon_chain::{BeaconChain, BeaconChainTypes};
use lighthouse_network::types::{BackFillState, NetworkGlobals};
use lighthouse_network::{PeerAction, PeerId};
@ -55,7 +55,7 @@ impl BatchConfig for BackFillBatchConfig {
fn max_batch_processing_attempts() -> u8 {
MAX_BATCH_PROCESSING_ATTEMPTS
}
fn batch_attempt_hash<T: EthSpec>(blocks: &[BlockWrapper<T>]) -> u64 {
fn batch_attempt_hash<T: EthSpec>(blocks: &[RpcBlock<T>]) -> u64 {
use std::collections::hash_map::DefaultHasher;
use std::hash::{Hash, Hasher};
let mut hasher = DefaultHasher::new();
@ -392,7 +392,7 @@ impl<T: BeaconChainTypes> BackFillSync<T> {
batch_id: BatchId,
peer_id: &PeerId,
request_id: Id,
beacon_block: Option<BlockWrapper<T::EthSpec>>,
beacon_block: Option<RpcBlock<T::EthSpec>>,
) -> Result<ProcessResult, BackFillError> {
// check if we have this batch
let batch = match self.batches.get_mut(&batch_id) {

View File

@ -10,7 +10,7 @@ use super::{
use crate::metrics;
use crate::network_beacon_processor::ChainSegmentProcessId;
use crate::sync::block_lookups::single_block_lookup::LookupId;
use beacon_chain::blob_verification::{AsBlock, BlockWrapper};
use beacon_chain::block_verification_types::{AsBlock, RpcBlock};
use beacon_chain::data_availability_checker::{AvailabilityCheckError, DataAvailabilityChecker};
use beacon_chain::{AvailabilityProcessingStatus, BeaconChainTypes, BlockError};
use lighthouse_network::rpc::RPCError;
@ -34,7 +34,7 @@ mod single_block_lookup;
#[cfg(test)]
mod tests;
pub type DownloadedBlocks<T> = (Hash256, BlockWrapper<T>);
pub type DownloadedBlocks<T> = (Hash256, RpcBlock<T>);
pub type RootBlockTuple<T> = (Hash256, Arc<SignedBeaconBlock<T>>);
pub type RootBlobsTuple<T> = (Hash256, FixedBlobSidecarList<T>);
@ -381,7 +381,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
if !has_pending_parent_request {
let rpc_block = request_ref
.get_downloaded_block()
.unwrap_or(BlockWrapper::Block(block));
.unwrap_or(RpcBlock::new_without_blobs(block));
// This is the correct block, send it for processing
match self.send_block_for_processing(
block_root,
@ -910,11 +910,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
BlockError::ParentUnknown(block) => {
let slot = block.slot();
let parent_root = block.parent_root();
let (block, blobs) = block.deconstruct();
request_ref.add_unknown_parent_components(UnknownParentComponents::new(
Some(block),
blobs,
));
request_ref.add_unknown_parent_components(block.into());
self.search_parent(slot, root, parent_root, peer_id.to_peer_id(), cx);
ShouldRemoveLookup::False
}
@ -1226,7 +1222,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
fn send_block_for_processing(
&mut self,
block_root: Hash256,
block: BlockWrapper<T::EthSpec>,
block: RpcBlock<T::EthSpec>,
duration: Duration,
process_type: BlockProcessType,
cx: &mut SyncNetworkContext<T>,

View File

@ -3,8 +3,8 @@ use super::{BlobRequestId, BlockRequestId, DownloadedBlocks, PeerShouldHave, Res
use crate::sync::block_lookups::single_block_lookup::{State, UnknownParentComponents};
use crate::sync::block_lookups::{RootBlobsTuple, RootBlockTuple};
use crate::sync::{manager::SLOT_IMPORT_TOLERANCE, network_context::SyncNetworkContext};
use beacon_chain::blob_verification::AsBlock;
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::block_verification_types::AsBlock;
use beacon_chain::block_verification_types::RpcBlock;
use beacon_chain::data_availability_checker::DataAvailabilityChecker;
use beacon_chain::BeaconChainTypes;
use lighthouse_network::PeerId;
@ -147,7 +147,7 @@ impl<T: BeaconChainTypes> ParentLookup<T> {
.check_peer_disconnected(peer_id)
}
pub fn add_unknown_parent_block(&mut self, block: BlockWrapper<T::EthSpec>) {
pub fn add_unknown_parent_block(&mut self, block: RpcBlock<T::EthSpec>) {
let next_parent = block.parent_root();
// Cache the block.
@ -203,7 +203,7 @@ impl<T: BeaconChainTypes> ParentLookup<T> {
self,
) -> (
Hash256,
Vec<BlockWrapper<T::EthSpec>>,
Vec<RpcBlock<T::EthSpec>>,
Vec<Hash256>,
SingleBlockLookup<PARENT_FAIL_TOLERANCE, T>,
) {

View File

@ -1,6 +1,6 @@
use crate::sync::block_lookups::{BlobRequestId, BlockRequestId, RootBlobsTuple, RootBlockTuple};
use crate::sync::network_context::SyncNetworkContext;
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::block_verification_types::RpcBlock;
use beacon_chain::data_availability_checker::DataAvailabilityChecker;
use beacon_chain::{get_block_root, BeaconChainTypes};
use lighthouse_network::rpc::methods::BlobsByRootRequest;
@ -138,6 +138,16 @@ pub struct UnknownParentComponents<E: EthSpec> {
pub downloaded_blobs: FixedBlobSidecarList<E>,
}
impl<E: EthSpec> From<RpcBlock<E>> for UnknownParentComponents<E> {
fn from(value: RpcBlock<E>) -> Self {
let (block, blobs) = value.deconstruct();
let fixed_blobs = blobs.map(|blobs| {
FixedBlobSidecarList::from(blobs.into_iter().map(Some).collect::<Vec<_>>())
});
Self::new(Some(block), fixed_blobs)
}
}
impl<E: EthSpec> UnknownParentComponents<E> {
pub fn new(
block: Option<Arc<SignedBeaconBlock<E>>>,
@ -284,7 +294,7 @@ impl<const MAX_ATTEMPTS: u8, T: BeaconChainTypes> SingleBlockLookup<MAX_ATTEMPTS
};
}
pub fn get_downloaded_block(&mut self) -> Option<BlockWrapper<T::EthSpec>> {
pub fn get_downloaded_block(&mut self) -> Option<RpcBlock<T::EthSpec>> {
self.unknown_parent_components
.as_mut()
.and_then(|components| {
@ -302,8 +312,16 @@ impl<const MAX_ATTEMPTS: u8, T: BeaconChainTypes> SingleBlockLookup<MAX_ATTEMPTS
downloaded_block,
downloaded_blobs,
} = components;
downloaded_block.as_ref().map(|block| {
BlockWrapper::BlockAndBlobs(block.clone(), std::mem::take(downloaded_blobs))
downloaded_block.as_ref().and_then(|block| {
//TODO(sean) figure out how to properly deal with a consistency error here,
// should we downscore the peer sending blobs?
let blobs = std::mem::take(downloaded_blobs);
let filtered = blobs
.into_iter()
.filter_map(|b| b.clone())
.collect::<Vec<_>>();
let blobs = VariableList::from(filtered);
RpcBlock::new(block.clone(), Some(blobs)).ok()
})
} else {
None

View File

@ -1474,7 +1474,7 @@ mod deneb_only {
fn parent_block_unknown_parent(mut self) -> Self {
self.bl.parent_block_processed(
self.block_root,
BlockProcessingResult::Err(BlockError::ParentUnknown(BlockWrapper::Block(
BlockProcessingResult::Err(BlockError::ParentUnknown(RpcBlock::new_without_blobs(
self.parent_block.clone().expect("parent block"),
))),
ResponseType::Block,

View File

@ -1,5 +1,5 @@
use beacon_chain::blob_verification::BlockWrapper;
use ssz_types::FixedVector;
use beacon_chain::block_verification_types::RpcBlock;
use ssz_types::VariableList;
use std::{collections::VecDeque, sync::Arc};
use types::{BlobSidecar, EthSpec, SignedBeaconBlock};
@ -16,28 +16,28 @@ pub struct BlocksAndBlobsRequestInfo<T: EthSpec> {
}
impl<T: EthSpec> BlocksAndBlobsRequestInfo<T> {
pub fn add_block_response(&mut self, maybe_block: Option<Arc<SignedBeaconBlock<T>>>) {
match maybe_block {
pub fn add_block_response(&mut self, block_opt: Option<Arc<SignedBeaconBlock<T>>>) {
match block_opt {
Some(block) => self.accumulated_blocks.push_back(block),
None => self.is_blocks_stream_terminated = true,
}
}
pub fn add_sidecar_response(&mut self, maybe_sidecar: Option<Arc<BlobSidecar<T>>>) {
match maybe_sidecar {
pub fn add_sidecar_response(&mut self, sidecar_opt: Option<Arc<BlobSidecar<T>>>) {
match sidecar_opt {
Some(sidecar) => self.accumulated_sidecars.push_back(sidecar),
None => self.is_sidecars_stream_terminated = true,
}
}
pub fn into_responses(self) -> Result<Vec<BlockWrapper<T>>, &'static str> {
pub fn into_responses(self) -> Result<Vec<RpcBlock<T>>, &'static str> {
let BlocksAndBlobsRequestInfo {
accumulated_blocks,
accumulated_sidecars,
..
} = self;
// ASSUMPTION: There can't be more more blobs than blocks. i.e. sending any blob (empty
// There can't be more more blobs than blocks. i.e. sending any blob (empty
// included) for a skipped slot is not permitted.
let mut responses = Vec::with_capacity(accumulated_blocks.len());
let mut blob_iter = accumulated_sidecars.into_iter().peekable();
@ -50,29 +50,23 @@ impl<T: EthSpec> BlocksAndBlobsRequestInfo<T> {
.unwrap_or(false);
pair_next_blob
} {
blob_list.push(blob_iter.next().expect("iterator is not empty"));
blob_list.push(blob_iter.next().ok_or("Missing next blob")?);
}
if blob_list.is_empty() {
responses.push(BlockWrapper::Block(block))
} else {
let mut blobs_fixed = vec![None; T::max_blobs_per_block()];
for blob in blob_list {
let blob_index = blob.index as usize;
let Some(blob_opt) = blobs_fixed.get_mut(blob_index) else {
let mut blobs_buffer = vec![None; T::max_blobs_per_block()];
for blob in blob_list {
let blob_index = blob.index as usize;
let Some(blob_opt) = blobs_buffer.get_mut(blob_index) else {
return Err("Invalid blob index");
};
if blob_opt.is_some() {
return Err("Repeat blob index");
} else {
*blob_opt = Some(blob);
}
if blob_opt.is_some() {
return Err("Repeat blob index");
} else {
*blob_opt = Some(blob);
}
responses.push(BlockWrapper::BlockAndBlobs(
block,
FixedVector::from(blobs_fixed),
))
}
let blobs = VariableList::from(blobs_buffer.into_iter().flatten().collect::<Vec<_>>());
responses.push(RpcBlock::new(block, Some(blobs))?)
}
// if accumulated sidecars is not empty, throw an error.

View File

@ -46,8 +46,8 @@ use crate::sync::block_lookups::delayed_lookup::DelayedLookupMessage;
pub use crate::sync::block_lookups::ResponseType;
use crate::sync::block_lookups::UnknownParentComponents;
use crate::sync::range_sync::ByRangeRequestType;
use beacon_chain::blob_verification::AsBlock;
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::block_verification_types::AsBlock;
use beacon_chain::block_verification_types::RpcBlock;
use beacon_chain::{
AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes, BlockError, EngineState,
MAXIMUM_GOSSIP_CLOCK_DISPARITY,
@ -127,7 +127,7 @@ pub enum SyncMessage<T: EthSpec> {
},
/// A block with an unknown parent has been received.
UnknownParentBlock(PeerId, BlockWrapper<T>, Hash256),
UnknownParentBlock(PeerId, RpcBlock<T>, Hash256),
/// A blob with an unknown parent has been received.
UnknownParentBlob(PeerId, Arc<BlobSidecar<T>>),
@ -614,15 +614,13 @@ impl<T: BeaconChainTypes> SyncManager<T> {
} => self.rpc_blob_received(request_id, peer_id, blob_sidecar, seen_timestamp),
SyncMessage::UnknownParentBlock(peer_id, block, block_root) => {
let block_slot = block.slot();
let (block, blobs) = block.deconstruct();
let parent_root = block.parent_root();
let parent_components = UnknownParentComponents::new(Some(block), blobs);
self.handle_unknown_parent(
peer_id,
block_root,
parent_root,
block_slot,
Some(parent_components),
Some(block.into()),
);
}
SyncMessage::UnknownParentBlob(peer_id, blob) => {
@ -910,7 +908,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
batch_id,
&peer_id,
id,
block.map(BlockWrapper::Block),
block.map(Into::into),
) {
Ok(ProcessResult::SyncCompleted) => self.update_sync_state(),
Ok(ProcessResult::Successful) => {}
@ -934,7 +932,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
chain_id,
batch_id,
id,
block.map(BlockWrapper::Block),
block.map(Into::into),
);
self.update_sync_state();
}

View File

@ -8,7 +8,7 @@ use crate::network_beacon_processor::NetworkBeaconProcessor;
use crate::service::{NetworkMessage, RequestId};
use crate::status::ToStatusMessage;
use crate::sync::block_lookups::{BlobRequestId, BlockRequestId};
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::block_verification_types::RpcBlock;
use beacon_chain::{BeaconChain, BeaconChainTypes, EngineState};
use fnv::FnvHashMap;
use lighthouse_network::rpc::methods::{BlobsByRangeRequest, BlobsByRootRequest};
@ -22,7 +22,7 @@ use types::{BlobSidecar, EthSpec, SignedBeaconBlock};
pub struct BlocksAndBlobsByRangeResponse<T: EthSpec> {
pub batch_id: BatchId,
pub responses: Result<Vec<BlockWrapper<T>>, &'static str>,
pub responses: Result<Vec<RpcBlock<T>>, &'static str>,
}
pub struct BlocksAndBlobsByRangeRequest<T: EthSpec> {

View File

@ -1,5 +1,5 @@
use crate::sync::manager::Id;
use beacon_chain::blob_verification::{AsBlock, BlockWrapper};
use beacon_chain::block_verification_types::{AsBlock, RpcBlock};
use lighthouse_network::rpc::methods::BlocksByRangeRequest;
use lighthouse_network::PeerId;
use std::collections::HashSet;
@ -56,7 +56,7 @@ pub trait BatchConfig {
/// Note that simpler hashing functions considered in the past (hash of first block, hash of last
/// block, number of received blocks) are not good enough to differentiate attempts. For this
/// reason, we hash the complete set of blocks both in RangeSync and BackFillSync.
fn batch_attempt_hash<T: EthSpec>(blocks: &[BlockWrapper<T>]) -> u64;
fn batch_attempt_hash<T: EthSpec>(blocks: &[RpcBlock<T>]) -> u64;
}
pub struct RangeSyncBatchConfig {}
@ -68,7 +68,7 @@ impl BatchConfig for RangeSyncBatchConfig {
fn max_batch_processing_attempts() -> u8 {
MAX_BATCH_PROCESSING_ATTEMPTS
}
fn batch_attempt_hash<T: EthSpec>(blocks: &[BlockWrapper<T>]) -> u64 {
fn batch_attempt_hash<T: EthSpec>(blocks: &[RpcBlock<T>]) -> u64 {
let mut hasher = std::collections::hash_map::DefaultHasher::new();
blocks.hash(&mut hasher);
hasher.finish()
@ -116,9 +116,9 @@ pub enum BatchState<T: EthSpec> {
/// The batch has failed either downloading or processing, but can be requested again.
AwaitingDownload,
/// The batch is being downloaded.
Downloading(PeerId, Vec<BlockWrapper<T>>, Id),
Downloading(PeerId, Vec<RpcBlock<T>>, Id),
/// The batch has been completely downloaded and is ready for processing.
AwaitingProcessing(PeerId, Vec<BlockWrapper<T>>),
AwaitingProcessing(PeerId, Vec<RpcBlock<T>>),
/// The batch is being processed.
Processing(Attempt),
/// The batch was successfully processed and is waiting to be validated.
@ -251,7 +251,7 @@ impl<T: EthSpec, B: BatchConfig> BatchInfo<T, B> {
}
/// Adds a block to a downloading batch.
pub fn add_block(&mut self, block: BlockWrapper<T>) -> Result<(), WrongState> {
pub fn add_block(&mut self, block: RpcBlock<T>) -> Result<(), WrongState> {
match self.state.poison() {
BatchState::Downloading(peer, mut blocks, req_id) => {
blocks.push(block);
@ -383,7 +383,7 @@ impl<T: EthSpec, B: BatchConfig> BatchInfo<T, B> {
}
}
pub fn start_processing(&mut self) -> Result<Vec<BlockWrapper<T>>, WrongState> {
pub fn start_processing(&mut self) -> Result<Vec<RpcBlock<T>>, WrongState> {
match self.state.poison() {
BatchState::AwaitingProcessing(peer, blocks) => {
self.state = BatchState::Processing(Attempt::new::<B, T>(peer, &blocks));
@ -481,7 +481,7 @@ pub struct Attempt {
}
impl Attempt {
fn new<B: BatchConfig, T: EthSpec>(peer_id: PeerId, blocks: &[BlockWrapper<T>]) -> Self {
fn new<B: BatchConfig, T: EthSpec>(peer_id: PeerId, blocks: &[RpcBlock<T>]) -> Self {
let hash = B::batch_attempt_hash(blocks);
Attempt { peer_id, hash }
}

View File

@ -3,7 +3,7 @@ use crate::network_beacon_processor::ChainSegmentProcessId;
use crate::sync::{
manager::Id, network_context::SyncNetworkContext, BatchOperationOutcome, BatchProcessResult,
};
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::block_verification_types::RpcBlock;
use beacon_chain::BeaconChainTypes;
use fnv::FnvHashMap;
use lighthouse_network::{PeerAction, PeerId};
@ -221,7 +221,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
batch_id: BatchId,
peer_id: &PeerId,
request_id: Id,
beacon_block: Option<BlockWrapper<T::EthSpec>>,
beacon_block: Option<RpcBlock<T::EthSpec>>,
) -> ProcessingResult {
// check if we have this batch
let batch = match self.batches.get_mut(&batch_id) {

View File

@ -47,7 +47,7 @@ use crate::status::ToStatusMessage;
use crate::sync::manager::Id;
use crate::sync::network_context::SyncNetworkContext;
use crate::sync::BatchProcessResult;
use beacon_chain::blob_verification::BlockWrapper;
use beacon_chain::block_verification_types::RpcBlock;
use beacon_chain::{BeaconChain, BeaconChainTypes};
use lighthouse_network::rpc::GoodbyeReason;
use lighthouse_network::PeerId;
@ -210,7 +210,7 @@ where
chain_id: ChainId,
batch_id: BatchId,
request_id: Id,
beacon_block: Option<BlockWrapper<T::EthSpec>>,
beacon_block: Option<RpcBlock<T::EthSpec>>,
) {
// check if this chunk removes the chain
match self.chains.call_by_id(chain_id, |chain| {
@ -387,19 +387,18 @@ mod tests {
use beacon_chain::builder::Witness;
use beacon_chain::eth1_chain::CachingEth1Backend;
use beacon_chain::parking_lot::RwLock;
use beacon_chain::test_utils::{BeaconChainHarness, EphemeralHarnessType};
use beacon_chain::EngineState;
use beacon_processor::WorkEvent as BeaconWorkEvent;
use lighthouse_network::rpc::BlocksByRangeRequest;
use lighthouse_network::Request;
use lighthouse_network::{rpc::StatusMessage, NetworkGlobals};
use slog::{o, Drain};
use tokio::sync::mpsc;
use beacon_chain::test_utils::{BeaconChainHarness, EphemeralHarnessType};
use slot_clock::TestingSlotClock;
use std::collections::HashSet;
use std::sync::Arc;
use store::MemoryStore;
use tokio::sync::mpsc;
use types::{Hash256, MinimalEthSpec as E};
#[derive(Debug)]

View File

@ -1,9 +1,5 @@
#![cfg(not(debug_assertions))]
use std::fmt;
use std::sync::Mutex;
use std::time::Duration;
use beacon_chain::test_utils::{
AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType,
};
@ -14,6 +10,9 @@ use beacon_chain::{
use fork_choice::{
ForkChoiceStore, InvalidAttestation, InvalidBlock, PayloadVerificationStatus, QueuedAttestation,
};
use std::fmt;
use std::sync::Mutex;
use std::time::Duration;
use store::MemoryStore;
use types::{
test_utils::generate_deterministic_keypair, BeaconBlockRef, BeaconState, ChainSpec, Checkpoint,
@ -195,17 +194,18 @@ impl ForkChoiceTest {
let validators = self.harness.get_all_validators();
loop {
let slot = self.harness.get_current_slot();
let (block, state_) = self.harness.make_block(state, slot).await;
let (block_contents, state_) = self.harness.make_block(state, slot).await;
state = state_;
if !predicate(block.0.message(), &state) {
if !predicate(block_contents.0.message(), &state) {
break;
}
if let Ok(block_hash) = self.harness.process_block_result(block.clone()).await {
let block = block_contents.0.clone();
if let Ok(block_hash) = self.harness.process_block_result(block_contents).await {
self.harness.attest_block(
&state,
block.0.state_root(),
block.state_root(),
block_hash,
&block.0,
&block,
&validators,
);
self.harness.advance_slot();

View File

@ -21,8 +21,6 @@ pub struct ConsensusContext<T: EthSpec> {
#[ssz(skip_serializing, skip_deserializing)]
indexed_attestations:
HashMap<(AttestationData, BitList<T::MaxValidatorsPerCommittee>), IndexedAttestation<T>>,
/// Whether `verify_kzg_commitments_against_transactions` has successfully passed.
kzg_commitments_consistent: bool,
}
#[derive(Debug, PartialEq, Clone)]
@ -45,7 +43,6 @@ impl<T: EthSpec> ConsensusContext<T> {
proposer_index: None,
current_block_root: None,
indexed_attestations: HashMap::new(),
kzg_commitments_consistent: false,
}
}
@ -161,13 +158,4 @@ impl<T: EthSpec> ConsensusContext<T> {
pub fn num_cached_indexed_attestations(&self) -> usize {
self.indexed_attestations.len()
}
pub fn set_kzg_commitments_consistent(mut self, kzg_commitments_consistent: bool) -> Self {
self.kzg_commitments_consistent = kzg_commitments_consistent;
self
}
pub fn kzg_commitments_consistent(&self) -> bool {
self.kzg_commitments_consistent
}
}

View File

@ -169,9 +169,10 @@ pub use crate::selection_proof::SelectionProof;
pub use crate::shuffling_id::AttestationShufflingId;
pub use crate::signed_aggregate_and_proof::SignedAggregateAndProof;
pub use crate::signed_beacon_block::{
ssz_tagged_signed_beacon_block, SignedBeaconBlock, SignedBeaconBlockAltair,
SignedBeaconBlockBase, SignedBeaconBlockCapella, SignedBeaconBlockDeneb, SignedBeaconBlockHash,
SignedBeaconBlockMerge, SignedBlindedBeaconBlock,
ssz_tagged_signed_beacon_block, ssz_tagged_signed_beacon_block_arc, SignedBeaconBlock,
SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockCapella,
SignedBeaconBlockDeneb, SignedBeaconBlockHash, SignedBeaconBlockMerge,
SignedBlindedBeaconBlock,
};
pub use crate::signed_beacon_block_header::SignedBeaconBlockHeader;
pub use crate::signed_blob::*;

View File

@ -642,6 +642,27 @@ pub mod ssz_tagged_signed_beacon_block {
}
}
pub mod ssz_tagged_signed_beacon_block_arc {
use super::*;
pub mod encode {
pub use super::ssz_tagged_signed_beacon_block::encode::*;
}
pub mod decode {
pub use super::ssz_tagged_signed_beacon_block::decode::{is_ssz_fixed_len, ssz_fixed_len};
use super::*;
#[allow(unused_imports)]
use ssz::*;
use std::sync::Arc;
pub fn from_ssz_bytes<E: EthSpec, Payload: AbstractExecPayload<E>>(
bytes: &[u8],
) -> Result<Arc<SignedBeaconBlock<E, Payload>>, DecodeError> {
ssz_tagged_signed_beacon_block::decode::from_ssz_bytes(bytes).map(Arc::new)
}
}
}
#[cfg(test)]
mod test {
use super::*;