block and blob handling progress

This commit is contained in:
realbigsean 2022-11-19 16:53:34 -05:00
parent 45897ad4e1
commit dc87156641
No known key found for this signature in database
GPG Key ID: B372B64D866BF8CC
11 changed files with 237 additions and 43 deletions

View File

@ -915,6 +915,28 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
Ok(self.get_block(block_root).await?.map(Arc::new))
}
pub async fn get_block_and_blobs_checking_early_attester_cache(
&self,
block_root: &Hash256,
) -> Result<
(
Option<Arc<SignedBeaconBlock<T::EthSpec>>>,
Option<Arc<BlobsSidecar<T::EthSpec>>>,
),
Error,
> {
if let (Some(block), Some(blobs)) = (
self.early_attester_cache.get_block(*block_root),
self.early_attester_cache.get_blobs(*block_root),
) {
return Ok((Some(block), Some(blobs)));
}
Ok((
self.get_block(block_root).await?.map(Arc::new),
self.get_blobs(block_root).await?.map(Arc::new),
))
}
/// Returns the block at the given root, if any.
///
/// ## Errors
@ -923,7 +945,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
pub async fn get_block(
&self,
block_root: &Hash256,
) -> Result<Option<Arc<SignedBeaconBlock<T::EthSpec>>>, Error> {
) -> Result<Option<SignedBeaconBlock<T::EthSpec>>, Error> {
// Load block from database, returning immediately if we have the full block w payload
// stored.
let blinded_block = match self.store.try_get_full_block(block_root)? {
@ -981,6 +1003,18 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.map(Some)
}
/// Returns the blobs at the given root, if any.
///
/// ## Errors
///
/// May return a database error.
pub async fn get_blobs(
&self,
block_root: &Hash256,
) -> Result<Option<BlobsSidecar<T::EthSpec>>, Error> {
Ok(self.store.get_blobs(block_root)?)
}
pub fn get_blinded_block(
&self,
block_root: &Hash256,
@ -2338,7 +2372,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
let last_index = filtered_chain_segment
.iter()
.position(|(_root, block)| {
block.block().slot().epoch(T::EthSpec::slots_per_epoch()) > start_epoch
block.slot().epoch(T::EthSpec::slots_per_epoch()) > start_epoch
})
.unwrap_or(filtered_chain_segment.len());
@ -2405,14 +2439,15 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
pub async fn verify_block_for_gossip(
self: &Arc<Self>,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
blobs: Option<Arc<BlobsSidecar<T::EthSpec>>>,
) -> Result<GossipVerifiedBlock<T>, BlockError<T::EthSpec>> {
let chain = self.clone();
self.task_executor
.clone()
.spawn_blocking_handle(
move || {
let slot = block.block().slot();
let graffiti_string = block.block().message().body().graffiti().as_utf8_lossy();
let slot = block.slot();
let graffiti_string = block.message().body().graffiti().as_utf8_lossy();
match GossipVerifiedBlock::new(block, &chain) {
Ok(verified) => {
@ -2490,7 +2525,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
self.log,
"Beacon block imported";
"block_root" => ?block_root,
"block_slot" => slot,
"block_slot" => %block.slot(),
);
// Increment the Prometheus counter for block processing successes.
@ -2540,6 +2575,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
) -> Result<Hash256, BlockError<T::EthSpec>> {
let ExecutionPendingBlock {
block,
blobs,
block_root,
state,
parent_block,
@ -2592,6 +2628,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
move || {
chain.import_block(
block,
blobs,
block_root,
state,
confirmed_state_roots,
@ -2616,7 +2653,8 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
#[allow(clippy::too_many_arguments)]
fn import_block(
&self,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
signed_block: Arc<SignedBeaconBlock<T::EthSpec>>,
blobs: Option<Arc<BlobsSidecar<T::EthSpec>>>,
block_root: Hash256,
mut state: BeaconState<T::EthSpec>,
confirmed_state_roots: Vec<Hash256>,
@ -2625,7 +2663,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
parent_block: SignedBlindedBeaconBlock<T::EthSpec>,
parent_eth1_finalization_data: Eth1FinalizationData,
) -> Result<Hash256, BlockError<T::EthSpec>> {
let signed_block = block.block();
let current_slot = self.slot()?;
let current_epoch = current_slot.epoch(T::EthSpec::slots_per_epoch());
@ -2867,6 +2904,7 @@ 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,
@ -2959,7 +2997,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
ops.push(StoreOp::PutBlock(block_root, signed_block.clone()));
ops.push(StoreOp::PutState(block.state_root(), &state));
if let Some(blobs) = block.blobs() {
if let Some(blobs) = blobs {
ops.push(StoreOp::PutBlobs(block_root, blobs));
};
let txn_lock = self.store.hot_db.begin_rw_transaction();

View File

@ -625,6 +625,7 @@ type PayloadVerificationHandle<E> =
/// `BeaconChain` immediately after it is instantiated.
pub struct ExecutionPendingBlock<T: BeaconChainTypes> {
pub block: Arc<SignedBeaconBlock<T::EthSpec>>,
pub blobs: Option<Box<BlobsSidecar<T::EthSpec>>>,
pub block_root: Hash256,
pub state: BeaconState<T::EthSpec>,
pub parent_block: SignedBeaconBlock<T::EthSpec, BlindedPayload<T::EthSpec>>,
@ -670,6 +671,7 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
/// Returns an error if the block is invalid, or if the block was unable to be verified.
pub fn new(
block: Arc<SignedBeaconBlock<T::EthSpec>>,
blobs: Option<Arc<BlobsSidecar<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
@ -677,7 +679,7 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
// 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, chain).map_err(|e| {
Self::new_without_slasher_checks(block, blobs, chain).map_err(|e| {
process_block_slash_info(chain, BlockSlashInfo::from_early_error(header, e))
})
}
@ -685,9 +687,9 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
/// 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>>>,
chain: &BeaconChain<T>,
) -> Result<Self, BlockError<T::EthSpec>> {
let block = block.block();
// Ensure the block is the correct structure for the fork at `block.slot()`.
block
.fork_name(&chain.spec)
@ -881,18 +883,20 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
// Validate the block's execution_payload (if any).
validate_execution_payload_for_gossip(&parent_block, block.message(), chain)?;
//FIXME(sean)
// if let Some(blobs_sidecar) = block.blobs() {
// validate_blob_for_gossip(blobs_sidecar, chain)?;
// }
if let Some(blobs_sidecar) = blobs.as_ref() {
validate_blob_for_gossip(blobs_sidecar, chain)?;
//FIXME(sean) validate blobs sidecar
}
// 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.message().proposer_index());
.set_proposer_index(block.message().proposer_index())
//FIXME(sean) set blobs sidecar validation results
.set_blobs_sidecar(blobs);
Ok(Self {
block: block,
block,
block_root,
parent,
consensus_context,
@ -1054,6 +1058,7 @@ impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for SignatureVerifiedBloc
ExecutionPendingBlock::from_signature_verified_components(
block,
self.consensus_context.blobs(),
block_root,
parent,
self.consensus_context,
@ -1084,10 +1089,7 @@ impl<T: BeaconChainTypes> IntoExecutionPendingBlock<T> for Arc<SignedBeaconBlock
}
fn block(&self) -> &SignedBeaconBlock<T::EthSpec> {
match self {
Self::Block(block) => block,
Self::BlockAndBlobs(block) => &block.beacon_block,
}
self
}
}
@ -1101,13 +1103,12 @@ 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>>,
blobs: Option<Box<BlobsSidecar<T::EthSpec>>>,
block_root: Hash256,
parent: PreProcessingSnapshot<T::EthSpec>,
mut consensus_context: ConsensusContext<T::EthSpec>,
chain: &Arc<BeaconChain<T>>,
) -> Result<Self, BlockError<T::EthSpec>> {
let block = block.block();
if let Some(parent) = chain
.canonical_head
.fork_choice_read_lock()
@ -1437,8 +1438,11 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
});
}
//FIXME(sean) validate blobs sidecar
Ok(Self {
block,
blobs,
block_root,
state,
parent_block: parent.beacon_block,
@ -1752,7 +1756,7 @@ fn load_parent<T: BeaconChainTypes>(
pre_state: parent_state,
beacon_state_root: Some(parent_state_root),
},
beacon_block,
block,
))
};

View File

@ -20,6 +20,7 @@ pub struct CacheItem<E: EthSpec> {
* Values used to make the block available.
*/
block: Arc<SignedBeaconBlock<E>>,
blobs: Option<Arc<BlobsSidecar<E>>>,
proto_block: ProtoBlock,
}
@ -50,6 +51,7 @@ impl<E: EthSpec> EarlyAttesterCache<E> {
&self,
beacon_block_root: Hash256,
block: Arc<SignedBeaconBlock<E>>,
blobs: Option<Arc<BlobsSidecar<E>>>,
proto_block: ProtoBlock,
state: &BeaconState<E>,
spec: &ChainSpec,
@ -74,6 +76,7 @@ impl<E: EthSpec> EarlyAttesterCache<E> {
source,
target,
block,
blobs,
proto_block,
};
@ -155,6 +158,16 @@ impl<E: EthSpec> EarlyAttesterCache<E> {
.map(|item| item.block.clone())
}
/// Returns the blobs, if `block_root` matches the cached item.
pub fn get_blobs(&self, block_root: Hash256) -> Option<Arc<BlobsSidecar<E>>> {
self.item
.read()
.as_ref()
.filter(|item| item.beacon_block_root == block_root)
.map(|item| item.blobs.clone())
.flatten()
}
/// Returns the proto-array block, if `block_root` matches the cached item.
pub fn get_proto_block(&self, block_root: Hash256) -> Option<ProtoBlock> {
self.item

View File

@ -142,6 +142,7 @@ async fn produces_attestations() {
.add_head_block(
block_root,
Arc::new(block.clone()),
None,
proto_block,
&state,
&chain.spec,
@ -198,6 +199,7 @@ async fn early_attester_cache_old_request() {
.add_head_block(
head.beacon_block_root,
head.beacon_block.clone(),
None,
head_proto_block,
&head.beacon_state,
&harness.chain.spec,

View File

@ -287,7 +287,7 @@ impl<T: EthSpec> std::fmt::Display for PubsubMessage<T> {
PubsubMessage::BeaconBlockAndBlobsSidecars(block_and_blob) => write!(
f,
"Beacon block and Blobs Sidecar: slot: {}, blobs: {}",
block_and_blob.beacon_block.message.slot,
block_and_blob.beacon_block.message().slot(),
block_and_blob.blobs_sidecar.blobs.len(),
),
PubsubMessage::AggregateAndProofAttestation(att) => write!(

View File

@ -1541,6 +1541,7 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
peer_id,
peer_client,
block,
None,
work_reprocessing_tx,
duplicate_cache,
seen_timestamp,
@ -1558,11 +1559,14 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
seen_timestamp,
} => task_spawner.spawn_async(async move {
worker
.process_gossip_block_and_blobs_sidecar(
.process_gossip_block(
message_id,
peer_id,
peer_client,
block_and_blobs,
block_and_blobs.beacon_block.clone(),
Some(block_and_blobs.blobs_sidecar.clone()),
work_reprocessing_tx,
duplicate_cache,
seen_timestamp,
)
.await
@ -1720,6 +1724,20 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
)
}),
Work::BlobsByRootsRequest {
peer_id,
request_id,
request,
} => task_spawner.spawn_blocking_with_manual_send_idle(move |send_idle_on_drop| {
worker.handle_blocks_by_root_request(
sub_executor,
send_idle_on_drop,
peer_id,
request_id,
request,
)
}),
Work::UnknownBlockAttestation {
message_id,
peer_id,

View File

@ -658,6 +658,7 @@ impl<T: BeaconChainTypes> Worker<T> {
peer_id: PeerId,
peer_client: Client,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
blobs: Option<Arc<BlobsSidecar<T::EthSpec>>>,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
duplicate_cache: DuplicateCache,
seen_duration: Duration,
@ -668,6 +669,7 @@ impl<T: BeaconChainTypes> Worker<T> {
peer_id,
peer_client,
block,
blobs,
reprocess_tx.clone(),
seen_duration,
)
@ -705,6 +707,7 @@ impl<T: BeaconChainTypes> Worker<T> {
peer_id: PeerId,
peer_client: Client,
block: Arc<SignedBeaconBlock<T::EthSpec>>,
blobs: Option<Arc<BlobsSidecar<T::EthSpec>>>,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
seen_duration: Duration,
) -> Option<GossipVerifiedBlock<T>> {
@ -719,7 +722,7 @@ impl<T: BeaconChainTypes> Worker<T> {
let verification_result = self
.chain
.clone()
.verify_block_for_gossip(block.clone())
.verify_block_for_gossip(block.clone(), blobs)
.await;
let block_root = if let Ok(verified_block) = &verification_result {

View File

@ -4,7 +4,9 @@ use crate::status::ToStatusMessage;
use crate::sync::SyncMessage;
use beacon_chain::{BeaconChainError, BeaconChainTypes, HistoricalBlockError, WhenSlotSkipped};
use itertools::process_results;
use lighthouse_network::rpc::methods::{BlobsByRangeRequest, MAX_REQUEST_BLOBS_SIDECARS};
use lighthouse_network::rpc::methods::{
BlobsByRangeRequest, BlobsByRootRequest, MAX_REQUEST_BLOBS_SIDECARS,
};
use lighthouse_network::rpc::StatusMessage;
use lighthouse_network::rpc::*;
use lighthouse_network::{PeerId, PeerRequestId, ReportSource, Response, SyncInfo};
@ -12,7 +14,7 @@ use slog::{debug, error};
use slot_clock::SlotClock;
use std::sync::Arc;
use task_executor::TaskExecutor;
use types::{Epoch, EthSpec, Hash256, Slot};
use types::{Epoch, EthSpec, Hash256, SignedBeaconBlockAndBlobsSidecar, Slot};
use super::Worker;
@ -204,6 +206,106 @@ impl<T: BeaconChainTypes> Worker<T> {
"load_blocks_by_root_blocks",
)
}
/// Handle a `BlobsByRoot` request from the peer.
pub fn handle_blobs_by_root_request(
self,
executor: TaskExecutor,
send_on_drop: SendOnDrop,
peer_id: PeerId,
request_id: PeerRequestId,
request: BlobsByRootRequest,
) {
// Fetching blocks is async because it may have to hit the execution layer for payloads.
executor.spawn(
async move {
let mut send_block_count = 0;
let mut send_response = true;
for root in request.block_roots.iter() {
match self
.chain
.get_block_and_blobs_checking_early_attester_cache(root)
.await
{
Ok((Some(block), Some(blobs))) => {
self.send_response(
peer_id,
Response::BlobsByRoot(Some(Arc::new(SignedBeaconBlockAndBlobsSidecar {
beacon_block: block,
blobs_sidecar: blobs,
}))),
request_id,
);
send_block_count += 1;
}
Ok((None, None)) => {
debug!(
self.log,
"Peer requested unknown block and blobs";
"peer" => %peer_id,
"request_root" => ?root
);
}
Ok((Some(_), None)) => {
debug!(
self.log,
"Peer requested block and blob, but no blob found";
"peer" => %peer_id,
"request_root" => ?root
);
}
Ok((None, Some(_))) => {
debug!(
self.log,
"Peer requested block and blob, but no block found";
"peer" => %peer_id,
"request_root" => ?root
);
}
Err(BeaconChainError::BlockHashMissingFromExecutionLayer(_)) => {
debug!(
self.log,
"Failed to fetch execution payload for block and blobs by root request";
"block_root" => ?root,
"reason" => "execution layer not synced",
);
// send the stream terminator
self.send_error_response(
peer_id,
RPCResponseErrorCode::ResourceUnavailable,
"Execution layer not synced".into(),
request_id,
);
send_response = false;
break;
}
Err(e) => {
debug!(
self.log,
"Error fetching block for peer";
"peer" => %peer_id,
"request_root" => ?root,
"error" => ?e,
);
}
}
}
debug!(
self.log,
"Received BlobsByRoot Request";
"peer" => %peer_id,
"requested" => request.block_roots.len(),
"returned" => %send_block_count
);
// send stream termination
if send_response {
self.send_response(peer_id, Response::BlocksByRoot(None), request_id);
}
drop(send_on_drop);
},
"load_blobs_by_root_blocks",
)
}
/// Handle a `BlocksByRange` request from the peer.
pub fn handle_blocks_by_range_request(

View File

@ -68,16 +68,6 @@ pub const SLOT_IMPORT_TOLERANCE: usize = 32;
pub type Id = u32;
pub struct SeansBlock {}
pub struct SeansBlob {}
/// This is the one that has them both and goes to range.
pub struct SeansBlockBlob {
block: SeansBlock,
blob: SeansBlob,
}
/// Id of rpc requests sent by sync to the network.
#[derive(Debug, Hash, PartialEq, Eq, Clone, Copy)]
pub enum RequestId {

View File

@ -1,4 +1,5 @@
use std::marker::PhantomData;
use std::sync::Arc;
use tree_hash::TreeHash;
use types::{
AbstractExecPayload, BeaconState, BeaconStateError, BlobsSidecar, ChainSpec, EthSpec,
@ -14,12 +15,11 @@ pub struct ConsensusContext<T: EthSpec> {
/// Block root of the block at `slot`.
current_block_root: Option<Hash256>,
/// Should only be populated if the sidecar has not been validated.
blobs_sidecar: Option<Box<BlobsSidecar<T>>>,
blobs_sidecar: Option<Arc<BlobsSidecar<T>>>,
/// Whether `validate_blobs_sidecar` has successfully passed.
blobs_sidecar_validated: bool,
/// Whether `verify_kzg_commitments_against_transactions` has successfully passed.
blobs_verified_vs_txs: bool,
_phantom: PhantomData<T>,
}
#[derive(Debug, PartialEq, Clone)]
@ -43,7 +43,6 @@ impl<T: EthSpec> ConsensusContext<T> {
blobs_sidecar: None,
blobs_sidecar_validated: false,
blobs_verified_vs_txs: false,
_phantom: PhantomData,
}
}
@ -116,4 +115,9 @@ impl<T: EthSpec> ConsensusContext<T> {
pub fn blobs_verified_vs_txs(&self) -> bool {
self.blobs_verified_vs_txs
}
pub fn set_blobs_sidecar(mut self, blobs_sidecar: Option<Arc<BlobsSidecar<T>>>) -> Self {
self.blobs_sidecar = blobs_sidecar;
self
}
}

View File

@ -7,7 +7,27 @@ use tree_hash_derive::TreeHash;
#[derive(Debug, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, PartialEq)]
#[serde(bound = "T: EthSpec")]
pub struct SignedBeaconBlockAndBlobsSidecar<T: EthSpec> {
pub struct SignedBeaconBlockAndBlobsSidecarDecode<T: EthSpec> {
pub beacon_block: SignedBeaconBlockEip4844<T>,
pub blobs_sidecar: BlobsSidecar<T>,
}
#[derive(Debug, Clone, Serialize, Deserialize, Encode, TreeHash, PartialEq)]
#[serde(bound = "T: EthSpec")]
pub struct SignedBeaconBlockAndBlobsSidecar<T: EthSpec> {
pub beacon_block: Arc<SignedBeaconBlock<T>>,
pub blobs_sidecar: Arc<BlobsSidecar<T>>,
}
impl<T: EthSpec> SignedBeaconBlockAndBlobsSidecar<T> {
pub fn from_ssz_bytes(bytes: &[u8]) -> Result<Self, DecodeError> {
let SignedBeaconBlockAndBlobsSidecarDecode {
beacon_block,
blobs_sidecar,
} = SignedBeaconBlockAndBlobsSidecarDecode::from_ssz_bytes(bytes)?;
Ok(SignedBeaconBlockAndBlobsSidecar {
beacon_block: Arc::new(SignedBeaconBlock::Eip4844(beacon_block)),
blobs_sidecar: Arc::new(blobs_sidecar),
})
}
}