This commit is contained in:
realbigsean 2022-10-03 21:48:02 -04:00
parent 88006735c4
commit e81dbbfea4
No known key found for this signature in database
GPG Key ID: B372B64D866BF8CC
46 changed files with 373 additions and 571 deletions

View File

@ -17,9 +17,9 @@ use crate::early_attester_cache::EarlyAttesterCache;
use crate::errors::{BeaconChainError as Error, BlockProductionError};
use crate::eth1_chain::{Eth1Chain, Eth1ChainBackend};
use crate::events::ServerSentEventHandler;
use crate::execution_payload::{ PreparePayloadHandle};
use crate::execution_payload::get_execution_payload;
use crate::execution_payload::PreparePayloadHandle;
use crate::fork_choice_signal::{ForkChoiceSignalRx, ForkChoiceSignalTx, ForkChoiceWaitResult};
use crate::execution_payload::{get_execution_payload, get_execution_payload_and_blobs};
use crate::head_tracker::HeadTracker;
use crate::historical_blocks::HistoricalBlockError;
use crate::migrate::BackgroundMigrator;
@ -377,8 +377,6 @@ pub struct BeaconChain<T: BeaconChainTypes> {
/// Sender given to tasks, so that if they encounter a state in which execution cannot
/// continue they can request that everything shuts down.
pub shutdown_sender: Sender<ShutdownReason>,
pub block_waiting_for_sidecar: Mutex<Option<GossipVerifiedBlock<T>>>,
pub sidecar_waiting_for_block: Mutex<Option<Arc<SignedBlobsSidecar<T::EthSpec>>>>,
/// Logging to CLI, etc.
pub(crate) log: Logger,
/// Arbitrary bytes included in the blocks.
@ -2440,7 +2438,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
self: &Arc<Self>,
block_root: Hash256,
unverified_block: B,
sidecar: Option<Arc<SignedBlobsSidecar<T::EthSpec>>>,
count_unrealized: CountUnrealized,
) -> Result<Hash256, BlockError<T::EthSpec>> {
// Start the Prometheus timer.
@ -2458,7 +2455,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
let execution_pending =
unverified_block.into_execution_pending_block(block_root, &chain)?;
chain
.import_execution_pending_block(execution_pending, sidecar, count_unrealized)
.import_execution_pending_block(execution_pending, count_unrealized)
.await
};
@ -2516,7 +2513,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
async fn import_execution_pending_block(
self: Arc<Self>,
execution_pending_block: ExecutionPendingBlock<T>,
sidecar: Option<Arc<SignedBlobsSidecar<T::EthSpec>>>,
count_unrealized: CountUnrealized,
) -> Result<Hash256, BlockError<T::EthSpec>> {
let ExecutionPendingBlock {
@ -2572,7 +2568,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
move || {
chain.import_block(
block,
sidecar,
block_root,
state,
confirmed_state_roots,
@ -2595,7 +2590,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
fn import_block(
&self,
signed_block: Arc<SignedBeaconBlock<T::EthSpec>>,
sidecar: Option<Arc<SignedBlobsSidecar<T::EthSpec>>>,
block_root: Hash256,
mut state: BeaconState<T::EthSpec>,
confirmed_state_roots: Vec<Hash256>,
@ -2934,9 +2928,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.collect();
ops.push(StoreOp::PutBlock(block_root, signed_block.clone()));
ops.push(StoreOp::PutState(block.state_root(), &state));
if let Some(sidecar) = sidecar {
ops.push(StoreOp::PutBlobs(block_root, sidecar));
}
let txn_lock = self.store.hot_db.begin_rw_transaction();
if let Err(e) = self.store.do_atomically(ops) {

View File

@ -134,7 +134,10 @@ pub enum BlockError<T: EthSpec> {
/// its parent.
ParentUnknown(Arc<SignedBeaconBlock<T>>),
/// The block skips too many slots and is a DoS risk.
TooManySkippedSlots { parent_slot: Slot, block_slot: Slot },
TooManySkippedSlots {
parent_slot: Slot,
block_slot: Slot,
},
/// The block slot is greater than the present slot.
///
/// ## Peer scoring
@ -150,7 +153,10 @@ pub enum BlockError<T: EthSpec> {
/// ## Peer scoring
///
/// The peer has incompatible state transition logic and is faulty.
StateRootMismatch { block: Hash256, local: Hash256 },
StateRootMismatch {
block: Hash256,
local: Hash256,
},
/// The block was a genesis block, these blocks cannot be re-imported.
GenesisBlock,
/// The slot is finalized, no need to import.
@ -169,7 +175,9 @@ pub enum BlockError<T: EthSpec> {
///
/// It's unclear if this block is valid, but it conflicts with finality and shouldn't be
/// imported.
NotFinalizedDescendant { block_parent_root: Hash256 },
NotFinalizedDescendant {
block_parent_root: Hash256,
},
/// Block is already known, no need to re-import.
///
/// ## Peer scoring
@ -182,7 +190,10 @@ pub enum BlockError<T: EthSpec> {
///
/// The `proposer` has already proposed a block at this slot. The existing block may or may not
/// be equal to the given block.
RepeatProposal { proposer: u64, slot: Slot },
RepeatProposal {
proposer: u64,
slot: Slot,
},
/// The block slot exceeds the MAXIMUM_BLOCK_SLOT_NUMBER.
///
/// ## Peer scoring
@ -197,7 +208,10 @@ pub enum BlockError<T: EthSpec> {
/// ## Peer scoring
///
/// The block is invalid and the peer is faulty.
IncorrectBlockProposer { block: u64, local_shuffling: u64 },
IncorrectBlockProposer {
block: u64,
local_shuffling: u64,
},
/// The proposal signature in invalid.
///
/// ## Peer scoring
@ -221,7 +235,10 @@ pub enum BlockError<T: EthSpec> {
/// ## Peer scoring
///
/// The block is invalid and the peer is faulty.
BlockIsNotLaterThanParent { block_slot: Slot, parent_slot: Slot },
BlockIsNotLaterThanParent {
block_slot: Slot,
parent_slot: Slot,
},
/// At least one block in the chain segment did not have it's parent root set to the root of
/// the prior block.
///
@ -277,8 +294,9 @@ pub enum BlockError<T: EthSpec> {
///
/// The peer sent us an invalid block, but I'm not really sure how to score this in an
/// "optimistic" sync world.
ParentExecutionPayloadInvalid { parent_root: Hash256 },
ParentExecutionPayloadInvalid {
parent_root: Hash256,
},
}
/// Returned when block validation failed due to some issue verifying

View File

@ -801,8 +801,6 @@ where
validator_pubkey_cache: TimeoutRwLock::new(validator_pubkey_cache),
attester_cache: <_>::default(),
early_attester_cache: <_>::default(),
block_waiting_for_sidecar: <_>::default(),
sidecar_waiting_for_block: <_>::default(),
shutdown_sender: self
.shutdown_sender
.ok_or("Cannot build without a shutdown sender.")?,

View File

@ -17,6 +17,7 @@ use fork_choice::{InvalidationOperation, PayloadVerificationStatus};
use proto_array::{Block as ProtoBlock, ExecutionStatus};
use slog::debug;
use slot_clock::SlotClock;
use ssz_types::VariableList;
use state_processing::per_block_processing::{
compute_timestamp_at_slot, is_execution_enabled, is_merge_transition_complete,
partially_verify_execution_payload,
@ -24,7 +25,10 @@ use state_processing::per_block_processing::{
use std::sync::Arc;
use tokio::task::JoinHandle;
use tree_hash::TreeHash;
use types::{*, execution_payload::BlobsBundle};
use types::{
BeaconBlockRef, BeaconState, BeaconStateError, EthSpec, ExecPayload, ExecutionBlockHash,
Hash256, KzgCommitment, SignedBeaconBlock, Slot,
};
pub type PreparePayloadResult<Payload> = Result<Payload, BlockProductionError>;
pub type PreparePayloadHandle<Payload> = JoinHandle<Option<PreparePayloadResult<Payload>>>;
@ -387,36 +391,6 @@ pub fn get_execution_payload<
Ok(join_handle)
}
/// Wraps the async `prepare_execution_payload` function as a blocking task.
pub fn prepare_execution_payload_and_blobs_blocking<
T: BeaconChainTypes,
Payload: ExecPayload<T::EthSpec>,
>(
chain: &BeaconChain<T>,
state: &BeaconState<T::EthSpec>,
proposer_index: u64,
) -> Result<
Option<(
Payload,
VariableList<
KzgCommitment,
<<T as BeaconChainTypes>::EthSpec as EthSpec>::MaxBlobsPerBlock,
>,
)>,
BlockProductionError,
> {
let execution_layer = chain
.execution_layer
.as_ref()
.ok_or(BlockProductionError::ExecutionLayerMissing)?;
execution_layer
.block_on_generic(|_| async {
prepare_execution_payload_and_blobs::<T, Payload>(chain, state, proposer_index).await
})
.map_err(BlockProductionError::BlockingFailed)?
}
/// Prepares an execution payload for inclusion in a block.
///
/// Will return `Ok(None)` if the merge fork has occurred, but a terminal block has not been found.
@ -513,7 +487,7 @@ where
.await
.map_err(BlockProductionError::GetPayloadFailed)?;
/*
/*
TODO: fetch blob bundles from el engine for block building
let suggested_fee_recipient = execution_layer.get_suggested_fee_recipient(proposer_index).await;
let blobs = execution_layer.get_blob_bundles(parent_hash, timestamp, random, suggested_fee_recipient)
@ -523,23 +497,3 @@ where
Ok(execution_payload)
}
pub async fn prepare_execution_payload_and_blobs<
T: BeaconChainTypes,
Payload: ExecPayload<T::EthSpec>,
>(
_chain: &BeaconChain<T>,
_state: &BeaconState<T::EthSpec>,
_proposer_index: u64,
) -> Result<
Option<(
Payload,
VariableList<
KzgCommitment,
<<T as BeaconChainTypes>::EthSpec as EthSpec>::MaxBlobsPerBlock,
>,
)>,
BlockProductionError,
> {
todo!()
}

View File

@ -6,11 +6,9 @@ use reqwest::StatusCode;
use serde::{Deserialize, Serialize};
use strum::IntoStaticStr;
pub use types::{
Address, EthSpec, ExecutionBlockHash, ExecutionPayload, ExecutionPayloadHeader, FixedVector,
Hash256, Uint256, VariableList, kzg_proof::KzgProof, kzg_commitment::KzgCommitment, blob::Blob,
blob::Blob, Address, EthSpec, ExecutionBlockHash, ExecutionPayload, ExecutionPayloadHeader,
FixedVector, Hash256, KzgCommitment, KzgProof, Uint256, VariableList,
};
use types::{KzgCommitment};
pub mod auth;
pub mod http;

View File

@ -10,7 +10,7 @@ use serde::de::DeserializeOwned;
use serde_json::json;
use std::time::Duration;
use types::{EthSpec, FullPayload, execution_payload::BlobsBundle};
use types::{EthSpec, FullPayload};
pub use deposit_log::{DepositLog, Log};
pub use reqwest::Client;
@ -671,14 +671,18 @@ impl HttpJsonRpc {
pub async fn get_blobs_bundle_v1<T: EthSpec>(
&self,
payload_id: PayloadId,
) -> Result<BlobsBundle<T>, Error> {
) -> Result<JsonBlobBundlesV1<T>, Error> {
let params = json!([JsonPayloadIdRequest::from(payload_id)]);
let response: JsonBlobBundlesV1<T> = self
.rpc_request(ENGINE_GET_BLOBS_BUNDLE_V1, params, ENGINE_GET_BLOBS_BUNDLE_TIMEOUT)
.rpc_request(
ENGINE_GET_BLOBS_BUNDLE_V1,
params,
ENGINE_GET_BLOBS_BUNDLE_TIMEOUT,
)
.await?;
Ok(response.into())
Ok(response)
}
pub async fn forkchoice_updated_v1(

View File

@ -1,6 +1,6 @@
use super::*;
use serde::{Deserialize, Serialize};
use types::{EthSpec, ExecutionBlockHash, FixedVector, Transaction, Unsigned, VariableList, execution_payload::BlobsBundle};
use types::{EthSpec, ExecutionBlockHash, FixedVector, Transaction, Unsigned, VariableList};
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
@ -278,45 +278,6 @@ pub struct JsonBlobBundlesV1<T: EthSpec> {
pub aggregated_proof: KzgProof,
}
impl <T: EthSpec> From<BlobsBundle<T>> for JsonBlobBundlesV1<T> {
fn from(p: BlobsBundle<T>) -> Self {
// Use this verbose deconstruction pattern to ensure no field is left unused.
let BlobsBundle {
block_hash,
aggregated_proof,
blobs,
kzgs,
} = p;
Self {
block_hash,
aggregated_proof,
blobs,
kzgs,
}
}
}
impl <T: EthSpec> From<JsonBlobBundlesV1<T>> for BlobsBundle<T> {
fn from(j: JsonBlobBundlesV1<T>) -> Self {
// Use this verbose deconstruction pattern to ensure no field is left unused.
let JsonBlobBundlesV1 {
block_hash,
aggregated_proof,
blobs,
kzgs,
} = j;
Self {
block_hash,
aggregated_proof,
blobs,
kzgs,
}
}
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub struct JsonForkChoiceStateV1 {

View File

@ -4,6 +4,7 @@
//! This crate only provides useful functionality for "The Merge", it does not provide any of the
//! deposit-contract functionality that the `beacon_node/eth1` crate already provides.
use crate::json_structures::JsonBlobBundlesV1;
use crate::payload_cache::PayloadCache;
use auth::{strip_prefix, Auth, JwtKey};
use builder_client::BuilderHttpClient;
@ -787,10 +788,10 @@ impl<T: EthSpec> ExecutionLayer<T> {
timestamp: u64,
prev_randao: Hash256,
suggested_fee_recipient: Address,
) -> Result<BlobsBundle<T>, Error> {
) -> Result<JsonBlobBundlesV1<T>, Error> {
debug!(
self.log(),
"Issuing engine_getPayload";
"Issuing engine_getBlobsBundle";
"suggested_fee_recipient" => ?suggested_fee_recipient,
"prev_randao" => ?prev_randao,
"timestamp" => timestamp,
@ -808,22 +809,15 @@ impl<T: EthSpec> ExecutionLayer<T> {
&[metrics::HIT],
);
id
} else {
} else {
error!(
self.log(),
"Exec engine unable to produce blobs, did you call get_payload before?",
);
return Err(ApiError::PayloadIdUnavailable);
return Err(ApiError::PayloadIdUnavailable);
};
engine
.api
.get_blobs_bundle_v1::<T>(payload_id)
.await
.map(|bundle| {
// TODO verify the blob bundle here?
bundle.into()
})
engine.api.get_blobs_bundle_v1::<T>(payload_id).await
})
.await
.map_err(Box::new)
@ -937,18 +931,6 @@ impl<T: EthSpec> ExecutionLayer<T> {
.map_err(Error::EngineError)
}
pub async fn get_blob<T: EthSpec>(
&self,
_parent_hash: Hash256,
_timestamp: u64,
_random: Hash256,
_finalized_block_hash: Hash256,
_proposer_index: u64,
_versioned_hash: Hash256,
) -> Result<BlobDetailsV1, Error> {
todo!()
}
/// Maps to the `engine_newPayload` JSON-RPC call.
///
/// ## Fallback Behaviour

View File

@ -4,7 +4,7 @@ use lru::LruCache;
use slog::{debug, warn, Logger};
use state_processing::BlockReplayer;
use std::sync::Arc;
use types::BlindedBeaconBlock;
use types::beacon_block::BlindedBeaconBlock;
use warp_utils::reject::{
beacon_chain_error, beacon_state_error, custom_bad_request, custom_server_error,
};

View File

@ -21,9 +21,6 @@ const GOSSIP_MAX_SIZE: usize = 1_048_576; // 1M
/// The maximum transmit size of gossip messages in bytes post-merge.
const GOSSIP_MAX_SIZE_POST_MERGE: usize = 10 * 1_048_576; // 10M
const MAX_REQUEST_BLOBS_SIDECARS: usize = 128;
const MIN_EPOCHS_FOR_BLOBS_SIDECARS_REQUESTS: usize = 128;
/// The cache time is set to accommodate the circulation time of an attestation.
///
/// The p2p spec declares that we accept attestations within the following range:

View File

@ -516,7 +516,6 @@ impl<TSpec: EthSpec> PeerManager<TSpec> {
match protocol {
Protocol::Ping => PeerAction::Fatal,
Protocol::BlocksByRange => return,
Protocol::TxBlobsByRange => return,
Protocol::BlocksByRoot => return,
Protocol::BlobsByRange => return,
Protocol::Goodbye => return,
@ -533,7 +532,6 @@ impl<TSpec: EthSpec> PeerManager<TSpec> {
ConnectionDirection::Outgoing => match protocol {
Protocol::Ping => PeerAction::LowToleranceError,
Protocol::BlocksByRange => PeerAction::MidToleranceError,
Protocol::TxBlobsByRange => PeerAction::MidToleranceError,
Protocol::BlocksByRoot => PeerAction::MidToleranceError,
Protocol::BlobsByRange => PeerAction::MidToleranceError,
Protocol::Goodbye => return,

View File

@ -16,8 +16,8 @@ use std::marker::PhantomData;
use std::sync::Arc;
use tokio_util::codec::{Decoder, Encoder};
use types::{
EthSpec, ForkContext, ForkName, SignedBeaconBlock, SignedBeaconBlockAltair,
SignedBeaconBlockBase, SignedBeaconBlockMerge, SignedBeaconBlockEip4844
BlobsSidecar, EthSpec, ForkContext, ForkName, SignedBeaconBlock, SignedBeaconBlockAltair,
SignedBeaconBlockBase, SignedBeaconBlockEip4844, SignedBeaconBlockMerge,
};
use unsigned_varint::codec::Uvi;
@ -550,9 +550,7 @@ fn handle_v1_response<T: EthSpec>(
Protocol::BlocksByRoot => Ok(Some(RPCResponse::BlocksByRoot(Arc::new(
SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?),
)))),
Protocol::BlobsByRange => Err(RPCError::InvalidData(
"blobs by range via v1".to_string(),
)),
Protocol::BlobsByRange => Err(RPCError::InvalidData("blobs by range via v1".to_string())),
Protocol::Ping => Ok(Some(RPCResponse::Pong(Ping {
data: u64::from_ssz_bytes(decoded_buffer)?,
}))),
@ -627,15 +625,15 @@ fn handle_v2_response<T: EthSpec>(
)?),
)))),
},
Protocol::BlobsByRange => match fork_name {
Protocol::BlobsByRange => match fork_name {
ForkName::Eip4844 => Ok(Some(RPCResponse::BlobsByRange(Arc::new(
VariableList::from_ssz_bytes(decoded_buffer)?,
BlobsSidecar::from_ssz_bytes(decoded_buffer)?,
)))),
_ => Err(RPCError::ErrorResponse(
RPCResponseErrorCode::InvalidRequest,
"Invalid forkname for blobsbyrange".to_string(),
)),
}
},
_ => Err(RPCError::ErrorResponse(
RPCResponseErrorCode::InvalidRequest,
"Invalid v2 request".to_string(),

View File

@ -12,8 +12,8 @@ use std::ops::Deref;
use std::sync::Arc;
use strum::IntoStaticStr;
use superstruct::superstruct;
use types::{Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot};
use types::blobs_sidecar::BlobsSidecar;
use types::{Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot};
/// Maximum number of blocks in a single request.
pub type MaxRequestBlocks = U1024;
@ -23,6 +23,9 @@ pub const MAX_REQUEST_BLOCKS: u64 = 1024;
pub type MaxErrorLen = U256;
pub const MAX_ERROR_LEN: u64 = 256;
pub type MaxRequestBlobsSidecars = U1024;
pub const MAX_REQUEST_BLOBS_SIDECARS: u64 = 1024;
/// Wrapper over SSZ List to represent error message in rpc responses.
#[derive(Debug, Clone)]
pub struct ErrorType(pub VariableList<u8, MaxErrorLen>);
@ -232,12 +235,6 @@ pub struct OldBlocksByRangeRequest {
pub step: u64,
}
#[derive(Encode, Decode, Clone, Debug, PartialEq)]
pub struct TxBlobsByRangeRequest {
pub execution_block_number: u64,
pub count: u64,
}
/// Request a number of beacon block bodies from a peer.
#[derive(Clone, Debug, PartialEq)]
pub struct BlocksByRootRequest {
@ -257,13 +254,11 @@ pub enum RPCResponse<T: EthSpec> {
/// batch.
BlocksByRange(Arc<SignedBeaconBlock<T>>),
TxBlobsByRange(Box<BlobsSidecar<T>>),
/// A response to a get BLOCKS_BY_ROOT request.
BlocksByRoot(Arc<SignedBeaconBlock<T>>),
/// A response to a get BLOBS_BY_RANGE request
BlobsByRange(Arc<VariableList<BlobsSidecar<T>, T::MaxRequestBlobsSidecars>>),
BlobsByRange(Arc<BlobsSidecar<T>>),
/// A PONG response to a PING request.
Pong(Ping),
@ -278,13 +273,11 @@ pub enum ResponseTermination {
/// Blocks by range stream termination.
BlocksByRange,
TxBlobsByRange,
/// Blocks by root stream termination.
BlocksByRoot,
// Blobs by range stream termination.
BlobsByRange
/// Blobs by range stream termination.
BlobsByRange,
}
/// The structured response containing a result/code indicating success or failure
@ -345,7 +338,6 @@ impl<T: EthSpec> RPCCodedResponse<T> {
RPCCodedResponse::Success(resp) => match resp {
RPCResponse::Status(_) => false,
RPCResponse::BlocksByRange(_) => true,
RPCResponse::TxBlobsByRange(_) => true,
RPCResponse::BlocksByRoot(_) => true,
RPCResponse::BlobsByRange(_) => true,
RPCResponse::Pong(_) => false,
@ -415,14 +407,11 @@ impl<T: EthSpec> std::fmt::Display for RPCResponse<T> {
RPCResponse::BlocksByRange(block) => {
write!(f, "BlocksByRange: Block slot: {}", block.slot())
}
RPCResponse::TxBlobsByRange(blob) => {
write!(f, "TxBlobsByRange: Block slot: {}", blob.beacon_block_slot)
}
RPCResponse::BlocksByRoot(block) => {
write!(f, "BlocksByRoot: Block slot: {}", block.slot())
}
RPCResponse::BlobsByRange(blob) => {
write!(f, "BlobsByRange: Blob slot: {}", blob.len())
write!(f, "BlobsByRange: Blob slot: {}", blob.beacon_block_slot)
}
RPCResponse::Pong(ping) => write!(f, "Pong: {}", ping.data),
RPCResponse::MetaData(metadata) => write!(f, "Metadata: {}", metadata.seq_number()),
@ -472,13 +461,9 @@ impl std::fmt::Display for OldBlocksByRangeRequest {
}
}
impl std::fmt::Display for TxBlobsByRangeRequest {
impl std::fmt::Display for BlobsByRangeRequest {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(
f,
"Execution block number: {}, Count: {}",
self.execution_block_number, self.count
)
write!(f, "Start Slot: {}, Count: {}", self.start_slot, self.count)
}
}

View File

@ -24,6 +24,7 @@ pub(crate) use handler::HandlerErr;
pub(crate) use methods::{MetaData, MetaDataV1, MetaDataV2, Ping, RPCCodedResponse, RPCResponse};
pub(crate) use protocol::{InboundRequest, RPCProtocol};
use crate::rpc::methods::MAX_REQUEST_BLOBS_SIDECARS;
pub use handler::SubstreamId;
pub use methods::{
BlocksByRangeRequest, BlocksByRootRequest, GoodbyeReason, MaxRequestBlocks,
@ -125,14 +126,12 @@ impl<Id: ReqId, TSpec: EthSpec> RPC<Id, TSpec> {
methods::MAX_REQUEST_BLOCKS,
Duration::from_secs(10),
)
//FIXME(sean)
.n_every(Protocol::BlocksByRoot, 128, Duration::from_secs(10))
.n_every(
Protocol::TxBlobsByRange,
methods::MAX_REQUEST_BLOCKS,
Protocol::BlobsByRange,
MAX_REQUEST_BLOBS_SIDECARS,
Duration::from_secs(10),
)
.n_every(Protocol::BlocksByRoot, 128, Duration::from_secs(10))
.n_every(Protocol::BlobsByRange, 128, Duration::from_secs(10))
.build()
.expect("Configuration parameters are valid");
RPC {

View File

@ -37,7 +37,6 @@ pub enum OutboundRequest<TSpec: EthSpec> {
Status(StatusMessage),
Goodbye(GoodbyeReason),
BlocksByRange(OldBlocksByRangeRequest),
TxBlobsByRange(TxBlobsByRangeRequest),
BlocksByRoot(BlocksByRootRequest),
BlobsByRange(BlobsByRangeRequest),
Ping(Ping),
@ -73,19 +72,15 @@ impl<TSpec: EthSpec> OutboundRequest<TSpec> {
ProtocolId::new(Protocol::BlocksByRange, Version::V2, Encoding::SSZSnappy),
ProtocolId::new(Protocol::BlocksByRange, Version::V1, Encoding::SSZSnappy),
],
//FIXME(sean) what should the protocol version be?
OutboundRequest::TxBlobsByRange(_) => vec![ProtocolId::new(
Protocol::TxBlobsByRange,
Version::V2,
Encoding::SSZSnappy,
)],
OutboundRequest::BlocksByRoot(_) => vec![
ProtocolId::new(Protocol::BlocksByRoot, Version::V2, Encoding::SSZSnappy),
ProtocolId::new(Protocol::BlocksByRoot, Version::V1, Encoding::SSZSnappy),
],
OutboundRequest::BlobsByRange(_) => vec![
ProtocolId::new(Protocol::BlocksByRoot, Version::V1, Encoding::SSZSnappy),
],
OutboundRequest::BlobsByRange(_) => vec![ProtocolId::new(
Protocol::BlobsByRange,
Version::V1,
Encoding::SSZSnappy,
)],
OutboundRequest::Ping(_) => vec![ProtocolId::new(
Protocol::Ping,
Version::V1,
@ -106,7 +101,6 @@ impl<TSpec: EthSpec> OutboundRequest<TSpec> {
OutboundRequest::Status(_) => 1,
OutboundRequest::Goodbye(_) => 0,
OutboundRequest::BlocksByRange(req) => req.count,
OutboundRequest::TxBlobsByRange(req) => req.count,
OutboundRequest::BlocksByRoot(req) => req.block_roots.len() as u64,
OutboundRequest::BlobsByRange(req) => req.count,
OutboundRequest::Ping(_) => 1,
@ -120,7 +114,6 @@ impl<TSpec: EthSpec> OutboundRequest<TSpec> {
OutboundRequest::Status(_) => Protocol::Status,
OutboundRequest::Goodbye(_) => Protocol::Goodbye,
OutboundRequest::BlocksByRange(_) => Protocol::BlocksByRange,
OutboundRequest::TxBlobsByRange(_) => Protocol::TxBlobsByRange,
OutboundRequest::BlocksByRoot(_) => Protocol::BlocksByRoot,
OutboundRequest::BlobsByRange(_) => Protocol::BlobsByRange,
OutboundRequest::Ping(_) => Protocol::Ping,
@ -135,7 +128,6 @@ impl<TSpec: EthSpec> OutboundRequest<TSpec> {
// this only gets called after `multiple_responses()` returns true. Therefore, only
// variants that have `multiple_responses()` can have values.
OutboundRequest::BlocksByRange(_) => ResponseTermination::BlocksByRange,
OutboundRequest::TxBlobsByRange(_) => ResponseTermination::TxBlobsByRange,
OutboundRequest::BlocksByRoot(_) => ResponseTermination::BlocksByRoot,
OutboundRequest::BlobsByRange(_) => ResponseTermination::BlobsByRange,
OutboundRequest::Status(_) => unreachable!(),
@ -192,7 +184,6 @@ impl<TSpec: EthSpec> std::fmt::Display for OutboundRequest<TSpec> {
OutboundRequest::Status(status) => write!(f, "Status Message: {}", status),
OutboundRequest::Goodbye(reason) => write!(f, "Goodbye: {}", reason),
OutboundRequest::BlocksByRange(req) => write!(f, "Blocks by range: {}", req),
OutboundRequest::TxBlobsByRange(req) => write!(f, "Blobs by range: {}", req),
OutboundRequest::BlocksByRoot(req) => write!(f, "Blocks by root: {:?}", req),
OutboundRequest::BlobsByRange(req) => write!(f, "Blobs by range: {:?}", req),
OutboundRequest::Ping(ping) => write!(f, "Ping: {}", ping.data),

View File

@ -71,11 +71,7 @@ lazy_static! {
+ types::ExecutionPayload::<MainnetEthSpec>::max_execution_payload_size() // adding max size of execution payload (~16gb)
+ ssz::BYTES_PER_LENGTH_OFFSET; // Adding the additional ssz offset for the `ExecutionPayload` field
pub static ref BLOB_MIN: usize = BlobsSidecar::<MainnetEthSpec>::empty()
.as_ssz_bytes()
.len();
pub static ref BLOB_MAX: usize = BlobsSidecar::<MainnetEthSpec>::max_size();
pub static ref SIGNED_BEACON_BLOCK_EIP4844_MAX: usize = *SIGNED_BEACON_BLOCK_MERGE_MAX + (48 * <MainnetEthSpec>::max_blobs_per_block());
pub static ref BLOCKS_BY_ROOT_REQUEST_MIN: usize =
VariableList::<Hash256, MaxRequestBlocks>::from(Vec::<Hash256>::new())
@ -108,7 +104,7 @@ lazy_static! {
pub(crate) const MAX_RPC_SIZE: usize = 1_048_576; // 1M
/// The maximum bytes that can be sent across the RPC post-merge.
pub(crate) const MAX_RPC_SIZE_POST_MERGE: usize = 10 * 1_048_576; // 10M
//TODO(sean) check
//TODO(sean) check
pub(crate) const MAX_RPC_SIZE_POST_EIP4844: usize = 20 * 1_048_576; // 10M
/// The protocol prefix the RPC protocol id.
const PROTOCOL_PREFIX: &str = "/eth2/beacon_chain/req";
@ -160,7 +156,6 @@ pub enum Protocol {
Goodbye,
/// The `BlocksByRange` protocol name.
BlocksByRange,
TxBlobsByRange,
/// The `BlocksByRoot` protocol name.
BlocksByRoot,
/// The `BlobsByRange` protocol name.
@ -302,9 +297,10 @@ impl ProtocolId {
Protocol::BlocksByRoot => {
RpcLimits::new(*BLOCKS_BY_ROOT_REQUEST_MIN, *BLOCKS_BY_ROOT_REQUEST_MAX)
}
Protocol::BlobsByRange => {
RpcLimits::new(*BLOCKS_BY_ROOT_REQUEST_MIN, *BLOCKS_BY_ROOT_REQUEST_MAX)
}
Protocol::BlobsByRange => RpcLimits::new(
<BlobsByRangeRequest as Encode>::ssz_fixed_len(),
<BlobsByRangeRequest as Encode>::ssz_fixed_len(),
),
Protocol::Ping => RpcLimits::new(
<Ping as Encode>::ssz_fixed_len(),
<Ping as Encode>::ssz_fixed_len(),
@ -478,9 +474,11 @@ impl<TSpec: EthSpec> InboundRequest<TSpec> {
ProtocolId::new(Protocol::BlocksByRoot, Version::V2, Encoding::SSZSnappy),
ProtocolId::new(Protocol::BlocksByRoot, Version::V1, Encoding::SSZSnappy),
],
InboundRequest::BlobsByRange(_) => vec![
ProtocolId::new(Protocol::BlocksByRoot, Version::V1, Encoding::SSZSnappy),
],
InboundRequest::BlobsByRange(_) => vec![ProtocolId::new(
Protocol::BlocksByRoot,
Version::V1,
Encoding::SSZSnappy,
)],
InboundRequest::Ping(_) => vec![ProtocolId::new(
Protocol::Ping,
Version::V1,
@ -501,7 +499,6 @@ impl<TSpec: EthSpec> InboundRequest<TSpec> {
InboundRequest::Status(_) => 1,
InboundRequest::Goodbye(_) => 0,
InboundRequest::BlocksByRange(req) => req.count,
InboundRequest::TxBlobsByRange(req) => req.count,
InboundRequest::BlocksByRoot(req) => req.block_roots.len() as u64,
InboundRequest::BlobsByRange(req) => req.count,
InboundRequest::Ping(_) => 1,
@ -515,7 +512,6 @@ impl<TSpec: EthSpec> InboundRequest<TSpec> {
InboundRequest::Status(_) => Protocol::Status,
InboundRequest::Goodbye(_) => Protocol::Goodbye,
InboundRequest::BlocksByRange(_) => Protocol::BlocksByRange,
InboundRequest::TxBlobsByRange(_) => Protocol::TxBlobsByRange,
InboundRequest::BlocksByRoot(_) => Protocol::BlocksByRoot,
InboundRequest::BlobsByRange(_) => Protocol::BlobsByRange,
InboundRequest::Ping(_) => Protocol::Ping,
@ -530,7 +526,6 @@ impl<TSpec: EthSpec> InboundRequest<TSpec> {
// this only gets called after `multiple_responses()` returns true. Therefore, only
// variants that have `multiple_responses()` can have values.
InboundRequest::BlocksByRange(_) => ResponseTermination::BlocksByRange,
InboundRequest::TxBlobsByRange(_) => ResponseTermination::TxBlobsByRange,
InboundRequest::BlocksByRoot(_) => ResponseTermination::BlocksByRoot,
InboundRequest::BlobsByRange(_) => ResponseTermination::BlobsByRange,
InboundRequest::Status(_) => unreachable!(),

View File

@ -161,7 +161,9 @@ impl RPCRateLimiterBuilder {
.bbrange_quota
.ok_or("BlocksByRange quota not specified")?;
let blbrange_quota = self.blbrange_quota.ok_or("BlobsByRange quota not specified")?;
let blbrange_quota = self
.blbrange_quota
.ok_or("BlobsByRange quota not specified")?;
// create the rate limiters
let ping_rl = Limiter::from_quota(ping_quota)?;

View File

@ -1,8 +1,9 @@
use std::sync::Arc;
use libp2p::core::connection::ConnectionId;
use types::{EthSpec, SignedBeaconBlock};
use types::{BlobsSidecar, EthSpec, SignedBeaconBlock};
use crate::rpc::methods::BlobsByRangeRequest;
use crate::rpc::{
methods::{
BlocksByRangeRequest, BlocksByRootRequest, OldBlocksByRangeRequest, RPCCodedResponse,
@ -32,6 +33,8 @@ pub enum Request {
Status(StatusMessage),
/// A blocks by range request.
BlocksByRange(BlocksByRangeRequest),
/// A bloibs by range request.
BlobsByRange(BlobsByRangeRequest),
/// A request blocks root request.
BlocksByRoot(BlocksByRootRequest),
}
@ -47,6 +50,7 @@ impl<TSpec: EthSpec> std::convert::From<Request> for OutboundRequest<TSpec> {
step: 1,
})
}
Request::BlobsByRange(r) => OutboundRequest::BlobsByRange(r),
Request::Status(s) => OutboundRequest::Status(s),
}
}
@ -64,6 +68,8 @@ pub enum Response<TSpec: EthSpec> {
Status(StatusMessage),
/// A response to a get BLOCKS_BY_RANGE request. A None response signals the end of the batch.
BlocksByRange(Option<Arc<SignedBeaconBlock<TSpec>>>),
/// A response to a get BLOBS_BY_RANGE request. A None response signals the end of the batch.
BlobsByRange(Option<Arc<BlobsSidecar<TSpec>>>),
/// A response to a get BLOCKS_BY_ROOT request.
BlocksByRoot(Option<Arc<SignedBeaconBlock<TSpec>>>),
}
@ -79,6 +85,10 @@ impl<TSpec: EthSpec> std::convert::From<Response<TSpec>> for RPCCodedResponse<TS
Some(b) => RPCCodedResponse::Success(RPCResponse::BlocksByRange(b)),
None => RPCCodedResponse::StreamTermination(ResponseTermination::BlocksByRange),
},
Response::BlobsByRange(r) => match r {
Some(b) => RPCCodedResponse::Success(RPCResponse::BlobsByRange(b)),
None => RPCCodedResponse::StreamTermination(ResponseTermination::BlobsByRange),
},
Response::Status(s) => RPCCodedResponse::Success(RPCResponse::Status(s)),
}
}

View File

@ -1,3 +1,5 @@
use self::behaviour::Behaviour;
use self::gossip_cache::GossipCache;
use crate::config::{gossipsub_config, NetworkLoad};
use crate::discovery::{
subnet_predicate, DiscoveredPeers, Discovery, FIND_NODE_QUERY_CLOSEST_PEERS,
@ -7,11 +9,10 @@ use crate::peer_manager::{
ConnectionDirection, PeerManager, PeerManagerEvent,
};
use crate::peer_manager::{MIN_OUTBOUND_ONLY_FACTOR, PEER_EXCESS_FACTOR, PRIORITY_PEER_EXCESS};
use crate::rpc::methods::BlobsByRangeRequest;
use crate::rpc::*;
use crate::service::behaviour::BehaviourEvent;
pub use crate::service::behaviour::Gossipsub;
use crate::rpc::*;
use crate::rpc::methods::BlobsByRangeRequest;
use crate::service::{Context as ServiceContext, METADATA_FILENAME};
use crate::types::{
subnet_from_topic_hash, GossipEncoding, GossipKind, GossipTopic, SnappyTransform, Subnet,
SubnetDiscovery,
@ -29,13 +30,17 @@ use libp2p::gossipsub::subscription_filter::MaxCountSubscriptionFilter;
use libp2p::gossipsub::{
GossipsubEvent, IdentTopic as Topic, MessageAcceptance, MessageAuthenticity, MessageId,
};
use slog::{crit, debug, o, trace, warn};
use libp2p::identify::{Identify, IdentifyConfig, IdentifyEvent};
use libp2p::multiaddr::{Multiaddr, Protocol as MProtocol};
use libp2p::swarm::{ConnectionLimits, Swarm, SwarmBuilder, SwarmEvent};
use libp2p::PeerId;
use slog::{crit, debug, info, o, trace, warn};
use ssz::Encode;
use types::blobs_sidecar::BlobsSidecar;
use std::collections::HashSet;
use std::fs::File;
use std::io::Write;
use std::path::{Path, PathBuf};
use std::pin::Pin;
use std::{
collections::VecDeque,
marker::PhantomData,
@ -44,13 +49,9 @@ use std::{
};
use types::{
consts::altair::SYNC_COMMITTEE_SUBNET_COUNT, EnrForkId, EthSpec, ForkContext,
SignedBeaconBlock, Slot, SubnetId, SyncSubnetId, VariableList
SignedBeaconBlock, Slot, SubnetId, SyncSubnetId, VariableList,
};
use crate::rpc::methods::TxBlobsByRangeRequest;
use utils::{build_transport, strip_peer_id, MAX_CONNECTIONS_PER_PEER};
use self::behaviour::Behaviour;
use self::gossip_cache::GossipCache;
use utils::{build_transport, strip_peer_id, Context as ServiceContext, MAX_CONNECTIONS_PER_PEER};
pub mod api_types;
mod behaviour;
@ -988,9 +989,6 @@ impl<AppReqId: ReqId, TSpec: EthSpec> Network<AppReqId, TSpec> {
Request::BlocksByRange { .. } => {
metrics::inc_counter_vec(&metrics::TOTAL_RPC_REQUESTS, &["blocks_by_range"])
}
Request::TxBlobsByRange { .. } => {
metrics::inc_counter_vec(&metrics::TOTAL_RPC_REQUESTS, &["tx_blobs_by_range"])
}
Request::BlocksByRoot { .. } => {
metrics::inc_counter_vec(&metrics::TOTAL_RPC_REQUESTS, &["blocks_by_root"])
}
@ -1261,7 +1259,12 @@ impl<AppReqId: ReqId, TSpec: EthSpec> Network<AppReqId, TSpec> {
Some(event)
}
InboundRequest::BlobsByRange(req) => {
self.propagate_request(peer_request_id, peer_id, Request::BlobsByRange(req))
let event = self.build_request(
peer_request_id,
peer_id,
Request::BlobsByRange(req),
);
Some(event)
}
}
}
@ -1287,21 +1290,17 @@ impl<AppReqId: ReqId, TSpec: EthSpec> Network<AppReqId, TSpec> {
RPCResponse::BlocksByRange(resp) => {
self.build_response(id, peer_id, Response::BlocksByRange(Some(resp)))
}
RPCResponse::TxBlobsByRange(resp) => {
self.propagate_response(id, peer_id, Response::TxBlobsByRange(Some(resp)))
RPCResponse::BlobsByRange(resp) => {
self.build_response(id, peer_id, Response::BlobsByRange(Some(resp)))
}
RPCResponse::BlocksByRoot(resp) => {
self.build_response(id, peer_id, Response::BlocksByRoot(Some(resp)))
}
RPCResponse::BlobsByRange(resp) => {
self.propagate_response(id, peer_id, Response::BlobsByRange(Some(resp)))
}
}
}
Ok(RPCReceived::EndOfStream(id, termination)) => {
let response = match termination {
ResponseTermination::BlocksByRange => Response::BlocksByRange(None),
ResponseTermination::TxBlobsByRange => Response::TxBlobsByRange(None),
ResponseTermination::BlocksByRoot => Response::BlocksByRoot(None),
ResponseTermination::BlobsByRange => Response::BlobsByRange(None),
};

View File

@ -8,14 +8,13 @@ use ssz::{Decode, Encode};
use std::boxed::Box;
use std::io::{Error, ErrorKind};
use std::sync::Arc;
use types::signed_blobs_sidecar::SignedBlobsSidecar;
use types::{
Attestation, AttesterSlashing, BlobsSidecar, EthSpec, ForkContext, ForkName, ProposerSlashing,
SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase,
SignedBeaconBlockMerge, SignedBeaconBlockEip4844, SignedContributionAndProof, SignedVoluntaryExit, SubnetId,
SyncCommitteeMessage, SyncSubnetId,
SignedBeaconBlockEip4844, SignedBeaconBlockMerge, SignedContributionAndProof,
SignedVoluntaryExit, SubnetId, SyncCommitteeMessage, SyncSubnetId,
};
use types::blobs_sidecar::BlobsSidecar;
use types::signed_blobs_sidecar::SignedBlobsSidecar;
#[derive(Debug, Clone, PartialEq)]
pub enum PubsubMessage<T: EthSpec> {

View File

@ -570,21 +570,6 @@ impl<T: BeaconChainTypes> WorkEvent<T> {
}
}
pub fn tx_blob_by_range_request(
peer_id: PeerId,
request_id: PeerRequestId,
request: TxBlobsByRangeRequest,
) -> Self {
Self {
drop_during_sync: false,
work: Work::TxBlobsByRangeRequest {
peer_id,
request_id,
request,
},
}
}
/// Create a new work event to process `BlocksByRootRequest`s from the RPC network.
pub fn blocks_by_roots_request(
peer_id: PeerId,
@ -741,13 +726,6 @@ pub enum Work<T: BeaconChainTypes> {
blobs: Arc<SignedBlobsSidecar<T::EthSpec>>,
seen_timestamp: Duration,
},
GossipBlob {
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
blob: Box<BlobsSidecar<T::EthSpec>>,
seen_timestamp: Duration,
},
DelayedImportBlock {
peer_id: PeerId,
block: Box<GossipVerifiedBlock<T>>,
@ -801,11 +779,6 @@ pub enum Work<T: BeaconChainTypes> {
request_id: PeerRequestId,
request: BlocksByRangeRequest,
},
TxBlobsByRangeRequest {
peer_id: PeerId,
request_id: PeerRequestId,
request: TxBlobsByRangeRequest,
},
BlocksByRootsRequest {
peer_id: PeerId,
request_id: PeerRequestId,
@ -838,7 +811,6 @@ impl<T: BeaconChainTypes> Work<T> {
Work::ChainSegment { .. } => CHAIN_SEGMENT,
Work::Status { .. } => STATUS_PROCESSING,
Work::BlocksByRangeRequest { .. } => BLOCKS_BY_RANGE_REQUEST,
Work::TxBlobsByRangeRequest { .. } => TX_BLOBS_BY_RANGE_REQUEST,
Work::BlocksByRootsRequest { .. } => BLOCKS_BY_ROOTS_REQUEST,
Work::BlobsByRangeRequest {..} => BLOBS_BY_RANGE_REQUEST,
Work::UnknownBlockAttestation { .. } => UNKNOWN_BLOCK_ATTESTATION,
@ -1090,7 +1062,7 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
} else if let Some(item) = gossip_block_queue.pop() {
self.spawn_worker(item, toolbox);
//FIXME(sean)
} else if let Some(item) = gossip_blob_queue.pop() {
} else if let Some(item) = gossip_blobs_sidecar_queue.pop() {
self.spawn_worker(item, toolbox);
// Check the aggregates, *then* the unaggregates since we assume that
// aggregates are more valuable to local validators and effectively give us
@ -1331,9 +1303,6 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
Work::BlocksByRangeRequest { .. } => {
bbrange_queue.push(work, work_id, &self.log)
}
Work::TxBlobsByRangeRequest { .. } => {
txbbrange_queue.push(work, work_id, &self.log)
}
Work::BlocksByRootsRequest { .. } => {
bbroots_queue.push(work, work_id, &self.log)
}
@ -1571,7 +1540,7 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
seen_timestamp,
} => task_spawner.spawn_async(async move {
worker
.process_gossip_blobs_sidecar(
.process_gossip_blob(
message_id,
peer_id,
peer_client,

View File

@ -695,12 +695,12 @@ impl<T: BeaconChainTypes> Worker<T> {
}
#[allow(clippy::too_many_arguments)]
pub fn process_gossip_blob(
pub async fn process_gossip_blob(
self,
_message_id: MessageId,
_peer_id: PeerId,
_peer_client: Client,
_blob: BlobsSidecar<T::EthSpec>,
_blob: Arc<SignedBlobsSidecar<T::EthSpec>>,
_reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
_duplicate_cache: DuplicateCache,
_seen_duration: Duration,
@ -951,22 +951,6 @@ impl<T: BeaconChainTypes> Worker<T> {
let block: Arc<_> = verified_block.block.clone();
let block_root = verified_block.block_root;
let sidecar = if verified_block.block.message()
.body().blob_kzg_commitments().map(|committments| committments.is_empty()).unwrap_or(true) {
None
} else if let Some(sidecar) = self.chain.sidecar_waiting_for_block.lock().as_ref() {
if sidecar.message.beacon_block_root == verified_block.block_root() {
Some(sidecar.clone())
} else {
*self.chain.block_waiting_for_sidecar.lock() = Some(verified_block);
return
}
} else {
*self.chain.block_waiting_for_sidecar.lock() = Some(verified_block);
// we need the sidecar but dont have it yet
return
};
match self
.chain
.process_block(block_root, verified_block, CountUnrealized::True)
@ -1012,7 +996,7 @@ impl<T: BeaconChainTypes> Worker<T> {
"Failed to verify execution payload";
"error" => %e
);
}
},
other => {
debug!(
self.log,
@ -1034,10 +1018,6 @@ impl<T: BeaconChainTypes> Worker<T> {
}
};
}
} else {
*self.chain.sidecar_waiting_for_block.lock() = Some(blobs);
}
}
pub fn process_gossip_voluntary_exit(
self,

View File

@ -6,10 +6,9 @@ use beacon_chain::{BeaconChainError, BeaconChainTypes, HistoricalBlockError, Whe
use itertools::process_results;
use lighthouse_network::rpc::StatusMessage;
use lighthouse_network::rpc::*;
use lighthouse_network::rpc::methods::BlobsByRangeRequest;
use lighthouse_network::rpc::methods::{BlobsByRangeRequest, MAX_REQUEST_BLOBS_SIDECARS};
use lighthouse_network::{PeerId, PeerRequestId, ReportSource, Response, SyncInfo};
use slog::{debug, error};
use lighthouse_network::rpc::methods::TxBlobsByRangeRequest;
use slot_clock::SlotClock;
use std::sync::Arc;
use task_executor::TaskExecutor;
@ -124,15 +123,6 @@ impl<T: BeaconChainTypes> Worker<T> {
}
}
pub fn handle_tx_blobs_by_range_request(
&self,
_peer_id: PeerId,
_request_id: PeerRequestId,
_req: TxBlobsByRangeRequest,
) {
//FIXME(sean)
}
/// Handle a `BlocksByRoot` request from the peer.
pub fn handle_blocks_by_root_request(
self,
@ -400,133 +390,135 @@ impl<T: BeaconChainTypes> Worker<T> {
);
// Should not send more than max request blocks
if req.count > MAX_REQUEST_BLOCKS {
req.count = MAX_REQUEST_BLOCKS;
if req.count > MAX_REQUEST_BLOBS_SIDECARS {
req.count = MAX_REQUEST_BLOBS_SIDECARS;
}
let forwards_block_root_iter = match self
.chain
.forwards_iter_block_roots(Slot::from(req.start_slot))
{
Ok(iter) => iter,
Err(BeaconChainError::HistoricalBlockError(
HistoricalBlockError::BlockOutOfRange {
slot,
oldest_block_slot,
},
)) => {
debug!(self.log, "Range request failed during backfill"; "requested_slot" => slot, "oldest_known_slot" => oldest_block_slot);
return self.send_error_response(
peer_id,
RPCResponseErrorCode::ResourceUnavailable,
"Backfilling".into(),
request_id,
);
}
Err(e) => return error!(self.log, "Unable to obtain root iter"; "error" => ?e),
};
//FIXME(sean) create the blobs iter
// Pick out the required blocks, ignoring skip-slots.
let mut last_block_root = None;
let maybe_block_roots = process_results(forwards_block_root_iter, |iter| {
iter.take_while(|(_, slot)| slot.as_u64() < req.start_slot.saturating_add(req.count))
// map skip slots to None
.map(|(root, _)| {
let result = if Some(root) == last_block_root {
None
} else {
Some(root)
};
last_block_root = Some(root);
result
})
.collect::<Vec<Option<Hash256>>>()
});
let block_roots = match maybe_block_roots {
Ok(block_roots) => block_roots,
Err(e) => return error!(self.log, "Error during iteration over blocks"; "error" => ?e),
};
// remove all skip slots
let block_roots = block_roots.into_iter().flatten().collect::<Vec<_>>();
// Fetching blocks is async because it may have to hit the execution layer for payloads.
executor.spawn(
async move {
let mut blocks_sent = 0;
let mut send_response = true;
for root in block_roots {
match self.chain.store.get_blobs(&root) {
Ok(Some(blob)) => {
blocks_sent += 1;
self.send_network_message(NetworkMessage::SendResponse {
peer_id,
response: Response::BlobsByRange(Some(Arc::new(VariableList::new(vec![blob.message]).unwrap()))),
id: request_id,
});
}
Ok(None) => {
error!(
self.log,
"Blob in the chain is not in the store";
"request_root" => ?root
);
break;
}
Err(e) => {
error!(
self.log,
"Error fetching block for peer";
"block_root" => ?root,
"error" => ?e
);
break;
}
}
}
let current_slot = self
.chain
.slot()
.unwrap_or_else(|_| self.chain.slot_clock.genesis_slot());
if blocks_sent < (req.count as usize) {
debug!(
self.log,
"BlocksByRange Response processed";
"peer" => %peer_id,
"msg" => "Failed to return all requested blocks",
"start_slot" => req.start_slot,
"current_slot" => current_slot,
"requested" => req.count,
"returned" => blocks_sent
);
} else {
debug!(
self.log,
"BlocksByRange Response processed";
"peer" => %peer_id,
"start_slot" => req.start_slot,
"current_slot" => current_slot,
"requested" => req.count,
"returned" => blocks_sent
);
}
if send_response {
// send the stream terminator
self.send_network_message(NetworkMessage::SendResponse {
peer_id,
response: Response::BlobsByRange(None),
id: request_id,
});
}
drop(send_on_drop);
},
"load_blocks_by_range_blocks",
);
// let forwards_block_root_iter = match self
// .chain
// .forwards_iter_block_roots(Slot::from(req.start_slot))
// {
// Ok(iter) => iter,
// Err(BeaconChainError::HistoricalBlockError(
// HistoricalBlockError::BlockOutOfRange {
// slot,
// oldest_block_slot,
// },
// )) => {
// debug!(self.log, "Range request failed during backfill"; "requested_slot" => slot, "oldest_known_slot" => oldest_block_slot);
// return self.send_error_response(
// peer_id,
// RPCResponseErrorCode::ResourceUnavailable,
// "Backfilling".into(),
// request_id,
// );
// }
// Err(e) => return error!(self.log, "Unable to obtain root iter"; "error" => ?e),
// };
//
// // Pick out the required blocks, ignoring skip-slots.
// let mut last_block_root = None;
// let maybe_block_roots = process_results(forwards_block_root_iter, |iter| {
// iter.take_while(|(_, slot)| slot.as_u64() < req.start_slot.saturating_add(req.count))
// // map skip slots to None
// .map(|(root, _)| {
// let result = if Some(root) == last_block_root {
// None
// } else {
// Some(root)
// };
// last_block_root = Some(root);
// result
// })
// .collect::<Vec<Option<Hash256>>>()
// });
//
// let block_roots = match maybe_block_roots {
// Ok(block_roots) => block_roots,
// Err(e) => return error!(self.log, "Error during iteration over blocks"; "error" => ?e),
// };
//
// // remove all skip slots
// let block_roots = block_roots.into_iter().flatten().collect::<Vec<_>>();
//
// // Fetching blocks is async because it may have to hit the execution layer for payloads.
// executor.spawn(
// async move {
// let mut blocks_sent = 0;
// let mut send_response = true;
//
// for root in block_roots {
// match self.chain.store.get_blobs(&root) {
// Ok(Some(blob)) => {
// blocks_sent += 1;
// self.send_network_message(NetworkMessage::SendResponse {
// peer_id,
// response: Response::BlobsByRange(Some(Arc::new(VariableList::new(vec![blob.message]).unwrap()))),
// id: request_id,
// });
// }
// Ok(None) => {
// error!(
// self.log,
// "Blob in the chain is not in the store";
// "request_root" => ?root
// );
// break;
// }
// Err(e) => {
// error!(
// self.log,
// "Error fetching block for peer";
// "block_root" => ?root,
// "error" => ?e
// );
// break;
// }
// }
// }
//
// let current_slot = self
// .chain
// .slot()
// .unwrap_or_else(|_| self.chain.slot_clock.genesis_slot());
//
// if blocks_sent < (req.count as usize) {
// debug!(
// self.log,
// "BlocksByRange Response processed";
// "peer" => %peer_id,
// "msg" => "Failed to return all requested blocks",
// "start_slot" => req.start_slot,
// "current_slot" => current_slot,
// "requested" => req.count,
// "returned" => blocks_sent
// );
// } else {
// debug!(
// self.log,
// "BlocksByRange Response processed";
// "peer" => %peer_id,
// "start_slot" => req.start_slot,
// "current_slot" => current_slot,
// "requested" => req.count,
// "returned" => blocks_sent
// );
// }
//
// if send_response {
// // send the stream terminator
// self.send_network_message(NetworkMessage::SendResponse {
// peer_id,
// response: Response::BlobsByRange(None),
// id: request_id,
// });
// }
//
// drop(send_on_drop);
// },
// "load_blocks_by_range_blocks",
// );
}
}

View File

@ -6,7 +6,6 @@ use crate::status::status_message;
use crate::sync::manager::RequestId as SyncId;
use crate::sync::SyncMessage;
use beacon_chain::{BeaconChain, BeaconChainError, BeaconChainTypes};
use lighthouse_network::rpc::methods::TxBlobsByRangeRequest;
use lighthouse_network::rpc::*;
use lighthouse_network::rpc::methods::BlobsByRangeRequest;
use lighthouse_network::{
@ -218,40 +217,29 @@ impl<T: BeaconChainTypes> Processor<T> {
});
}
/// Handle a `BlocksByRange` request from the peer.
pub fn on_tx_blobs_by_range_request(
&mut self,
peer_id: PeerId,
request_id: PeerRequestId,
req: TxBlobsByRangeRequest,
) {
self.send_beacon_processor_work(BeaconWorkEvent::tx_blob_by_range_request(
peer_id, request_id, req,
))
}
pub fn on_tx_blobs_by_range_response(
pub fn on_blobs_by_range_response(
&mut self,
peer_id: PeerId,
request_id: RequestId,
blob_wrapper: Option<Box<BlobsSidecar<T::EthSpec>>>,
blob_wrapper: Option<Arc<BlobsSidecar<T::EthSpec>>>,
) {
trace!(
self.log,
"Received TxBlobsByRange Response";
"Received BlobsByRange Response";
"peer" => %peer_id,
);
if let RequestId::Sync(id) = request_id {
self.send_to_sync(SyncMessage::TxBlobsByRangeResponse {
self.send_to_sync(SyncMessage::RpcBlob {
peer_id,
request_id: id,
blob_wrapper,
blob_sidecar: blob_wrapper,
seen_timestamp: timestamp_now(),
});
} else {
debug!(
self.log,
"All tx blobs by range responses should belong to sync"
"All blobs by range responses should belong to sync"
);
}
}
@ -286,15 +274,6 @@ impl<T: BeaconChainTypes> Processor<T> {
});
}
pub fn on_blobs_by_range_response(
&mut self,
peer_id: PeerId,
request_id: RequestId,
beacon_blob: Option<Arc<VariableList<BlobsSidecar<T::EthSpec>, <<T as BeaconChainTypes>::EthSpec as EthSpec>::MaxRequestBlobsSidecars>>>,
) {
}
/// Process a gossip message declaring a new block.
///
/// Attempts to apply to block to the beacon chain. May queue the block for later processing.

View File

@ -89,21 +89,15 @@ pub enum SyncMessage<T: EthSpec> {
RpcBlock {
request_id: RequestId,
peer_id: PeerId,
beacon_block: Option<Box<SignedBeaconBlock<T>>>,
beacon_block: Option<Arc<SignedBeaconBlock<T>>>,
seen_timestamp: Duration,
},
/// A [`TxBlobsByRangeResponse`] response has been received.
TxBlobsByRangeResponse {
/// A blob has been received from RPC.
RpcBlob {
peer_id: PeerId,
request_id: RequestId,
blob_wrapper: Option<Box<BlobsSidecar<T>>>,
},
/// A [`BlocksByRoot`] response has been received.
BlocksByRootResponse {
peer_id: PeerId,
beacon_block: Option<Arc<SignedBeaconBlock<T>>>,
blob_sidecar: Option<Arc<BlobsSidecar<T>>>,
seen_timestamp: Duration,
},
@ -598,6 +592,8 @@ impl<T: BeaconChainTypes> SyncManager<T> {
.block_lookups
.parent_chain_processed(chain_hash, result, &mut self.network),
},
//FIXME(sean)
SyncMessage::RpcBlob { .. } => todo!()
}
}

View File

@ -38,8 +38,8 @@ use std::marker::PhantomData;
use std::path::Path;
use std::sync::Arc;
use std::time::Duration;
use types::*;
use types::signed_blobs_sidecar::SignedBlobsSidecar;
use types::*;
/// On-disk database that stores finalized states efficiently.
///
@ -480,22 +480,28 @@ impl<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> HotColdDB<E, Hot, Cold>
.key_delete(DBColumn::ExecPayload.into(), block_root.as_bytes())
}
pub fn put_blobs(&self,
block_root: &Hash256,
blobs: SignedBlobsSidecar<E>,
pub fn put_blobs(
&self,
block_root: &Hash256,
blobs: SignedBlobsSidecar<E>,
) -> Result<(), Error> {
self.hot_db.put_bytes(DBColumn::BeaconBlob.into(), block_root.as_bytes(), &blobs.as_ssz_bytes())?;
self.hot_db.put_bytes(
DBColumn::BeaconBlob.into(),
block_root.as_bytes(),
&blobs.as_ssz_bytes(),
)?;
self.blob_cache.lock().push(*block_root, blobs);
Ok(())
}
pub fn get_blobs(&self,
block_root: &Hash256,
) -> Result<Option<SignedBlobsSidecar<E>>, Error> {
pub fn get_blobs(&self, block_root: &Hash256) -> Result<Option<SignedBlobsSidecar<E>>, Error> {
if let Some(blobs) = self.blob_cache.lock().get(block_root) {
Ok(Some(blobs.clone()))
} else {
if let Some(bytes) = self.hot_db.get_bytes(DBColumn::BeaconBlob.into(), block_root.as_bytes())? {
if let Some(bytes) = self
.hot_db
.get_bytes(DBColumn::BeaconBlob.into(), block_root.as_bytes())?
{
let ret = SignedBlobsSidecar::from_ssz_bytes(&bytes)?;
self.blob_cache.lock().put(*block_root, ret.clone());
Ok(Some(ret))
@ -512,10 +518,7 @@ impl<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> HotColdDB<E, Hot, Cold>
ops: &mut Vec<KeyValueStoreOp>,
) {
let db_key = get_key_for_col(DBColumn::BeaconBlob.into(), key.as_bytes());
ops.push(KeyValueStoreOp::PutKeyValue(
db_key,
blobs.as_ssz_bytes(),
));
ops.push(KeyValueStoreOp::PutKeyValue(db_key, blobs.as_ssz_bytes()));
}
pub fn put_state_summary(
@ -746,11 +749,7 @@ impl<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> HotColdDB<E, Hot, Cold>
}
StoreOp::PutBlobs(block_root, blobs) => {
self.blobs_as_kv_store_ops(
&block_root,
&blobs,
&mut key_value_batch,
);
self.blobs_as_kv_store_ops(&block_root, &blobs, &mut key_value_batch);
}
StoreOp::PutStateSummary(state_root, summary) => {

View File

@ -41,8 +41,8 @@ pub use metrics::scrape_for_metrics;
use parking_lot::MutexGuard;
use std::sync::Arc;
use strum::{EnumString, IntoStaticStr};
pub use types::*;
use types::signed_blobs_sidecar::SignedBlobsSidecar;
pub use types::*;
pub type ColumnIter<'a> = Box<dyn Iterator<Item = Result<(Hash256, Vec<u8>), Error>> + 'a>;
pub type ColumnKeyIter<'a> = Box<dyn Iterator<Item = Result<Hash256, Error>> + 'a>;

View File

@ -308,9 +308,5 @@ define_hardcoded_nets!(
// directory.
GENESIS_STATE_IS_KNOWN
),
(
eip4844,
"eip4844",
GENESIS_STATE_IS_KNOWN
)
(eip4844, "eip4844", GENESIS_STATE_IS_KNOWN)
);

View File

@ -777,7 +777,9 @@ where
(parent_justified, parent_finalized)
} else {
let justification_and_finalization_state = match block {
BeaconBlockRef::Eip4844(_) | BeaconBlockRef::Merge(_) | BeaconBlockRef::Altair(_) => {
BeaconBlockRef::Eip4844(_)
| BeaconBlockRef::Merge(_)
| BeaconBlockRef::Altair(_) => {
let participation_cache =
per_epoch_processing::altair::ParticipationCache::new(state, spec)
.map_err(Error::ParticipationCacheBuild)?;

View File

@ -1,6 +1,6 @@
use crate::beacon_block_body::{
BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyMerge, BeaconBlockBodyRef,
BeaconBlockBodyRefMut, BeaconBlockBodyEip4844
BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyEip4844, BeaconBlockBodyMerge,
BeaconBlockBodyRef, BeaconBlockBodyRefMut,
};
use crate::test_utils::TestRandom;
use crate::*;

View File

@ -1,3 +1,4 @@
use crate::kzg_commitment::KzgCommitment;
use crate::test_utils::TestRandom;
use crate::*;
use derivative::Derivative;
@ -8,7 +9,6 @@ use std::marker::PhantomData;
use superstruct::superstruct;
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
use crate::kzg_commitment::KzgCommitment;
/// The body of a `BeaconChain` block, containing operations.
///
@ -256,10 +256,10 @@ impl<E: EthSpec> From<BeaconBlockBodyMerge<E, FullPayload<E>>>
}
impl<E: EthSpec> From<BeaconBlockBodyEip4844<E, FullPayload<E>>>
for (
BeaconBlockBodyEip4844<E, BlindedPayload<E>>,
Option<ExecutionPayload<E>>,
)
for (
BeaconBlockBodyEip4844<E, BlindedPayload<E>>,
Option<ExecutionPayload<E>>,
)
{
fn from(body: BeaconBlockBodyEip4844<E, FullPayload<E>>) -> Self {
let BeaconBlockBodyEip4844 {
@ -272,7 +272,7 @@ for (
deposits,
voluntary_exits,
sync_aggregate,
execution_payload: FullPayload { execution_payload},
execution_payload: FullPayload { execution_payload },
blob_kzg_commitments,
} = body;

View File

@ -1,22 +1,27 @@
use ssz_types::VariableList;
use crate::bls_field_element::BlsFieldElement;
use crate::test_utils::RngCore;
use crate::test_utils::TestRandom;
use crate::{EthSpec, Uint256};
use serde::{Deserialize, Deserializer, Serialize, Serializer};
use ssz::{Decode, DecodeError, Encode};
use ssz_types::VariableList;
use tree_hash::{PackedEncoding, TreeHash};
use crate::test_utils::RngCore;
use crate::bls_field_element::BlsFieldElement;
use crate::{EthSpec, Uint256};
use crate::test_utils::TestRandom;
#[derive(Default, Debug, PartialEq, Hash, Clone, Serialize, Deserialize)]
#[serde(transparent)]
pub struct Blob<T: EthSpec>(pub VariableList<BlsFieldElement, T::FieldElementsPerBlob>);
impl <T: EthSpec> TestRandom for Blob<T> {
impl<T: EthSpec> TestRandom for Blob<T> {
fn random_for_test(rng: &mut impl RngCore) -> Self {
let mut res = Blob(VariableList::empty());
for i in 0..4096 {
let slice = ethereum_types::U256([rng.next_u64(), rng.next_u64(), rng.next_u64(), rng.next_u64()]);
let elem =BlsFieldElement(slice);
let slice = ethereum_types::U256([
rng.next_u64(),
rng.next_u64(),
rng.next_u64(),
rng.next_u64(),
]);
let elem = BlsFieldElement(slice);
res.0.push(elem);
}
res
@ -60,7 +65,7 @@ impl<T: EthSpec> TreeHash for Blob<T> {
<VariableList<BlsFieldElement, T::FieldElementsPerBlob>>::tree_hash_type()
}
fn tree_hash_packed_encoding(&self) -> PackedEncoding{
fn tree_hash_packed_encoding(&self) -> PackedEncoding {
self.0.tree_hash_packed_encoding()
}

View File

@ -1,3 +1,4 @@
use crate::kzg_proof::KzgProof;
use crate::{Blob, EthSpec, Hash256, Slot};
use serde_derive::{Deserialize, Serialize};
use ssz::Encode;
@ -5,7 +6,6 @@ use ssz_derive::{Decode, Encode};
use ssz_types::VariableList;
use tree_hash::TreeHash;
use tree_hash_derive::TreeHash;
use crate::kzg_proof::KzgProof;
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Debug, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, PartialEq, Default)]
@ -24,6 +24,6 @@ impl<E: EthSpec> BlobsSidecar<E> {
// Fixed part
Self::empty().as_ssz_bytes().len()
// Max size of variable length `blobs` field
+ (E::max_object_list_size() * <Blob<E::FieldElementsPerBlob> as Encode>::ssz_fixed_len())
+ (E::max_blobs_per_block() * <Blob<E::FieldElementsPerBlob> as Encode>::ssz_fixed_len())
}
}

View File

@ -7,7 +7,6 @@ use tree_hash::{PackedEncoding, TreeHash};
#[serde(transparent)]
pub struct BlsFieldElement(pub Uint256);
impl Encode for BlsFieldElement {
fn is_ssz_fixed_len() -> bool {
<Uint256 as Encode>::is_ssz_fixed_len()
@ -45,7 +44,7 @@ impl TreeHash for BlsFieldElement {
<Uint256>::tree_hash_type()
}
fn tree_hash_packed_encoding(&self) -> PackedEncoding{
fn tree_hash_packed_encoding(&self) -> PackedEncoding {
self.0.tree_hash_packed_encoding()
}

View File

@ -246,9 +246,8 @@ impl ChainSpec {
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Altair,
_ => ForkName::Base,
},
}
},
}
}
/// Returns the fork version for a named fork.

View File

@ -228,7 +228,7 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq +
Self::BytesPerLogsBloom::to_usize()
}
fn max_object_list_size() -> usize {
fn max_blobs_per_block() -> usize {
Self::MaxBlobsPerBlock::to_usize()
}

View File

@ -1,4 +1,7 @@
use crate::{test_utils::TestRandom, test_utils::RngCore, *, kzg_commitment::KzgCommitment, kzg_proof::KzgProof, blob::Blob};
use crate::{
blob::Blob, kzg_commitment::KzgCommitment, kzg_proof::KzgProof, test_utils::RngCore,
test_utils::TestRandom, *,
};
use derivative::Derivative;
use serde_derive::{Deserialize, Serialize};
use ssz::Encode;

View File

@ -50,10 +50,7 @@ impl ForkContext {
if spec.eip4844_fork_epoch.is_some() {
fork_to_digest.push((
ForkName::Eip4844,
ChainSpec::compute_fork_digest(
spec.eip4844_fork_version,
genesis_validators_root,
),
ChainSpec::compute_fork_digest(spec.eip4844_fork_version, genesis_validators_root),
));
}

View File

@ -11,12 +11,17 @@ pub enum ForkName {
Base,
Altair,
Merge,
Eip4844
Eip4844,
}
impl ForkName {
pub fn list_all() -> Vec<ForkName> {
vec![ForkName::Base, ForkName::Altair, ForkName::Merge, ForkName::Eip4844]
vec![
ForkName::Base,
ForkName::Altair,
ForkName::Merge,
ForkName::Eip4844,
]
}
/// Set the activation slots in the given `ChainSpec` so that the fork named by `self`

View File

@ -14,7 +14,7 @@ impl TreeHash for KzgCommitment {
<[u8; 48] as TreeHash>::tree_hash_type()
}
fn tree_hash_packed_encoding(&self) -> PackedEncoding{
fn tree_hash_packed_encoding(&self) -> PackedEncoding {
self.0.tree_hash_packed_encoding()
}

View File

@ -1,8 +1,8 @@
use std::fmt;
use crate::test_utils::{RngCore, TestRandom};
use serde::{Deserialize, Deserializer, Serialize, Serializer};
use ssz::{Decode, DecodeError, Encode};
use std::fmt;
use tree_hash::{PackedEncoding, TreeHash};
use crate::test_utils::{RngCore, TestRandom};
const KZG_PROOF_BYTES_LEN: usize = 48;
@ -35,19 +35,19 @@ impl Into<[u8; KZG_PROOF_BYTES_LEN]> for KzgProof {
}
pub mod serde_kzg_proof {
use serde::de::Error;
use super::*;
use serde::de::Error;
pub fn serialize<S>(bytes: &[u8; KZG_PROOF_BYTES_LEN], serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
where
S: Serializer,
{
serializer.serialize_str(&eth2_serde_utils::hex::encode(bytes))
}
pub fn deserialize<'de, D>(deserializer: D) -> Result<[u8; KZG_PROOF_BYTES_LEN], D::Error>
where
D: Deserializer<'de>,
where
D: Deserializer<'de>,
{
let s: String = Deserialize::deserialize(deserializer)?;

View File

@ -90,12 +90,12 @@ pub mod slot_data;
#[cfg(feature = "sqlite")]
pub mod sqlite;
pub mod blob;
pub mod blobs_sidecar;
pub mod bls_field_element;
pub mod kzg_commitment;
pub mod kzg_proof;
pub mod bls_field_element;
pub mod blob;
pub mod signed_blobs_sidecar;
pub mod blobs_sidecar;
use ethereum_types::{H160, H256};
@ -105,12 +105,12 @@ pub use crate::attestation_data::AttestationData;
pub use crate::attestation_duty::AttestationDuty;
pub use crate::attester_slashing::AttesterSlashing;
pub use crate::beacon_block::{
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, BeaconBlockRef,
BeaconBlockRefMut, BeaconBlockEip4844
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockEip4844, BeaconBlockMerge,
BeaconBlockRef, BeaconBlockRefMut,
};
pub use crate::beacon_block_body::{
BeaconBlockBody, BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyMerge,
BeaconBlockBodyRef, BeaconBlockBodyRefMut, BeaconBlockBodyEip4844
BeaconBlockBody, BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyEip4844,
BeaconBlockBodyMerge, BeaconBlockBodyRef, BeaconBlockBodyRefMut,
};
pub use crate::beacon_block_header::BeaconBlockHeader;
pub use crate::beacon_committee::{BeaconCommittee, OwnedBeaconCommittee};
@ -139,6 +139,8 @@ pub use crate::free_attestation::FreeAttestation;
pub use crate::graffiti::{Graffiti, GRAFFITI_BYTES_LEN};
pub use crate::historical_batch::HistoricalBatch;
pub use crate::indexed_attestation::IndexedAttestation;
pub use crate::kzg_commitment::KzgCommitment;
pub use crate::kzg_proof::KzgProof;
pub use crate::participation_flags::ParticipationFlags;
pub use crate::participation_list::ParticipationList;
pub use crate::payload::{BlindedPayload, BlockType, ExecPayload, FullPayload};
@ -151,8 +153,8 @@ 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::{
SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockHash,
SignedBeaconBlockMerge, SignedBlindedBeaconBlock, SignedBeaconBlockEip4844,
SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockEip4844,
SignedBeaconBlockHash, SignedBeaconBlockMerge, SignedBlindedBeaconBlock,
};
pub use crate::signed_beacon_block_header::SignedBeaconBlockHeader;
pub use crate::signed_contribution_and_proof::SignedContributionAndProof;

View File

@ -1,4 +1,5 @@
use crate::{test_utils::TestRandom, *};
use core::hash::Hasher;
use derivative::Derivative;
use serde::de::DeserializeOwned;
use serde::{Deserialize, Serialize};
@ -8,7 +9,6 @@ use std::fmt::Debug;
use std::hash::Hash;
use test_random_derive::TestRandom;
use tree_hash::{PackedEncoding, TreeHash};
use core::hash::Hasher;
#[derive(Debug)]
pub enum BlockType {

View File

@ -319,26 +319,26 @@ impl<E: EthSpec> SignedBeaconBlockEip4844<E, BlindedPayload<E>> {
) -> SignedBeaconBlockEip4844<E, FullPayload<E>> {
let SignedBeaconBlockEip4844 {
message:
BeaconBlockEip4844 {
slot,
proposer_index,
parent_root,
state_root,
body:
BeaconBlockBodyEip4844 {
randao_reveal,
eth1_data,
graffiti,
proposer_slashings,
attester_slashings,
attestations,
deposits,
voluntary_exits,
sync_aggregate,
execution_payload: BlindedPayload { .. },
blob_kzg_commitments,
BeaconBlockEip4844 {
slot,
proposer_index,
parent_root,
state_root,
body:
BeaconBlockBodyEip4844 {
randao_reveal,
eth1_data,
graffiti,
proposer_slashings,
attester_slashings,
attestations,
deposits,
voluntary_exits,
sync_aggregate,
execution_payload: BlindedPayload { .. },
blob_kzg_commitments,
},
},
},
signature,
} = self;
SignedBeaconBlockEip4844 {

View File

@ -280,7 +280,7 @@ impl<E: EthSpec, T: EpochTransition<E>> Case for EpochProcessing<E, T> {
}
// No phase0 tests for Altair and later.
ForkName::Altair | ForkName::Merge => T::name() != "participation_record_updates",
ForkName::Eip4844 => false, // TODO: revisit when tests are out
ForkName::Eip4844 => false, // TODO: revisit when tests are out
}
}