handle unknown parents for block-blob pairs

wip

handle unknown parents for block-blob pairs
This commit is contained in:
Diva M 2022-11-30 13:31:58 -05:00
parent 2157d91b43
commit 979a95d62f
No known key found for this signature in database
GPG Key ID: 1BAE5E01126680FE
11 changed files with 99 additions and 102 deletions

View File

@ -2484,8 +2484,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// Returns an `Err` if the given block was invalid, or an error was encountered during
pub async fn verify_block_for_gossip(
self: &Arc<Self>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
blobs: Option<Arc<BlobsSidecar<T::EthSpec>>>,
block: BlockWrapper<T::EthSpec>,
) -> Result<GossipVerifiedBlock<T>, BlockError<T::EthSpec>> {
let chain = self.clone();
self.task_executor
@ -2495,7 +2494,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
let slot = block.slot();
let graffiti_string = block.message().body().graffiti().as_utf8_lossy();
match GossipVerifiedBlock::new(block, blobs, &chain) {
match GossipVerifiedBlock::new(block, &chain) {
Ok(verified) => {
debug!(
chain.log,
@ -2621,7 +2620,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
) -> Result<Hash256, BlockError<T::EthSpec>> {
let ExecutionPendingBlock {
block,
blobs,
block_root,
state,
parent_block,
@ -2674,7 +2672,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
move || {
chain.import_block(
block,
blobs,
block_root,
state,
confirmed_state_roots,
@ -2699,8 +2696,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
#[allow(clippy::too_many_arguments)]
fn import_block(
&self,
signed_block: Arc<SignedBeaconBlock<T::EthSpec>>,
blobs: Option<Arc<BlobsSidecar<T::EthSpec>>>,
signed_block: BlockWrapper<T::EthSpec>,
block_root: Hash256,
mut state: BeaconState<T::EthSpec>,
confirmed_state_roots: Vec<Hash256>,
@ -2833,7 +2829,8 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
let mut fork_choice = self.canonical_head.fork_choice_write_lock();
// Do not import a block that doesn't descend from the finalized root.
check_block_is_finalized_descendant(self, &fork_choice, &signed_block)?;
let signed_block = check_block_is_finalized_descendant(self, &fork_choice, signed_block)?;
let block = signed_block.message();
// Register the new block with the fork choice service.
{
@ -2950,7 +2947,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
if let Err(e) = self.early_attester_cache.add_head_block(
block_root,
signed_block.clone(),
blobs.clone(),
proto_block,
&state,
&self.spec,
@ -3036,6 +3032,8 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
// If the write fails, revert fork choice to the version from disk, else we can
// end up with blocks in fork choice that are missing from disk.
// See https://github.com/sigp/lighthouse/issues/2028
let (signed_block, blobs) = signed_block.deconstruct();
let block = signed_block.message();
let mut ops: Vec<_> = confirmed_state_roots
.into_iter()
.map(StoreOp::DeleteStateTemporaryFlag)

View File

@ -140,7 +140,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(Arc<SignedBeaconBlock<T>>),
ParentUnknown(BlockWrapper<T>),
/// The block skips too many slots and is a DoS risk.
TooManySkippedSlots {
parent_slot: Slot,
@ -556,11 +556,10 @@ pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
return Ok(vec![]);
}
let (first_root, first_block_wrapper) = chain_segment.remove(0);
let (mut parent, first_block) =
load_parent(first_root, first_block_wrapper.block_cloned(), chain)?;
let (first_root, first_block) = chain_segment.remove(0);
let (mut parent, first_block) = load_parent(first_root, first_block, chain)?;
let slot = first_block.slot();
chain_segment.insert(0, (first_root, first_block_wrapper));
chain_segment.insert(0, (first_root, first_block));
let highest_slot = chain_segment
.last()
@ -594,9 +593,9 @@ pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
let consensus_context = ConsensusContext::new(block.slot())
.set_current_block_root(block_root)
.set_proposer_index(block.block().message().proposer_index())
.set_blobs_sidecar(block.blocks_sidecar());
.set_blobs_sidecar(block.blobs_sidecar());
SignatureVerifiedBlock {
block: block.block_cloned(),
block,
block_root,
parent: None,
consensus_context,
@ -616,7 +615,7 @@ pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
#[derive(Derivative)]
#[derivative(Debug(bound = "T: BeaconChainTypes"))]
pub struct GossipVerifiedBlock<T: BeaconChainTypes> {
pub block: Arc<SignedBeaconBlock<T::EthSpec>>,
pub block: BlockWrapper<T::EthSpec>,
pub block_root: Hash256,
parent: Option<PreProcessingSnapshot<T::EthSpec>>,
consensus_context: ConsensusContext<T::EthSpec>,
@ -625,7 +624,7 @@ pub struct GossipVerifiedBlock<T: BeaconChainTypes> {
/// A wrapper around a `SignedBeaconBlock` that indicates that all signatures (except the deposit
/// signatures) have been verified.
pub struct SignatureVerifiedBlock<T: BeaconChainTypes> {
block: Arc<SignedBeaconBlock<T::EthSpec>>,
block: BlockWrapper<T::EthSpec>,
block_root: Hash256,
parent: Option<PreProcessingSnapshot<T::EthSpec>>,
consensus_context: ConsensusContext<T::EthSpec>,
@ -648,8 +647,7 @@ type PayloadVerificationHandle<E> =
/// due to finality or some other event. A `ExecutionPendingBlock` should be imported into the
/// `BeaconChain` immediately after it is instantiated.
pub struct ExecutionPendingBlock<T: BeaconChainTypes> {
pub block: Arc<SignedBeaconBlock<T::EthSpec>>,
pub blobs: Option<Arc<BlobsSidecar<T::EthSpec>>>,
pub block: BlockWrapper<T::EthSpec>,
pub block_root: Hash256,
pub state: BeaconState<T::EthSpec>,
pub parent_block: SignedBeaconBlock<T::EthSpec, BlindedPayload<T::EthSpec>>,
@ -671,7 +669,8 @@ pub trait IntoExecutionPendingBlock<T: BeaconChainTypes>: Sized {
.map(|execution_pending| {
// Supply valid block to slasher.
if let Some(slasher) = chain.slasher.as_ref() {
slasher.accept_block_header(execution_pending.block.signed_block_header());
slasher
.accept_block_header(execution_pending.block.block().signed_block_header());
}
execution_pending
})
@ -692,30 +691,29 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
/// Instantiates `Self`, a wrapper that indicates the given `block` is safe to be re-gossiped
/// on the p2p network.
///
/// 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 i8f the block was unable to be verified.
pub fn new(
block: Arc<SignedBeaconBlock<T::EthSpec>>,
blobs: Option<Arc<BlobsSidecar<T::EthSpec>>>,
block: BlockWrapper<T::EthSpec>,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError<T::EthSpec>> {
// If the block is valid for gossip we don't supply it to the slasher here because
// we assume it will be transformed into a fully verified block. We *do* need to supply
// it to the slasher if an error occurs, because that's the end of this block's journey,
// and it could be a repeat proposal (a likely cause for slashing!).
let header = block.signed_block_header();
Self::new_without_slasher_checks(block, blobs, chain).map_err(|e| {
let header = block.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: Arc<SignedBeaconBlock<T::EthSpec>>,
blobs: Option<Arc<BlobsSidecar<T::EthSpec>>>,
block: BlockWrapper<T::EthSpec>,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError<T::EthSpec>> {
// Ensure the block is the correct structure for the fork at `block.slot()`.
block
.block()
.fork_name(&chain.spec)
.map_err(BlockError::InconsistentFork)?;
@ -731,7 +729,7 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
});
}
let block_root = get_block_root(&block);
let block_root = get_block_root(block.block());
// Disallow blocks that conflict with the anchor (weak subjectivity checkpoint), if any.
check_block_against_anchor_slot(block.message(), chain)?;
@ -770,10 +768,10 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
// Do not process a block that doesn't descend from the finalized root.
//
// We check this *before* we load the parent so that we can return a more detailed error.
check_block_is_finalized_descendant(
let block = check_block_is_finalized_descendant(
chain,
&chain.canonical_head.fork_choice_write_lock(),
&block,
block,
)?;
let block_epoch = block.slot().epoch(T::EthSpec::slots_per_epoch());
@ -867,7 +865,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.verify_signature(
block.block().verify_signature(
Some(block_root),
pubkey,
&fork,
@ -907,7 +905,7 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
// Validate the block's execution_payload (if any).
validate_execution_payload_for_gossip(&parent_block, block.message(), chain)?;
if let Some(blobs_sidecar) = blobs.as_ref() {
if let Some(blobs_sidecar) = block.blobs() {
let kzg_commitments = block
.message()
.body()
@ -937,7 +935,7 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
.set_proposer_index(block.message().proposer_index())
.set_blobs_sidecar_validated(true) // Validated in `validate_blob_for_gossip`
.set_blobs_verified_vs_txs(true) // Validated in `validate_blob_for_gossip`
.set_blobs_sidecar(blobs);
.set_blobs_sidecar(block.blobs_sidecar()); // TODO: potentially remove
Ok(Self {
block,
@ -965,7 +963,7 @@ impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for GossipVerifiedBlock<T
}
fn block(&self) -> &SignedBeaconBlock<T::EthSpec> {
&self.block
self.block.block()
}
}
@ -975,13 +973,13 @@ 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_wrapper: BlockWrapper<T::EthSpec>,
block: BlockWrapper<T::EthSpec>,
block_root: Hash256,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError<T::EthSpec>> {
let (block, blobs_sidecar) = block_wrapper.deconstruct();
// Ensure the block is the correct structure for the fork at `block.slot()`.
block
.block()
.fork_name(&chain.spec)
.map_err(BlockError::InconsistentFork)?;
@ -1004,14 +1002,14 @@ impl<T: BeaconChainTypes> SignatureVerifiedBlock<T> {
let mut signature_verifier = get_signature_verifier(&state, &pubkey_cache, &chain.spec);
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_ok() {
Ok(Self {
consensus_context: ConsensusContext::new(block.slot())
.set_current_block_root(block_root)
.set_proposer_index(block.message().proposer_index())
.set_blobs_sidecar(blobs_sidecar),
.set_blobs_sidecar(block.blobs_sidecar()),
block,
block_root,
parent: Some(parent),
@ -1058,7 +1056,7 @@ impl<T: BeaconChainTypes> SignatureVerifiedBlock<T> {
// signature.
let verified_proposer_index = Some(block.message().proposer_index());
signature_verifier
.include_all_signatures_except_proposal(&block, verified_proposer_index)?;
.include_all_signatures_except_proposal(block.block(), verified_proposer_index)?;
if signature_verifier.verify().is_ok() {
Ok(Self {
@ -1077,7 +1075,7 @@ impl<T: BeaconChainTypes> SignatureVerifiedBlock<T> {
from: GossipVerifiedBlock<T>,
chain: &BeaconChain<T>,
) -> Result<Self, BlockSlashInfo<BlockError<T::EthSpec>>> {
let header = from.block.signed_block_header();
let header = from.block.block().signed_block_header();
Self::from_gossip_verified_block(from, chain)
.map_err(|e| BlockSlashInfo::from_early_error(header, e))
}
@ -1094,7 +1092,7 @@ impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for SignatureVerifiedBloc
block_root: Hash256,
chain: &Arc<BeaconChain<T>>,
) -> Result<ExecutionPendingBlock<T>, BlockSlashInfo<BlockError<T::EthSpec>>> {
let header = self.block.signed_block_header();
let header = self.block.block().signed_block_header();
let (parent, block) = if let Some(parent) = self.parent {
(parent, self.block)
} else {
@ -1113,7 +1111,7 @@ impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for SignatureVerifiedBloc
}
fn block(&self) -> &SignedBeaconBlock<T::EthSpec> {
&self.block
&self.block.block()
}
}
@ -1173,7 +1171,7 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
///
/// Returns an error if the block is invalid, or if the block was unable to be verified.
pub fn from_signature_verified_components(
block: Arc<SignedBeaconBlock<T::EthSpec>>,
block: BlockWrapper<T::EthSpec>,
block_root: Hash256,
parent: PreProcessingSnapshot<T::EthSpec>,
mut consensus_context: ConsensusContext<T::EthSpec>,
@ -1212,7 +1210,7 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
* Perform cursory checks to see if the block is even worth processing.
*/
check_block_relevancy(&block, block_root, chain)?;
check_block_relevancy(block.block(), block_root, chain)?;
/*
* Advance the given `parent.beacon_state` to the slot of the given `block`.
@ -1324,7 +1322,7 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
// Define a future that will verify the execution payload with an execution engine (but
// don't execute it yet).
let payload_notifier = PayloadNotifier::new(chain.clone(), block.clone(), &state)?;
let payload_notifier = PayloadNotifier::new(chain.clone(), block.block_cloned(), &state)?;
let is_valid_merge_transition_block =
is_merge_transition_block(&state, block.message().body());
let payload_verification_future = async move {
@ -1458,13 +1456,13 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
&state,
&chain.log,
);
write_block(&block, block_root, &chain.log);
write_block(block.block(), block_root, &chain.log);
let core_timer = metrics::start_timer(&metrics::BLOCK_PROCESSING_CORE);
if let Err(err) = per_block_processing(
&mut state,
&block,
block.block(),
// Signatures were verified earlier in this function.
BlockSignatureStrategy::NoVerification,
VerifyBlockRoot::True,
@ -1501,9 +1499,9 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
* Check to ensure the state root on the block matches the one we have calculated.
*/
if block.state_root() != state_root {
if block.block().state_root() != state_root {
return Err(BlockError::StateRootMismatch {
block: block.state_root(),
block: block.block().state_root(),
local: state_root,
});
}
@ -1559,7 +1557,6 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
Ok(Self {
block,
blobs: consensus_context.blobs_sidecar(),
block_root,
state,
parent_block: parent.beacon_block,
@ -1646,10 +1643,10 @@ fn check_block_against_finalized_slot<T: BeaconChainTypes>(
pub fn check_block_is_finalized_descendant<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
fork_choice: &BeaconForkChoice<T>,
block: &Arc<SignedBeaconBlock<T::EthSpec>>,
) -> Result<(), BlockError<T::EthSpec>> {
block: BlockWrapper<T::EthSpec>,
) -> Result<BlockWrapper<T::EthSpec>, BlockError<T::EthSpec>> {
if fork_choice.is_descendant_of_finalized(block.parent_root()) {
Ok(())
Ok(block)
} else {
// If fork choice does *not* consider the parent to be a descendant of the finalized block,
// then there are two more cases:
@ -1668,8 +1665,7 @@ pub fn check_block_is_finalized_descendant<T: BeaconChainTypes>(
block_parent_root: block.parent_root(),
})
} else {
//FIXME(sean) does this matter if it only returns a block?
Err(BlockError::ParentUnknown(block.clone()))
Err(BlockError::ParentUnknown(block))
}
}
}
@ -1741,8 +1737,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: Arc<SignedBeaconBlock<T::EthSpec>>,
) -> Result<(ProtoBlock, Arc<SignedBeaconBlock<T::EthSpec>>), BlockError<T::EthSpec>> {
block: BlockWrapper<T::EthSpec>,
) -> Result<(ProtoBlock, BlockWrapper<T::EthSpec>), BlockError<T::EthSpec>> {
if let Some(proto_block) = chain
.canonical_head
.fork_choice_read_lock()
@ -1761,15 +1757,9 @@ fn verify_parent_block_is_known<T: BeaconChainTypes>(
#[allow(clippy::type_complexity)]
fn load_parent<T: BeaconChainTypes>(
block_root: Hash256,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
block: BlockWrapper<T::EthSpec>,
chain: &BeaconChain<T>,
) -> Result<
(
PreProcessingSnapshot<T::EthSpec>,
Arc<SignedBeaconBlock<T::EthSpec>>,
),
BlockError<T::EthSpec>,
> {
) -> Result<(PreProcessingSnapshot<T::EthSpec>, BlockWrapper<T::EthSpec>), BlockError<T::EthSpec>> {
let spec = &chain.spec;
// Reject any block if its parent is not known to fork choice.

View File

@ -5,6 +5,7 @@ use crate::{
use parking_lot::RwLock;
use proto_array::Block as ProtoBlock;
use std::sync::Arc;
use store::signed_block_and_blobs::BlockWrapper;
use types::*;
pub struct CacheItem<E: EthSpec> {
@ -50,8 +51,7 @@ impl<E: EthSpec> EarlyAttesterCache<E> {
pub fn add_head_block(
&self,
beacon_block_root: Hash256,
block: Arc<SignedBeaconBlock<E>>,
blobs: Option<Arc<BlobsSidecar<E>>>,
block: BlockWrapper<E>,
proto_block: ProtoBlock,
state: &BeaconState<E>,
spec: &ChainSpec,
@ -69,6 +69,7 @@ impl<E: EthSpec> EarlyAttesterCache<E> {
},
};
let (block, blobs) = block.deconstruct();
let item = CacheItem {
epoch,
committee_lengths,

View File

@ -33,10 +33,10 @@ pub async fn publish_block<T: BeaconChainTypes>(
// specification is very clear that this is the desired behaviour.
let message = if matches!(block.as_ref(), &SignedBeaconBlock::Eip4844(_)) {
if let Some(sidecar) = chain.blob_cache.pop(&block_root) {
PubsubMessage::BeaconBlockAndBlobsSidecars(Arc::new(SignedBeaconBlockAndBlobsSidecar {
PubsubMessage::BeaconBlockAndBlobsSidecars(SignedBeaconBlockAndBlobsSidecar {
beacon_block: block.clone(),
blobs_sidecar: Arc::new(sidecar),
}))
})
} else {
//FIXME(sean): This should probably return a specific no-blob-cached error code, beacon API coordination required
return Err(warp_utils::reject::broadcast_without_import(format!(

View File

@ -24,7 +24,7 @@ pub enum PubsubMessage<T: EthSpec> {
/// Gossipsub message providing notification of a new block.
BeaconBlock(Arc<SignedBeaconBlock<T>>),
/// Gossipsub message providing notification of a new SignedBeaconBlock coupled with a blobs sidecar.
BeaconBlockAndBlobsSidecars(Arc<SignedBeaconBlockAndBlobsSidecar<T>>),
BeaconBlockAndBlobsSidecars(SignedBeaconBlockAndBlobsSidecar<T>),
/// Gossipsub message providing notification of a Aggregate attestation and associated proof.
AggregateAndProofAttestation(Box<SignedAggregateAndProof<T>>),
/// Gossipsub message providing notification of a raw un-aggregated attestation with its shard id.
@ -204,9 +204,9 @@ impl<T: EthSpec> PubsubMessage<T> {
let block_and_blobs_sidecar =
SignedBeaconBlockAndBlobsSidecar::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?;
Ok(PubsubMessage::BeaconBlockAndBlobsSidecars(Arc::new(
Ok(PubsubMessage::BeaconBlockAndBlobsSidecars(
block_and_blobs_sidecar,
)))
))
}
Some(
ForkName::Base

View File

@ -424,7 +424,7 @@ impl<T: BeaconChainTypes> WorkEvent<T> {
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
block_and_blobs: Arc<SignedBeaconBlockAndBlobsSidecar<T::EthSpec>>,
block_and_blobs: SignedBeaconBlockAndBlobsSidecar<T::EthSpec>,
seen_timestamp: Duration,
) -> Self {
Self {
@ -764,7 +764,7 @@ pub enum Work<T: BeaconChainTypes> {
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
block_and_blobs: Arc<SignedBeaconBlockAndBlobsSidecar<T::EthSpec>>,
block_and_blobs: SignedBeaconBlockAndBlobsSidecar<T::EthSpec>,
seen_timestamp: Duration,
},
DelayedImportBlock {
@ -1594,8 +1594,7 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
message_id,
peer_id,
peer_client,
block,
None,
BlockWrapper::Block { block },
work_reprocessing_tx,
duplicate_cache,
seen_timestamp,
@ -1609,7 +1608,7 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
message_id,
peer_id,
peer_client,
block_and_blobs,
block_and_blobs: block_sidecar_pair,
seen_timestamp,
} => task_spawner.spawn_async(async move {
worker
@ -1617,8 +1616,7 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
message_id,
peer_id,
peer_client,
block_and_blobs.beacon_block.clone(),
Some(block_and_blobs.blobs_sidecar.clone()),
BlockWrapper::BlockAndBlob { block_sidecar_pair },
work_reprocessing_tx,
duplicate_cache,
seen_timestamp,

View File

@ -18,6 +18,7 @@ use std::sync::Arc;
use std::time::{Duration, SystemTime, UNIX_EPOCH};
use store::hot_cold_store::HotColdDBError;
use tokio::sync::mpsc;
use types::signed_block_and_blobs::BlockWrapper;
use types::{
Attestation, AttesterSlashing, BlobsSidecar, EthSpec, Hash256, IndexedAttestation,
ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockAndBlobsSidecar,
@ -657,8 +658,7 @@ impl<T: BeaconChainTypes> Worker<T> {
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
blobs: Option<Arc<BlobsSidecar<T::EthSpec>>>,
block: BlockWrapper<T::EthSpec>,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
duplicate_cache: DuplicateCache,
seen_duration: Duration,
@ -669,7 +669,6 @@ impl<T: BeaconChainTypes> Worker<T> {
peer_id,
peer_client,
block,
blobs,
reprocess_tx.clone(),
seen_duration,
)
@ -706,8 +705,7 @@ impl<T: BeaconChainTypes> Worker<T> {
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
blobs: Option<Arc<BlobsSidecar<T::EthSpec>>>,
block: BlockWrapper<T::EthSpec>,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
seen_duration: Duration,
) -> Option<GossipVerifiedBlock<T>> {
@ -722,13 +720,13 @@ impl<T: BeaconChainTypes> Worker<T> {
let verification_result = self
.chain
.clone()
.verify_block_for_gossip(block.clone(), blobs)
.verify_block_for_gossip(block.clone())
.await;
let block_root = if let Ok(verified_block) = &verification_result {
verified_block.block_root
} else {
block.canonical_root()
block.block().canonical_root()
};
// Write the time the block was observed into delay cache.
@ -936,7 +934,7 @@ impl<T: BeaconChainTypes> Worker<T> {
// This value is not used presently, but it might come in handy for debugging.
_seen_duration: Duration,
) {
let block: Arc<_> = verified_block.block.clone();
let block = verified_block.block.block_cloned();
let block_root = verified_block.block_root;
match self
@ -968,7 +966,7 @@ impl<T: BeaconChainTypes> Worker<T> {
self.chain.recompute_head_at_current_slot().await;
}
Err(BlockError::ParentUnknown { .. }) => {
Err(BlockError::ParentUnknown(block)) => {
// Inform the sync manager to find parents for this block
// This should not occur. It should be checked by `should_forward_block`
error!(

View File

@ -351,7 +351,7 @@ impl<T: BeaconChainTypes> Processor<T> {
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
block_and_blobs: Arc<SignedBeaconBlockAndBlobsSidecar<T::EthSpec>>,
block_and_blobs: SignedBeaconBlockAndBlobsSidecar<T::EthSpec>,
) {
self.send_beacon_processor_work(BeaconWorkEvent::gossip_block_and_blobs_sidecar(
message_id,

View File

@ -429,7 +429,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
error!(self.log, "Beacon chain error processing single block"; "block_root" => %root, "error" => ?e);
}
BlockError::ParentUnknown(block) => {
self.search_parent(root, BlockWrapper::Block { block }, peer_id, cx);
self.search_parent(root, block, peer_id, cx);
}
ref e @ BlockError::ExecutionPayloadError(ref epe) if !epe.penalize_peer() => {
// These errors indicate that the execution layer is offline
@ -509,7 +509,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
BlockProcessResult::Err(BlockError::ParentUnknown(block)) => {
// need to keep looking for parents
// add the block back to the queue and continue the search
parent_lookup.add_block(BlockWrapper::Block { block });
parent_lookup.add_block(block);
self.request_parent(parent_lookup, cx);
}
BlockProcessResult::Ok

View File

@ -118,7 +118,7 @@ pub enum SyncMessage<T: EthSpec> {
},
/// A block with an unknown parent has been received.
UnknownBlock(PeerId, Arc<SignedBeaconBlock<T>>, Hash256),
UnknownBlock(PeerId, BlockWrapper<T>, Hash256),
/// A peer has sent an object that references a block that is unknown. This triggers the
/// manager to attempt to find the block matching the unknown hash.
@ -582,14 +582,8 @@ impl<T: BeaconChainTypes> SyncManager<T> {
if self.network_globals.peers.read().is_connected(&peer_id)
&& self.network.is_execution_engine_online()
{
// TODO: here it would be ideal if unknown block carried either the block or
// the block and blob since for block lookups we don't care.
self.block_lookups.search_parent(
block_root,
BlockWrapper::Block { block },
peer_id,
&mut self.network,
);
self.block_lookups
.search_parent(block_root, block, peer_id, &mut self.network);
}
}
SyncMessage::UnknownBlockHash(peer_id, block_hash) => {

View File

@ -66,7 +66,7 @@ impl<T: EthSpec> BlockWrapper<T> {
}
}
}
pub fn blocks_sidecar(&self) -> Option<Arc<BlobsSidecar<T>>> {
pub fn blobs_sidecar(&self) -> Option<Arc<BlobsSidecar<T>>> {
match self {
BlockWrapper::Block { block: _ } => None,
BlockWrapper::BlockAndBlob { block_sidecar_pair } => {
@ -75,6 +75,24 @@ impl<T: EthSpec> BlockWrapper<T> {
}
}
pub fn blobs(&self) -> Option<&BlobsSidecar<T>> {
match self {
BlockWrapper::Block { .. } => None,
BlockWrapper::BlockAndBlob { block_sidecar_pair } => {
Some(&block_sidecar_pair.blobs_sidecar)
}
}
}
pub fn message(&self) -> crate::BeaconBlockRef<T> {
match self {
BlockWrapper::Block { block } => block.message(),
BlockWrapper::BlockAndBlob { block_sidecar_pair } => {
block_sidecar_pair.beacon_block.message()
}
}
}
pub fn parent_root(&self) -> Hash256 {
self.block().parent_root()
}