gossip validate blobs prior to publish

This commit is contained in:
realbigsean 2023-07-12 12:32:14 -04:00
parent 1599487933
commit c016f5d787
No known key found for this signature in database
GPG Key ID: BE1B3DB104F6C788
11 changed files with 135 additions and 82 deletions

View File

@ -1981,7 +1981,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
self: &Arc<Self>,
blob_sidecar: SignedBlobSidecar<T::EthSpec>,
subnet_id: u64,
) -> Result<GossipVerifiedBlob<T::EthSpec>, BlobError<T::EthSpec>> {
) -> Result<GossipVerifiedBlob<T>, BlobError<T::EthSpec>> {
blob_verification::validate_blob_sidecar_for_gossip(blob_sidecar, subnet_id, self)
}
@ -2711,7 +2711,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: BlockWrapper<T::EthSpec>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
) -> Result<GossipVerifiedBlock<T>, BlockError<T::EthSpec>> {
let chain = self.clone();
self.task_executor
@ -2755,7 +2755,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
pub async fn process_blob(
self: &Arc<Self>,
blob: GossipVerifiedBlob<T::EthSpec>,
blob: GossipVerifiedBlob<T>,
) -> Result<AvailabilityProcessingStatus, BlockError<T::EthSpec>> {
self.check_availability_and_maybe_import(blob.slot(), |chain| {
chain.data_availability_checker.put_gossip_blob(blob)

View File

@ -16,7 +16,7 @@ use eth2::types::BlockContentsTuple;
use kzg::Kzg;
use slog::{debug, warn};
use ssz_derive::{Decode, Encode};
use ssz_types::FixedVector;
use ssz_types::{FixedVector, VariableList};
use std::borrow::Cow;
use types::blob_sidecar::{BlobIdentifier, FixedBlobSidecarList};
use types::{
@ -125,25 +125,40 @@ impl<T: EthSpec> From<BeaconStateError> for BlobError<T> {
}
}
pub type GossipVerifiedBlobList<T> = VariableList<
GossipVerifiedBlob<T>,
<<T as BeaconChainTypes>::EthSpec as EthSpec>::MaxBlobsPerBlock,
>;
/// A wrapper around a `BlobSidecar` that indicates it has been approved for re-gossiping on
/// the p2p network.
#[derive(Debug, Clone)]
pub struct GossipVerifiedBlob<T: EthSpec> {
blob: Arc<BlobSidecar<T>>,
#[derive(Debug)]
pub struct GossipVerifiedBlob<T: BeaconChainTypes> {
blob: SignedBlobSidecar<T::EthSpec>,
}
impl<T: EthSpec> GossipVerifiedBlob<T> {
impl<T: BeaconChainTypes> GossipVerifiedBlob<T> {
pub fn new(
blob: SignedBlobSidecar<T::EthSpec>,
chain: &BeaconChain<T>,
) -> Result<Self, BlobError<T::EthSpec>> {
let blob_index = blob.message.index;
validate_blob_sidecar_for_gossip(blob, blob_index, chain)
}
pub fn id(&self) -> BlobIdentifier {
self.blob.id()
self.blob.message.id()
}
pub fn block_root(&self) -> Hash256 {
self.blob.block_root
self.blob.message.block_root
}
pub fn to_blob(self) -> Arc<BlobSidecar<T>> {
self.blob
pub fn to_blob(self) -> Arc<BlobSidecar<T::EthSpec>> {
self.blob.message
}
pub fn signed_blob(&self) -> SignedBlobSidecar<T::EthSpec> {
self.blob.clone()
}
pub fn slot(&self) -> Slot {
self.blob.slot
self.blob.message.slot
}
}
@ -151,7 +166,7 @@ pub fn validate_blob_sidecar_for_gossip<T: BeaconChainTypes>(
signed_blob_sidecar: SignedBlobSidecar<T::EthSpec>,
subnet: u64,
chain: &BeaconChain<T>,
) -> Result<GossipVerifiedBlob<T::EthSpec>, BlobError<T::EthSpec>> {
) -> Result<GossipVerifiedBlob<T>, BlobError<T::EthSpec>> {
let blob_slot = signed_blob_sidecar.message.slot;
let blob_index = signed_blob_sidecar.message.index;
let block_parent_root = signed_blob_sidecar.message.block_parent_root;
@ -366,7 +381,7 @@ pub fn validate_blob_sidecar_for_gossip<T: BeaconChainTypes>(
// Note: If this BlobSidecar goes on to fail full verification, we do not evict it from the seen_cache
// as alternate blob_sidecars for the same identifier can still be retrieved
// over rpc. Evicting them from this cache would allow faster propagation over gossip. So we allow
// retreieval of potentially valid blocks over rpc, but try to punish the proposer for signing
// retrieval of potentially valid blocks over rpc, but try to punish the proposer for signing
// invalid messages. Issue for more background
// https://github.com/ethereum/consensus-specs/issues/3261
if chain
@ -383,7 +398,7 @@ pub fn validate_blob_sidecar_for_gossip<T: BeaconChainTypes>(
}
Ok(GossipVerifiedBlob {
blob: signed_blob_sidecar.message,
blob: signed_blob_sidecar,
})
}

View File

@ -48,7 +48,10 @@
// returned alongside.
#![allow(clippy::result_large_err)]
use crate::blob_verification::{AsBlock, BlobError, BlockWrapper, MaybeAvailableBlock};
use crate::blob_verification::{
AsBlock, BlobError, BlockWrapper, GossipVerifiedBlob, GossipVerifiedBlobList,
MaybeAvailableBlock,
};
use crate::data_availability_checker::{
AvailabilityCheckError, AvailabilityPendingBlock, AvailableBlock,
};
@ -79,6 +82,7 @@ use slog::{debug, error, warn, Logger};
use slot_clock::SlotClock;
use ssz::Encode;
use ssz_derive::{Decode, Encode};
use ssz_types::VariableList;
use state_processing::per_block_processing::{errors::IntoWithIndex, is_merge_transition_block};
use state_processing::{
block_signature_verifier::{BlockSignatureVerifier, Error as BlockSignatureVerifierError},
@ -803,48 +807,65 @@ pub struct BlockImportData<E: EthSpec> {
pub consensus_context: ConsensusContext<E>,
}
pub trait IntoGossipVerifiedBlock<T: BeaconChainTypes>: Sized {
pub type GossipVerifiedBlockContents<T> =
(GossipVerifiedBlock<T>, Option<GossipVerifiedBlobList<T>>);
pub trait IntoGossipVerifiedBlockContents<T: BeaconChainTypes>: Sized {
fn into_gossip_verified_block(
self,
chain: &BeaconChain<T>,
) -> Result<GossipVerifiedBlock<T>, BlockError<T::EthSpec>>;
fn inner(&self) -> &SignedBeaconBlock<T::EthSpec>;
fn blobs(&self) -> Option<SignedBlobSidecarList<T::EthSpec>>;
) -> Result<GossipVerifiedBlockContents<T>, BlockError<T::EthSpec>>;
fn inner_block(&self) -> &SignedBeaconBlock<T::EthSpec>;
fn inner_blobs(&self) -> Option<SignedBlobSidecarList<T::EthSpec>>;
}
impl<T: BeaconChainTypes> IntoGossipVerifiedBlock<T>
for (
GossipVerifiedBlock<T>,
Option<SignedBlobSidecarList<T::EthSpec>>,
)
{
impl<T: BeaconChainTypes> IntoGossipVerifiedBlockContents<T> for GossipVerifiedBlockContents<T> {
fn into_gossip_verified_block(
self,
_chain: &BeaconChain<T>,
) -> Result<GossipVerifiedBlock<T>, BlockError<T::EthSpec>> {
Ok(self.0)
) -> Result<GossipVerifiedBlockContents<T>, BlockError<T::EthSpec>> {
Ok(self)
}
fn inner(&self) -> &SignedBeaconBlock<T::EthSpec> {
fn inner_block(&self) -> &SignedBeaconBlock<T::EthSpec> {
self.0.block.as_block()
}
fn blobs(&self) -> Option<SignedBlobSidecarList<T::EthSpec>> {
self.1.clone()
fn inner_blobs(&self) -> Option<SignedBlobSidecarList<T::EthSpec>> {
self.1.as_ref().map(|blobs| {
VariableList::from(
blobs
.into_iter()
.map(GossipVerifiedBlob::signed_blob)
.collect::<Vec<_>>(),
)
})
}
}
impl<T: BeaconChainTypes> IntoGossipVerifiedBlock<T> for SignedBlockContents<T::EthSpec> {
impl<T: BeaconChainTypes> IntoGossipVerifiedBlockContents<T> for SignedBlockContents<T::EthSpec> {
fn into_gossip_verified_block(
self,
chain: &BeaconChain<T>,
) -> Result<GossipVerifiedBlock<T>, BlockError<T::EthSpec>> {
GossipVerifiedBlock::new(self.deconstruct().into(), chain)
) -> Result<GossipVerifiedBlockContents<T>, BlockError<T::EthSpec>> {
let (block, blobs) = self.deconstruct();
let gossip_verified_block = GossipVerifiedBlock::new(Arc::new(block), chain)?;
let gossip_verified_blobs = blobs
.map(|blobs| {
Ok::<_, BlobError<T::EthSpec>>(VariableList::from(
blobs
.into_iter()
.map(|blob| GossipVerifiedBlob::new(blob, chain))
.collect::<Result<Vec<_>, BlobError<T::EthSpec>>>()?,
))
})
.transpose()?;
Ok((gossip_verified_block, gossip_verified_blobs))
}
fn inner(&self) -> &SignedBeaconBlock<T::EthSpec> {
fn inner_block(&self) -> &SignedBeaconBlock<T::EthSpec> {
self.signed_block()
}
fn blobs(&self) -> Option<SignedBlobSidecarList<T::EthSpec>> {
fn inner_blobs(&self) -> Option<SignedBlobSidecarList<T::EthSpec>> {
self.blobs_cloned()
}
}
@ -887,10 +908,12 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
///
/// Returns an error if the block is invalid, or i8f the block was unable to be verified.
pub fn new(
block: BlockWrapper<T::EthSpec>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError<T::EthSpec>> {
let maybe_available = chain.data_availability_checker.check_availability(block)?;
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,

View File

@ -205,7 +205,7 @@ impl<T: BeaconChainTypes> DataAvailabilityChecker<T> {
/// This should only accept gossip verified blobs, so we should not have to worry about dupes.
pub fn put_gossip_blob(
&self,
gossip_blob: GossipVerifiedBlob<T::EthSpec>,
gossip_blob: GossipVerifiedBlob<T>,
) -> Result<Availability<T::EthSpec>, AvailabilityCheckError> {
// Verify the KZG commitments.
let kzg_verified_blob = if let Some(kzg) = self.kzg.as_ref() {

View File

@ -1075,7 +1075,7 @@ mod test {
log: Logger,
) -> (
AvailabilityPendingExecutedBlock<E>,
Vec<GossipVerifiedBlob<E>>,
Vec<GossipVerifiedBlob<BaseHarnessType<E, Hot, Cold>>>,
)
where
E: EthSpec,

View File

@ -71,7 +71,7 @@ pub use beacon_fork_choice_store::{BeaconForkChoiceStore, Error as ForkChoiceSto
pub use block_verification::{
get_block_root, AvailabilityPendingExecutedBlock, BlockError, ExecutedBlock,
ExecutionPayloadError, ExecutionPendingBlock, GossipVerifiedBlock, IntoExecutionPendingBlock,
IntoGossipVerifiedBlock, PayloadVerificationOutcome, PayloadVerificationStatus,
IntoGossipVerifiedBlockContents, PayloadVerificationOutcome, PayloadVerificationStatus,
};
pub use canonical_head::{CachedHead, CanonicalHead, CanonicalHeadRwLock};
pub use eth1_chain::{Eth1Chain, Eth1ChainBackend};

View File

@ -851,7 +851,7 @@ async fn block_gossip_verification() {
{
let gossip_verified = harness
.chain
.verify_block_for_gossip(snapshot.beacon_block.clone().into())
.verify_block_for_gossip(snapshot.beacon_block.clone())
.await
.expect("should obtain gossip verified block");
@ -1079,11 +1079,7 @@ async fn block_gossip_verification() {
let block = chain_segment[block_index].beacon_block.clone();
assert!(
harness
.chain
.verify_block_for_gossip(block.into())
.await
.is_ok(),
harness.chain.verify_block_for_gossip(block).await.is_ok(),
"the valid block should be processed"
);
@ -1134,7 +1130,7 @@ async fn verify_block_for_gossip_slashing_detection() {
let verified_block = harness
.chain
.verify_block_for_gossip(Arc::new(block1).into())
.verify_block_for_gossip(Arc::new(block1))
.await
.unwrap();
@ -1161,7 +1157,7 @@ async fn verify_block_for_gossip_slashing_detection() {
unwrap_err(
harness
.chain
.verify_block_for_gossip(Arc::new(block2).into())
.verify_block_for_gossip(Arc::new(block2))
.await,
);
@ -1184,7 +1180,7 @@ async fn verify_block_for_gossip_doppelganger_detection() {
let verified_block = harness
.chain
.verify_block_for_gossip(Arc::new(block).into())
.verify_block_for_gossip(Arc::new(block))
.await
.unwrap();
let attestations = verified_block.block.message().body().attestations().clone();

View File

@ -4,7 +4,7 @@ use beacon_chain::blob_verification::AsBlock;
use beacon_chain::validator_monitor::{get_block_delay_ms, timestamp_now};
use beacon_chain::{
AvailabilityProcessingStatus, BeaconChain, BeaconChainError, BeaconChainTypes, BlockError,
IntoGossipVerifiedBlock, NotifyExecutionLayer,
IntoGossipVerifiedBlockContents, NotifyExecutionLayer,
};
use eth2::types::BroadcastValidation;
use eth2::types::SignedBlockContents;
@ -24,7 +24,7 @@ use types::{
};
use warp::Rejection;
pub enum ProvenancedBlock<T: BeaconChainTypes, B: IntoGossipVerifiedBlock<T>> {
pub enum ProvenancedBlock<T: BeaconChainTypes, B: IntoGossipVerifiedBlockContents<T>> {
/// The payload was built using a local EE.
Local(B, PhantomData<T>),
/// The payload was build using a remote builder (e.g., via a mev-boost
@ -32,7 +32,7 @@ pub enum ProvenancedBlock<T: BeaconChainTypes, B: IntoGossipVerifiedBlock<T>> {
Builder(B, PhantomData<T>),
}
impl<T: BeaconChainTypes, B: IntoGossipVerifiedBlock<T>> ProvenancedBlock<T, B> {
impl<T: BeaconChainTypes, B: IntoGossipVerifiedBlockContents<T>> ProvenancedBlock<T, B> {
pub fn local(block: B) -> Self {
Self::Local(block, PhantomData)
}
@ -43,7 +43,7 @@ impl<T: BeaconChainTypes, B: IntoGossipVerifiedBlock<T>> ProvenancedBlock<T, B>
}
/// Handles a request from the HTTP API for full blocks.
pub async fn publish_block<T: BeaconChainTypes, B: IntoGossipVerifiedBlock<T>>(
pub async fn publish_block<T: BeaconChainTypes, B: IntoGossipVerifiedBlockContents<T>>(
block_root: Option<Hash256>,
provenanced_block: ProvenancedBlock<T, B>,
chain: Arc<BeaconChain<T>>,
@ -57,7 +57,7 @@ pub async fn publish_block<T: BeaconChainTypes, B: IntoGossipVerifiedBlock<T>>(
ProvenancedBlock::Local(block_contents, _) => (block_contents, true),
ProvenancedBlock::Builder(block_contents, _) => (block_contents, false),
};
let block = block_contents.inner();
let block = block_contents.inner_block();
let delay = get_block_delay_ms(seen_timestamp, block.message(), &chain.slot_clock);
debug!(log, "Signed block received in HTTP API"; "slot" => block.slot());
@ -111,10 +111,10 @@ pub async fn publish_block<T: BeaconChainTypes, B: IntoGossipVerifiedBlock<T>>(
// We can clone this because the blobs are `Arc`'d in `BlockContents`, but the block is not,
// so we avoid cloning the block at this point.
let blobs_opt = block_contents.blobs();
let blobs_opt = block_contents.inner_blobs();
/* if we can form a `GossipVerifiedBlock`, we've passed our basic gossip checks */
let gossip_verified_block = block_contents
let (gossip_verified_block, gossip_verified_blobs) = block_contents
.into_gossip_verified_block(&chain)
.map_err(|e| {
warn!(log, "Not publishing block, not gossip verified"; "slot" => slot, "error" => ?e);
@ -175,6 +175,16 @@ pub async fn publish_block<T: BeaconChainTypes, B: IntoGossipVerifiedBlock<T>>(
}
};
if let Some(gossip_verified_blobs) = gossip_verified_blobs {
for blob in gossip_verified_blobs {
if let Err(e) = chain.process_blob(blob).await {
return Err(warp_utils::reject::custom_bad_request(format!(
"Invalid blob: {e}"
)));
}
}
}
match chain
.process_block(
block_root,

View File

@ -1,12 +1,13 @@
use beacon_chain::{
test_utils::{AttestationStrategy, BlockStrategy},
GossipVerifiedBlock,
GossipVerifiedBlock, IntoGossipVerifiedBlockContents,
};
use eth2::types::{
BroadcastValidation, SignedBeaconBlock, SignedBlindedBeaconBlock, SignedBlockContents,
};
use http_api::test_utils::InteractiveTester;
use http_api::{publish_blinded_block, publish_block, reconstruct_block, ProvenancedBlock};
use std::sync::Arc;
use tree_hash::TreeHash;
use types::{Hash256, MainnetEthSpec, Slot};
use warp::Rejection;
@ -314,14 +315,16 @@ pub async fn consensus_partial_pass_only_consensus() {
tester.harness.make_block(state_a.clone(), slot_b).await;
let ((block_b, blobs_b), state_after_b): ((SignedBeaconBlock<E>, _), _) =
tester.harness.make_block(state_a, slot_b).await;
let block_b_root = block_b.canonical_root();
/* check for `make_block` curios */
assert_eq!(block_a.state_root(), state_after_a.tree_hash_root());
assert_eq!(block_b.state_root(), state_after_b.tree_hash_root());
assert_ne!(block_a.state_root(), block_b.state_root());
let gossip_block_b = GossipVerifiedBlock::new(block_b.clone().into(), &tester.harness.chain);
assert!(gossip_block_b.is_ok());
let gossip_block_contents_b = SignedBlockContents::new(block_b, blobs_b)
.into_gossip_verified_block(&tester.harness.chain);
assert!(gossip_block_contents_b.is_ok());
let gossip_block_a = GossipVerifiedBlock::new(block_a.clone().into(), &tester.harness.chain);
assert!(gossip_block_a.is_err());
@ -330,7 +333,7 @@ pub async fn consensus_partial_pass_only_consensus() {
let publication_result: Result<(), Rejection> = publish_block(
None,
ProvenancedBlock::local((gossip_block_b.unwrap(), blobs_b)),
ProvenancedBlock::local(gossip_block_contents_b.unwrap()),
tester.harness.chain.clone(),
&channel.0,
test_logger,
@ -342,7 +345,7 @@ pub async fn consensus_partial_pass_only_consensus() {
assert!(tester
.harness
.chain
.block_is_known_to_fork_choice(&block_b.canonical_root()));
.block_is_known_to_fork_choice(&block_b_root));
}
/// This test checks that a block that is valid from both a gossip and consensus perspective is accepted when using `broadcast_validation=consensus`.
@ -600,7 +603,7 @@ pub async fn equivocation_consensus_late_equivocation() {
let slot_b = slot_a + 1;
let state_a = tester.harness.get_current_state();
let ((block_a, _), state_after_a): ((SignedBeaconBlock<E>, _), _) =
let ((block_a, blobs_a), state_after_a): ((SignedBeaconBlock<E>, _), _) =
tester.harness.make_block(state_a.clone(), slot_b).await;
let ((block_b, blobs_b), state_after_b): ((SignedBeaconBlock<E>, _), _) =
tester.harness.make_block(state_a, slot_b).await;
@ -610,16 +613,18 @@ pub async fn equivocation_consensus_late_equivocation() {
assert_eq!(block_b.state_root(), state_after_b.tree_hash_root());
assert_ne!(block_a.state_root(), block_b.state_root());
let gossip_block_b = GossipVerifiedBlock::new(block_b.clone().into(), &tester.harness.chain);
assert!(gossip_block_b.is_ok());
let gossip_block_a = GossipVerifiedBlock::new(block_a.clone().into(), &tester.harness.chain);
assert!(gossip_block_a.is_err());
let gossip_block_contents_b = SignedBlockContents::new(block_b, blobs_b)
.into_gossip_verified_block(&tester.harness.chain);
assert!(gossip_block_contents_b.is_ok());
let gossip_block_contents_a = SignedBlockContents::new(block_a, blobs_a)
.into_gossip_verified_block(&tester.harness.chain);
assert!(gossip_block_contents_a.is_err());
let channel = tokio::sync::mpsc::unbounded_channel();
let publication_result: Result<(), Rejection> = publish_block(
None,
ProvenancedBlock::local((gossip_block_b.unwrap(), blobs_b)),
ProvenancedBlock::local(gossip_block_contents_b.unwrap()),
tester.harness.chain,
&channel.0,
test_logger,
@ -1224,11 +1229,15 @@ pub async fn blinded_equivocation_consensus_late_equivocation() {
ProvenancedBlock::Builder(b, _) => b,
};
let gossip_block_b =
GossipVerifiedBlock::new(inner_block_b.deconstruct().into(), &tester.harness.chain);
let gossip_block_b = GossipVerifiedBlock::new(
Arc::new(inner_block_b.clone().deconstruct().0),
&tester.harness.chain,
);
assert!(gossip_block_b.is_ok());
let gossip_block_a =
GossipVerifiedBlock::new(inner_block_a.deconstruct().into(), &tester.harness.chain);
let gossip_block_a = GossipVerifiedBlock::new(
Arc::new(inner_block_a.clone().deconstruct().0),
&tester.harness.chain,
);
assert!(gossip_block_a.is_err());
let channel = tokio::sync::mpsc::unbounded_channel();

View File

@ -1852,7 +1852,7 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
message_id,
peer_id,
peer_client,
block.into(),
block,
work_reprocessing_tx,
duplicate_cache,
invalid_block_storage,

View File

@ -1,6 +1,6 @@
use crate::{metrics, service::NetworkMessage, sync::SyncMessage};
use beacon_chain::blob_verification::{AsBlock, BlobError, BlockWrapper, GossipVerifiedBlob};
use beacon_chain::blob_verification::{AsBlock, BlobError, GossipVerifiedBlob};
use beacon_chain::store::Error;
use beacon_chain::{
attestation_verification::{self, Error as AttnError, VerifiedAttestation},
@ -20,6 +20,7 @@ use ssz::Encode;
use std::fs;
use std::io::Write;
use std::path::PathBuf;
use std::sync::Arc;
use std::time::{Duration, SystemTime, UNIX_EPOCH};
use store::hot_cold_store::HotColdDBError;
use tokio::sync::mpsc;
@ -754,13 +755,13 @@ impl<T: BeaconChainTypes> Worker<T> {
pub async fn process_gossip_verified_blob(
self,
peer_id: PeerId,
verified_blob: GossipVerifiedBlob<T::EthSpec>,
verified_blob: GossipVerifiedBlob<T>,
// This value is not used presently, but it might come in handy for debugging.
_seen_duration: Duration,
) {
let blob_root = verified_blob.block_root();
let blob_slot = verified_blob.slot();
let blob_clone = verified_blob.clone().to_blob();
let blob_index = verified_blob.id().index;
match self.chain.process_blob(verified_blob).await {
Ok(AvailabilityProcessingStatus::Imported(_hash)) => {
//TODO(sean) add metrics and logging
@ -778,7 +779,7 @@ impl<T: BeaconChainTypes> Worker<T> {
"outcome" => ?err,
"block root" => ?blob_root,
"block slot" => blob_slot,
"blob index" => blob_clone.index,
"blob index" => blob_index,
);
self.gossip_penalize_peer(
peer_id,
@ -788,7 +789,6 @@ impl<T: BeaconChainTypes> Worker<T> {
trace!(
self.log,
"Invalid gossip blob ssz";
"ssz" => format_args!("0x{}", hex::encode(blob_clone.as_ssz_bytes())),
);
}
}
@ -807,7 +807,7 @@ impl<T: BeaconChainTypes> Worker<T> {
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
block: BlockWrapper<T::EthSpec>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
duplicate_cache: DuplicateCache,
invalid_block_storage: InvalidBlockStorage,
@ -856,7 +856,7 @@ impl<T: BeaconChainTypes> Worker<T> {
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
block: BlockWrapper<T::EthSpec>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
seen_duration: Duration,
) -> Option<GossipVerifiedBlock<T>> {
@ -881,7 +881,7 @@ impl<T: BeaconChainTypes> Worker<T> {
let block_root = if let Ok(verified_block) = &verification_result {
verified_block.block_root
} else {
block.as_block().canonical_root()
block.canonical_root()
};
// Write the time the block was observed into delay cache.