chain segment processing for blobs
This commit is contained in:
parent
8b56446b64
commit
422d145902
@ -104,12 +104,13 @@ use store::{
|
|||||||
use task_executor::{ShutdownReason, TaskExecutor};
|
use task_executor::{ShutdownReason, TaskExecutor};
|
||||||
use tree_hash::TreeHash;
|
use tree_hash::TreeHash;
|
||||||
use types::beacon_state::CloneConfig;
|
use types::beacon_state::CloneConfig;
|
||||||
|
use types::signed_block_and_blobs::BlockWrapper;
|
||||||
use types::*;
|
use types::*;
|
||||||
|
|
||||||
pub type ForkChoiceError = fork_choice::Error<crate::ForkChoiceStoreError>;
|
pub type ForkChoiceError = fork_choice::Error<crate::ForkChoiceStoreError>;
|
||||||
|
|
||||||
/// Alias to appease clippy.
|
/// Alias to appease clippy.
|
||||||
type HashBlockTuple<E> = (Hash256, Arc<SignedBeaconBlock<E>>);
|
type HashBlockTuple<E> = (Hash256, BlockWrapper<E>);
|
||||||
|
|
||||||
/// The time-out before failure during an operation to take a read/write RwLock on the block
|
/// The time-out before failure during an operation to take a read/write RwLock on the block
|
||||||
/// processing cache.
|
/// processing cache.
|
||||||
@ -2278,7 +2279,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
|||||||
/// This method is potentially long-running and should not run on the core executor.
|
/// This method is potentially long-running and should not run on the core executor.
|
||||||
pub fn filter_chain_segment(
|
pub fn filter_chain_segment(
|
||||||
self: &Arc<Self>,
|
self: &Arc<Self>,
|
||||||
chain_segment: Vec<Arc<SignedBeaconBlock<T::EthSpec>>>,
|
chain_segment: Vec<BlockWrapper<T::EthSpec>>,
|
||||||
) -> Result<Vec<HashBlockTuple<T::EthSpec>>, ChainSegmentResult<T::EthSpec>> {
|
) -> Result<Vec<HashBlockTuple<T::EthSpec>>, ChainSegmentResult<T::EthSpec>> {
|
||||||
// This function will never import any blocks.
|
// This function will never import any blocks.
|
||||||
let imported_blocks = 0;
|
let imported_blocks = 0;
|
||||||
@ -2290,19 +2291,19 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
|||||||
let children = chain_segment
|
let children = chain_segment
|
||||||
.iter()
|
.iter()
|
||||||
.skip(1)
|
.skip(1)
|
||||||
.map(|block| (block.parent_root(), block.slot()))
|
.map(|block| (block.block().parent_root(), block.slot()))
|
||||||
.collect::<Vec<_>>();
|
.collect::<Vec<_>>();
|
||||||
|
|
||||||
for (i, block) in chain_segment.into_iter().enumerate() {
|
for (i, block) in chain_segment.into_iter().enumerate() {
|
||||||
// Ensure the block is the correct structure for the fork at `block.slot()`.
|
// Ensure the block is the correct structure for the fork at `block.slot()`.
|
||||||
if let Err(e) = block.fork_name(&self.spec) {
|
if let Err(e) = block.block().fork_name(&self.spec) {
|
||||||
return Err(ChainSegmentResult::Failed {
|
return Err(ChainSegmentResult::Failed {
|
||||||
imported_blocks,
|
imported_blocks,
|
||||||
error: BlockError::InconsistentFork(e),
|
error: BlockError::InconsistentFork(e),
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
let block_root = get_block_root(&block);
|
let block_root = get_block_root(block.block());
|
||||||
|
|
||||||
if let Some((child_parent_root, child_slot)) = children.get(i) {
|
if let Some((child_parent_root, child_slot)) = children.get(i) {
|
||||||
// If this block has a child in this chain segment, ensure that its parent root matches
|
// If this block has a child in this chain segment, ensure that its parent root matches
|
||||||
@ -2326,7 +2327,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
match check_block_relevancy(&block, block_root, self) {
|
match check_block_relevancy(block.block(), block_root, self) {
|
||||||
// If the block is relevant, add it to the filtered chain segment.
|
// If the block is relevant, add it to the filtered chain segment.
|
||||||
Ok(_) => filtered_chain_segment.push((block_root, block)),
|
Ok(_) => filtered_chain_segment.push((block_root, block)),
|
||||||
// If the block is already known, simply ignore this block.
|
// If the block is already known, simply ignore this block.
|
||||||
@ -2384,7 +2385,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
|||||||
/// `Self::process_block`.
|
/// `Self::process_block`.
|
||||||
pub async fn process_chain_segment(
|
pub async fn process_chain_segment(
|
||||||
self: &Arc<Self>,
|
self: &Arc<Self>,
|
||||||
chain_segment: Vec<Arc<SignedBeaconBlock<T::EthSpec>>>,
|
chain_segment: Vec<BlockWrapper<T::EthSpec>>,
|
||||||
count_unrealized: CountUnrealized,
|
count_unrealized: CountUnrealized,
|
||||||
) -> ChainSegmentResult<T::EthSpec> {
|
) -> ChainSegmentResult<T::EthSpec> {
|
||||||
let mut imported_blocks = 0;
|
let mut imported_blocks = 0;
|
||||||
@ -5246,6 +5247,14 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
|||||||
|
|
||||||
gossip_attested || block_attested || aggregated || produced_block
|
gossip_attested || block_attested || aggregated || produced_block
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// The epoch at which we require a data availability check in block processing.
|
||||||
|
/// `None` if the `Eip4844` fork is disabled.
|
||||||
|
pub fn data_availability_boundary(&self) -> Option<Epoch> {
|
||||||
|
self.spec
|
||||||
|
.eip4844_fork_epoch
|
||||||
|
.map(|e| std::cmp::max(e, self.head().finalized_checkpoint().epoch))
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<T: BeaconChainTypes> Drop for BeaconChain<T> {
|
impl<T: BeaconChainTypes> Drop for BeaconChain<T> {
|
||||||
|
@ -87,6 +87,7 @@ use std::time::Duration;
|
|||||||
use store::{Error as DBError, HotStateSummary, KeyValueStore, StoreOp};
|
use store::{Error as DBError, HotStateSummary, KeyValueStore, StoreOp};
|
||||||
use task_executor::JoinHandle;
|
use task_executor::JoinHandle;
|
||||||
use tree_hash::TreeHash;
|
use tree_hash::TreeHash;
|
||||||
|
use types::signed_block_and_blobs::BlockWrapper;
|
||||||
use types::{
|
use types::{
|
||||||
BeaconBlockRef, BeaconState, BeaconStateError, BlindedPayload, ChainSpec, CloneConfig, Epoch,
|
BeaconBlockRef, BeaconState, BeaconStateError, BlindedPayload, ChainSpec, CloneConfig, Epoch,
|
||||||
EthSpec, ExecutionBlockHash, Hash256, InconsistentFork, PublicKey, PublicKeyBytes,
|
EthSpec, ExecutionBlockHash, Hash256, InconsistentFork, PublicKey, PublicKeyBytes,
|
||||||
@ -548,17 +549,18 @@ fn process_block_slash_info<T: BeaconChainTypes>(
|
|||||||
/// The given `chain_segment` must contain only blocks from the same epoch, otherwise an error
|
/// The given `chain_segment` must contain only blocks from the same epoch, otherwise an error
|
||||||
/// will be returned.
|
/// will be returned.
|
||||||
pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
|
pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
|
||||||
mut chain_segment: Vec<(Hash256, Arc<SignedBeaconBlock<T::EthSpec>>)>,
|
mut chain_segment: Vec<(Hash256, BlockWrapper<T::EthSpec>)>,
|
||||||
chain: &BeaconChain<T>,
|
chain: &BeaconChain<T>,
|
||||||
) -> Result<Vec<SignatureVerifiedBlock<T>>, BlockError<T::EthSpec>> {
|
) -> Result<Vec<SignatureVerifiedBlock<T>>, BlockError<T::EthSpec>> {
|
||||||
if chain_segment.is_empty() {
|
if chain_segment.is_empty() {
|
||||||
return Ok(vec![]);
|
return Ok(vec![]);
|
||||||
}
|
}
|
||||||
|
|
||||||
let (first_root, first_block) = chain_segment.remove(0);
|
let (first_root, first_block_wrapper) = chain_segment.remove(0);
|
||||||
let (mut parent, first_block) = load_parent(first_root, first_block, chain)?;
|
let (mut parent, first_block) =
|
||||||
|
load_parent(first_root, first_block_wrapper.block_cloned(), chain)?;
|
||||||
let slot = first_block.slot();
|
let slot = first_block.slot();
|
||||||
chain_segment.insert(0, (first_root, first_block));
|
chain_segment.insert(0, (first_root, first_block_wrapper));
|
||||||
|
|
||||||
let highest_slot = chain_segment
|
let highest_slot = chain_segment
|
||||||
.last()
|
.last()
|
||||||
@ -576,7 +578,7 @@ pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
|
|||||||
let mut signature_verifier = get_signature_verifier(&state, &pubkey_cache, &chain.spec);
|
let mut signature_verifier = get_signature_verifier(&state, &pubkey_cache, &chain.spec);
|
||||||
|
|
||||||
for (block_root, block) in &chain_segment {
|
for (block_root, block) in &chain_segment {
|
||||||
signature_verifier.include_all_signatures(block, Some(*block_root), None)?;
|
signature_verifier.include_all_signatures(block.block(), Some(*block_root), None)?;
|
||||||
}
|
}
|
||||||
|
|
||||||
if signature_verifier.verify().is_err() {
|
if signature_verifier.verify().is_err() {
|
||||||
@ -585,19 +587,16 @@ pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
|
|||||||
|
|
||||||
drop(pubkey_cache);
|
drop(pubkey_cache);
|
||||||
|
|
||||||
//FIXME(sean) batch verify kzg blobs
|
|
||||||
// TODO(pawan): do not have the sidecar here, maybe validate kzg proofs in a different function
|
|
||||||
// with relevant block params?
|
|
||||||
|
|
||||||
let mut signature_verified_blocks = chain_segment
|
let mut signature_verified_blocks = chain_segment
|
||||||
.into_iter()
|
.into_iter()
|
||||||
.map(|(block_root, block)| {
|
.map(|(block_root, block)| {
|
||||||
// Proposer index has already been verified above during signature verification.
|
// Proposer index has already been verified above during signature verification.
|
||||||
let consensus_context = ConsensusContext::new(block.slot())
|
let consensus_context = ConsensusContext::new(block.slot())
|
||||||
.set_current_block_root(block_root)
|
.set_current_block_root(block_root)
|
||||||
.set_proposer_index(block.message().proposer_index());
|
.set_proposer_index(block.block().message().proposer_index())
|
||||||
|
.set_blobs_sidecar(block.blocks_sidecar());
|
||||||
SignatureVerifiedBlock {
|
SignatureVerifiedBlock {
|
||||||
block,
|
block: block.block_cloned(),
|
||||||
block_root,
|
block_root,
|
||||||
parent: None,
|
parent: None,
|
||||||
consensus_context,
|
consensus_context,
|
||||||
@ -1103,7 +1102,6 @@ impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for SignatureVerifiedBloc
|
|||||||
|
|
||||||
ExecutionPendingBlock::from_signature_verified_components(
|
ExecutionPendingBlock::from_signature_verified_components(
|
||||||
block,
|
block,
|
||||||
self.consensus_context.blobs_sidecar(),
|
|
||||||
block_root,
|
block_root,
|
||||||
parent,
|
parent,
|
||||||
self.consensus_context,
|
self.consensus_context,
|
||||||
@ -1148,7 +1146,6 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
|
|||||||
/// Returns an error if the block is invalid, or if the block was unable to be verified.
|
/// Returns an error if the block is invalid, or if the block was unable to be verified.
|
||||||
pub fn from_signature_verified_components(
|
pub fn from_signature_verified_components(
|
||||||
block: Arc<SignedBeaconBlock<T::EthSpec>>,
|
block: Arc<SignedBeaconBlock<T::EthSpec>>,
|
||||||
blobs: Option<Arc<BlobsSidecar<T::EthSpec>>>,
|
|
||||||
block_root: Hash256,
|
block_root: Hash256,
|
||||||
parent: PreProcessingSnapshot<T::EthSpec>,
|
parent: PreProcessingSnapshot<T::EthSpec>,
|
||||||
mut consensus_context: ConsensusContext<T::EthSpec>,
|
mut consensus_context: ConsensusContext<T::EthSpec>,
|
||||||
@ -1483,9 +1480,12 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO: are we guaranteed that consensus_context.blobs_sidecar == blobs ?
|
/*
|
||||||
/* Verify kzg proofs and kzg commitments against transactions if required */
|
* Verify kzg proofs and kzg commitments against transactions if required
|
||||||
if let Some(ref sidecar) = blobs {
|
*/
|
||||||
|
if let Some(ref sidecar) = consensus_context.blobs_sidecar() {
|
||||||
|
if let Some(data_availability_boundary) = chain.data_availability_boundary() {
|
||||||
|
if block_slot.epoch(T::EthSpec::slots_per_epoch()) > data_availability_boundary {
|
||||||
let kzg = chain.kzg.as_ref().ok_or(BlockError::BlobValidation(
|
let kzg = chain.kzg.as_ref().ok_or(BlockError::BlobValidation(
|
||||||
BlobError::TrustedSetupNotInitialized,
|
BlobError::TrustedSetupNotInitialized,
|
||||||
))?;
|
))?;
|
||||||
@ -1496,11 +1496,10 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
|
|||||||
.map(|payload| payload.transactions())
|
.map(|payload| payload.transactions())
|
||||||
.map_err(|_| BlockError::BlobValidation(BlobError::TransactionsMissing))?
|
.map_err(|_| BlockError::BlobValidation(BlobError::TransactionsMissing))?
|
||||||
.ok_or(BlockError::BlobValidation(BlobError::TransactionsMissing))?;
|
.ok_or(BlockError::BlobValidation(BlobError::TransactionsMissing))?;
|
||||||
let kzg_commitments = block
|
let kzg_commitments =
|
||||||
.message()
|
block.message().body().blob_kzg_commitments().map_err(|_| {
|
||||||
.body()
|
BlockError::BlobValidation(BlobError::KzgCommitmentMissing)
|
||||||
.blob_kzg_commitments()
|
})?;
|
||||||
.map_err(|_| BlockError::BlobValidation(BlobError::KzgCommitmentMissing))?;
|
|
||||||
if !consensus_context.blobs_sidecar_validated() {
|
if !consensus_context.blobs_sidecar_validated() {
|
||||||
if !kzg_utils::validate_blobs_sidecar(
|
if !kzg_utils::validate_blobs_sidecar(
|
||||||
&kzg,
|
&kzg,
|
||||||
@ -1519,22 +1518,20 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
|
|||||||
transactions,
|
transactions,
|
||||||
kzg_commitments,
|
kzg_commitments,
|
||||||
)
|
)
|
||||||
|
//FIXME(sean) we should maybe just map this error so we have more info about the mismatch
|
||||||
.is_err()
|
.is_err()
|
||||||
{
|
{
|
||||||
return Err(BlockError::BlobValidation(
|
return Err(BlockError::BlobValidation(
|
||||||
BlobError::TransactionCommitmentMismatch,
|
BlobError::TransactionCommitmentMismatch,
|
||||||
));
|
));
|
||||||
}
|
}
|
||||||
// TODO(pawan): confirm with sean. are we expected to set the context here? because the ConsensusContext
|
}
|
||||||
// setters don't take mutable references.
|
}
|
||||||
// Set the consensus context after completing the required kzg valdiations
|
|
||||||
// consensus_context.set_blobs_sidecar_validated(true);
|
|
||||||
// consensus_context.set_blobs_verified_vs_txs(true);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
Ok(Self {
|
Ok(Self {
|
||||||
block,
|
block,
|
||||||
blobs,
|
blobs: consensus_context.blobs_sidecar(),
|
||||||
block_root,
|
block_root,
|
||||||
state,
|
state,
|
||||||
parent_block: parent.beacon_block,
|
parent_block: parent.beacon_block,
|
||||||
|
@ -61,6 +61,7 @@ use std::time::Duration;
|
|||||||
use std::{cmp, collections::HashSet};
|
use std::{cmp, collections::HashSet};
|
||||||
use task_executor::TaskExecutor;
|
use task_executor::TaskExecutor;
|
||||||
use tokio::sync::mpsc;
|
use tokio::sync::mpsc;
|
||||||
|
use types::signed_block_and_blobs::BlockWrapper;
|
||||||
use types::{
|
use types::{
|
||||||
Attestation, AttesterSlashing, Hash256, ProposerSlashing, SignedAggregateAndProof,
|
Attestation, AttesterSlashing, Hash256, ProposerSlashing, SignedAggregateAndProof,
|
||||||
SignedBeaconBlock, SignedBeaconBlockAndBlobsSidecar, SignedBlsToExecutionChange,
|
SignedBeaconBlock, SignedBeaconBlockAndBlobsSidecar, SignedBlsToExecutionChange,
|
||||||
@ -1762,8 +1763,13 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
|
|||||||
/*
|
/*
|
||||||
* Verification for a chain segment (multiple blocks).
|
* Verification for a chain segment (multiple blocks).
|
||||||
*/
|
*/
|
||||||
Work::ChainSegment { process_id, blocks } => task_spawner
|
Work::ChainSegment { process_id, blocks } => task_spawner.spawn_async(async move {
|
||||||
.spawn_async(async move { worker.process_chain_segment(process_id, blocks).await }),
|
let wrapped = blocks
|
||||||
|
.into_iter()
|
||||||
|
.map(|block| BlockWrapper::Block { block })
|
||||||
|
.collect();
|
||||||
|
worker.process_chain_segment(process_id, wrapped).await
|
||||||
|
}),
|
||||||
/*
|
/*
|
||||||
* Processing of Status Messages.
|
* Processing of Status Messages.
|
||||||
*/
|
*/
|
||||||
@ -1867,9 +1873,13 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
|
|||||||
process_id,
|
process_id,
|
||||||
blocks_and_blobs,
|
blocks_and_blobs,
|
||||||
} => task_spawner.spawn_async(async move {
|
} => task_spawner.spawn_async(async move {
|
||||||
worker
|
let wrapped = blocks_and_blobs
|
||||||
.process_blob_chain_segment(process_id, blocks_and_blobs)
|
.into_iter()
|
||||||
.await
|
.map(|b| BlockWrapper::BlockAndBlob {
|
||||||
|
block_sidecar_pair: b,
|
||||||
|
})
|
||||||
|
.collect();
|
||||||
|
worker.process_chain_segment(process_id, wrapped).await
|
||||||
}),
|
}),
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
@ -15,7 +15,11 @@ use lighthouse_network::PeerAction;
|
|||||||
use slog::{debug, error, info, warn};
|
use slog::{debug, error, info, warn};
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
use tokio::sync::mpsc;
|
use tokio::sync::mpsc;
|
||||||
use types::{Epoch, Hash256, SignedBeaconBlock, SignedBeaconBlockAndBlobsSidecar};
|
use types::signed_block_and_blobs::BlockWrapper;
|
||||||
|
use types::{
|
||||||
|
Epoch, Hash256, SignedBeaconBlock, SignedBeaconBlockAndBlobsSidecar,
|
||||||
|
SignedBeaconBlockAndBlobsSidecarDecode,
|
||||||
|
};
|
||||||
|
|
||||||
/// Id associated to a batch processing request, either a sync batch or a parent lookup.
|
/// Id associated to a batch processing request, either a sync batch or a parent lookup.
|
||||||
#[derive(Clone, Debug, PartialEq)]
|
#[derive(Clone, Debug, PartialEq)]
|
||||||
@ -126,7 +130,7 @@ impl<T: BeaconChainTypes> Worker<T> {
|
|||||||
pub async fn process_chain_segment(
|
pub async fn process_chain_segment(
|
||||||
&self,
|
&self,
|
||||||
sync_type: ChainSegmentProcessId,
|
sync_type: ChainSegmentProcessId,
|
||||||
downloaded_blocks: Vec<Arc<SignedBeaconBlock<T::EthSpec>>>,
|
downloaded_blocks: Vec<BlockWrapper<T::EthSpec>>,
|
||||||
) {
|
) {
|
||||||
let result = match sync_type {
|
let result = match sync_type {
|
||||||
// this a request from the range sync
|
// this a request from the range sync
|
||||||
@ -176,7 +180,18 @@ impl<T: BeaconChainTypes> Worker<T> {
|
|||||||
let end_slot = downloaded_blocks.last().map(|b| b.slot().as_u64());
|
let end_slot = downloaded_blocks.last().map(|b| b.slot().as_u64());
|
||||||
let sent_blocks = downloaded_blocks.len();
|
let sent_blocks = downloaded_blocks.len();
|
||||||
|
|
||||||
match self.process_backfill_blocks(downloaded_blocks) {
|
let unwrapped = downloaded_blocks
|
||||||
|
.into_iter()
|
||||||
|
.map(|block| match block {
|
||||||
|
BlockWrapper::Block { block } => block,
|
||||||
|
//FIXME(sean) handle blobs in backfill
|
||||||
|
BlockWrapper::BlockAndBlob {
|
||||||
|
block_sidecar_pair: _,
|
||||||
|
} => todo!(),
|
||||||
|
})
|
||||||
|
.collect();
|
||||||
|
|
||||||
|
match self.process_backfill_blocks(unwrapped) {
|
||||||
(_, Ok(_)) => {
|
(_, Ok(_)) => {
|
||||||
debug!(self.log, "Backfill batch processed";
|
debug!(self.log, "Backfill batch processed";
|
||||||
"batch_epoch" => epoch,
|
"batch_epoch" => epoch,
|
||||||
@ -241,24 +256,13 @@ impl<T: BeaconChainTypes> Worker<T> {
|
|||||||
self.send_sync_message(SyncMessage::BatchProcessed { sync_type, result });
|
self.send_sync_message(SyncMessage::BatchProcessed { sync_type, result });
|
||||||
}
|
}
|
||||||
|
|
||||||
pub async fn process_blob_chain_segment(
|
|
||||||
&self,
|
|
||||||
sync_type: ChainSegmentProcessId,
|
|
||||||
downloaded_blocks: Vec<SignedBeaconBlockAndBlobsSidecar<T::EthSpec>>,
|
|
||||||
) {
|
|
||||||
warn!(self.log, "FAKE PROCESSING A BLOBS SEGMENT!!!");
|
|
||||||
let result = BatchProcessResult::Success {
|
|
||||||
was_non_empty: !downloaded_blocks.is_empty(),
|
|
||||||
};
|
|
||||||
self.send_sync_message(SyncMessage::BatchProcessed { sync_type, result });
|
|
||||||
}
|
|
||||||
/// Helper function to process blocks batches which only consumes the chain and blocks to process.
|
/// Helper function to process blocks batches which only consumes the chain and blocks to process.
|
||||||
async fn process_blocks<'a>(
|
async fn process_blocks<'a>(
|
||||||
&self,
|
&self,
|
||||||
downloaded_blocks: impl Iterator<Item = &'a Arc<SignedBeaconBlock<T::EthSpec>>>,
|
downloaded_blocks: impl Iterator<Item = &'a BlockWrapper<T::EthSpec>>,
|
||||||
count_unrealized: CountUnrealized,
|
count_unrealized: CountUnrealized,
|
||||||
) -> (usize, Result<(), ChainSegmentFailed>) {
|
) -> (usize, Result<(), ChainSegmentFailed>) {
|
||||||
let blocks: Vec<Arc<_>> = downloaded_blocks.cloned().collect();
|
let blocks: Vec<_> = downloaded_blocks.cloned().collect();
|
||||||
match self
|
match self
|
||||||
.chain
|
.chain
|
||||||
.process_chain_segment(blocks, count_unrealized)
|
.process_chain_segment(blocks, count_unrealized)
|
||||||
|
@ -647,6 +647,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
|
|||||||
WorkEvent::rpc_beacon_block(block_root, block, duration, process_type)
|
WorkEvent::rpc_beacon_block(block_root, block, duration, process_type)
|
||||||
}
|
}
|
||||||
BlockTy::BlockAndBlob { block_sidecar_pair } => {
|
BlockTy::BlockAndBlob { block_sidecar_pair } => {
|
||||||
|
//FIXME(sean)
|
||||||
// WorkEvent::rpc_block_and_glob(block_sidecar_pair)
|
// WorkEvent::rpc_block_and_glob(block_sidecar_pair)
|
||||||
todo!("we also need to process block-glob pairs for rpc")
|
todo!("we also need to process block-glob pairs for rpc")
|
||||||
}
|
}
|
||||||
|
@ -108,7 +108,7 @@ impl<const MAX_ATTEMPTS: u8> SingleBlockRequest<MAX_ATTEMPTS> {
|
|||||||
pub fn verify_block<T: EthSpec>(
|
pub fn verify_block<T: EthSpec>(
|
||||||
&mut self,
|
&mut self,
|
||||||
block: Option<BlockTy<T>>,
|
block: Option<BlockTy<T>>,
|
||||||
) -> Result<Option<(RootBlockTuple<T>)>, VerifyError> {
|
) -> Result<Option<RootBlockTuple<T>>, VerifyError> {
|
||||||
match self.state {
|
match self.state {
|
||||||
State::AwaitingDownload => {
|
State::AwaitingDownload => {
|
||||||
self.register_failure_downloading();
|
self.register_failure_downloading();
|
||||||
|
@ -35,5 +35,5 @@ procinfo = { version = "0.4.2", optional = true }
|
|||||||
[features]
|
[features]
|
||||||
default = ["lighthouse"]
|
default = ["lighthouse"]
|
||||||
lighthouse = ["proto_array", "psutil", "procinfo", "store", "slashing_protection"]
|
lighthouse = ["proto_array", "psutil", "procinfo", "store", "slashing_protection"]
|
||||||
withdrawals = ["store/withdrawals"]
|
withdrawals = ["store/withdrawals", "types/withdrawals"]
|
||||||
withdrawals-processing = ["store/withdrawals-processing"]
|
withdrawals-processing = ["store/withdrawals-processing"]
|
@ -1,4 +1,4 @@
|
|||||||
use crate::{BlobsSidecar, EthSpec, SignedBeaconBlock, SignedBeaconBlockEip4844};
|
use crate::{BlobsSidecar, EthSpec, SignedBeaconBlock, SignedBeaconBlockEip4844, Slot};
|
||||||
use serde_derive::{Deserialize, Serialize};
|
use serde_derive::{Deserialize, Serialize};
|
||||||
use ssz::{Decode, DecodeError};
|
use ssz::{Decode, DecodeError};
|
||||||
use ssz_derive::{Decode, Encode};
|
use ssz_derive::{Decode, Encode};
|
||||||
@ -33,7 +33,7 @@ impl<T: EthSpec> SignedBeaconBlockAndBlobsSidecar<T> {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// A wrapper over a [`SignedBeaconBlock`] or a [`SignedBeaconBlockAndBlobsSidecar`].
|
/// A wrapper over a [`SignedBeaconBlock`] or a [`SignedBeaconBlockAndBlobsSidecar`].
|
||||||
#[derive(Debug)]
|
#[derive(Clone, Debug)]
|
||||||
pub enum BlockWrapper<T: EthSpec> {
|
pub enum BlockWrapper<T: EthSpec> {
|
||||||
Block {
|
Block {
|
||||||
block: Arc<SignedBeaconBlock<T>>,
|
block: Arc<SignedBeaconBlock<T>>,
|
||||||
@ -44,10 +44,34 @@ pub enum BlockWrapper<T: EthSpec> {
|
|||||||
}
|
}
|
||||||
|
|
||||||
impl<T: EthSpec> BlockWrapper<T> {
|
impl<T: EthSpec> BlockWrapper<T> {
|
||||||
|
pub fn slot(&self) -> Slot {
|
||||||
|
match self {
|
||||||
|
BlockWrapper::Block { block } => block.slot(),
|
||||||
|
BlockWrapper::BlockAndBlob { block_sidecar_pair } => {
|
||||||
|
block_sidecar_pair.beacon_block.slot()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
pub fn block(&self) -> &SignedBeaconBlock<T> {
|
pub fn block(&self) -> &SignedBeaconBlock<T> {
|
||||||
match self {
|
match self {
|
||||||
BlockWrapper::Block { block } => &block,
|
BlockWrapper::Block { block } => &block,
|
||||||
BlockWrapper::BlockAndBlob { block_sidecar_pair } => &block_sidecar_pair.beacon_block,
|
BlockWrapper::BlockAndBlob { block_sidecar_pair } => &block_sidecar_pair.beacon_block,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
pub fn block_cloned(&self) -> Arc<SignedBeaconBlock<T>> {
|
||||||
|
match self {
|
||||||
|
BlockWrapper::Block { block } => block.clone(),
|
||||||
|
BlockWrapper::BlockAndBlob { block_sidecar_pair } => {
|
||||||
|
block_sidecar_pair.beacon_block.clone()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
pub fn blocks_sidecar(&self) -> Option<Arc<BlobsSidecar<T>>> {
|
||||||
|
match self {
|
||||||
|
BlockWrapper::Block { block: _ } => None,
|
||||||
|
BlockWrapper::BlockAndBlob { block_sidecar_pair } => {
|
||||||
|
Some(block_sidecar_pair.blobs_sidecar.clone())
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user