This commit is contained in:
realbigsean 2022-10-03 10:06:04 -04:00
commit 88006735c4
No known key found for this signature in database
GPG Key ID: B372B64D866BF8CC
64 changed files with 1349 additions and 321 deletions

View File

@ -101,6 +101,8 @@ use types::*;
pub use crate::canonical_head::{CanonicalHead, CanonicalHeadRwLock};
pub use fork_choice::CountUnrealized;
use types::kzg_commitment::KzgCommitment;
use types::signed_blobs_sidecar::SignedBlobsSidecar;
pub type ForkChoiceError = fork_choice::Error<crate::ForkChoiceStoreError>;
@ -375,6 +377,8 @@ 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.
@ -2436,6 +2440,7 @@ 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.
@ -2453,7 +2458,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, count_unrealized)
.import_execution_pending_block(execution_pending, sidecar, count_unrealized)
.await
};
@ -2511,6 +2516,7 @@ 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 {
@ -2566,6 +2572,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
move || {
chain.import_block(
block,
sidecar,
block_root,
state,
confirmed_state_roots,
@ -2588,6 +2595,7 @@ 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>,
@ -2926,6 +2934,9 @@ 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) {
@ -3364,7 +3375,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
// allows it to run concurrently with things like attestation packing.
let prepare_payload_handle = match &state {
BeaconState::Base(_) | BeaconState::Altair(_) => None,
BeaconState::Merge(_) => {
BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
let prepare_payload_handle =
get_execution_payload(self.clone(), &state, proposer_index, builder_params)?;
Some(prepare_payload_handle)
@ -3620,31 +3631,28 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.ok_or(BlockProductionError::MissingExecutionPayload)?,
},
}),
BeaconState::Capella(_) => {
let sync_aggregate = get_sync_aggregate()?;
let (execution_payload, _blobs) =
get_execution_payload_and_blobs(self, &state, proposer_index)?;
//FIXME(sean) get blobs
BeaconBlock::Capella(BeaconBlockCapella {
slot,
proposer_index,
parent_root,
state_root: Hash256::zero(),
body: BeaconBlockBodyCapella {
randao_reveal,
eth1_data,
graffiti,
proposer_slashings: proposer_slashings.into(),
attester_slashings: attester_slashings.into(),
attestations,
deposits,
voluntary_exits: voluntary_exits.into(),
sync_aggregate,
execution_payload,
blob_kzgs: VariableList::empty(),
},
})
}
BeaconState::Eip4844(_) => BeaconBlock::Eip4844(BeaconBlockEip4844 {
slot,
proposer_index,
parent_root,
state_root: Hash256::zero(),
body: BeaconBlockBodyEip4844 {
randao_reveal,
eth1_data,
graffiti,
proposer_slashings: proposer_slashings.into(),
attester_slashings: attester_slashings.into(),
attestations: attestations.into(),
deposits: deposits.into(),
voluntary_exits: voluntary_exits.into(),
sync_aggregate: sync_aggregate
.ok_or(BlockProductionError::MissingSyncAggregate)?,
execution_payload: execution_payload
.ok_or(BlockProductionError::MissingExecutionPayload)?,
//FIXME(sean) get blobs
blob_kzg_commitments: VariableList::empty(),
},
}),
};
let block = SignedBeaconBlock::from_block(

View File

@ -144,6 +144,7 @@ pub enum BlockError<T: EthSpec> {
present_slot: Slot,
block_slot: Slot,
},
MissingSidecar,
/// The block state_root does not match the generated state.
///
/// ## Peer scoring
@ -277,6 +278,7 @@ 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 },
}
/// Returned when block validation failed due to some issue verifying

View File

@ -801,6 +801,8 @@ 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

@ -24,7 +24,7 @@ use state_processing::per_block_processing::{
use std::sync::Arc;
use tokio::task::JoinHandle;
use tree_hash::TreeHash;
use types::*;
use types::{*, execution_payload::BlobsBundle};
pub type PreparePayloadResult<Payload> = Result<Payload, BlockProductionError>;
pub type PreparePayloadHandle<Payload> = JoinHandle<Option<PreparePayloadResult<Payload>>>;
@ -399,7 +399,7 @@ pub fn prepare_execution_payload_and_blobs_blocking<
Option<(
Payload,
VariableList<
KZGCommitment,
KzgCommitment,
<<T as BeaconChainTypes>::EthSpec as EthSpec>::MaxBlobsPerBlock,
>,
)>,
@ -513,6 +513,14 @@ 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)
.await
.map_err(BlockProductionError::GetPayloadFailed)?;
*/
Ok(execution_payload)
}
@ -527,7 +535,7 @@ pub async fn prepare_execution_payload_and_blobs<
Option<(
Payload,
VariableList<
KZGCommitment,
KzgCommitment,
<<T as BeaconChainTypes>::EthSpec as EthSpec>::MaxBlobsPerBlock,
>,
)>,

View File

@ -16,7 +16,7 @@ pub const DEFAULT_SNAPSHOT_CACHE_SIZE: usize = 4;
const MINIMUM_BLOCK_DELAY_FOR_CLONE: Duration = Duration::from_secs(6);
/// This snapshot is to be used for verifying a child of `self.beacon_block`.
#[derive(Debug)]
#[derive(Debug, Clone)]
pub struct PreProcessingSnapshot<T: EthSpec> {
/// This state is equivalent to the `self.beacon_block.state_root()` state that has been
/// advanced forward one slot using `per_slot_processing`. This state is "primed and ready" for

View File

@ -7,9 +7,10 @@ use serde::{Deserialize, Serialize};
use strum::IntoStaticStr;
pub use types::{
Address, EthSpec, ExecutionBlockHash, ExecutionPayload, ExecutionPayloadHeader, FixedVector,
Hash256, Uint256, VariableList,
Hash256, Uint256, VariableList, kzg_proof::KzgProof, kzg_commitment::KzgCommitment, blob::Blob,
};
use types::{KZGCommitment};
use types::{KzgCommitment};
pub mod auth;
pub mod http;
@ -170,6 +171,6 @@ pub struct ProposeBlindedBlockResponse {
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)]
pub struct BlobDetailsV1 {
kzg: KZGCommitment,
kzg: KzgCommitment,
blob: Vec<Hash256>,
}

View File

