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:
parent
3735450749
commit
33dd13c798
@ -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
|
||||
///
|
||||
|
@ -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)),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
|
438
beacon_node/beacon_chain/src/block_verification_types.rs
Normal file
438
beacon_node/beacon_chain/src/block_verification_types.rs
Normal 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
|
||||
}
|
||||
}
|
@ -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>>),
|
||||
}
|
||||
|
@ -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()
|
||||
|
@ -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;
|
||||
|
@ -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(
|
||||
|
@ -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")
|
||||
|
@ -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!(
|
||||
|
@ -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) => (),
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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,
|
||||
|
@ -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();
|
||||
|
@ -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));
|
||||
|
@ -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},
|
||||
|
@ -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();
|
||||
|
@ -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<_>, _>>()
|
||||
{
|
||||
|
@ -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) {
|
||||
|
@ -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>,
|
||||
|
@ -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>,
|
||||
) {
|
||||
|
@ -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
|
||||
|
@ -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,
|
||||
|
@ -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.
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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> {
|
||||
|
@ -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 }
|
||||
}
|
||||
|
@ -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) {
|
||||
|
@ -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)]
|
||||
|
@ -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();
|
||||
|
@ -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
|
||||
}
|
||||
}
|
||||
|
@ -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::*;
|
||||
|
@ -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::*;
|
||||
|
Loading…
Reference in New Issue
Block a user