@ -3,13 +3,14 @@
use super::*;
use crate::auth::Auth;
use crate::json_structures::*;
use eth2::lighthouse::Eth1Block;
use reqwest::header::CONTENT_TYPE;
use sensitive_url::SensitiveUrl;
use serde::de::DeserializeOwned;
use serde_json::json;
use std::time::Duration;
use types::EthSpec;
use types::{EthSpec, FullPayload, execution_payload::BlobsBundle};
pub use deposit_log::{DepositLog, Log};
pub use reqwest::Client;
@ -34,8 +35,8 @@ pub const ENGINE_NEW_PAYLOAD_TIMEOUT: Duration = Duration::from_secs(8);
pub const ENGINE_GET_PAYLOAD_V1: &str = "engine_getPayloadV1";
pub const ENGINE_GET_PAYLOAD_TIMEOUT: Duration = Duration::from_secs(2);
pub const ENGINE_GET_BLOB_V1: &str = "engine_getBlobV1";
pub const ENGINE_GET_BLOB_TIMEOUT: Duration = Duration::from_secs(2);
pub const ENGINE_GET_BLOBS_BUNDLE_V1: &str = "engine_getBlobsBundleV1";
pub const ENGINE_GET_BLOBS_BUNDLE_TIMEOUT: Duration = Duration::from_secs(2);
pub const ENGINE_FORKCHOICE_UPDATED_V1: &str = "engine_forkchoiceUpdatedV1";
pub const ENGINE_FORKCHOICE_UPDATED_TIMEOUT: Duration = Duration::from_secs(8);
@ -667,15 +668,14 @@ impl HttpJsonRpc {
Ok(response.into())
}
pub async fn get_blob_v1<T: EthSpec>(
pub async fn get_blobs_bundle_v1<T: EthSpec>(
&self,
payload_id: PayloadId,
versioned_hash: ExecutionBlockHash,
) -> Result<BlobDetailsV1, Error> {
let params = json!([JsonPayloadIdRequest::from(payload_id), versioned_hash]);
) -> Result<BlobsBundle<T>, Error> {
let params = json!([JsonPayloadIdRequest::from(payload_id)]);
let response: BlobDetailsV1 = self
.rpc_request(ENGINE_GET_BLOB_V1, params, ENGINE_GET_BLOB_TIMEOUT)
let response: JsonBlobBundlesV1<T> = self
.rpc_request(ENGINE_GET_BLOBS_BUNDLE_V1, params, ENGINE_GET_BLOBS_BUNDLE_TIMEOUT)
.await?;
Ok(response.into())

View File

@ -1,6 +1,6 @@
use super::*;
use serde::{Deserialize, Serialize};
use types::{EthSpec, ExecutionBlockHash, FixedVector, Transaction, Unsigned, VariableList};
use types::{EthSpec, ExecutionBlockHash, FixedVector, Transaction, Unsigned, VariableList, execution_payload::BlobsBundle};
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
@ -269,6 +269,54 @@ impl From<JsonPayloadAttributesV1> for PayloadAttributes {
}
}
#[derive(Debug, PartialEq, Serialize, Deserialize)]
#[serde(bound = "T: EthSpec", rename_all = "camelCase")]
pub struct JsonBlobBundlesV1<T: EthSpec> {
pub block_hash: Hash256,
pub kzgs: Vec<KzgCommitment>,
pub blobs: Vec<Blob<T>>,
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

@ -781,6 +781,55 @@ impl<T: EthSpec> ExecutionLayer<T> {
.await
}
pub async fn get_blob_bundles(
&self,
parent_hash: ExecutionBlockHash,
timestamp: u64,
prev_randao: Hash256,
suggested_fee_recipient: Address,
) -> Result<BlobsBundle<T>, Error> {
debug!(
self.log(),
"Issuing engine_getPayload";
"suggested_fee_recipient" => ?suggested_fee_recipient,
"prev_randao" => ?prev_randao,
"timestamp" => timestamp,
"parent_hash" => ?parent_hash,
);
self.engine()
.request(|engine| async move {
let payload_id = if let Some(id) = engine
.get_payload_id(parent_hash, timestamp, prev_randao, suggested_fee_recipient)
.await
{
// The payload id has been cached for this engine.
metrics::inc_counter_vec(
&metrics::EXECUTION_LAYER_PRE_PREPARED_PAYLOAD_ID,
&[metrics::HIT],
);
id
} else {
error!(
self.log(),
"Exec engine unable to produce blobs, did you call get_payload before?",
);
return Err(ApiError::PayloadIdUnavailable);
};
engine
.api
.get_blobs_bundle_v1::<T>(payload_id)
.await
.map(|bundle| {
// TODO verify the blob bundle here?
bundle.into()
})
})
.await
.map_err(Box::new)
.map_err(Error::EngineError)
}
async fn get_full_payload_with<Payload: ExecPayload<T>>(
&self,
parent_hash: ExecutionBlockHash,

View File

@ -300,7 +300,7 @@ pub fn gossipsub_config(network_load: u8, fork_context: Arc<ForkContext>) -> Gos
// according to: https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/p2p-interface.md#the-gossip-domain-gossipsub
// the derivation of the message-id remains the same in the merge
//TODO(sean): figure this out
ForkName::Altair | ForkName::Merge | ForkName::Capella => {
ForkName::Altair | ForkName::Merge | ForkName::Eip4844 => {
let topic_len_bytes = topic_bytes.len().to_le_bytes();
let mut vec = Vec::with_capacity(
prefix.len() + topic_len_bytes.len() + topic_bytes.len() + message.data.len(),

View File

@ -501,7 +501,7 @@ impl<TSpec: EthSpec> PeerManager<TSpec> {
Protocol::Ping => PeerAction::MidToleranceError,
Protocol::BlocksByRange => PeerAction::MidToleranceError,
Protocol::BlocksByRoot => PeerAction::MidToleranceError,
Protocol::TxBlobsByRange => PeerAction::MidToleranceError,
Protocol::BlobsByRange => PeerAction::MidToleranceError,
Protocol::Goodbye => PeerAction::LowToleranceError,
Protocol::MetaData => PeerAction::LowToleranceError,
Protocol::Status => PeerAction::LowToleranceError,
@ -518,6 +518,7 @@ impl<TSpec: EthSpec> PeerManager<TSpec> {
Protocol::BlocksByRange => return,
Protocol::TxBlobsByRange => return,
Protocol::BlocksByRoot => return,
Protocol::BlobsByRange => return,
Protocol::Goodbye => return,
Protocol::MetaData => PeerAction::LowToleranceError,
Protocol::Status => PeerAction::LowToleranceError,
@ -534,6 +535,7 @@ impl<TSpec: EthSpec> PeerManager<TSpec> {
Protocol::BlocksByRange => PeerAction::MidToleranceError,
Protocol::TxBlobsByRange => PeerAction::MidToleranceError,
Protocol::BlocksByRoot => PeerAction::MidToleranceError,
Protocol::BlobsByRange => PeerAction::MidToleranceError,
Protocol::Goodbye => return,
Protocol::MetaData => return,
Protocol::Status => return,

View File

@ -193,17 +193,17 @@ mod tests {
let mut chain_spec = Spec::default_spec();
let altair_fork_epoch = Epoch::new(1);
let merge_fork_epoch = Epoch::new(2);
let capella_fork_epoch = Epoch::new(3);
let eip4844_fork_epoch = Epoch::new(3);
chain_spec.altair_fork_epoch = Some(altair_fork_epoch);
chain_spec.bellatrix_fork_epoch = Some(merge_fork_epoch);
chain_spec.capella_fork_epoch = Some(capella_fork_epoch);
chain_spec.eip4844_fork_epoch = Some(eip4844_fork_epoch);
let current_slot = match fork_name {
ForkName::Base => Slot::new(0),
ForkName::Altair => altair_fork_epoch.start_slot(Spec::slots_per_epoch()),
ForkName::Merge => merge_fork_epoch.start_slot(Spec::slots_per_epoch()),
ForkName::Capella => capella_fork_epoch.start_slot(Spec::slots_per_epoch()),
ForkName::Eip4844 => eip4844_fork_epoch.start_slot(Spec::slots_per_epoch()),
};
ForkContext::new::<Spec>(current_slot, Hash256::zero(), &chain_spec)
}

View File

@ -16,8 +16,8 @@ use std::marker::PhantomData;
use std::sync::Arc;
use tokio_util::codec::{Decoder, Encoder};
use types::{
BlobsSidecar, EthSpec, ForkContext, ForkName, SignedBeaconBlock, SignedBeaconBlockAltair,
SignedBeaconBlockBase, SignedBeaconBlockCapella, SignedBeaconBlockMerge,
EthSpec, ForkContext, ForkName, SignedBeaconBlock, SignedBeaconBlockAltair,
SignedBeaconBlockBase, SignedBeaconBlockMerge, SignedBeaconBlockEip4844
};
use unsigned_varint::codec::Uvi;
@ -69,8 +69,8 @@ impl<TSpec: EthSpec> Encoder<RPCCodedResponse<TSpec>> for SSZSnappyInboundCodec<
RPCCodedResponse::Success(resp) => match &resp {
RPCResponse::Status(res) => res.as_ssz_bytes(),
RPCResponse::BlocksByRange(res) => res.as_ssz_bytes(),
RPCResponse::TxBlobsByRange(res) => res.as_ssz_bytes(),
RPCResponse::BlocksByRoot(res) => res.as_ssz_bytes(),
RPCResponse::BlobsByRange(res) => res.as_ssz_bytes(),
RPCResponse::Pong(res) => res.data.as_ssz_bytes(),
RPCResponse::MetaData(res) =>
// Encode the correct version of the MetaData response based on the negotiated version.
@ -228,8 +228,8 @@ impl<TSpec: EthSpec> Encoder<OutboundRequest<TSpec>> for SSZSnappyOutboundCodec<
OutboundRequest::Status(req) => req.as_ssz_bytes(),
OutboundRequest::Goodbye(req) => req.as_ssz_bytes(),
OutboundRequest::BlocksByRange(req) => req.as_ssz_bytes(),
OutboundRequest::TxBlobsByRange(req) => req.as_ssz_bytes(),
OutboundRequest::BlocksByRoot(req) => req.block_roots.as_ssz_bytes(),
OutboundRequest::BlobsByRange(req) => req.as_ssz_bytes(),
OutboundRequest::Ping(req) => req.as_ssz_bytes(),
OutboundRequest::MetaData(_) => return Ok(()), // no metadata to encode
};
@ -409,8 +409,9 @@ fn context_bytes<T: EthSpec>(
return match **ref_box_block {
// NOTE: If you are adding another fork type here, be sure to modify the
// `fork_context.to_context_bytes()` function to support it as well!
SignedBeaconBlock::Capella { .. } => {
fork_context.to_context_bytes(ForkName::Capella)
SignedBeaconBlock::Eip4844 { .. } => {
// Merge context being `None` implies that "merge never happened".
fork_context.to_context_bytes(ForkName::Eip4844)
}
SignedBeaconBlock::Merge { .. } => {
// Merge context being `None` implies that "merge never happened".
@ -471,12 +472,12 @@ fn handle_v1_request<T: EthSpec>(
Protocol::BlocksByRange => Ok(Some(InboundRequest::BlocksByRange(
OldBlocksByRangeRequest::from_ssz_bytes(decoded_buffer)?,
))),
Protocol::TxBlobsByRange => Ok(Some(InboundRequest::TxBlobsByRange(
TxBlobsByRangeRequest::from_ssz_bytes(decoded_buffer)?,
))),
Protocol::BlocksByRoot => Ok(Some(InboundRequest::BlocksByRoot(BlocksByRootRequest {
block_roots: VariableList::from_ssz_bytes(decoded_buffer)?,
}))),
Protocol::BlobsByRange => Ok(Some(InboundRequest::BlobsByRange(
BlobsByRangeRequest::from_ssz_bytes(decoded_buffer)?,
))),
Protocol::Ping => Ok(Some(InboundRequest::Ping(Ping {
data: u64::from_ssz_bytes(decoded_buffer)?,
}))),
@ -506,12 +507,12 @@ fn handle_v2_request<T: EthSpec>(
Protocol::BlocksByRange => Ok(Some(InboundRequest::BlocksByRange(
OldBlocksByRangeRequest::from_ssz_bytes(decoded_buffer)?,
))),
Protocol::TxBlobsByRange => Ok(Some(InboundRequest::TxBlobsByRange(
TxBlobsByRangeRequest::from_ssz_bytes(decoded_buffer)?,
))),
Protocol::BlocksByRoot => Ok(Some(InboundRequest::BlocksByRoot(BlocksByRootRequest {
block_roots: VariableList::from_ssz_bytes(decoded_buffer)?,
}))),
Protocol::BlobsByRange => Ok(Some(InboundRequest::BlobsByRange(
BlobsByRangeRequest::from_ssz_bytes(decoded_buffer)?,
))),
// MetaData requests return early from InboundUpgrade and do not reach the decoder.
// Handle this case just for completeness.
Protocol::MetaData => {
@ -546,12 +547,12 @@ fn handle_v1_response<T: EthSpec>(
Protocol::BlocksByRange => Ok(Some(RPCResponse::BlocksByRange(Arc::new(
SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?),
)))),
Protocol::TxBlobsByRange => Ok(Some(RPCResponse::TxBlobsByRange(Arc::new(
BlobsSidecar::from_ssz_bytes(decoded_buffer)?),
))),
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::Ping => Ok(Some(RPCResponse::Pong(Ping {
data: u64::from_ssz_bytes(decoded_buffer)?,
}))),
@ -600,15 +601,12 @@ fn handle_v2_response<T: EthSpec>(
decoded_buffer,
)?),
)))),
ForkName::Capella => Ok(Some(RPCResponse::BlocksByRange(Box::new(
SignedBeaconBlock::Capella(SignedBeaconBlockCapella::from_ssz_bytes(
ForkName::Eip4844 => Ok(Some(RPCResponse::BlocksByRange(Arc::new(
SignedBeaconBlock::Eip4844(SignedBeaconBlockEip4844::from_ssz_bytes(
decoded_buffer,
)?),
)))),
},
Protocol::TxBlobsByRange => Ok(Some(RPCResponse::TxBlobsByRange(Box::new(
BlobsSidecar::from_ssz_bytes(decoded_buffer)?,
)))),
Protocol::BlocksByRoot => match fork_name {
ForkName::Altair => Ok(Some(RPCResponse::BlocksByRoot(Arc::new(
SignedBeaconBlock::Altair(SignedBeaconBlockAltair::from_ssz_bytes(
@ -623,12 +621,21 @@ fn handle_v2_response<T: EthSpec>(
decoded_buffer,
)?),
)))),
ForkName::Capella => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
SignedBeaconBlock::Capella(SignedBeaconBlockCapella::from_ssz_bytes(
ForkName::Eip4844 => Ok(Some(RPCResponse::BlocksByRange(Arc::new(
SignedBeaconBlock::Eip4844(SignedBeaconBlockEip4844::from_ssz_bytes(
decoded_buffer,
)?),
)))),
},
Protocol::BlobsByRange => match fork_name {
ForkName::Eip4844 => Ok(Some(RPCResponse::BlobsByRange(Arc::new(
VariableList::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(),
@ -677,17 +684,17 @@ mod tests {
let mut chain_spec = Spec::default_spec();
let altair_fork_epoch = Epoch::new(1);
let merge_fork_epoch = Epoch::new(2);
let capella_fork_epoch = Epoch::new(3);
let eip4844_fork_epoch = Epoch::new(3);
chain_spec.altair_fork_epoch = Some(altair_fork_epoch);
chain_spec.bellatrix_fork_epoch = Some(merge_fork_epoch);
chain_spec.capella_fork_epoch = Some(capella_fork_epoch);
chain_spec.eip4844_fork_epoch = Some(eip4844_fork_epoch);
let current_slot = match fork_name {
ForkName::Base => Slot::new(0),
ForkName::Altair => altair_fork_epoch.start_slot(Spec::slots_per_epoch()),
ForkName::Merge => merge_fork_epoch.start_slot(Spec::slots_per_epoch()),
ForkName::Capella => capella_fork_epoch.start_slot(Spec::slots_per_epoch()),
ForkName::Eip4844 => eip4844_fork_epoch.start_slot(Spec::slots_per_epoch()),
};
ForkContext::new::<Spec>(current_slot, Hash256::zero(), &chain_spec)
}
@ -891,6 +898,9 @@ mod tests {
OutboundRequest::BlocksByRoot(bbroot) => {
assert_eq!(decoded, InboundRequest::BlocksByRoot(bbroot))
}
OutboundRequest::BlobsByRange(blbrange) => {
assert_eq!(decoded, InboundRequest::BlobsByRange(blbrange))
}
OutboundRequest::Ping(ping) => {
assert_eq!(decoded, InboundRequest::Ping(ping))
}

View File

@ -12,7 +12,8 @@ use std::ops::Deref;
use std::sync::Arc;
use strum::IntoStaticStr;
use superstruct::superstruct;
use types::{BlobsSidecar, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot};
use types::{Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot};
use types::blobs_sidecar::BlobsSidecar;
/// Maximum number of blocks in a single request.
pub type MaxRequestBlocks = U1024;
@ -204,6 +205,16 @@ pub struct BlocksByRangeRequest {
pub count: u64,
}
/// Request a number of beacon blobs from a peer.
#[derive(Encode, Decode, Clone, Debug, PartialEq)]
pub struct BlobsByRangeRequest {
/// The starting slot to request blobs.
pub start_slot: u64,
/// The number of blobs from the start slot.
pub count: u64,
}
/// Request a number of beacon block roots from a peer.
#[derive(Encode, Decode, Clone, Debug, PartialEq)]
pub struct OldBlocksByRangeRequest {
@ -251,6 +262,9 @@ pub enum RPCResponse<T: EthSpec> {
/// 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>>),
/// A PONG response to a PING request.
Pong(Ping),
@ -268,6 +282,9 @@ pub enum ResponseTermination {
/// Blocks by root stream termination.
BlocksByRoot,
// Blobs by range stream termination.
BlobsByRange
}
/// The structured response containing a result/code indicating success or failure
@ -330,6 +347,7 @@ impl<T: EthSpec> RPCCodedResponse<T> {
RPCResponse::BlocksByRange(_) => true,
RPCResponse::TxBlobsByRange(_) => true,
RPCResponse::BlocksByRoot(_) => true,
RPCResponse::BlobsByRange(_) => true,
RPCResponse::Pong(_) => false,
RPCResponse::MetaData(_) => false,
},
@ -364,6 +382,7 @@ impl<T: EthSpec> RPCResponse<T> {
RPCResponse::Status(_) => Protocol::Status,
RPCResponse::BlocksByRange(_) => Protocol::BlocksByRange,
RPCResponse::BlocksByRoot(_) => Protocol::BlocksByRoot,
RPCResponse::BlobsByRange(_) => Protocol::BlobsByRange,
RPCResponse::Pong(_) => Protocol::Ping,
RPCResponse::MetaData(_) => Protocol::MetaData,
}
@ -402,6 +421,9 @@ impl<T: EthSpec> std::fmt::Display for RPCResponse<T> {
RPCResponse::BlocksByRoot(block) => {
write!(f, "BlocksByRoot: Block slot: {}", block.slot())
}
RPCResponse::BlobsByRange(blob) => {
write!(f, "BlobsByRange: Blob slot: {}", blob.len())
}
RPCResponse::Pong(ping) => write!(f, "Pong: {}", ping.data),
RPCResponse::MetaData(metadata) => write!(f, "Metadata: {}", metadata.seq_number()),
}

View File

@ -132,6 +132,7 @@ impl<Id: ReqId, TSpec: EthSpec> RPC<Id, TSpec> {
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 {
@ -299,6 +300,7 @@ where
match end {
ResponseTermination::BlocksByRange => Protocol::BlocksByRange,
ResponseTermination::BlocksByRoot => Protocol::BlocksByRoot,
ResponseTermination::BlobsByRange => Protocol::BlobsByRange,
},
),
},

View File

@ -39,6 +39,7 @@ pub enum OutboundRequest<TSpec: EthSpec> {
BlocksByRange(OldBlocksByRangeRequest),
TxBlobsByRange(TxBlobsByRangeRequest),
BlocksByRoot(BlocksByRootRequest),
BlobsByRange(BlobsByRangeRequest),
Ping(Ping),
MetaData(PhantomData<TSpec>),
}
@ -82,6 +83,9 @@ impl<TSpec: EthSpec> OutboundRequest<TSpec> {
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::Ping(_) => vec![ProtocolId::new(
Protocol::Ping,
Version::V1,
@ -104,6 +108,7 @@ impl<TSpec: EthSpec> OutboundRequest<TSpec> {
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,
OutboundRequest::MetaData(_) => 1,
}
@ -117,6 +122,7 @@ impl<TSpec: EthSpec> OutboundRequest<TSpec> {
OutboundRequest::BlocksByRange(_) => Protocol::BlocksByRange,
OutboundRequest::TxBlobsByRange(_) => Protocol::TxBlobsByRange,
OutboundRequest::BlocksByRoot(_) => Protocol::BlocksByRoot,
OutboundRequest::BlobsByRange(_) => Protocol::BlobsByRange,
OutboundRequest::Ping(_) => Protocol::Ping,
OutboundRequest::MetaData(_) => Protocol::MetaData,
}
@ -131,6 +137,7 @@ impl<TSpec: EthSpec> OutboundRequest<TSpec> {
OutboundRequest::BlocksByRange(_) => ResponseTermination::BlocksByRange,
OutboundRequest::TxBlobsByRange(_) => ResponseTermination::TxBlobsByRange,
OutboundRequest::BlocksByRoot(_) => ResponseTermination::BlocksByRoot,
OutboundRequest::BlobsByRange(_) => ResponseTermination::BlobsByRange,
OutboundRequest::Status(_) => unreachable!(),
OutboundRequest::Goodbye(_) => unreachable!(),
OutboundRequest::Ping(_) => unreachable!(),
@ -187,6 +194,7 @@ impl<TSpec: EthSpec> std::fmt::Display for OutboundRequest<TSpec> {
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),
OutboundRequest::MetaData(_) => write!(f, "MetaData request"),
}

View File

@ -108,6 +108,8 @@ 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
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";
/// Time allowed for the first byte of a request to arrive before we time out (Time To First Byte).
@ -119,9 +121,9 @@ const REQUEST_TIMEOUT: u64 = 15;
/// Returns the maximum bytes that can be sent across the RPC.
pub fn max_rpc_size(fork_context: &ForkContext) -> usize {
match fork_context.current_fork() {
ForkName::Eip4844 => MAX_RPC_SIZE_POST_EIP4844,
ForkName::Merge => MAX_RPC_SIZE_POST_MERGE,
//FIXME(sean) check this
ForkName::Altair | ForkName::Base | ForkName::Capella => MAX_RPC_SIZE,
ForkName::Altair | ForkName::Base => MAX_RPC_SIZE,
}
}
@ -142,6 +144,10 @@ pub fn rpc_block_limits_by_fork(current_fork: ForkName) -> RpcLimits {
*SIGNED_BEACON_BLOCK_BASE_MIN, // Base block is smaller than altair and merge blocks
*SIGNED_BEACON_BLOCK_MERGE_MAX, // Merge block is larger than base and altair blocks
),
ForkName::Eip4844 => RpcLimits::new(
*SIGNED_BEACON_BLOCK_BASE_MIN, // Base block is smaller than altair and merge blocks
*SIGNED_BEACON_BLOCK_EIP4844_MAX, // Merge block is larger than base and altair blocks
),
}
}
@ -157,6 +163,8 @@ pub enum Protocol {
TxBlobsByRange,
/// The `BlocksByRoot` protocol name.
BlocksByRoot,
/// The `BlobsByRange` protocol name.
BlobsByRange,
/// The `Ping` protocol name.
Ping,
/// The `MetaData` protocol name.
@ -184,9 +192,8 @@ impl std::fmt::Display for Protocol {
Protocol::Status => "status",
Protocol::Goodbye => "goodbye",
Protocol::BlocksByRange => "beacon_blocks_by_range",
//FIXME(sean) verify
Protocol::TxBlobsByRange => "tx_blobs_by_range",
Protocol::BlocksByRoot => "beacon_blocks_by_root",
Protocol::BlobsByRange => "blobs_sidecars_by_range",
Protocol::Ping => "ping",
Protocol::MetaData => "metadata",
};
@ -292,13 +299,12 @@ impl ProtocolId {
<OldBlocksByRangeRequest as Encode>::ssz_fixed_len(),
<OldBlocksByRangeRequest as Encode>::ssz_fixed_len(),
),
Protocol::TxBlobsByRange => RpcLimits::new(
<TxBlobsByRangeRequest as Encode>::ssz_fixed_len(),
<TxBlobsByRangeRequest as Encode>::ssz_fixed_len(),
),
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::Ping => RpcLimits::new(
<Ping as Encode>::ssz_fixed_len(),
<Ping as Encode>::ssz_fixed_len(),
@ -317,6 +323,7 @@ impl ProtocolId {
Protocol::Goodbye => RpcLimits::new(0, 0), // Goodbye request has no response
Protocol::BlocksByRange => rpc_block_limits_by_fork(fork_context.current_fork()),
Protocol::BlocksByRoot => rpc_block_limits_by_fork(fork_context.current_fork()),
Protocol::BlobsByRange => rpc_block_limits_by_fork(fork_context.current_fork()),
Protocol::Ping => RpcLimits::new(
<Ping as Encode>::ssz_fixed_len(),
@ -431,6 +438,7 @@ pub enum InboundRequest<TSpec: EthSpec> {
Goodbye(GoodbyeReason),
BlocksByRange(OldBlocksByRangeRequest),
BlocksByRoot(BlocksByRootRequest),
BlobsByRange(BlobsByRangeRequest),
Ping(Ping),
MetaData(PhantomData<TSpec>),
}
@ -465,16 +473,14 @@ impl<TSpec: EthSpec> InboundRequest<TSpec> {
ProtocolId::new(Protocol::BlocksByRange, Version::V2, Encoding::SSZSnappy),
ProtocolId::new(Protocol::BlocksByRange, Version::V1, Encoding::SSZSnappy),
],
//FIXME(sean) do I need v1
InboundRequest::TxBlobsByRange(_) => vec![
// V2 has higher preference when negotiating a stream
ProtocolId::new(Protocol::TxBlobsByRange, Version::V2, Encoding::SSZSnappy),
],
InboundRequest::BlocksByRoot(_) => vec![
// V2 has higher preference when negotiating a stream
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::Ping(_) => vec![ProtocolId::new(
Protocol::Ping,
Version::V1,
@ -497,6 +503,7 @@ impl<TSpec: EthSpec> InboundRequest<TSpec> {
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,
InboundRequest::MetaData(_) => 1,
}
@ -510,6 +517,7 @@ impl<TSpec: EthSpec> InboundRequest<TSpec> {
InboundRequest::BlocksByRange(_) => Protocol::BlocksByRange,
InboundRequest::TxBlobsByRange(_) => Protocol::TxBlobsByRange,
InboundRequest::BlocksByRoot(_) => Protocol::BlocksByRoot,
InboundRequest::BlobsByRange(_) => Protocol::BlobsByRange,
InboundRequest::Ping(_) => Protocol::Ping,
InboundRequest::MetaData(_) => Protocol::MetaData,
}
@ -524,6 +532,7 @@ impl<TSpec: EthSpec> InboundRequest<TSpec> {
InboundRequest::BlocksByRange(_) => ResponseTermination::BlocksByRange,
InboundRequest::TxBlobsByRange(_) => ResponseTermination::TxBlobsByRange,
InboundRequest::BlocksByRoot(_) => ResponseTermination::BlocksByRoot,
InboundRequest::BlobsByRange(_) => ResponseTermination::BlobsByRange,
InboundRequest::Status(_) => unreachable!(),
InboundRequest::Goodbye(_) => unreachable!(),
InboundRequest::Ping(_) => unreachable!(),
@ -628,8 +637,8 @@ impl<TSpec: EthSpec> std::fmt::Display for InboundRequest<TSpec> {
InboundRequest::Status(status) => write!(f, "Status Message: {}", status),
InboundRequest::Goodbye(reason) => write!(f, "Goodbye: {}", reason),
InboundRequest::BlocksByRange(req) => write!(f, "Blocks by range: {}", req),
InboundRequest::TxBlobsByRange(req) => write!(f, "Blobs by range: {}", req),
InboundRequest::BlocksByRoot(req) => write!(f, "Blocks by root: {:?}", req),
InboundRequest::BlobsByRange(req) => write!(f, "Blobs by range: {:?}", req),
InboundRequest::Ping(ping) => write!(f, "Ping: {}", ping.data),
InboundRequest::MetaData(_) => write!(f, "MetaData request"),
}

View File

@ -71,9 +71,10 @@ pub struct RPCRateLimiter {
status_rl: Limiter<PeerId>,
/// BlocksByRange rate limiter.
bbrange_rl: Limiter<PeerId>,
txbbrange_rl: Limiter<PeerId>,
/// BlocksByRoot rate limiter.
bbroots_rl: Limiter<PeerId>,
/// BlobsByRange rate limiter.
blbrange_rl: Limiter<PeerId>,
}
/// Error type for non conformant requests
@ -97,9 +98,10 @@ pub struct RPCRateLimiterBuilder {
status_quota: Option<Quota>,
/// Quota for the BlocksByRange protocol.
bbrange_quota: Option<Quota>,
txbbrange_quota: Option<Quota>,
/// Quota for the BlocksByRoot protocol.
bbroots_quota: Option<Quota>,
/// Quota for the BlocksByRange protocol.
blbrange_quota: Option<Quota>,
}
impl RPCRateLimiterBuilder {
@ -117,8 +119,8 @@ impl RPCRateLimiterBuilder {
Protocol::MetaData => self.metadata_quota = q,
Protocol::Goodbye => self.goodbye_quota = q,
Protocol::BlocksByRange => self.bbrange_quota = q,
Protocol::TxBlobsByRange => self.txbbrange_quota = q,
Protocol::BlocksByRoot => self.bbroots_quota = q,
Protocol::BlobsByRange => self.blbrange_quota = q,
}
self
}
@ -158,9 +160,8 @@ impl RPCRateLimiterBuilder {
let bbrange_quota = self
.bbrange_quota
.ok_or("BlocksByRange quota not specified")?;
let txbbrange_quota = self
.txbbrange_quota
.ok_or("TxBlobsByRange 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)?;
@ -169,7 +170,7 @@ impl RPCRateLimiterBuilder {
let goodbye_rl = Limiter::from_quota(goodbye_quota)?;
let bbroots_rl = Limiter::from_quota(bbroots_quota)?;
let bbrange_rl = Limiter::from_quota(bbrange_quota)?;
let txbbrange_rl = Limiter::from_quota(txbbrange_quota)?;
let blbrange_rl = Limiter::from_quota(blbrange_quota)?;
// check for peers to prune every 30 seconds, starting in 30 seconds
let prune_every = tokio::time::Duration::from_secs(30);
@ -183,7 +184,7 @@ impl RPCRateLimiterBuilder {
goodbye_rl,
bbroots_rl,
bbrange_rl,
txbbrange_rl,
blbrange_rl,
init_time: Instant::now(),
})
}
@ -206,8 +207,8 @@ impl RPCRateLimiter {
Protocol::MetaData => &mut self.metadata_rl,
Protocol::Goodbye => &mut self.goodbye_rl,
Protocol::BlocksByRange => &mut self.bbrange_rl,
Protocol::TxBlobsByRange => &mut self.txbbrange_rl,
Protocol::BlocksByRoot => &mut self.bbroots_rl,
Protocol::BlobsByRange => &mut self.blbrange_rl,
};
check(limiter)
}
@ -220,6 +221,7 @@ impl RPCRateLimiter {
self.goodbye_rl.prune(time_since_start);
self.bbrange_rl.prune(time_since_start);
self.bbroots_rl.prune(time_since_start);
self.blbrange_rl.prune(time_since_start);
}
}

View File

@ -20,6 +20,8 @@ pub struct GossipCache {
topic_msgs: HashMap<GossipTopic, HashMap<Vec<u8>, Key>>,
/// Timeout for blocks.
beacon_block: Option<Duration>,
/// Timeout for blobs.
blobs_sidecar: Option<Duration>,
/// Timeout for aggregate attestations.
aggregates: Option<Duration>,
/// Timeout for attestations.
@ -41,6 +43,8 @@ pub struct GossipCacheBuilder {
default_timeout: Option<Duration>,
/// Timeout for blocks.
beacon_block: Option<Duration>,
/// Timeout for blob sidecars.
blobs_sidecar: Option<Duration>,
/// Timeout for aggregate attestations.
aggregates: Option<Duration>,
/// Timeout for attestations.
@ -117,6 +121,7 @@ impl GossipCacheBuilder {
let GossipCacheBuilder {
default_timeout,
beacon_block,
blobs_sidecar,
aggregates,
attestation,
voluntary_exit,
@ -129,6 +134,7 @@ impl GossipCacheBuilder {
expirations: DelayQueue::default(),
topic_msgs: HashMap::default(),
beacon_block: beacon_block.or(default_timeout),
blobs_sidecar: blobs_sidecar.or(default_timeout),
aggregates: aggregates.or(default_timeout),
attestation: attestation.or(default_timeout),
voluntary_exit: voluntary_exit.or(default_timeout),
@ -151,8 +157,7 @@ impl GossipCache {
pub fn insert(&mut self, topic: GossipTopic, data: Vec<u8>) {
let expire_timeout = match topic.kind() {
GossipKind::BeaconBlock => self.beacon_block,
//FIXME(sean) use its own timeout
GossipKind::Blob => self.beacon_block,
GossipKind::BlobsSidecar => self.blobs_sidecar,
GossipKind::BeaconAggregateAndProof => self.aggregates,
GossipKind::Attestation(_) => self.attestation,
GossipKind::VoluntaryExit => self.voluntary_exit,

View File

@ -10,6 +10,7 @@ use crate::peer_manager::{MIN_OUTBOUND_ONLY_FACTOR, PEER_EXCESS_FACTOR, PRIORITY
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,
@ -28,20 +29,22 @@ use libp2p::gossipsub::subscription_filter::MaxCountSubscriptionFilter;
use libp2p::gossipsub::{
GossipsubEvent, IdentTopic as Topic, MessageAcceptance, MessageAuthenticity, MessageId,
};
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 std::marker::PhantomData;
use std::path::PathBuf;
use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};
use slog::{crit, debug, 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::{
collections::VecDeque,
marker::PhantomData,
sync::Arc,
task::{Context, Poll},
};
use types::{
consts::altair::SYNC_COMMITTEE_SUBNET_COUNT, EnrForkId, EthSpec, ForkContext, Slot, SubnetId,
BlobsSidecar, SignedBeaconBlock, SyncSubnetId
consts::altair::SYNC_COMMITTEE_SUBNET_COUNT, EnrForkId, EthSpec, ForkContext,
SignedBeaconBlock, Slot, SubnetId, SyncSubnetId, VariableList
};
use crate::rpc::methods::TxBlobsByRangeRequest;
use utils::{build_transport, strip_peer_id, MAX_CONNECTIONS_PER_PEER};
@ -991,6 +994,9 @@ impl<AppReqId: ReqId, TSpec: EthSpec> Network<AppReqId, TSpec> {
Request::BlocksByRoot { .. } => {
metrics::inc_counter_vec(&metrics::TOTAL_RPC_REQUESTS, &["blocks_by_root"])
}
Request::BlobsByRange { .. } => {
metrics::inc_counter_vec(&metrics::TOTAL_RPC_REQUESTS, &["blobs_by_range"])
}
}
NetworkEvent::RequestReceived {
peer_id,
@ -1254,6 +1260,9 @@ impl<AppReqId: ReqId, TSpec: EthSpec> Network<AppReqId, TSpec> {
);
Some(event)
}
InboundRequest::BlobsByRange(req) => {
self.propagate_request(peer_request_id, peer_id, Request::BlobsByRange(req))
}
}
}
Ok(RPCReceived::Response(id, resp)) => {
@ -1284,6 +1293,9 @@ impl<AppReqId: ReqId, TSpec: EthSpec> Network<AppReqId, TSpec> {
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)) => {
@ -1291,6 +1303,7 @@ impl<AppReqId: ReqId, TSpec: EthSpec> Network<AppReqId, TSpec> {
ResponseTermination::BlocksByRange => Response::BlocksByRange(None),
ResponseTermination::TxBlobsByRange => Response::TxBlobsByRange(None),
ResponseTermination::BlocksByRoot => Response::BlocksByRoot(None),
ResponseTermination::BlobsByRange => Response::BlobsByRange(None),
};
self.build_response(id, peer_id, response)
}

View File

@ -11,14 +11,18 @@ use std::sync::Arc;
use types::{
Attestation, AttesterSlashing, BlobsSidecar, EthSpec, ForkContext, ForkName, ProposerSlashing,
SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase,
SignedBeaconBlockCapella, SignedBeaconBlockMerge, SignedContributionAndProof,
SignedVoluntaryExit, SubnetId, SyncCommitteeMessage, SyncSubnetId,
SignedBeaconBlockMerge, SignedBeaconBlockEip4844, 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> {
/// Gossipsub message providing notification of a new block.
BeaconBlock(Arc<SignedBeaconBlock<T>>),
/// Gossipsub message providing notification of a new blobs sidecar.
BlobsSidecars(Arc<SignedBlobsSidecar<T>>),
/// Gossipsub message providing notification of a Aggregate attestation and associated proof.
AggregateAndProofAttestation(Box<SignedAggregateAndProof<T>>),
/// Gossipsub message providing notification of a raw un-aggregated attestation with its shard id.
@ -106,7 +110,7 @@ impl<T: EthSpec> PubsubMessage<T> {
pub fn kind(&self) -> GossipKind {
match self {
PubsubMessage::BeaconBlock(_) => GossipKind::BeaconBlock,
PubsubMessage::Blob(_) => GossipKind::Blob,
PubsubMessage::BlobsSidecars(_) => GossipKind::BlobsSidecar,
PubsubMessage::AggregateAndProofAttestation(_) => GossipKind::BeaconAggregateAndProof,
PubsubMessage::Attestation(attestation_data) => {
GossipKind::Attestation(attestation_data.0)
@ -168,8 +172,12 @@ impl<T: EthSpec> PubsubMessage<T> {
SignedBeaconBlockMerge::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?,
),
Some(ForkName::Capella) => SignedBeaconBlock::<T>::Capella(
SignedBeaconBlockCapella::from_ssz_bytes(data)
Some(ForkName::Eip4844) => SignedBeaconBlock::<T>::Eip4844(
SignedBeaconBlockEip4844::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?,
),
Some(ForkName::Eip4844) => SignedBeaconBlock::<T>::Eip4844(
SignedBeaconBlockEip4844::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?,
),
None => {
@ -181,11 +189,10 @@ impl<T: EthSpec> PubsubMessage<T> {
};
Ok(PubsubMessage::BeaconBlock(Arc::new(beacon_block)))
}
GossipKind::Blob => {
//FIXME(sean) verify against fork context
let blob =
BlobsSidecar::from_ssz_bytes(data).map_err(|e| format!("{:?}", e))?;
Ok(PubsubMessage::Blob(Box::new(blob)))
GossipKind::BlobsSidecar => {
let blobs_sidecar = SignedBlobsSidecar::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?;
Ok(PubsubMessage::BlobsSidecars(Arc::new(blobs_sidecar)))
}
GossipKind::VoluntaryExit => {
let voluntary_exit = SignedVoluntaryExit::from_ssz_bytes(data)
@ -231,7 +238,7 @@ impl<T: EthSpec> PubsubMessage<T> {
// messages for us.
match &self {
PubsubMessage::BeaconBlock(data) => data.as_ssz_bytes(),
PubsubMessage::Blob(data) => data.as_ssz_bytes(),
PubsubMessage::BlobsSidecars(data) => data.as_ssz_bytes(),
PubsubMessage::AggregateAndProofAttestation(data) => data.as_ssz_bytes(),
PubsubMessage::VoluntaryExit(data) => data.as_ssz_bytes(),
PubsubMessage::ProposerSlashing(data) => data.as_ssz_bytes(),
@ -252,10 +259,11 @@ impl<T: EthSpec> std::fmt::Display for PubsubMessage<T> {
block.slot(),
block.message().proposer_index()
),
PubsubMessage::Blob(blob) => write!(
PubsubMessage::BlobsSidecars(blobs) => write!(
f,
"Tx Blob: slot: {}, beacon_block_root: {}",
blob.beacon_block_slot, blob.beacon_block_root
"Blobs Sidecar: slot: {}, blobs: {}",
blobs.message.beacon_block_slot,
blobs.message.blobs.len(),
),
PubsubMessage::AggregateAndProofAttestation(att) => write!(
f,

View File

@ -11,8 +11,7 @@ use crate::Subnet;
pub const TOPIC_PREFIX: &str = "eth2";
pub const SSZ_SNAPPY_ENCODING_POSTFIX: &str = "ssz_snappy";
pub const BEACON_BLOCK_TOPIC: &str = "beacon_block";
//FIXME(sean) check this name
pub const BLOB_TOPIC: &str = "tx_blob";
pub const BLOBS_SIDECAR_TOPIC: &str = "blobs_sidecar";
pub const BEACON_AGGREGATE_AND_PROOF_TOPIC: &str = "beacon_aggregate_and_proof";
pub const BEACON_ATTESTATION_PREFIX: &str = "beacon_attestation_";
pub const VOLUNTARY_EXIT_TOPIC: &str = "voluntary_exit";
@ -21,8 +20,9 @@ pub const ATTESTER_SLASHING_TOPIC: &str = "attester_slashing";
pub const SIGNED_CONTRIBUTION_AND_PROOF_TOPIC: &str = "sync_committee_contribution_and_proof";
pub const SYNC_COMMITTEE_PREFIX_TOPIC: &str = "sync_committee_";
pub const CORE_TOPICS: [GossipKind; 6] = [
pub const CORE_TOPICS: [GossipKind; 7] = [
GossipKind::BeaconBlock,
GossipKind::BlobsSidecar,
GossipKind::BeaconAggregateAndProof,
GossipKind::VoluntaryExit,
GossipKind::ProposerSlashing,
@ -49,7 +49,8 @@ pub struct GossipTopic {
pub enum GossipKind {
/// Topic for publishing beacon blocks.
BeaconBlock,
Blob,
/// Topic for publishing blob sidecars.
BlobsSidecar,
/// Topic for publishing aggregate attestations and proofs.
BeaconAggregateAndProof,
/// Topic for publishing raw attestations on a particular subnet.
@ -181,7 +182,7 @@ impl From<GossipTopic> for String {
let kind = match topic.kind {
GossipKind::BeaconBlock => BEACON_BLOCK_TOPIC.into(),
GossipKind::Blob => BLOB_TOPIC.into(),
GossipKind::BlobsSidecar => BLOBS_SIDECAR_TOPIC.into(),
GossipKind::BeaconAggregateAndProof => BEACON_AGGREGATE_AND_PROOF_TOPIC.into(),
GossipKind::VoluntaryExit => VOLUNTARY_EXIT_TOPIC.into(),
GossipKind::ProposerSlashing => PROPOSER_SLASHING_TOPIC.into(),
@ -210,7 +211,7 @@ impl std::fmt::Display for GossipTopic {
let kind = match self.kind {
GossipKind::BeaconBlock => BEACON_BLOCK_TOPIC.into(),
GossipKind::Blob => BLOB_TOPIC.into(),
GossipKind::BlobsSidecar => BLOBS_SIDECAR_TOPIC.into(),
GossipKind::BeaconAggregateAndProof => BEACON_AGGREGATE_AND_PROOF_TOPIC.into(),
GossipKind::VoluntaryExit => VOLUNTARY_EXIT_TOPIC.into(),
GossipKind::ProposerSlashing => PROPOSER_SLASHING_TOPIC.into(),

View File

@ -32,17 +32,17 @@ pub fn fork_context(fork_name: ForkName) -> ForkContext {
let mut chain_spec = E::default_spec();
let altair_fork_epoch = Epoch::new(1);
let merge_fork_epoch = Epoch::new(2);
let capella_fork_epoch = Epoch::new(3);
let eip4844_fork_epoch = Epoch::new(3);
chain_spec.altair_fork_epoch = Some(altair_fork_epoch);
chain_spec.bellatrix_fork_epoch = Some(merge_fork_epoch);
chain_spec.capella_fork_epoch = Some(capella_fork_epoch);
chain_spec.eip4844_fork_epoch = Some(eip4844_fork_epoch);
let current_slot = match fork_name {
ForkName::Base => Slot::new(0),
ForkName::Altair => altair_fork_epoch.start_slot(E::slots_per_epoch()),
ForkName::Merge => merge_fork_epoch.start_slot(E::slots_per_epoch()),
ForkName::Capella => capella_fork_epoch.start_slot(E::slots_per_epoch()),
ForkName::Eip4844 => eip4844_fork_epoch.start_slot(E::slots_per_epoch()),
};
ForkContext::new::<E>(current_slot, Hash256::zero(), &chain_spec)
}

View File

@ -45,7 +45,7 @@ use beacon_chain::{BeaconChain, BeaconChainTypes, GossipVerifiedBlock};
use derivative::Derivative;
use futures::stream::{Stream, StreamExt};
use futures::task::Poll;
use lighthouse_network::rpc::methods::TxBlobsByRangeRequest;
use lighthouse_network::rpc::methods::BlobsByRangeRequest;
use lighthouse_network::{
rpc::{BlocksByRangeRequest, BlocksByRootRequest, StatusMessage},
Client, MessageId, NetworkGlobals, PeerId, PeerRequestId,
@ -66,6 +66,7 @@ use types::{
SignedAggregateAndProof, SignedBeaconBlock, SignedContributionAndProof, SignedVoluntaryExit,
SubnetId, SyncCommitteeMessage, SyncSubnetId,
};
use types::signed_blobs_sidecar::SignedBlobsSidecar;
use work_reprocessing_queue::{
spawn_reprocess_scheduler, QueuedAggregate, QueuedRpcBlock, QueuedUnaggregate, ReadyWork,
};
@ -190,7 +191,7 @@ pub const GOSSIP_ATTESTATION_BATCH: &str = "gossip_attestation_batch";
pub const GOSSIP_AGGREGATE: &str = "gossip_aggregate";
pub const GOSSIP_AGGREGATE_BATCH: &str = "gossip_aggregate_batch";
pub const GOSSIP_BLOCK: &str = "gossip_block";
pub const GOSSIP_BLOB: &str = "gossip_blob";
pub const GOSSIP_BLOBS_SIDECAR: &str = "gossip_blobs_sidecar";
pub const DELAYED_IMPORT_BLOCK: &str = "delayed_import_block";
pub const GOSSIP_VOLUNTARY_EXIT: &str = "gossip_voluntary_exit";
pub const GOSSIP_PROPOSER_SLASHING: &str = "gossip_proposer_slashing";
@ -201,8 +202,8 @@ pub const RPC_BLOCK: &str = "rpc_block";
pub const CHAIN_SEGMENT: &str = "chain_segment";
pub const STATUS_PROCESSING: &str = "status_processing";
pub const BLOCKS_BY_RANGE_REQUEST: &str = "blocks_by_range_request";
pub const TX_BLOBS_BY_RANGE_REQUEST: &str = "tx_blobs_by_range_request";
pub const BLOCKS_BY_ROOTS_REQUEST: &str = "blocks_by_roots_request";
pub const BLOBS_BY_RANGE_REQUEST: &str = "blobs_by_range_request";
pub const UNKNOWN_BLOCK_ATTESTATION: &str = "unknown_block_attestation";
pub const UNKNOWN_BLOCK_AGGREGATE: &str = "unknown_block_aggregate";
@ -408,20 +409,21 @@ impl<T: BeaconChainTypes> WorkEvent<T> {
}
}
pub fn gossip_tx_blob_block(
/// Create a new `Work` event for some blobs sidecar.
pub fn gossip_blobs_sidecar(
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
blob: Box<BlobsSidecar<T::EthSpec>>,
blobs: Arc<SignedBlobsSidecar<T::EthSpec>>,
seen_timestamp: Duration,
) -> Self {
Self {
drop_during_sync: false,
work: Work::GossipBlob {
work: Work::GossipBlobsSidecar {
message_id,
peer_id,
peer_client,
blob,
blobs,
seen_timestamp,
},
}
@ -599,6 +601,21 @@ impl<T: BeaconChainTypes> WorkEvent<T> {
}
}
pub fn blobs_by_range_request(
peer_id: PeerId,
request_id: PeerRequestId,
request: BlobsByRangeRequest,
) -> Self {
Self {
drop_during_sync: false,
work: Work::BlobsByRangeRequest {
peer_id,
request_id,
request,
},
}
}
/// Get a `str` representation of the type of work this `WorkEvent` contains.
pub fn work_type(&self) -> &'static str {
self.work.str_id()
@ -717,6 +734,13 @@ pub enum Work<T: BeaconChainTypes> {
block: Arc<SignedBeaconBlock<T::EthSpec>>,
seen_timestamp: Duration,
},
GossipBlobsSidecar {
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
blobs: Arc<SignedBlobsSidecar<T::EthSpec>>,
seen_timestamp: Duration,
},
GossipBlob {
message_id: MessageId,
peer_id: PeerId,
@ -787,6 +811,11 @@ pub enum Work<T: BeaconChainTypes> {
request_id: PeerRequestId,
request: BlocksByRootRequest,
},
BlobsByRangeRequest {
peer_id: PeerId,
request_id: PeerRequestId,
request: BlobsByRangeRequest,
}
}
impl<T: BeaconChainTypes> Work<T> {
@ -798,7 +827,7 @@ impl<T: BeaconChainTypes> Work<T> {
Work::GossipAggregate { .. } => GOSSIP_AGGREGATE,
Work::GossipAggregateBatch { .. } => GOSSIP_AGGREGATE_BATCH,
Work::GossipBlock { .. } => GOSSIP_BLOCK,
Work::GossipBlob { .. } => GOSSIP_BLOB,
Work::GossipBlobsSidecar { .. } => GOSSIP_BLOBS_SIDECAR,
Work::DelayedImportBlock { .. } => DELAYED_IMPORT_BLOCK,
Work::GossipVoluntaryExit { .. } => GOSSIP_VOLUNTARY_EXIT,
Work::GossipProposerSlashing { .. } => GOSSIP_PROPOSER_SLASHING,
@ -811,6 +840,7 @@ impl<T: BeaconChainTypes> Work<T> {
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,
Work::UnknownBlockAggregate { .. } => UNKNOWN_BLOCK_AGGREGATE,
}
@ -949,13 +979,14 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
let mut chain_segment_queue = FifoQueue::new(MAX_CHAIN_SEGMENT_QUEUE_LEN);
let mut backfill_chain_segment = FifoQueue::new(MAX_CHAIN_SEGMENT_QUEUE_LEN);
let mut gossip_block_queue = FifoQueue::new(MAX_GOSSIP_BLOCK_QUEUE_LEN);
let mut gossip_blob_queue = FifoQueue::new(MAX_GOSSIP_BLOB_QUEUE_LEN);
let mut gossip_blobs_sidecar_queue = FifoQueue::new(MAX_GOSSIP_BLOCK_QUEUE_LEN);
let mut delayed_block_queue = FifoQueue::new(MAX_DELAYED_BLOCK_QUEUE_LEN);
let mut status_queue = FifoQueue::new(MAX_STATUS_QUEUE_LEN);
let mut bbrange_queue = FifoQueue::new(MAX_BLOCKS_BY_RANGE_QUEUE_LEN);
let mut txbbrange_queue = FifoQueue::new(MAX_TX_BLOBS_BY_RANGE_QUEUE_LEN);
let mut bbroots_queue = FifoQueue::new(MAX_BLOCKS_BY_ROOTS_QUEUE_LEN);
let mut blbrange_queue = FifoQueue::new(MAX_BLOCKS_BY_ROOTS_QUEUE_LEN);
// Channels for sending work to the re-process scheduler (`work_reprocessing_tx`) and to
// receive them back once they are ready (`ready_work_rx`).
@ -1267,8 +1298,8 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
Work::GossipBlock { .. } => {
gossip_block_queue.push(work, work_id, &self.log)
}
Work::GossipBlob { .. } => {
gossip_blob_queue.push(work, work_id, &self.log)
Work::GossipBlobsSidecar { .. } => {
gossip_blobs_sidecar_queue.push(work, work_id, &self.log)
}
Work::DelayedImportBlock { .. } => {
delayed_block_queue.push(work, work_id, &self.log)
@ -1306,6 +1337,9 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
Work::BlocksByRootsRequest { .. } => {
bbroots_queue.push(work, work_id, &self.log)
}
Work::BlobsByRangeRequest { .. } => {
blbrange_queue.push(work, work_id, &self.log)
}
Work::UnknownBlockAttestation { .. } => {
unknown_block_attestation_queue.push(work)
}
@ -1526,6 +1560,28 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
)
.await
}),
/*
* Verification for blobs sidecars received on gossip.
*/
Work::GossipBlobsSidecar {
message_id,
peer_id,
peer_client,
blobs,
seen_timestamp,
} => task_spawner.spawn_async(async move {
worker
.process_gossip_blobs_sidecar(
message_id,
peer_id,
peer_client,
blobs,
work_reprocessing_tx,
duplicate_cache,
seen_timestamp,
)
.await
}),
/*
* Import for blocks that we received earlier than their intended slot.
*/
@ -1664,6 +1720,21 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
request,
)
}),
Work::BlobsByRangeRequest {
peer_id,
request_id,
request
} => task_spawner.spawn_blocking_with_manual_send_idle(move |send_idle_on_drop| {
worker.handle_blobs_by_range_request(
sub_executor,
send_idle_on_drop,
peer_id,
request_id,
request,
)
}),
Work::UnknownBlockAttestation {
message_id,
peer_id,

View File

@ -22,6 +22,7 @@ use types::{
ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedContributionAndProof,
SignedVoluntaryExit, Slot, SubnetId, SyncCommitteeMessage, SyncSubnetId,
};
use types::signed_blobs_sidecar::SignedBlobsSidecar;
use super::{
super::work_reprocessing_queue::{
@ -672,6 +673,7 @@ impl<T: BeaconChainTypes> Worker<T> {
.await
{
let block_root = gossip_verified_block.block_root;
if let Some(handle) = duplicate_cache.check_and_insert(block_root) {
self.process_gossip_verified_block(
peer_id,
@ -784,6 +786,9 @@ impl<T: BeaconChainTypes> Worker<T> {
verified_block
}
Err(BlockError::MissingSidecar) => {
todo!(); //is relevant?
}
Err(BlockError::ParentUnknown(block)) => {
debug!(
self.log,
@ -946,6 +951,22 @@ 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)
@ -954,31 +975,31 @@ impl<T: BeaconChainTypes> Worker<T> {
Ok(block_root) => {
metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL);
if reprocess_tx
.try_send(ReprocessQueueMessage::BlockImported(block_root))
.is_err()
{
error!(
if reprocess_tx
.try_send(ReprocessQueueMessage::BlockImported(block_root))
.is_err()
{
error!(
self.log,
"Failed to inform block import";
"source" => "gossip",
"block_root" => ?block_root,
)
};
};
debug!(
debug!(
self.log,
"Gossipsub block processed";
"block" => ?block_root,
"peer_id" => %peer_id
);
self.chain.recompute_head_at_current_slot().await;
}
Err(BlockError::ParentUnknown { .. }) => {
// Inform the sync manager to find parents for this block
// This should not occur. It should be checked by `should_forward_block`
error!(
self.chain.recompute_head_at_current_slot().await;
}
Err(BlockError::ParentUnknown { .. }) => {
// Inform the sync manager to find parents for this block
// This should not occur. It should be checked by `should_forward_block`
error!(
self.log,
"Block with unknown parent attempted to be processed";
"peer_id" => %peer_id
@ -991,27 +1012,31 @@ impl<T: BeaconChainTypes> Worker<T> {
"Failed to verify execution payload";
"error" => %e
);
}
other => {
debug!(
}
other => {
debug!(
self.log,
"Invalid gossip beacon block";
"outcome" => ?other,
"block root" => ?block_root,
"block slot" => block.slot()
);
self.gossip_penalize_peer(
peer_id,
PeerAction::MidToleranceError,
"bad_gossip_block_ssz",
);
trace!(
self.gossip_penalize_peer(
peer_id,
PeerAction::MidToleranceError,
"bad_gossip_block_ssz",
);
trace!(
self.log,
"Invalid gossip beacon block ssz";
"ssz" => format_args!("0x{}", hex::encode(block.as_ssz_bytes())),
);
}
};
}
};
} else {
*self.chain.sidecar_waiting_for_block.lock() = Some(blobs);
}
}
pub fn process_gossip_voluntary_exit(

View File

@ -6,13 +6,14 @@ 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::{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;
use types::{Epoch, EthSpec, Hash256, Slot};
use types::{Epoch, EthSpec, Hash256, Slot, VariableList};
use super::Worker;
@ -382,4 +383,150 @@ impl<T: BeaconChainTypes> Worker<T> {
"load_blocks_by_range_blocks",
);
}
/// Handle a `BlobsByRange` request from the peer.
pub fn handle_blobs_by_range_request(
self,
executor: TaskExecutor,
send_on_drop: SendOnDrop,
peer_id: PeerId,
request_id: PeerRequestId,
mut req: BlobsByRangeRequest,
) {
debug!(self.log, "Received BlobsByRange Request";
"peer_id" => %peer_id,
"count" => req.count,
"start_slot" => req.start_slot,
);
// Should not send more than max request blocks
if req.count > MAX_REQUEST_BLOCKS {
req.count = MAX_REQUEST_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

@ -165,12 +165,12 @@ impl<T: BeaconChainTypes> Router<T> {
Request::BlocksByRange(request) => self
.processor
.on_blocks_by_range_request(peer_id, id, request),
Request::TxBlobsByRange(request) => self
.processor
.on_tx_blobs_by_range_request(peer_id, id, request),
Request::BlocksByRoot(request) => self
.processor
.on_blocks_by_root_request(peer_id, id, request),
Request::BlobsByRange(request) => self
.processor
.on_blobs_by_range_request(peer_id, id, request),
}
}
@ -191,14 +191,14 @@ impl<T: BeaconChainTypes> Router<T> {
self.processor
.on_blocks_by_range_response(peer_id, request_id, beacon_block);
}
Response::TxBlobsByRange(blob_wrapper) => {
self.processor
.on_tx_blobs_by_range_response(peer_id, request_id, blob_wrapper);
}
Response::BlocksByRoot(beacon_block) => {
self.processor
.on_blocks_by_root_response(peer_id, request_id, beacon_block);
}
Response::BlobsByRange(beacon_blob) => {
self.processor
.on_blobs_by_range_response(peer_id, request_id, beacon_blob);
}
}
}
@ -236,12 +236,12 @@ impl<T: BeaconChainTypes> Router<T> {
block,
);
}
PubsubMessage::Blob(blob) => {
self.processor.on_tx_blob_gossip(
PubsubMessage::BlobsSidecars(blobs) => {
self.processor.on_blobs_gossip(
id,
peer_id,
self.network_globals.client(&peer_id),
blob,
blobs,
);
}
PubsubMessage::VoluntaryExit(exit) => {

View File

@ -8,6 +8,7 @@ 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::{
Client, MessageId, NetworkGlobals, PeerId, PeerRequestId, Request, Response,
};
@ -22,6 +23,7 @@ use types::{
SignedAggregateAndProof, SignedBeaconBlock, SignedContributionAndProof, SignedVoluntaryExit,
SubnetId, SyncSubnetId,
};
use types::signed_blobs_sidecar::SignedBlobsSidecar;
/// Processes validated messages from the network. It relays necessary data to the syncing thread
/// and processes blocks from the pubsub network.
@ -162,6 +164,16 @@ impl<T: BeaconChainTypes> Processor<T> {
))
}
pub fn on_blobs_by_range_request(
&mut self,
peer_id: PeerId,
request_id: PeerRequestId,
request: BlobsByRangeRequest,
) {
self.send_beacon_processor_work(BeaconWorkEvent::blobs_by_range_request(
peer_id, request_id, request,
))
}
/// Handle a `BlocksByRange` request from the peer.
pub fn on_blocks_by_range_request(
&mut self,
@ -274,6 +286,15 @@ 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.
@ -295,18 +316,18 @@ impl<T: BeaconChainTypes> Processor<T> {
))
}
pub fn on_tx_blob_gossip(
pub fn on_blobs_gossip(
&mut self,
message_id: MessageId,
peer_id: PeerId,
peer_client: Client,
blob: Box<BlobsSidecar<T::EthSpec>>,
blobs: Arc<SignedBlobsSidecar<T::EthSpec>>,
) {
self.send_beacon_processor_work(BeaconWorkEvent::gossip_tx_blob_block(
self.send_beacon_processor_work(BeaconWorkEvent::gossip_blobs_sidecar(
message_id,
peer_id,
peer_client,
blob,
blobs,
timestamp_now(),
))
}

View File

@ -7,6 +7,7 @@ use types::{EthSpec, MinimalEthSpec};
pub const PREV_DEFAULT_SLOTS_PER_RESTORE_POINT: u64 = 2048;
pub const DEFAULT_SLOTS_PER_RESTORE_POINT: u64 = 8192;
pub const DEFAULT_BLOCK_CACHE_SIZE: usize = 5;
pub const DEFAULT_BLOB_CACHE_SIZE: usize = 5;
/// Database configuration parameters.
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)]
@ -17,6 +18,8 @@ pub struct StoreConfig {
pub slots_per_restore_point_set_explicitly: bool,
/// Maximum number of blocks to store in the in-memory block cache.
pub block_cache_size: usize,
/// Maximum number of blobs to store in the in-memory block cache.
pub blob_cache_size: usize,
/// Whether to compact the database on initialization.
pub compact_on_init: bool,
/// Whether to compact the database during database pruning.
@ -43,6 +46,7 @@ impl Default for StoreConfig {
slots_per_restore_point: MinimalEthSpec::slots_per_historical_root() as u64,
slots_per_restore_point_set_explicitly: false,
block_cache_size: DEFAULT_BLOCK_CACHE_SIZE,
blob_cache_size: DEFAULT_BLOB_CACHE_SIZE,
compact_on_init: false,
compact_on_prune: true,
prune_payloads: true,

View File

@ -39,6 +39,7 @@ use std::path::Path;
use std::sync::Arc;
use std::time::Duration;
use types::*;
use types::signed_blobs_sidecar::SignedBlobsSidecar;
/// On-disk database that stores finalized states efficiently.
///
@ -60,6 +61,8 @@ pub struct HotColdDB<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> {
///
/// The hot database also contains all blocks.
pub hot_db: Hot,
/// LRU cache of deserialized blobs. Updated whenever a blob is loaded.
blob_cache: Mutex<LruCache<Hash256, SignedBlobsSidecar<E>>>,
/// LRU cache of deserialized blocks. Updated whenever a block is loaded.
block_cache: Mutex<LruCache<Hash256, SignedBeaconBlock<E>>>,
/// Chain spec.
@ -129,6 +132,7 @@ impl<E: EthSpec> HotColdDB<E, MemoryStore<E>, MemoryStore<E>> {
cold_db: MemoryStore::open(),
hot_db: MemoryStore::open(),
block_cache: Mutex::new(LruCache::new(config.block_cache_size)),
blob_cache: Mutex::new(LruCache::new(config.blob_cache_size)),
config,
spec,
log,
@ -162,6 +166,7 @@ impl<E: EthSpec> HotColdDB<E, LevelDB<E>, LevelDB<E>> {
cold_db: LevelDB::open(cold_path)?,
hot_db: LevelDB::open(hot_path)?,
block_cache: Mutex::new(LruCache::new(config.block_cache_size)),
blob_cache: Mutex::new(LruCache::new(config.blob_cache_size)),
config,
spec,
log,
@ -475,6 +480,44 @@ 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>,
) -> Result<(), Error> {
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> {
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())? {
let ret = SignedBlobsSidecar::from_ssz_bytes(&bytes)?;
self.blob_cache.lock().put(*block_root, ret.clone());
Ok(Some(ret))
} else {
Ok(None)
}
}
}
pub fn blobs_as_kv_store_ops(
&self,
key: &Hash256,
blobs: &SignedBlobsSidecar<E>,
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(),
));
}
pub fn put_state_summary(
&self,
state_root: &Hash256,
@ -702,6 +745,14 @@ impl<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> HotColdDB<E, Hot, Cold>
self.store_hot_state(&state_root, state, &mut key_value_batch)?;
}
StoreOp::PutBlobs(block_root, blobs) => {
self.blobs_as_kv_store_ops(
&block_root,
&blobs,
&mut key_value_batch,
);
}
StoreOp::PutStateSummary(state_root, summary) => {
key_value_batch.push(summary.as_kv_store_op(state_root));
}
@ -746,6 +797,7 @@ impl<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> HotColdDB<E, Hot, Cold>
// Update the block cache whilst holding a lock, to ensure that the cache updates atomically
// with the database.
let mut guard = self.block_cache.lock();
let mut guard_blob = self.blob_cache.lock();
for op in &batch {
match op {
@ -753,6 +805,10 @@ impl<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> HotColdDB<E, Hot, Cold>
guard.put(*block_root, (**block).clone());
}
StoreOp::PutBlobs(block_root, blobs) => {
guard_blob.put(*block_root, (**blobs).clone());
}
StoreOp::PutState(_, _) => (),
StoreOp::PutStateSummary(_, _) => (),

View File

@ -42,6 +42,7 @@ use parking_lot::MutexGuard;
use std::sync::Arc;
use strum::{EnumString, IntoStaticStr};
pub use types::*;
use types::signed_blobs_sidecar::SignedBlobsSidecar;
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>;
@ -155,6 +156,7 @@ pub trait ItemStore<E: EthSpec>: KeyValueStore<E> + Sync + Send + Sized + 'stati
pub enum StoreOp<'a, E: EthSpec> {
PutBlock(Hash256, Arc<SignedBeaconBlock<E>>),
PutState(Hash256, &'a BeaconState<E>),
PutBlobs(Hash256, Arc<SignedBlobsSidecar<E>>),
PutStateSummary(Hash256, HotStateSummary),
PutStateTemporaryFlag(Hash256),
DeleteStateTemporaryFlag(Hash256),
@ -171,6 +173,8 @@ pub enum DBColumn {
BeaconMeta,
#[strum(serialize = "blk")]
BeaconBlock,
#[strum(serialize = "blo")]
BeaconBlob,
/// For full `BeaconState`s in the hot database (finalized or fork-boundary states).
#[strum(serialize = "ste")]
BeaconState,

View File

@ -14,7 +14,7 @@ use types::*;
///
/// Utilises lazy-loading from separate storage for its vector fields.
#[superstruct(
variants(Base, Altair, Merge, Capella),
variants(Base, Altair, Merge, Eip4844),
variant_attributes(derive(Debug, PartialEq, Clone, Encode, Decode))
)]
#[derive(Debug, PartialEq, Clone, Encode)]
@ -66,9 +66,9 @@ where
pub current_epoch_attestations: VariableList<PendingAttestation<T>, T::MaxPendingAttestations>,
// Participation (Altair and later)
#[superstruct(only(Altair, Merge, Capella))]
#[superstruct(only(Altair, Merge, Eip4844))]
pub previous_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
#[superstruct(only(Altair, Merge, Capella))]
#[superstruct(only(Altair, Merge, Eip4844))]
pub current_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
// Finality
@ -78,17 +78,17 @@ where
pub finalized_checkpoint: Checkpoint,
// Inactivity
#[superstruct(only(Altair, Merge, Capella))]
#[superstruct(only(Altair, Merge, Eip4844))]
pub inactivity_scores: VariableList<u64, T::ValidatorRegistryLimit>,
// Light-client sync committees
#[superstruct(only(Altair, Merge, Capella))]
#[superstruct(only(Altair, Merge, Eip4844))]
pub current_sync_committee: Arc<SyncCommittee<T>>,
#[superstruct(only(Altair, Merge, Capella))]
#[superstruct(only(Altair, Merge, Eip4844))]
pub next_sync_committee: Arc<SyncCommittee<T>>,
// Execution
#[superstruct(only(Merge, Capella))]
#[superstruct(only(Merge, Eip4844))]
pub latest_execution_payload_header: ExecutionPayloadHeader<T>,
}
@ -178,11 +178,11 @@ impl<T: EthSpec> PartialBeaconState<T> {
latest_execution_payload_header
]
),
BeaconState::Capella(s) => impl_from_state_forgetful!(
BeaconState::Eip4844(s) => impl_from_state_forgetful!(
s,
outer,
Capella,
PartialBeaconStateCapella,
Eip4844,
PartialBeaconStateEip4844,
[
previous_epoch_participation,
current_epoch_participation,
@ -379,10 +379,10 @@ impl<E: EthSpec> TryInto<BeaconState<E>> for PartialBeaconState<E> {
latest_execution_payload_header
]
),
PartialBeaconState::Capella(inner) => impl_try_into_beacon_state!(
PartialBeaconState::Eip4844(inner) => impl_try_into_beacon_state!(
inner,
Capella,
BeaconStateCapella,
Eip4844,
BeaconStateEip4844,
[
previous_epoch_participation,
current_epoch_participation,

View File

@ -307,5 +307,10 @@ define_hardcoded_nets!(
// Set to `true` if the genesis state can be found in the `built_in_network_configs`
// directory.
GENESIS_STATE_IS_KNOWN
),
(
eip4844,
"eip4844",
GENESIS_STATE_IS_KNOWN
)
);

View File

@ -0,0 +1,3 @@
- enr:-MK4QLij8YaVQ6fIi09rDuD9fufxBlCZRXwfM1q6SbNJfy5ZZdAvtlnsfqhIeI0IqeOZdaPExVCfZfR4JJTIuKXFR76GAYJGrqHnh2F0dG5ldHOIAAAAAAAAAACEZXRoMpBCynldgwAP_QMAAAAAAAAAgmlkgnY0gmlwhCJ7uEyJc2VjcDI1NmsxoQJpeftU6RbmIhcFllICznlAMJXL3EwHEGhn73_Gk0wrCYhzeW5jbmV0cwCDdGNwgjLIg3VkcIIu4A
- enr:-JG4QK27MZvV3QbwdLt055Yhei27SjAsDXMFGCdl-Q7SDiCgR_qbiW3BmcOClehFVJgMa6IfjHeJBdbC0jvrr2NycOqGAYJLWb5kgmlkgnY0gmlwhCJE_eeJc2VjcDI1NmsxoQIecO7Y9C7J2Bs7RNxXaUkU6BfmPKIhEsDScKAoxENaRYN0Y3CCdl-DdWRwgnZf
- enr:-JG4QExcHW3vzBcE0f_r-93nSA4iBy4qNLthSyTw7p0tlPwjMl1JVTAgLSNHLLZJzOGtelJO4sw37LliuHyJ55zN5J6GAYJLWTvzgmlkgnY0gmlwhCKq1cmJc2VjcDI1NmsxoQJT2d4jtKQbHNw3tZPLhoMlR73o5LNdi-bk_bYq6siwuIN0Y3CCdl-DdWRwgnZf

View File

@ -0,0 +1,85 @@
# Prater config
# Extends the mainnet preset
CONFIG_NAME: 'eip4844'
PRESET_BASE: 'mainnet'
# Transition
# ---------------------------------------------------------------
TERMINAL_TOTAL_DIFFICULTY: 40
# By default, don't use these params
TERMINAL_BLOCK_HASH: 0x0000000000000000000000000000000000000000000000000000000000000000
TERMINAL_BLOCK_HASH_ACTIVATION_EPOCH: 18446744073709551615
# Genesis
# ---------------------------------------------------------------
# `2**14` (= 16,384)
MIN_GENESIS_ACTIVE_VALIDATOR_COUNT: 2
# Mar-01-2021 08:53:32 AM +UTC
MIN_GENESIS_TIME: 1653318000
# Prater area code (Vienna)
GENESIS_FORK_VERSION: 0x00000ffd
# Customized for Prater: 1919188 seconds (Mar-23-2021 02:00:00 PM +UTC)
GENESIS_DELAY: 0
# Forking
# ---------------------------------------------------------------
# Some forks are disabled for now:
# - These may be re-assigned to another fork-version later
# - Temporarily set to max uint64 value: 2**64 - 1
# Altair
ALTAIR_FORK_VERSION: 0x01000ffd
ALTAIR_FORK_EPOCH: 1
# Merge
BELLATRIX_FORK_VERSION: 0x02000ffd
BELLATRIX_FORK_EPOCH: 2
# Sharding
EIP4844_FORK_VERSION: 0x83000ffd
EIP4844_FORK_EPOCH: 3
# TBD, 2**32 is a placeholder. Merge transition approach is in active R&D.
TRANSITION_TOTAL_DIFFICULTY: 40
# Time parameters
# ---------------------------------------------------------------
# 12 seconds
SECONDS_PER_SLOT: 12
# 14 (estimate from Eth1 mainnet)
SECONDS_PER_ETH1_BLOCK: 14
# 2**8 (= 256) epochs ~27 hours
MIN_VALIDATOR_WITHDRAWABILITY_DELAY: 256
# 2**8 (= 256) epochs ~27 hours
SHARD_COMMITTEE_PERIOD: 256
# 2**11 (= 2,048) Eth1 blocks ~8 hours
ETH1_FOLLOW_DISTANCE: 15
# Validator cycle
# ---------------------------------------------------------------
# 2**2 (= 4)
INACTIVITY_SCORE_BIAS: 4
# 2**4 (= 16)
INACTIVITY_SCORE_RECOVERY_RATE: 16
# 2**4 * 10**9 (= 16,000,000,000) Gwei
EJECTION_BALANCE: 16000000000
# 2**2 (= 4)
MIN_PER_EPOCH_CHURN_LIMIT: 4
# 2**16 (= 65,536)
CHURN_LIMIT_QUOTIENT: 65536
# Fork choice
# ---------------------------------------------------------------
# 40%
PROPOSER_SCORE_BOOST: 40
# Deposit contract
# ---------------------------------------------------------------
# Ethereum Goerli testnet
DEPOSIT_CHAIN_ID: 1331
DEPOSIT_NETWORK_ID: 69
# Prater test deposit contract on Goerli Testnet
DEPOSIT_CONTRACT_ADDRESS: 0x8A04d14125D0FDCDc742F4A05C051De07232EDa4

View File

@ -28,9 +28,9 @@ TERMINAL_TOTAL_DIFFICULTY: 17000000000000000
TERMINAL_BLOCK_HASH: 0x0000000000000000000000000000000000000000000000000000000000000000
TERMINAL_BLOCK_HASH_ACTIVATION_EPOCH: 18446744073709551615
# Capella
CAPELLA_FORK_VERSION: 0x03001020
CAPELLA_FORK_EPOCH: 18446744073709551615
# Eip4844
EIP4844_FORK_VERSION: 0x03001020
EIP4844_FORK_EPOCH: 18446744073709551615
# Sharding
SHARDING_FORK_VERSION: 0x04001020

View File

@ -777,7 +777,7 @@ where
(parent_justified, parent_finalized)
} else {
let justification_and_finalization_state = match block {
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

@ -45,7 +45,7 @@ pub fn slash_validator<T: EthSpec>(
validator_effective_balance.safe_div(spec.whistleblower_reward_quotient)?;
let proposer_reward = match state {
BeaconState::Base(_) => whistleblower_reward.safe_div(spec.proposer_reward_quotient)?,
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
whistleblower_reward
.safe_mul(PROPOSER_WEIGHT)?
.safe_div(WEIGHT_DENOMINATOR)?

View File

@ -232,7 +232,7 @@ pub fn process_attestations<'a, T: EthSpec, Payload: ExecPayload<T>>(
}
BeaconBlockBodyRef::Altair(_)
| BeaconBlockBodyRef::Merge(_)
| BeaconBlockBodyRef::Capella(_) => {
| BeaconBlockBodyRef::Eip4844(_) => {
altair::process_attestations(
state,
block_body.attestations(),

View File

@ -37,7 +37,7 @@ pub fn process_epoch<T: EthSpec>(
match state {
BeaconState::Base(_) => base::process_epoch(state, spec),
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
altair::process_epoch(state, spec)
}
}

View File

@ -1,6 +1,6 @@
use crate::beacon_block_body::{
BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyCapella, BeaconBlockBodyMerge,
BeaconBlockBodyRef, BeaconBlockBodyRefMut,
BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyMerge, BeaconBlockBodyRef,
BeaconBlockBodyRefMut, BeaconBlockBodyEip4844
};
use crate::test_utils::TestRandom;
use crate::*;
@ -17,7 +17,7 @@ use tree_hash_derive::TreeHash;
/// A block of the `BeaconChain`.
#[superstruct(
variants(Base, Altair, Merge, Capella),
variants(Base, Altair, Merge, Eip4844),
variant_attributes(
derive(
Debug,
@ -64,8 +64,8 @@ pub struct BeaconBlock<T: EthSpec, Payload: ExecPayload<T> = FullPayload<T>> {
pub body: BeaconBlockBodyAltair<T, Payload>,
#[superstruct(only(Merge), partial_getter(rename = "body_merge"))]
pub body: BeaconBlockBodyMerge<T, Payload>,
#[superstruct(only(Capella), partial_getter(rename = "body_capella"))]
pub body: BeaconBlockBodyCapella<T, Payload>,
#[superstruct(only(Eip4844), partial_getter(rename = "body_eip4844"))]
pub body: BeaconBlockBodyEip4844<T, Payload>,
}
pub type BlindedBeaconBlock<E> = BeaconBlock<E, BlindedPayload<E>>;
@ -191,7 +191,7 @@ impl<'a, T: EthSpec, Payload: ExecPayload<T>> BeaconBlockRef<'a, T, Payload> {
BeaconBlockRef::Base { .. } => ForkName::Base,
BeaconBlockRef::Altair { .. } => ForkName::Altair,
BeaconBlockRef::Merge { .. } => ForkName::Merge,
BeaconBlockRef::Capella { .. } => ForkName::Capella,
BeaconBlockRef::Eip4844 { .. } => ForkName::Eip4844,
};
if fork_at_slot == object_fork {
@ -545,6 +545,7 @@ macro_rules! impl_from {
impl_from!(BeaconBlockBase, <E, FullPayload<E>>, <E, BlindedPayload<E>>, |body: BeaconBlockBodyBase<_, _>| body.into());
impl_from!(BeaconBlockAltair, <E, FullPayload<E>>, <E, BlindedPayload<E>>, |body: BeaconBlockBodyAltair<_, _>| body.into());
impl_from!(BeaconBlockMerge, <E, FullPayload<E>>, <E, BlindedPayload<E>>, |body: BeaconBlockBodyMerge<_, _>| body.into());
impl_from!(BeaconBlockEip4844, <E, FullPayload<E>>, <E, BlindedPayload<E>>, |body: BeaconBlockBodyEip4844<_, _>| body.into());
// We can clone blocks with payloads to blocks without payloads, without cloning the payload.
macro_rules! impl_clone_as_blinded {
@ -575,6 +576,7 @@ macro_rules! impl_clone_as_blinded {
impl_clone_as_blinded!(BeaconBlockBase, <E, FullPayload<E>>, <E, BlindedPayload<E>>);
impl_clone_as_blinded!(BeaconBlockAltair, <E, FullPayload<E>>, <E, BlindedPayload<E>>);
impl_clone_as_blinded!(BeaconBlockMerge, <E, FullPayload<E>>, <E, BlindedPayload<E>>);
impl_clone_as_blinded!(BeaconBlockEip4844, <E, FullPayload<E>>, <E, BlindedPayload<E>>);
// A reference to a full beacon block can be cloned into a blinded beacon block, without cloning the
// execution payload.

View File

@ -8,12 +8,13 @@ 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.
///
/// This *superstruct* abstracts over the hard-fork.
#[superstruct(
variants(Base, Altair, Merge, Capella),
variants(Base, Altair, Merge, Eip4844),
variant_attributes(
derive(
Debug,
@ -47,16 +48,16 @@ pub struct BeaconBlockBody<T: EthSpec, Payload: ExecPayload<T> = FullPayload<T>>
pub attestations: VariableList<Attestation<T>, T::MaxAttestations>,
pub deposits: VariableList<Deposit, T::MaxDeposits>,
pub voluntary_exits: VariableList<SignedVoluntaryExit, T::MaxVoluntaryExits>,
#[superstruct(only(Altair, Merge, Capella))]
#[superstruct(only(Altair, Merge, Eip4844))]
pub sync_aggregate: SyncAggregate<T>,
// We flatten the execution payload so that serde can use the name of the inner type,
// either `execution_payload` for full payloads, or `execution_payload_header` for blinded
// payloads.
#[superstruct(only(Merge, Capella))]
#[superstruct(only(Merge, Eip4844))]
#[serde(flatten)]
pub execution_payload: Payload,
#[superstruct(only(Capella))]
pub blob_kzgs: VariableList<KZGCommitment, T::MaxBlobsPerBlock>,
#[superstruct(only(Eip4844))]
pub blob_kzg_commitments: VariableList<KzgCommitment, T::MaxBlobsPerBlock>,
#[superstruct(only(Base, Altair))]
#[ssz(skip_serializing, skip_deserializing)]
#[tree_hash(skip_hashing)]
@ -71,7 +72,7 @@ impl<'a, T: EthSpec> BeaconBlockBodyRef<'a, T> {
BeaconBlockBodyRef::Base { .. } => ForkName::Base,
BeaconBlockBodyRef::Altair { .. } => ForkName::Altair,
BeaconBlockBodyRef::Merge { .. } => ForkName::Merge,
BeaconBlockBodyRef::Capella { .. } => ForkName::Capella,
BeaconBlockBodyRef::Eip4844 { .. } => ForkName::Eip4844,
}
}
}
@ -254,6 +255,48 @@ impl<E: EthSpec> From<BeaconBlockBodyMerge<E, FullPayload<E>>>
}
}
impl<E: EthSpec> From<BeaconBlockBodyEip4844<E, FullPayload<E>>>
for (
BeaconBlockBodyEip4844<E, BlindedPayload<E>>,
Option<ExecutionPayload<E>>,
)
{
fn from(body: BeaconBlockBodyEip4844<E, FullPayload<E>>) -> Self {
let BeaconBlockBodyEip4844 {
randao_reveal,
eth1_data,
graffiti,
proposer_slashings,
attester_slashings,
attestations,
deposits,
voluntary_exits,
sync_aggregate,
execution_payload: FullPayload { execution_payload},
blob_kzg_commitments,
} = body;
(
BeaconBlockBodyEip4844 {
randao_reveal,
eth1_data,
graffiti,
proposer_slashings,
attester_slashings,
attestations,
deposits,
voluntary_exits,
sync_aggregate,
execution_payload: BlindedPayload {
execution_payload_header: From::from(&execution_payload),
},
blob_kzg_commitments,
},
None,
)
}
}
// We can clone a full block into a blinded block, without cloning the payload.
impl<E: EthSpec> BeaconBlockBodyBase<E, FullPayload<E>> {
pub fn clone_as_blinded(&self) -> BeaconBlockBodyBase<E, BlindedPayload<E>> {
@ -301,6 +344,40 @@ impl<E: EthSpec> BeaconBlockBodyMerge<E, FullPayload<E>> {
}
}
impl<E: EthSpec> BeaconBlockBodyEip4844<E, FullPayload<E>> {
pub fn clone_as_blinded(&self) -> BeaconBlockBodyEip4844<E, BlindedPayload<E>> {
let BeaconBlockBodyEip4844 {
randao_reveal,
eth1_data,
graffiti,
proposer_slashings,
attester_slashings,
attestations,
deposits,
voluntary_exits,
sync_aggregate,
execution_payload: FullPayload { execution_payload },
blob_kzg_commitments,
} = self;
BeaconBlockBodyEip4844 {
randao_reveal: randao_reveal.clone(),
eth1_data: eth1_data.clone(),
graffiti: *graffiti,
proposer_slashings: proposer_slashings.clone(),
attester_slashings: attester_slashings.clone(),
attestations: attestations.clone(),
deposits: deposits.clone(),
voluntary_exits: voluntary_exits.clone(),
sync_aggregate: sync_aggregate.clone(),
execution_payload: BlindedPayload {
execution_payload_header: From::from(execution_payload),
},
blob_kzg_commitments: blob_kzg_commitments.clone(),
}
}
}
impl<E: EthSpec> From<BeaconBlockBody<E, FullPayload<E>>>
for (
BeaconBlockBody<E, BlindedPayload<E>>,

View File

@ -172,7 +172,7 @@ impl From<BeaconStateHash> for Hash256 {
/// The state of the `BeaconChain` at some slot.
#[superstruct(
variants(Base, Altair, Merge, Capella),
variants(Base, Altair, Merge, Eip4844),
variant_attributes(
derive(
Derivative,
@ -250,9 +250,9 @@ where
pub current_epoch_attestations: VariableList<PendingAttestation<T>, T::MaxPendingAttestations>,
// Participation (Altair and later)
#[superstruct(only(Altair, Merge, Capella))]
#[superstruct(only(Altair, Merge, Eip4844))]
pub previous_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
#[superstruct(only(Altair, Merge, Capella))]
#[superstruct(only(Altair, Merge, Eip4844))]
pub current_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
// Finality
@ -267,17 +267,17 @@ where
// Inactivity
#[serde(with = "ssz_types::serde_utils::quoted_u64_var_list")]
#[superstruct(only(Altair, Merge, Capella))]
#[superstruct(only(Altair, Merge, Eip4844))]
pub inactivity_scores: VariableList<u64, T::ValidatorRegistryLimit>,
// Light-client sync committees
#[superstruct(only(Altair, Merge, Capella))]
#[superstruct(only(Altair, Merge, Eip4844))]
pub current_sync_committee: Arc<SyncCommittee<T>>,
#[superstruct(only(Altair, Merge, Capella))]
#[superstruct(only(Altair, Merge, Eip4844))]
pub next_sync_committee: Arc<SyncCommittee<T>>,
// Execution
#[superstruct(only(Merge, Capella))]
#[superstruct(only(Merge, Eip4844))]
pub latest_execution_payload_header: ExecutionPayloadHeader<T>,
// Caching (not in the spec)
@ -389,7 +389,7 @@ impl<T: EthSpec> BeaconState<T> {
BeaconState::Base { .. } => ForkName::Base,
BeaconState::Altair { .. } => ForkName::Altair,
BeaconState::Merge { .. } => ForkName::Merge,
BeaconState::Capella { .. } => ForkName::Capella,
BeaconState::Eip4844 { .. } => ForkName::Eip4844,
};
if fork_at_slot == object_fork {
@ -1103,7 +1103,7 @@ impl<T: EthSpec> BeaconState<T> {
BeaconState::Base(state) => (&mut state.validators, &mut state.balances),
BeaconState::Altair(state) => (&mut state.validators, &mut state.balances),
BeaconState::Merge(state) => (&mut state.validators, &mut state.balances),
BeaconState::Capella(state) => (&mut state.validators, &mut state.balances),
BeaconState::Eip4844(state) => (&mut state.validators, &mut state.balances),
}
}
@ -1300,14 +1300,14 @@ impl<T: EthSpec> BeaconState<T> {
BeaconState::Base(_) => Err(BeaconStateError::IncorrectStateVariant),
BeaconState::Altair(state) => Ok(&mut state.current_epoch_participation),
BeaconState::Merge(state) => Ok(&mut state.current_epoch_participation),
BeaconState::Capella(state) => Ok(&mut state.current_epoch_participation),
BeaconState::Eip4844(state) => Ok(&mut state.current_epoch_participation),
}
} else if epoch == self.previous_epoch() {
match self {
BeaconState::Base(_) => Err(BeaconStateError::IncorrectStateVariant),
BeaconState::Altair(state) => Ok(&mut state.previous_epoch_participation),
BeaconState::Merge(state) => Ok(&mut state.previous_epoch_participation),
BeaconState::Capella(state) => Ok(&mut state.previous_epoch_participation),
BeaconState::Eip4844(state) => Ok(&mut state.previous_epoch_participation),
}
} else {
Err(BeaconStateError::EpochOutOfBounds)
@ -1612,7 +1612,7 @@ impl<T: EthSpec> BeaconState<T> {
BeaconState::Base(inner) => BeaconState::Base(inner.clone()),
BeaconState::Altair(inner) => BeaconState::Altair(inner.clone()),
BeaconState::Merge(inner) => BeaconState::Merge(inner.clone()),
BeaconState::Capella(inner) => BeaconState::Capella(inner.clone()),
BeaconState::Eip4844(inner) => BeaconState::Eip4844(inner.clone()),
};
if config.committee_caches {
*res.committee_caches_mut() = self.committee_caches().clone();

View File

@ -0,0 +1,74 @@
use ssz_types::VariableList;
use serde::{Deserialize, Deserializer, Serialize, Serializer};
use ssz::{Decode, DecodeError, Encode};
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> {
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);
res.0.push(elem);
}
res
}
}
impl<T: EthSpec> Encode for Blob<T> {
fn is_ssz_fixed_len() -> bool {
<VariableList<BlsFieldElement, T::FieldElementsPerBlob> as Encode>::is_ssz_fixed_len()
}
fn ssz_fixed_len() -> usize {
<VariableList<BlsFieldElement, T::FieldElementsPerBlob> as Encode>::ssz_fixed_len()
}
fn ssz_bytes_len(&self) -> usize {
self.0.ssz_bytes_len()
}
fn ssz_append(&self, buf: &mut Vec<u8>) {
self.0.ssz_append(buf)
}
}
impl<T: EthSpec> Decode for Blob<T> {
fn is_ssz_fixed_len() -> bool {
<VariableList<BlsFieldElement, T::FieldElementsPerBlob> as Decode>::is_ssz_fixed_len()
}
fn ssz_fixed_len() -> usize {
<VariableList<BlsFieldElement, T::FieldElementsPerBlob> as Decode>::ssz_fixed_len()
}
fn from_ssz_bytes(bytes: &[u8]) -> Result<Self, DecodeError> {
<VariableList<BlsFieldElement, T::FieldElementsPerBlob>>::from_ssz_bytes(bytes).map(Self)
}
}
impl<T: EthSpec> TreeHash for Blob<T> {
fn tree_hash_type() -> tree_hash::TreeHashType {
<VariableList<BlsFieldElement, T::FieldElementsPerBlob>>::tree_hash_type()
}
fn tree_hash_packed_encoding(&self) -> PackedEncoding{
self.0.tree_hash_packed_encoding()
}
fn tree_hash_packing_factor() -> usize {
<VariableList<BlsFieldElement, T::FieldElementsPerBlob>>::tree_hash_packing_factor()
}
fn tree_hash_root(&self) -> tree_hash::Hash256 {
self.0.tree_hash_root()
}
}

View File

@ -5,6 +5,7 @@ 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)]
@ -12,6 +13,7 @@ pub struct BlobsSidecar<E: EthSpec> {
pub beacon_block_root: Hash256,
pub beacon_block_slot: Slot,
pub blobs: VariableList<Blob<E::FieldElementsPerBlob>, E::MaxBlobsPerBlock>,
pub kzg_aggregate_proof: KzgProof,
}
impl<E: EthSpec> BlobsSidecar<E> {

View File

@ -0,0 +1,59 @@
use crate::{EthSpec, Uint256};
use serde::{Deserialize, Serialize};
use ssz::{Decode, DecodeError, Encode};
use tree_hash::{PackedEncoding, TreeHash};
#[derive(Default, Debug, PartialEq, Hash, Clone, Copy, Serialize, Deserialize)]
#[serde(transparent)]
pub struct BlsFieldElement(pub Uint256);
impl Encode for BlsFieldElement {
fn is_ssz_fixed_len() -> bool {
<Uint256 as Encode>::is_ssz_fixed_len()
}
fn ssz_fixed_len() -> usize {
<Uint256 as Encode>::ssz_fixed_len()
}
fn ssz_bytes_len(&self) -> usize {
self.0.ssz_bytes_len()
}
fn ssz_append(&self, buf: &mut Vec<u8>) {
self.0.ssz_append(buf)
}
}
impl Decode for BlsFieldElement {
fn is_ssz_fixed_len() -> bool {
<Uint256 as Decode>::is_ssz_fixed_len()
}
fn ssz_fixed_len() -> usize {
<Uint256 as Decode>::ssz_fixed_len()
}
fn from_ssz_bytes(bytes: &[u8]) -> Result<Self, DecodeError> {
<Uint256 as Decode>::from_ssz_bytes(bytes).map(Self)
}
}
impl TreeHash for BlsFieldElement {
fn tree_hash_type() -> tree_hash::TreeHashType {
<Uint256>::tree_hash_type()
}
fn tree_hash_packed_encoding(&self) -> PackedEncoding{
self.0.tree_hash_packed_encoding()
}
fn tree_hash_packing_factor() -> usize {
<Uint256>::tree_hash_packing_factor()
}
fn tree_hash_root(&self) -> tree_hash::Hash256 {
self.0.tree_hash_root()
}
}

View File

@ -153,10 +153,10 @@ pub struct ChainSpec {
pub safe_slots_to_import_optimistically: u64,
/*
* Capella hard fork params
* Eip4844 hard fork params
*/
pub capella_fork_version: [u8; 4],
pub capella_fork_epoch: Option<Epoch>,
pub eip4844_fork_version: [u8; 4],
pub eip4844_fork_epoch: Option<Epoch>,
/*
* Networking
@ -238,16 +238,17 @@ impl ChainSpec {
/// Returns the name of the fork which is active at `epoch`.
pub fn fork_name_at_epoch(&self, epoch: Epoch) -> ForkName {
match self.capella_fork_epoch {
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Capella,
match self.eip4844_fork_epoch {
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Eip4844,
_ => match self.bellatrix_fork_epoch {
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Merge,
_ => match self.altair_fork_epoch {
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Altair,
_ => ForkName::Base,
},
},
}
}
}
/// Returns the fork version for a named fork.
@ -256,7 +257,7 @@ impl ChainSpec {
ForkName::Base => self.genesis_fork_version,
ForkName::Altair => self.altair_fork_version,
ForkName::Merge => self.bellatrix_fork_version,
ForkName::Capella => self.capella_fork_version,
ForkName::Eip4844 => self.eip4844_fork_version,
}
}
@ -266,7 +267,7 @@ impl ChainSpec {
ForkName::Base => Some(Epoch::new(0)),
ForkName::Altair => self.altair_fork_epoch,
ForkName::Merge => self.bellatrix_fork_epoch,
ForkName::Capella => self.capella_fork_epoch,
ForkName::Eip4844 => self.eip4844_fork_epoch,
}
}
@ -276,7 +277,7 @@ impl ChainSpec {
BeaconState::Base(_) => self.inactivity_penalty_quotient,
BeaconState::Altair(_) => self.inactivity_penalty_quotient_altair,
BeaconState::Merge(_) => self.inactivity_penalty_quotient_bellatrix,
BeaconState::Capella(_) => self.inactivity_penalty_quotient_bellatrix,
BeaconState::Eip4844(_) => self.inactivity_penalty_quotient_bellatrix,
}
}
@ -289,7 +290,7 @@ impl ChainSpec {
BeaconState::Base(_) => self.proportional_slashing_multiplier,
BeaconState::Altair(_) => self.proportional_slashing_multiplier_altair,
BeaconState::Merge(_) => self.proportional_slashing_multiplier_bellatrix,
BeaconState::Capella(_) => self.proportional_slashing_multiplier_bellatrix,
BeaconState::Eip4844(_) => self.proportional_slashing_multiplier_bellatrix,
}
}
@ -302,7 +303,7 @@ impl ChainSpec {
BeaconState::Base(_) => self.min_slashing_penalty_quotient,
BeaconState::Altair(_) => self.min_slashing_penalty_quotient_altair,
BeaconState::Merge(_) => self.min_slashing_penalty_quotient_bellatrix,
BeaconState::Capella(_) => self.min_slashing_penalty_quotient_bellatrix,
BeaconState::Eip4844(_) => self.min_slashing_penalty_quotient_bellatrix,
}
}
@ -565,7 +566,7 @@ impl ChainSpec {
domain_sync_committee: 7,
domain_sync_committee_selection_proof: 8,
domain_contribution_and_proof: 9,
altair_fork_version: [0x01, 0x00, 0x00, 0x00],
altair_fork_version: [0x01, 0x00, 0x0f, 0xfd],
altair_fork_epoch: Some(Epoch::new(74240)),
/*
@ -576,7 +577,7 @@ impl ChainSpec {
min_slashing_penalty_quotient_bellatrix: u64::checked_pow(2, 5)
.expect("pow does not overflow"),
proportional_slashing_multiplier_bellatrix: 3,
bellatrix_fork_version: [0x02, 0x00, 0x00, 0x00],
bellatrix_fork_version: [0x02, 0x00, 0x0f, 0xfd],
bellatrix_fork_epoch: Some(Epoch::new(144896)),
terminal_total_difficulty: Uint256::from_dec_str("58750000000000000000000")
.expect("terminal_total_difficulty is a valid integer"),
@ -585,11 +586,11 @@ impl ChainSpec {
safe_slots_to_import_optimistically: 128u64,
/*
* Capella hardfork params
* Eip4844 hard fork params
*/
//FIXME(sean)
capella_fork_version: [0x03, 0x00, 0x00, 0x00],
capella_fork_epoch: None,
eip4844_fork_version: [0x04, 0x00, 0x00, 0xfd],
eip4844_fork_epoch: Some(Epoch::new(u64::MAX)),
/*
* Network specific
*/
@ -645,10 +646,10 @@ impl ChainSpec {
// `Uint256::MAX` which is `2*256- 1`.
.checked_add(Uint256::one())
.expect("addition does not overflow"),
// Capella
// Eip4844
//FIXME(sean)
capella_fork_version: [0x03, 0x00, 0x00, 0x01],
capella_fork_epoch: None,
eip4844_fork_version: [0x03, 0x00, 0x00, 0x01],
eip4844_fork_epoch: None,
// Other
network_id: 2, // lighthouse testnet network id
deposit_chain_id: 5,
@ -804,9 +805,8 @@ impl ChainSpec {
terminal_block_hash_activation_epoch: Epoch::new(u64::MAX),
safe_slots_to_import_optimistically: 128u64,
//FIXME(sean)
capella_fork_version: [0x03, 0x00, 0x00, 0x64],
capella_fork_epoch: None,
eip4844_fork_version: [0x04, 0x00, 0x00, 0x64],
eip4844_fork_epoch: Some(Epoch::new(u64::MAX)),
/*
* Network specific
@ -883,15 +883,12 @@ pub struct Config {
#[serde(deserialize_with = "deserialize_fork_epoch")]
pub bellatrix_fork_epoch: Option<MaybeQuoted<Epoch>>,
// FIXME(sean): remove this default
#[serde(default = "default_capella_fork_version")]
#[serde(default = "default_eip4844_fork_version")]
#[serde(with = "eth2_serde_utils::bytes_4_hex")]
capella_fork_version: [u8; 4],
// FIXME(sean): remove this default
#[serde(default = "default_capella_fork_epoch")]
eip4844_fork_version: [u8; 4],
#[serde(serialize_with = "serialize_fork_epoch")]
#[serde(deserialize_with = "deserialize_fork_epoch")]
pub capella_fork_epoch: Option<MaybeQuoted<Epoch>>,
pub eip4844_fork_epoch: Option<MaybeQuoted<Epoch>>,
#[serde(with = "eth2_serde_utils::quoted_u64")]
seconds_per_slot: u64,
@ -930,7 +927,7 @@ fn default_bellatrix_fork_version() -> [u8; 4] {
[0xff, 0xff, 0xff, 0xff]
}
fn default_capella_fork_version() -> [u8; 4] {
fn default_eip4844_fork_version() -> [u8; 4] {
// This value shouldn't be used.
[0xff, 0xff, 0xff, 0xff]
}
@ -1031,9 +1028,9 @@ impl Config {
bellatrix_fork_epoch: spec
.bellatrix_fork_epoch
.map(|epoch| MaybeQuoted { value: epoch }),
capella_fork_version: spec.capella_fork_version,
capella_fork_epoch: spec
.capella_fork_epoch
eip4844_fork_version: spec.eip4844_fork_version,
eip4844_fork_epoch: spec
.eip4844_fork_epoch
.map(|epoch| MaybeQuoted { value: epoch }),
seconds_per_slot: spec.seconds_per_slot,
@ -1080,8 +1077,8 @@ impl Config {
altair_fork_epoch,
bellatrix_fork_epoch,
bellatrix_fork_version,
capella_fork_epoch,
capella_fork_version,
eip4844_fork_epoch,
eip4844_fork_version,
seconds_per_slot,
seconds_per_eth1_block,
min_validator_withdrawability_delay,
@ -1112,8 +1109,8 @@ impl Config {
altair_fork_epoch: altair_fork_epoch.map(|q| q.value),
bellatrix_fork_epoch: bellatrix_fork_epoch.map(|q| q.value),
bellatrix_fork_version,
capella_fork_epoch: capella_fork_epoch.map(|q| q.value),
capella_fork_version,
eip4844_fork_epoch: eip4844_fork_epoch.map(|q| q.value),
eip4844_fork_version,
seconds_per_slot,
seconds_per_eth1_block,
min_validator_withdrawability_delay,

View File

@ -96,7 +96,7 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq +
type MinGasLimit: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type MaxExtraDataBytes: Unsigned + Clone + Sync + Send + Debug + PartialEq;
/*
* New in Capella
* New in Eip4844
*/
type MaxBlobsPerBlock: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type FieldElementsPerBlob: Unsigned + Clone + Sync + Send + Debug + PartialEq;

View File

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

View File

@ -47,10 +47,13 @@ impl ForkContext {
));
}
if spec.capella_fork_epoch.is_some() {
if spec.eip4844_fork_epoch.is_some() {
fork_to_digest.push((
ForkName::Capella,
ChainSpec::compute_fork_digest(spec.capella_fork_version, genesis_validators_root),
ForkName::Eip4844,
ChainSpec::compute_fork_digest(
spec.eip4844_fork_version,
genesis_validators_root,
),
));
}

View File

@ -11,12 +11,12 @@ pub enum ForkName {
Base,
Altair,
Merge,
Capella,
Eip4844
}
impl ForkName {
pub fn list_all() -> Vec<ForkName> {
vec![ForkName::Base, ForkName::Altair, ForkName::Merge]
vec![ForkName::Base, ForkName::Altair, ForkName::Merge, ForkName::Eip4844]
}
/// Set the activation slots in the given `ChainSpec` so that the fork named by `self`
@ -27,21 +27,25 @@ impl ForkName {
ForkName::Base => {
spec.altair_fork_epoch = None;
spec.bellatrix_fork_epoch = None;
spec.eip4844_fork_epoch = None;
spec
}
ForkName::Altair => {
spec.altair_fork_epoch = Some(Epoch::new(0));
spec.bellatrix_fork_epoch = None;
spec.eip4844_fork_epoch = None;
spec
}
ForkName::Merge => {
spec.altair_fork_epoch = Some(Epoch::new(0));
spec.bellatrix_fork_epoch = Some(Epoch::new(0));
spec.eip4844_fork_epoch = None;
spec
}
ForkName::Capella => {
ForkName::Eip4844 => {
spec.altair_fork_epoch = Some(Epoch::new(0));
spec.bellatrix_fork_epoch = Some(Epoch::new(0));
spec.capella_fork_epoch = Some(Epoch::new(0));
spec.eip4844_fork_epoch = Some(Epoch::new(0));
spec
}
}
@ -55,7 +59,7 @@ impl ForkName {
ForkName::Base => None,
ForkName::Altair => Some(ForkName::Base),
ForkName::Merge => Some(ForkName::Altair),
ForkName::Capella => Some(ForkName::Merge),
ForkName::Eip4844 => Some(ForkName::Merge),
}
}
@ -66,8 +70,8 @@ impl ForkName {
match self {
ForkName::Base => Some(ForkName::Altair),
ForkName::Altair => Some(ForkName::Merge),
ForkName::Merge => Some(ForkName::Capella),
ForkName::Capella => None,
ForkName::Merge => Some(ForkName::Eip4844),
ForkName::Eip4844 => None,
}
}
}
@ -109,10 +113,9 @@ macro_rules! map_fork_name_with {
let (value, extra_data) = $body;
($t::Merge(value), extra_data)
}
//TODO: don't have a beacon state variant for the new fork yet
ForkName::Capella => {
ForkName::Eip4844 => {
let (value, extra_data) = $body;
($t::Merge(value), extra_data)
($t::Eip4844(value), extra_data)
}
}
};
@ -126,6 +129,7 @@ impl FromStr for ForkName {
"phase0" | "base" => ForkName::Base,
"altair" => ForkName::Altair,
"bellatrix" | "merge" => ForkName::Merge,
"eip4844" => ForkName::Eip4844,
_ => return Err(format!("unknown fork name: {}", fork_name)),
})
}
@ -137,7 +141,7 @@ impl Display for ForkName {
ForkName::Base => "phase0".fmt(f),
ForkName::Altair => "altair".fmt(f),
ForkName::Merge => "bellatrix".fmt(f),
ForkName::Capella => "capella".fmt(f),
ForkName::Eip4844 => "eip4844".fmt(f),
}
}
}

View File

@ -3,18 +3,18 @@ use crate::*;
use derivative::Derivative;
use serde_derive::{Deserialize, Serialize};
use ssz::{Decode, DecodeError, Encode};
use tree_hash::TreeHash;
use tree_hash::{PackedEncoding, TreeHash};
//TODO: is there a way around this newtype
#[derive(Derivative, Debug, Clone, Serialize, Deserialize)]
#[derivative(PartialEq, Eq, Hash)]
pub struct KZGCommitment(#[serde(with = "BigArray")] [u8; 48]);
impl TreeHash for KZGCommitment {
pub struct KzgCommitment(#[serde(with = "BigArray")] [u8; 48]);
impl TreeHash for KzgCommitment {
fn tree_hash_type() -> tree_hash::TreeHashType {
<[u8; 48] as TreeHash>::tree_hash_type()
}
fn tree_hash_packed_encoding(&self) -> Vec<u8> {
fn tree_hash_packed_encoding(&self) -> PackedEncoding{
self.0.tree_hash_packed_encoding()
}
@ -27,23 +27,23 @@ impl TreeHash for KZGCommitment {
}
}
impl TestRandom for KZGCommitment {
impl TestRandom for KzgCommitment {
fn random_for_test(rng: &mut impl rand::RngCore) -> Self {
KZGCommitment(<[u8; 48] as TestRandom>::random_for_test(rng))
KzgCommitment(<[u8; 48] as TestRandom>::random_for_test(rng))
}
}
impl Decode for KZGCommitment {
impl Decode for KzgCommitment {
fn is_ssz_fixed_len() -> bool {
<[u8; 48] as Decode>::is_ssz_fixed_len()
}
fn from_ssz_bytes(bytes: &[u8]) -> Result<Self, DecodeError> {
<[u8; 48] as Decode>::from_ssz_bytes(bytes).map(KZGCommitment)
<[u8; 48] as Decode>::from_ssz_bytes(bytes).map(KzgCommitment)
}
}
impl Encode for KZGCommitment {
impl Encode for KzgCommitment {
fn is_ssz_fixed_len() -> bool {
<[u8; 48] as Encode>::is_ssz_fixed_len()
}

View File

@ -0,0 +1,127 @@
use std::fmt;
use serde::{Deserialize, Deserializer, Serialize, Serializer};
use ssz::{Decode, DecodeError, Encode};
use tree_hash::{PackedEncoding, TreeHash};
use crate::test_utils::{RngCore, TestRandom};
const KZG_PROOF_BYTES_LEN: usize = 48;
#[derive(Debug, PartialEq, Hash, Clone, Copy, Serialize, Deserialize)]
#[serde(transparent)]
pub struct KzgProof(#[serde(with = "serde_kzg_proof")] pub [u8; KZG_PROOF_BYTES_LEN]);
impl fmt::Display for KzgProof {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
write!(f, "{}", eth2_serde_utils::hex::encode(&self.0))
}
}
impl Default for KzgProof {
fn default() -> Self {
KzgProof([0; 48])
}
}
impl From<[u8; KZG_PROOF_BYTES_LEN]> for KzgProof {
fn from(bytes: [u8; KZG_PROOF_BYTES_LEN]) -> Self {
Self(bytes)
}
}
impl Into<[u8; KZG_PROOF_BYTES_LEN]> for KzgProof {
fn into(self) -> [u8; KZG_PROOF_BYTES_LEN] {
self.0
}
}
pub mod serde_kzg_proof {
use serde::de::Error;
use super::*;
pub fn serialize<S>(bytes: &[u8; KZG_PROOF_BYTES_LEN], serializer: S) -> Result<S::Ok, S::Error>
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>,
{
let s: String = Deserialize::deserialize(deserializer)?;
let bytes = eth2_serde_utils::hex::decode(&s).map_err(D::Error::custom)?;
if bytes.len() != KZG_PROOF_BYTES_LEN {
return Err(D::Error::custom(format!(
"incorrect byte length {}, expected {}",
bytes.len(),
KZG_PROOF_BYTES_LEN
)));
}
let mut array = [0; KZG_PROOF_BYTES_LEN];
array[..].copy_from_slice(&bytes);
Ok(array)
}
}
impl Encode for KzgProof {
fn is_ssz_fixed_len() -> bool {
<[u8; KZG_PROOF_BYTES_LEN] as Encode>::is_ssz_fixed_len()
}
fn ssz_fixed_len() -> usize {
<[u8; KZG_PROOF_BYTES_LEN] as Encode>::ssz_fixed_len()
}
fn ssz_bytes_len(&self) -> usize {
self.0.ssz_bytes_len()
}
fn ssz_append(&self, buf: &mut Vec<u8>) {
self.0.ssz_append(buf)
}
}
impl Decode for KzgProof {
fn is_ssz_fixed_len() -> bool {
<[u8; KZG_PROOF_BYTES_LEN] as Decode>::is_ssz_fixed_len()
}
fn ssz_fixed_len() -> usize {
<[u8; KZG_PROOF_BYTES_LEN] as Decode>::ssz_fixed_len()
}
fn from_ssz_bytes(bytes: &[u8]) -> Result<Self, DecodeError> {
<[u8; KZG_PROOF_BYTES_LEN]>::from_ssz_bytes(bytes).map(Self)
}
}
impl TreeHash for KzgProof {
fn tree_hash_type() -> tree_hash::TreeHashType {
<[u8; KZG_PROOF_BYTES_LEN]>::tree_hash_type()
}
fn tree_hash_packed_encoding(&self) -> PackedEncoding {
self.0.tree_hash_packed_encoding()
}
fn tree_hash_packing_factor() -> usize {
<[u8; KZG_PROOF_BYTES_LEN]>::tree_hash_packing_factor()
}
fn tree_hash_root(&self) -> tree_hash::Hash256 {
self.0.tree_hash_root()
}
}
impl TestRandom for KzgProof {
fn random_for_test(rng: &mut impl RngCore) -> Self {
let mut bytes = [0; KZG_PROOF_BYTES_LEN];
rng.fill_bytes(&mut bytes);
Self(bytes)
}
}

View File

@ -86,14 +86,16 @@ pub mod sync_subnet_id;
mod tree_hash_impls;
pub mod validator_registration_data;
mod blobs_sidecar;
mod kzg_commitment;
mod signed_blobs_sidecar;
pub mod slot_data;
#[cfg(feature = "sqlite")]
pub mod sqlite;
pub use kzg_commitment::KZGCommitment;
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};
@ -103,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, BeaconBlockCapella, BeaconBlockMerge, BeaconBlockRef,
BeaconBlockRefMut, BlindedBeaconBlock,
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, BeaconBlockRef,
BeaconBlockRefMut, BeaconBlockEip4844
};
pub use crate::beacon_block_body::{
BeaconBlockBody, BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyCapella,
BeaconBlockBodyMerge, BeaconBlockBodyRef, BeaconBlockBodyRefMut,
BeaconBlockBody, BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyMerge,
BeaconBlockBodyRef, BeaconBlockBodyRefMut, BeaconBlockBodyEip4844
};
pub use crate::beacon_block_header::BeaconBlockHeader;
pub use crate::beacon_committee::{BeaconCommittee, OwnedBeaconCommittee};
@ -150,7 +152,7 @@ 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,SignedBeaconBlockCapella
SignedBeaconBlockMerge, SignedBlindedBeaconBlock, SignedBeaconBlockEip4844,
};
pub use crate::signed_beacon_block_header::SignedBeaconBlockHeader;
pub use crate::signed_contribution_and_proof::SignedContributionAndProof;

View File

@ -8,6 +8,7 @@ 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

@ -38,7 +38,7 @@ impl From<SignedBeaconBlockHash> for Hash256 {
/// A `BeaconBlock` and a signature from its proposer.
#[superstruct(
variants(Base, Altair, Merge, Capella),
variants(Base, Altair, Merge, Eip4844),
variant_attributes(
derive(
Debug,
@ -72,8 +72,8 @@ pub struct SignedBeaconBlock<E: EthSpec, Payload: ExecPayload<E> = FullPayload<E
pub message: BeaconBlockAltair<E, Payload>,
#[superstruct(only(Merge), partial_getter(rename = "message_merge"))]
pub message: BeaconBlockMerge<E, Payload>,
#[superstruct(only(Capella), partial_getter(rename = "message_capella"))]
pub message: BeaconBlockCapella<E, Payload>,
#[superstruct(only(Eip4844), partial_getter(rename = "message_eip4844"))]
pub message: BeaconBlockEip4844<E, Payload>,
pub signature: Signature,
}
@ -131,8 +131,8 @@ impl<E: EthSpec, Payload: ExecPayload<E>> SignedBeaconBlock<E, Payload> {
BeaconBlock::Merge(message) => {
SignedBeaconBlock::Merge(SignedBeaconBlockMerge { message, signature })
}
BeaconBlock::Capella(message) => {
SignedBeaconBlock::Capella(SignedBeaconBlockCapella { message, signature })
BeaconBlock::Eip4844(message) => {
SignedBeaconBlock::Eip4844(SignedBeaconBlockEip4844 { message, signature })
}
}
}
@ -312,6 +312,60 @@ impl<E: EthSpec> SignedBeaconBlockMerge<E, BlindedPayload<E>> {
}
}
impl<E: EthSpec> SignedBeaconBlockEip4844<E, BlindedPayload<E>> {
pub fn into_full_block(
self,
execution_payload: ExecutionPayload<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,
},
},
signature,
} = self;
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: FullPayload { execution_payload },
blob_kzg_commitments,
},
},
signature,
}
}
}
impl<E: EthSpec> SignedBeaconBlock<E, BlindedPayload<E>> {
pub fn try_into_full_block(
self,
@ -323,6 +377,9 @@ impl<E: EthSpec> SignedBeaconBlock<E, BlindedPayload<E>> {
SignedBeaconBlock::Merge(block) => {
SignedBeaconBlock::Merge(block.into_full_block(execution_payload?))
}
SignedBeaconBlock::Eip4844(block) => {
SignedBeaconBlock::Eip4844(block.into_full_block(execution_payload?))
}
};
Some(full_block)
}

View File

@ -41,8 +41,8 @@ excluded_paths = [
"tests/.*/.*/ssz_static/LightClientFinalityUpdate",
# Merkle-proof tests for light clients
"tests/.*/.*/merkle/single_proof",
# Capella tests are disabled for now.
"tests/.*/capella",
# Eip4844 tests are disabled for now.
"tests/.*/eip4844",
# One of the EF researchers likes to pack the tarballs on a Mac
".*\.DS_Store.*",
# More Mac weirdness.

View File

@ -78,6 +78,6 @@ pub fn previous_fork(fork_name: ForkName) -> ForkName {
ForkName::Base => ForkName::Base,
ForkName::Altair => ForkName::Base,
ForkName::Merge => ForkName::Altair, // TODO: Check this when tests are released..
ForkName::Capella => ForkName::Merge, // TODO: Check this when tests are released..
ForkName::Eip4844 => ForkName::Merge, // TODO: Check this when tests are released..
}
}

View File

@ -97,7 +97,7 @@ impl<E: EthSpec> EpochTransition<E> for JustificationAndFinalization {
justification_and_finalization_state.apply_changes_to_state(state);
Ok(())
}
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
let justification_and_finalization_state =
altair::process_justification_and_finalization(
state,
@ -118,7 +118,7 @@ impl<E: EthSpec> EpochTransition<E> for RewardsAndPenalties {
validator_statuses.process_attestations(state)?;
base::process_rewards_and_penalties(state, &mut validator_statuses, spec)
}
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
altair::process_rewards_and_penalties(
state,
&altair::ParticipationCache::new(state, spec).unwrap(),
@ -147,7 +147,7 @@ impl<E: EthSpec> EpochTransition<E> for Slashings {
spec,
)?;
}
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
process_slashings(
state,
altair::ParticipationCache::new(state, spec)
@ -205,7 +205,7 @@ impl<E: EthSpec> EpochTransition<E> for SyncCommitteeUpdates {
fn run(state: &mut BeaconState<E>, spec: &ChainSpec) -> Result<(), EpochProcessingError> {
match state {
BeaconState::Base(_) => Ok(()),
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
altair::process_sync_committee_updates(state, spec)
}
}
@ -216,7 +216,7 @@ impl<E: EthSpec> EpochTransition<E> for InactivityUpdates {
fn run(state: &mut BeaconState<E>, spec: &ChainSpec) -> Result<(), EpochProcessingError> {
match state {
BeaconState::Base(_) => Ok(()),
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
altair::process_inactivity_updates(
state,
&altair::ParticipationCache::new(state, spec).unwrap(),
@ -231,7 +231,7 @@ impl<E: EthSpec> EpochTransition<E> for ParticipationFlagUpdates {
fn run(state: &mut BeaconState<E>, _: &ChainSpec) -> Result<(), EpochProcessingError> {
match state {
BeaconState::Base(_) => Ok(()),
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
altair::process_participation_flag_updates(state)
}
}
@ -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::Capella => false, // TODO: revisit when tests are out
ForkName::Eip4844 => false, // TODO: revisit when tests are out
}
}

View File

@ -61,7 +61,7 @@ impl<E: EthSpec> Case for ForkTest<E> {
ForkName::Base => panic!("phase0 not supported"),
ForkName::Altair => upgrade_to_altair(&mut result_state, spec).map(|_| result_state),
ForkName::Merge => upgrade_to_bellatrix(&mut result_state, spec).map(|_| result_state),
ForkName::Capella => panic!("capella not supported"),
ForkName::Eip4844 => panic!("eip4844 not supported"),
};
compare_beacon_state_results_without_caches(&mut result, &mut expected)

View File

@ -81,7 +81,7 @@ impl<E: EthSpec> Operation<E> for Attestation<E> {
BeaconState::Base(_) => {
base::process_attestations(state, &[self.clone()], VerifySignatures::True, spec)
}
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Eip4844(_) => {
altair::process_attestation(
state,
self,

View File

@ -42,9 +42,9 @@ impl<E: EthSpec> LoadCase for TransitionTest<E> {
spec.altair_fork_epoch = Some(Epoch::new(0));
spec.bellatrix_fork_epoch = Some(metadata.fork_epoch);
}
ForkName::Capella => {
ForkName::Eip4844 => {
spec.bellatrix_fork_epoch = Some(Epoch::new(0));
spec.capella_fork_epoch = Some(metadata.fork_epoch);
spec.eip4844_fork_epoch = Some(metadata.fork_epoch);
}
}

View File

@ -26,6 +26,7 @@ pub enum ForkName {
Phase0,
Altair,
Bellatrix,
Eip4844,
}
#[derive(Debug, PartialEq, Serialize)]
@ -90,8 +91,8 @@ impl<'a, T: EthSpec, Payload: ExecPayload<T>> Web3SignerObject<'a, T, Payload> {
block: None,
block_header: Some(block.block_header()),
}),
BeaconBlock::Capella(_) => Ok(Web3SignerObject::BeaconBlock {
version: ForkName::Capella,
BeaconBlock::Eip4844(_) => Ok(Web3SignerObject::BeaconBlock {
version: ForkName::Eip4844,
block: None,
block_header: Some(block.block_header()),
}),