fix compilation errors, rename capella -> shanghai, cleanup some rebase issues

This commit is contained in:
realbigsean 2022-04-05 16:55:42 -04:00
parent 809b52715e
commit fe6fc55449
No known key found for this signature in database
GPG Key ID: B372B64D866BF8CC
38 changed files with 218 additions and 311 deletions

View File

@ -3620,17 +3620,17 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.ok_or(BlockProductionError::MissingExecutionPayload)?, .ok_or(BlockProductionError::MissingExecutionPayload)?,
}, },
}), }),
BeaconState::Shanghai(_) => { BeaconState::Capella(_) => {
let sync_aggregate = get_sync_aggregate()?; let sync_aggregate = get_sync_aggregate()?;
let (execution_payload, blobs) = let (execution_payload, blobs) =
get_execution_payload_and_blobs(self, &state, proposer_index)?; get_execution_payload_and_blobs(self, &state, proposer_index)?;
//FIXME(sean) get blobs //FIXME(sean) get blobs
BeaconBlock::Shanghai(BeaconBlockShanghai { BeaconBlock::Capella(BeaconBlockCapella {
slot, slot,
proposer_index, proposer_index,
parent_root, parent_root,
state_root: Hash256::zero(), state_root: Hash256::zero(),
body: BeaconBlockBodyShanghai { body: BeaconBlockBodyCapella {
randao_reveal, randao_reveal,
eth1_data, eth1_data,
graffiti, graffiti,

View File

@ -388,16 +388,19 @@ pub fn get_execution_payload<
} }
/// Wraps the async `prepare_execution_payload` function as a blocking task. /// Wraps the async `prepare_execution_payload` function as a blocking task.
pub fn prepare_execution_payload_and_blobs_blocking<T: BeaconChainTypes>( pub fn prepare_execution_payload_and_blobs_blocking<
T: BeaconChainTypes,
Payload: ExecPayload<T::EthSpec>,
>(
chain: &BeaconChain<T>, chain: &BeaconChain<T>,
state: &BeaconState<T::EthSpec>, state: &BeaconState<T::EthSpec>,
proposer_index: u64, proposer_index: u64,
) -> Result< ) -> Result<
Option<( Option<(
ExecutionPayload<T::EthSpec>, Payload,
VariableList< VariableList<
KZGCommitment, KZGCommitment,
<<T as BeaconChainTypes>::EthSpec as EthSpec>::MaxObjectListSize, <<T as BeaconChainTypes>::EthSpec as EthSpec>::MaxBlobsPerBlock,
>, >,
)>, )>,
BlockProductionError, BlockProductionError,
@ -409,7 +412,7 @@ pub fn prepare_execution_payload_and_blobs_blocking<T: BeaconChainTypes>(
execution_layer execution_layer
.block_on_generic(|_| async { .block_on_generic(|_| async {
prepare_execution_payload_and_blobs(chain, state, proposer_index).await prepare_execution_payload_and_blobs::<T, Payload>(chain, state, proposer_index).await
}) })
.map_err(BlockProductionError::BlockingFailed)? .map_err(BlockProductionError::BlockingFailed)?
} }
@ -513,100 +516,22 @@ where
Ok(execution_payload) Ok(execution_payload)
} }
pub async fn prepare_execution_payload_and_blobs<T: BeaconChainTypes>( pub async fn prepare_execution_payload_and_blobs<
T: BeaconChainTypes,
Payload: ExecPayload<T::EthSpec>,
>(
chain: &BeaconChain<T>, chain: &BeaconChain<T>,
state: &BeaconState<T::EthSpec>, state: &BeaconState<T::EthSpec>,
proposer_index: u64, proposer_index: u64,
) -> Result< ) -> Result<
Option<( Option<(
ExecutionPayload<T::EthSpec>, Payload,
VariableList< VariableList<
KZGCommitment, KZGCommitment,
<<T as BeaconChainTypes>::EthSpec as EthSpec>::MaxObjectListSize, <<T as BeaconChainTypes>::EthSpec as EthSpec>::MaxBlobsPerBlock,
>, >,
)>, )>,
BlockProductionError, BlockProductionError,
> { > {
let spec = &chain.spec; todo!()
let execution_layer = chain
.execution_layer
.as_ref()
.ok_or(BlockProductionError::ExecutionLayerMissing)?;
let parent_hash = if !is_merge_transition_complete(state) {
let is_terminal_block_hash_set = spec.terminal_block_hash != Hash256::zero();
let is_activation_epoch_reached =
state.current_epoch() >= spec.terminal_block_hash_activation_epoch;
if is_terminal_block_hash_set && !is_activation_epoch_reached {
return Ok(None);
}
let terminal_pow_block_hash = execution_layer
.get_terminal_pow_block_hash(spec)
.await
.map_err(BlockProductionError::TerminalPoWBlockLookupFailed)?;
if let Some(terminal_pow_block_hash) = terminal_pow_block_hash {
terminal_pow_block_hash
} else {
return Ok(None);
}
} else {
state.latest_execution_payload_header()?.block_hash
};
let timestamp = compute_timestamp_at_slot(state, spec).map_err(BeaconStateError::from)?;
let random = *state.get_randao_mix(state.current_epoch())?;
let finalized_root = state.finalized_checkpoint().root;
// The finalized block hash is not included in the specification, however we provide this
// parameter so that the execution layer can produce a payload id if one is not already known
// (e.g., due to a recent reorg).
let finalized_block_hash =
if let Some(block) = chain.fork_choice.read().get_block(&finalized_root) {
block.execution_status.block_hash()
} else {
chain
.store
.get_block(&finalized_root)
.map_err(BlockProductionError::FailedToReadFinalizedBlock)?
.ok_or(BlockProductionError::MissingFinalizedBlock(finalized_root))?
.message()
.body()
.execution_payload()
.ok()
.map(|ep| ep.block_hash)
};
// Note: the suggested_fee_recipient is stored in the `execution_layer`, it will add this parameter.
let execution_payload = execution_layer
.get_payload(
parent_hash,
timestamp,
random,
finalized_block_hash.unwrap_or_else(Hash256::zero),
proposer_index,
)
.await
.map_err(BlockProductionError::GetPayloadFailed)?;
//FIXME(sean)
for tx in execution_payload.blob_txns_iter() {
let versioned_hash = Hash256::zero();
// get versioned hash
let blob = execution_layer
.get_blob::<T::EthSpec>(
parent_hash,
timestamp,
random,
finalized_root,
proposer_index,
versioned_hash,
)
.await
.map_err(BlockProductionError::GetPayloadFailed)?;
}
Ok(Some((execution_payload, VariableList::empty())))
} }

View File

@ -9,12 +9,9 @@ pub use types::{
Address, EthSpec, ExecutionBlockHash, ExecutionPayload, ExecutionPayloadHeader, FixedVector, Address, EthSpec, ExecutionBlockHash, ExecutionPayload, ExecutionPayloadHeader, FixedVector,
Hash256, Uint256, VariableList, Hash256, Uint256, VariableList,
}; };
pub mod auth;
use crate::engines::ForkChoiceState;
pub use types::{Address, EthSpec, ExecutionPayload, Hash256, Uint256};
use types::{Blob, KZGCommitment}; use types::{Blob, KZGCommitment};
pub mod auth;
pub mod http; pub mod http;
pub mod json_structures; pub mod json_structures;

View File

@ -667,10 +667,10 @@ impl HttpJsonRpc {
Ok(response.into()) Ok(response.into())
} }
async fn get_blob_v1<T: EthSpec>( pub async fn get_blob_v1<T: EthSpec>(
&self, &self,
payload_id: PayloadId, payload_id: PayloadId,
versioned_hash: Hash256, versioned_hash: ExecutionBlockHash,
) -> Result<BlobDetailsV1, Error> { ) -> Result<BlobDetailsV1, Error> {
let params = json!([JsonPayloadIdRequest::from(payload_id), versioned_hash]); let params = json!([JsonPayloadIdRequest::from(payload_id), versioned_hash]);

View File

@ -897,60 +897,7 @@ impl<T: EthSpec> ExecutionLayer<T> {
proposer_index: u64, proposer_index: u64,
versioned_hash: Hash256, versioned_hash: Hash256,
) -> Result<BlobDetailsV1, Error> { ) -> Result<BlobDetailsV1, Error> {
let suggested_fee_recipient = self.get_suggested_fee_recipient(proposer_index).await; todo!()
debug!(
self.log(),
"Issuing engine_getBlob";
"suggested_fee_recipient" => ?suggested_fee_recipient,
"random" => ?random,
"timestamp" => timestamp,
"parent_hash" => ?parent_hash,
);
self.engines()
.first_success(|engine| async move {
let payload_id = if let Some(id) = engine
.get_payload_id(parent_hash, timestamp, random, suggested_fee_recipient)
.await
{
// The payload id has been cached for this engine.
id
} else {
// The payload id has *not* been cached for this engine. Trigger an artificial
// fork choice update to retrieve a payload ID.
//
// TODO(merge): a better algorithm might try to favour a node that already had a
// cached payload id, since a payload that has had more time to produce is
// likely to be more profitable.
let fork_choice_state = ForkChoiceState {
head_block_hash: parent_hash,
safe_block_hash: parent_hash,
finalized_block_hash,
};
let payload_attributes = PayloadAttributes {
timestamp,
random,
suggested_fee_recipient,
};
engine
.notify_forkchoice_updated(
fork_choice_state,
Some(payload_attributes),
self.log(),
)
.await
.map(|response| response.payload_id)?
.ok_or(ApiError::PayloadIdUnavailable)?
};
engine
.api
.get_blob_v1::<T>(payload_id, versioned_hash)
.await
})
.await
.map_err(Error::EngineErrors)
} }
/// Maps to the `engine_newPayload` JSON-RPC call. /// Maps to the `engine_newPayload` JSON-RPC call.

View File

@ -21,6 +21,9 @@ const GOSSIP_MAX_SIZE: usize = 1_048_576; // 1M
/// The maximum transmit size of gossip messages in bytes post-merge. /// The maximum transmit size of gossip messages in bytes post-merge.
const GOSSIP_MAX_SIZE_POST_MERGE: usize = 10 * 1_048_576; // 10M const GOSSIP_MAX_SIZE_POST_MERGE: usize = 10 * 1_048_576; // 10M
const MAX_REQUEST_BLOBS_SIDECARS: usize = 128;
const MIN_EPOCHS_FOR_BLOBS_SIDECARS_REQUESTS: usize = 128;
/// The cache time is set to accommodate the circulation time of an attestation. /// The cache time is set to accommodate the circulation time of an attestation.
/// ///
/// The p2p spec declares that we accept attestations within the following range: /// The p2p spec declares that we accept attestations within the following range:
@ -297,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 // 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 // the derivation of the message-id remains the same in the merge
//TODO(sean): figure this out //TODO(sean): figure this out
ForkName::Altair | ForkName::Merge | ForkName::Shanghai => { ForkName::Altair | ForkName::Merge | ForkName::Capella => {
let topic_len_bytes = topic_bytes.len().to_le_bytes(); let topic_len_bytes = topic_bytes.len().to_le_bytes();
let mut vec = Vec::with_capacity( let mut vec = Vec::with_capacity(
prefix.len() + topic_len_bytes.len() + topic_bytes.len() + message.data.len(), prefix.len() + topic_len_bytes.len() + topic_bytes.len() + message.data.len(),

View File

@ -16,8 +16,8 @@ use std::marker::PhantomData;
use std::sync::Arc; use std::sync::Arc;
use tokio_util::codec::{Decoder, Encoder}; use tokio_util::codec::{Decoder, Encoder};
use types::{ use types::{
BlobWrapper, EthSpec, ForkContext, ForkName, SignedBeaconBlock, SignedBeaconBlockAltair, BlobsSidecar, EthSpec, ForkContext, ForkName, SignedBeaconBlock, SignedBeaconBlockAltair,
SignedBeaconBlockBase, SignedBeaconBlockMerge, SignedBeaconBlockShanghai, SignedBeaconBlockBase, SignedBeaconBlockCapella, SignedBeaconBlockMerge,
}; };
use unsigned_varint::codec::Uvi; use unsigned_varint::codec::Uvi;
@ -409,8 +409,8 @@ fn context_bytes<T: EthSpec>(
return match **ref_box_block { return match **ref_box_block {
// NOTE: If you are adding another fork type here, be sure to modify the // 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! // `fork_context.to_context_bytes()` function to support it as well!
SignedBeaconBlock::Shanghai { .. } => { SignedBeaconBlock::Capella { .. } => {
fork_context.to_context_bytes(ForkName::Shanghai) fork_context.to_context_bytes(ForkName::Capella)
} }
SignedBeaconBlock::Merge { .. } => { SignedBeaconBlock::Merge { .. } => {
// Merge context being `None` implies that "merge never happened". // Merge context being `None` implies that "merge never happened".
@ -547,7 +547,7 @@ fn handle_v1_response<T: EthSpec>(
SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?), SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?),
)))), )))),
Protocol::TxBlobsByRange => Ok(Some(RPCResponse::TxBlobsByRange(Arc::new( Protocol::TxBlobsByRange => Ok(Some(RPCResponse::TxBlobsByRange(Arc::new(
BlobWrapper::from_ssz_bytes(decoded_buffer)?), BlobsSidecar::from_ssz_bytes(decoded_buffer)?),
))), ))),
Protocol::BlocksByRoot => Ok(Some(RPCResponse::BlocksByRoot(Arc::new( Protocol::BlocksByRoot => Ok(Some(RPCResponse::BlocksByRoot(Arc::new(
SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?), SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?),
@ -600,14 +600,14 @@ fn handle_v2_response<T: EthSpec>(
decoded_buffer, decoded_buffer,
)?), )?),
)))), )))),
ForkName::Shanghai => Ok(Some(RPCResponse::BlocksByRange(Box::new( ForkName::Capella => Ok(Some(RPCResponse::BlocksByRange(Box::new(
SignedBeaconBlock::Shanghai(SignedBeaconBlockShanghai::from_ssz_bytes( SignedBeaconBlock::Capella(SignedBeaconBlockCapella::from_ssz_bytes(
decoded_buffer, decoded_buffer,
)?), )?),
)))), )))),
}, },
Protocol::TxBlobsByRange => Ok(Some(RPCResponse::TxBlobsByRange(Box::new( Protocol::TxBlobsByRange => Ok(Some(RPCResponse::TxBlobsByRange(Box::new(
BlobWrapper::from_ssz_bytes(decoded_buffer)?, BlobsSidecar::from_ssz_bytes(decoded_buffer)?,
)))), )))),
Protocol::BlocksByRoot => match fork_name { Protocol::BlocksByRoot => match fork_name {
ForkName::Altair => Ok(Some(RPCResponse::BlocksByRoot(Arc::new( ForkName::Altair => Ok(Some(RPCResponse::BlocksByRoot(Arc::new(
@ -623,8 +623,8 @@ fn handle_v2_response<T: EthSpec>(
decoded_buffer, decoded_buffer,
)?), )?),
)))), )))),
ForkName::Shanghai => Ok(Some(RPCResponse::BlocksByRoot(Box::new( ForkName::Capella => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
SignedBeaconBlock::Shanghai(SignedBeaconBlockShanghai::from_ssz_bytes( SignedBeaconBlock::Capella(SignedBeaconBlockCapella::from_ssz_bytes(
decoded_buffer, decoded_buffer,
)?), )?),
)))), )))),

View File

@ -12,7 +12,7 @@ use std::ops::Deref;
use std::sync::Arc; use std::sync::Arc;
use strum::IntoStaticStr; use strum::IntoStaticStr;
use superstruct::superstruct; use superstruct::superstruct;
use types::{BlobWrapper, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot}; use types::{BlobsSidecar, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot};
/// Maximum number of blocks in a single request. /// Maximum number of blocks in a single request.
pub type MaxRequestBlocks = U1024; pub type MaxRequestBlocks = U1024;
@ -246,7 +246,7 @@ pub enum RPCResponse<T: EthSpec> {
/// batch. /// batch.
BlocksByRange(Arc<SignedBeaconBlock<T>>), BlocksByRange(Arc<SignedBeaconBlock<T>>),
TxBlobsByRange(Box<BlobWrapper<T>>), TxBlobsByRange(Box<BlobsSidecar<T>>),
/// A response to a get BLOCKS_BY_ROOT request. /// A response to a get BLOCKS_BY_ROOT request.
BlocksByRoot(Arc<SignedBeaconBlock<T>>), BlocksByRoot(Arc<SignedBeaconBlock<T>>),

View File

@ -21,7 +21,7 @@ use tokio_util::{
compat::{Compat, FuturesAsyncReadCompatExt}, compat::{Compat, FuturesAsyncReadCompatExt},
}; };
use types::{ use types::{
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, BlobWrapper, EthSpec, BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, BlobsSidecar, EthSpec,
ForkContext, ForkName, Hash256, MainnetEthSpec, Signature, SignedBeaconBlock, ForkContext, ForkName, Hash256, MainnetEthSpec, Signature, SignedBeaconBlock,
}; };
@ -71,11 +71,11 @@ lazy_static! {
+ types::ExecutionPayload::<MainnetEthSpec>::max_execution_payload_size() // adding max size of execution payload (~16gb) + types::ExecutionPayload::<MainnetEthSpec>::max_execution_payload_size() // adding max size of execution payload (~16gb)
+ ssz::BYTES_PER_LENGTH_OFFSET; // Adding the additional ssz offset for the `ExecutionPayload` field + ssz::BYTES_PER_LENGTH_OFFSET; // Adding the additional ssz offset for the `ExecutionPayload` field
pub static ref BLOB_MIN: usize = BlobWrapper::<MainnetEthSpec>::empty() pub static ref BLOB_MIN: usize = BlobsSidecar::<MainnetEthSpec>::empty()
.as_ssz_bytes() .as_ssz_bytes()
.len(); .len();
pub static ref BLOB_MAX: usize = BlobWrapper::<MainnetEthSpec>::max_size(); pub static ref BLOB_MAX: usize = BlobsSidecar::<MainnetEthSpec>::max_size();
pub static ref BLOCKS_BY_ROOT_REQUEST_MIN: usize = pub static ref BLOCKS_BY_ROOT_REQUEST_MIN: usize =
VariableList::<Hash256, MaxRequestBlocks>::from(Vec::<Hash256>::new()) VariableList::<Hash256, MaxRequestBlocks>::from(Vec::<Hash256>::new())
@ -120,7 +120,8 @@ const REQUEST_TIMEOUT: u64 = 15;
pub fn max_rpc_size(fork_context: &ForkContext) -> usize { pub fn max_rpc_size(fork_context: &ForkContext) -> usize {
match fork_context.current_fork() { match fork_context.current_fork() {
ForkName::Merge => MAX_RPC_SIZE_POST_MERGE, ForkName::Merge => MAX_RPC_SIZE_POST_MERGE,
ForkName::Altair | ForkName::Base => MAX_RPC_SIZE, //FIXME(sean) check this
ForkName::Altair | ForkName::Base | ForkName::Capella => MAX_RPC_SIZE,
} }
} }

View File

@ -41,7 +41,7 @@ use std::sync::Arc;
use std::task::{Context, Poll}; use std::task::{Context, Poll};
use types::{ use types::{
consts::altair::SYNC_COMMITTEE_SUBNET_COUNT, EnrForkId, EthSpec, ForkContext, Slot, SubnetId, consts::altair::SYNC_COMMITTEE_SUBNET_COUNT, EnrForkId, EthSpec, ForkContext, Slot, SubnetId,
BlobWrapper, SignedBeaconBlock, SyncSubnetId BlobsSidecar, SignedBeaconBlock, SyncSubnetId
}; };
use crate::rpc::methods::TxBlobsByRangeRequest; use crate::rpc::methods::TxBlobsByRangeRequest;
use utils::{build_transport, strip_peer_id, MAX_CONNECTIONS_PER_PEER}; use utils::{build_transport, strip_peer_id, MAX_CONNECTIONS_PER_PEER};

View File

@ -9,9 +9,9 @@ use std::boxed::Box;
use std::io::{Error, ErrorKind}; use std::io::{Error, ErrorKind};
use std::sync::Arc; use std::sync::Arc;
use types::{ use types::{
Attestation, AttesterSlashing, BlobWrapper, EthSpec, ForkContext, ForkName, ProposerSlashing, Attestation, AttesterSlashing, BlobsSidecar, EthSpec, ForkContext, ForkName, ProposerSlashing,
SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase,
SignedBeaconBlockMerge, SignedBeaconBlockShanghai, SignedContributionAndProof, SignedBeaconBlockCapella, SignedBeaconBlockMerge, SignedContributionAndProof,
SignedVoluntaryExit, SubnetId, SyncCommitteeMessage, SyncSubnetId, SignedVoluntaryExit, SubnetId, SyncCommitteeMessage, SyncSubnetId,
}; };
@ -168,8 +168,8 @@ impl<T: EthSpec> PubsubMessage<T> {
SignedBeaconBlockMerge::from_ssz_bytes(data) SignedBeaconBlockMerge::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?, .map_err(|e| format!("{:?}", e))?,
), ),
Some(ForkName::Shanghai) => SignedBeaconBlock::<T>::Shanghai( Some(ForkName::Capella) => SignedBeaconBlock::<T>::Capella(
SignedBeaconBlockShanghai::from_ssz_bytes(data) SignedBeaconBlockCapella::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?, .map_err(|e| format!("{:?}", e))?,
), ),
None => { None => {
@ -184,7 +184,7 @@ impl<T: EthSpec> PubsubMessage<T> {
GossipKind::Blob => { GossipKind::Blob => {
//FIXME(sean) verify against fork context //FIXME(sean) verify against fork context
let blob = let blob =
BlobWrapper::from_ssz_bytes(data).map_err(|e| format!("{:?}", e))?; BlobsSidecar::from_ssz_bytes(data).map_err(|e| format!("{:?}", e))?;
Ok(PubsubMessage::Blob(Box::new(blob))) Ok(PubsubMessage::Blob(Box::new(blob)))
} }
GossipKind::VoluntaryExit => { GossipKind::VoluntaryExit => {

View File

@ -62,9 +62,9 @@ use std::{cmp, collections::HashSet};
use task_executor::TaskExecutor; use task_executor::TaskExecutor;
use tokio::sync::{mpsc, oneshot}; use tokio::sync::{mpsc, oneshot};
use types::{ use types::{
Attestation, AttesterSlashing, BlobWrapper, Hash256, ProposerSlashing, SignedAggregateAndProof, Attestation, AttesterSlashing, BlobsSidecar, Hash256, ProposerSlashing,
SignedBeaconBlock, SignedContributionAndProof, SignedVoluntaryExit, SubnetId, SignedAggregateAndProof, SignedBeaconBlock, SignedContributionAndProof, SignedVoluntaryExit,
SyncCommitteeMessage, SyncSubnetId, SubnetId, SyncCommitteeMessage, SyncSubnetId,
}; };
use work_reprocessing_queue::{ use work_reprocessing_queue::{
spawn_reprocess_scheduler, QueuedAggregate, QueuedRpcBlock, QueuedUnaggregate, ReadyWork, spawn_reprocess_scheduler, QueuedAggregate, QueuedRpcBlock, QueuedUnaggregate, ReadyWork,
@ -412,7 +412,7 @@ impl<T: BeaconChainTypes> WorkEvent<T> {
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
peer_client: Client, peer_client: Client,
blob: Box<BlobWrapper<T::EthSpec>>, blob: Box<BlobsSidecar<T::EthSpec>>,
seen_timestamp: Duration, seen_timestamp: Duration,
) -> Self { ) -> Self {
Self { Self {
@ -721,7 +721,7 @@ pub enum Work<T: BeaconChainTypes> {
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
peer_client: Client, peer_client: Client,
blob: Box<BlobWrapper<T::EthSpec>>, blob: Box<BlobsSidecar<T::EthSpec>>,
seen_timestamp: Duration, seen_timestamp: Duration,
}, },
DelayedImportBlock { DelayedImportBlock {

View File

@ -18,7 +18,7 @@ use std::time::{Duration, SystemTime, UNIX_EPOCH};
use store::hot_cold_store::HotColdDBError; use store::hot_cold_store::HotColdDBError;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use types::{ use types::{
Attestation, AttesterSlashing, BlobWrapper, EthSpec, Hash256, IndexedAttestation, Attestation, AttesterSlashing, BlobsSidecar, EthSpec, Hash256, IndexedAttestation,
ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedContributionAndProof, ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedContributionAndProof,
SignedVoluntaryExit, Slot, SubnetId, SyncCommitteeMessage, SyncSubnetId, SignedVoluntaryExit, Slot, SubnetId, SyncCommitteeMessage, SyncSubnetId,
}; };
@ -698,7 +698,7 @@ impl<T: BeaconChainTypes> Worker<T> {
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
peer_client: Client, peer_client: Client,
blob: BlobWrapper<T::EthSpec>, blob: BlobsSidecar<T::EthSpec>,
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>, reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
duplicate_cache: DuplicateCache, duplicate_cache: DuplicateCache,
seen_duration: Duration, seen_duration: Duration,

View File

@ -18,8 +18,9 @@ use std::time::{Duration, SystemTime, UNIX_EPOCH};
use store::SyncCommitteeMessage; use store::SyncCommitteeMessage;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use types::{ use types::{
Attestation, AttesterSlashing, BlobWrapper, EthSpec, ProposerSlashing, SignedAggregateAndProof, Attestation, AttesterSlashing, BlobsSidecar, EthSpec, ProposerSlashing,
SignedBeaconBlock, SignedContributionAndProof, SignedVoluntaryExit, SubnetId, SyncSubnetId, SignedAggregateAndProof, SignedBeaconBlock, SignedContributionAndProof, SignedVoluntaryExit,
SubnetId, SyncSubnetId,
}; };
/// Processes validated messages from the network. It relays necessary data to the syncing thread /// Processes validated messages from the network. It relays necessary data to the syncing thread
@ -221,7 +222,7 @@ impl<T: BeaconChainTypes> Processor<T> {
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
request_id: RequestId, request_id: RequestId,
blob_wrapper: Option<Box<BlobWrapper<T::EthSpec>>>, blob_wrapper: Option<Box<BlobsSidecar<T::EthSpec>>>,
) { ) {
trace!( trace!(
self.log, self.log,
@ -299,7 +300,7 @@ impl<T: BeaconChainTypes> Processor<T> {
message_id: MessageId, message_id: MessageId,
peer_id: PeerId, peer_id: PeerId,
peer_client: Client, peer_client: Client,
blob: Box<BlobWrapper<T::EthSpec>>, blob: Box<BlobsSidecar<T::EthSpec>>,
) { ) {
self.send_beacon_processor_work(BeaconWorkEvent::gossip_tx_blob_block( self.send_beacon_processor_work(BeaconWorkEvent::gossip_tx_blob_block(
message_id, message_id,

View File

@ -53,7 +53,7 @@ use std::ops::Sub;
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration; use std::time::Duration;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use types::{BlobWrapper, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot}; use types::{BlobsSidecar, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot};
/// The number of slots ahead of us that is allowed before requesting a long-range (batch) Sync /// The number of slots ahead of us that is allowed before requesting a long-range (batch) Sync
/// from a peer. If a peer is within this tolerance (forwards or backwards), it is treated as a /// from a peer. If a peer is within this tolerance (forwards or backwards), it is treated as a
@ -88,14 +88,16 @@ pub enum SyncMessage<T: EthSpec> {
/// A block has been received from the RPC. /// A block has been received from the RPC.
RpcBlock { RpcBlock {
request_id: RequestId, request_id: RequestId,
peer_id: PeerId,
beacon_block: Option<Box<SignedBeaconBlock<T>>>, beacon_block: Option<Box<SignedBeaconBlock<T>>>,
seen_timestamp: Duration,
}, },
/// A [`TxBlobsByRangeResponse`] response has been received. /// A [`TxBlobsByRangeResponse`] response has been received.
TxBlobsByRangeResponse { TxBlobsByRangeResponse {
peer_id: PeerId, peer_id: PeerId,
request_id: RequestId, request_id: RequestId,
blob_wrapper: Option<Box<BlobWrapper<T>>>, blob_wrapper: Option<Box<BlobsSidecar<T>>>,
}, },
/// A [`BlocksByRoot`] response has been received. /// A [`BlocksByRoot`] response has been received.

View File

@ -55,7 +55,7 @@ use lru_cache::LRUTimeCache;
use slog::{crit, debug, trace, warn}; use slog::{crit, debug, trace, warn};
use std::collections::HashMap; use std::collections::HashMap;
use std::sync::Arc; use std::sync::Arc;
use types::{BlobWrapper, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot}; use types::{BlobsSidecar, Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot};
/// For how long we store failed finalized chains to prevent retries. /// For how long we store failed finalized chains to prevent retries.
const FAILED_CHAINS_EXPIRY_SECONDS: u64 = 30; const FAILED_CHAINS_EXPIRY_SECONDS: u64 = 30;

View File

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

View File

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

View File

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

View File

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

View File

@ -1,6 +1,6 @@
use crate::beacon_block_body::{ use crate::beacon_block_body::{
BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyMerge, BeaconBlockBodyRef, BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyCapella, BeaconBlockBodyMerge,
BeaconBlockBodyRefMut, BeaconBlockBodyShanghai, BeaconBlockBodyRef, BeaconBlockBodyRefMut,
}; };
use crate::test_utils::TestRandom; use crate::test_utils::TestRandom;
use crate::*; use crate::*;
@ -17,7 +17,7 @@ use tree_hash_derive::TreeHash;
/// A block of the `BeaconChain`. /// A block of the `BeaconChain`.
#[superstruct( #[superstruct(
variants(Base, Altair, Merge, Shanghai), variants(Base, Altair, Merge, Capella),
variant_attributes( variant_attributes(
derive( derive(
Debug, Debug,
@ -64,8 +64,8 @@ pub struct BeaconBlock<T: EthSpec, Payload: ExecPayload<T> = FullPayload<T>> {
pub body: BeaconBlockBodyAltair<T, Payload>, pub body: BeaconBlockBodyAltair<T, Payload>,
#[superstruct(only(Merge), partial_getter(rename = "body_merge"))] #[superstruct(only(Merge), partial_getter(rename = "body_merge"))]
pub body: BeaconBlockBodyMerge<T, Payload>, pub body: BeaconBlockBodyMerge<T, Payload>,
#[superstruct(only(Shanghai), partial_getter(rename = "body_shanghai"))] #[superstruct(only(Capella), partial_getter(rename = "body_capella"))]
pub body: BeaconBlockBodyShanghai<T, Payload>, pub body: BeaconBlockBodyCapella<T, Payload>,
} }
pub type BlindedBeaconBlock<E> = BeaconBlock<E, BlindedPayload<E>>; 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::Base { .. } => ForkName::Base,
BeaconBlockRef::Altair { .. } => ForkName::Altair, BeaconBlockRef::Altair { .. } => ForkName::Altair,
BeaconBlockRef::Merge { .. } => ForkName::Merge, BeaconBlockRef::Merge { .. } => ForkName::Merge,
BeaconBlockRef::Shanghai { .. } => ForkName::Shanghai, BeaconBlockRef::Capella { .. } => ForkName::Capella,
}; };
if fork_at_slot == object_fork { if fork_at_slot == object_fork {

View File

@ -13,7 +13,7 @@ use tree_hash_derive::TreeHash;
/// ///
/// This *superstruct* abstracts over the hard-fork. /// This *superstruct* abstracts over the hard-fork.
#[superstruct( #[superstruct(
variants(Base, Altair, Merge, Shanghai), variants(Base, Altair, Merge, Capella),
variant_attributes( variant_attributes(
derive( derive(
Debug, Debug,
@ -47,16 +47,16 @@ pub struct BeaconBlockBody<T: EthSpec, Payload: ExecPayload<T> = FullPayload<T>>
pub attestations: VariableList<Attestation<T>, T::MaxAttestations>, pub attestations: VariableList<Attestation<T>, T::MaxAttestations>,
pub deposits: VariableList<Deposit, T::MaxDeposits>, pub deposits: VariableList<Deposit, T::MaxDeposits>,
pub voluntary_exits: VariableList<SignedVoluntaryExit, T::MaxVoluntaryExits>, pub voluntary_exits: VariableList<SignedVoluntaryExit, T::MaxVoluntaryExits>,
#[superstruct(only(Altair, Merge, Shanghai))] #[superstruct(only(Altair, Merge, Capella))]
pub sync_aggregate: SyncAggregate<T>, pub sync_aggregate: SyncAggregate<T>,
// We flatten the execution payload so that serde can use the name of the inner type, // 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 // either `execution_payload` for full payloads, or `execution_payload_header` for blinded
// payloads. // payloads.
#[superstruct(only(Merge, Shanghai))] #[superstruct(only(Merge, Capella))]
#[serde(flatten)] #[serde(flatten)]
pub execution_payload: Payload, pub execution_payload: Payload,
#[superstruct(only(Shanghai))] #[superstruct(only(Capella))]
pub blob_kzgs: VariableList<KZGCommitment, T::MaxObjectListSize>, pub blob_kzgs: VariableList<KZGCommitment, T::MaxBlobsPerBlock>,
#[superstruct(only(Base, Altair))] #[superstruct(only(Base, Altair))]
#[ssz(skip_serializing, skip_deserializing)] #[ssz(skip_serializing, skip_deserializing)]
#[tree_hash(skip_hashing)] #[tree_hash(skip_hashing)]
@ -71,7 +71,7 @@ impl<'a, T: EthSpec> BeaconBlockBodyRef<'a, T> {
BeaconBlockBodyRef::Base { .. } => ForkName::Base, BeaconBlockBodyRef::Base { .. } => ForkName::Base,
BeaconBlockBodyRef::Altair { .. } => ForkName::Altair, BeaconBlockBodyRef::Altair { .. } => ForkName::Altair,
BeaconBlockBodyRef::Merge { .. } => ForkName::Merge, BeaconBlockBodyRef::Merge { .. } => ForkName::Merge,
BeaconBlockBodyRef::Shanghai { .. } => ForkName::Shanghai, BeaconBlockBodyRef::Capella { .. } => ForkName::Capella,
} }
} }
} }

View File

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

View File

@ -8,13 +8,13 @@ use tree_hash_derive::TreeHash;
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))] #[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Debug, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, PartialEq, Default)] #[derive(Debug, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, PartialEq, Default)]
pub struct BlobWrapper<E: EthSpec> { pub struct BlobsSidecar<E: EthSpec> {
pub beacon_block_root: Hash256, pub beacon_block_root: Hash256,
pub beacon_block_slot: Slot, pub beacon_block_slot: Slot,
pub blobs: VariableList<Blob<E::ChunksPerBlob>, E::MaxObjectListSize>, pub blobs: VariableList<Blob<E::FieldElementsPerBlob>, E::MaxBlobsPerBlock>,
} }
impl<E: EthSpec> BlobWrapper<E> { impl<E: EthSpec> BlobsSidecar<E> {
pub fn empty() -> Self { pub fn empty() -> Self {
Self::default() Self::default()
} }
@ -22,6 +22,6 @@ impl<E: EthSpec> BlobWrapper<E> {
// Fixed part // Fixed part
Self::empty().as_ssz_bytes().len() Self::empty().as_ssz_bytes().len()
// Max size of variable length `blobs` field // Max size of variable length `blobs` field
+ (E::max_object_list_size() * <Blob<E::ChunksPerBlob> as Encode>::ssz_fixed_len()) + (E::max_object_list_size() * <Blob<E::FieldElementsPerBlob> as Encode>::ssz_fixed_len())
} }
} }

View File

@ -22,6 +22,8 @@ pub enum Domain {
ContributionAndProof, ContributionAndProof,
SyncCommitteeSelectionProof, SyncCommitteeSelectionProof,
ApplicationMask(ApplicationDomain), ApplicationMask(ApplicationDomain),
//FIXME(sean) add this domain
//BlobsSideCar,
} }
/// Lighthouse's internal configuration struct. /// Lighthouse's internal configuration struct.
@ -151,10 +153,10 @@ pub struct ChainSpec {
pub safe_slots_to_import_optimistically: u64, pub safe_slots_to_import_optimistically: u64,
/* /*
* Shanghai hard fork params * Capella hard fork params
*/ */
pub shanghai_fork_version: [u8; 4], pub capella_fork_version: [u8; 4],
pub shanghai_fork_epoch: Option<Epoch>, pub capella_fork_epoch: Option<Epoch>,
/* /*
* Networking * Networking
@ -236,8 +238,8 @@ impl ChainSpec {
/// Returns the name of the fork which is active at `epoch`. /// Returns the name of the fork which is active at `epoch`.
pub fn fork_name_at_epoch(&self, epoch: Epoch) -> ForkName { pub fn fork_name_at_epoch(&self, epoch: Epoch) -> ForkName {
match self.shanghai_fork_epoch { match self.capella_fork_epoch {
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Shanghai, Some(fork_epoch) if epoch >= fork_epoch => ForkName::Capella,
_ => match self.bellatrix_fork_epoch { _ => match self.bellatrix_fork_epoch {
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Merge, Some(fork_epoch) if epoch >= fork_epoch => ForkName::Merge,
_ => match self.altair_fork_epoch { _ => match self.altair_fork_epoch {
@ -254,7 +256,7 @@ impl ChainSpec {
ForkName::Base => self.genesis_fork_version, ForkName::Base => self.genesis_fork_version,
ForkName::Altair => self.altair_fork_version, ForkName::Altair => self.altair_fork_version,
ForkName::Merge => self.bellatrix_fork_version, ForkName::Merge => self.bellatrix_fork_version,
ForkName::Shanghai => self.shanghai_fork_version, ForkName::Capella => self.capella_fork_version,
} }
} }
@ -264,7 +266,7 @@ impl ChainSpec {
ForkName::Base => Some(Epoch::new(0)), ForkName::Base => Some(Epoch::new(0)),
ForkName::Altair => self.altair_fork_epoch, ForkName::Altair => self.altair_fork_epoch,
ForkName::Merge => self.bellatrix_fork_epoch, ForkName::Merge => self.bellatrix_fork_epoch,
ForkName::Shanghai => self.shanghai_fork_epoch, ForkName::Capella => self.capella_fork_epoch,
} }
} }
@ -274,7 +276,7 @@ impl ChainSpec {
BeaconState::Base(_) => self.inactivity_penalty_quotient, BeaconState::Base(_) => self.inactivity_penalty_quotient,
BeaconState::Altair(_) => self.inactivity_penalty_quotient_altair, BeaconState::Altair(_) => self.inactivity_penalty_quotient_altair,
BeaconState::Merge(_) => self.inactivity_penalty_quotient_bellatrix, BeaconState::Merge(_) => self.inactivity_penalty_quotient_bellatrix,
BeaconState::Shanghai(_) => self.inactivity_penalty_quotient_bellatrix, BeaconState::Capella(_) => self.inactivity_penalty_quotient_bellatrix,
} }
} }
@ -287,7 +289,7 @@ impl ChainSpec {
BeaconState::Base(_) => self.proportional_slashing_multiplier, BeaconState::Base(_) => self.proportional_slashing_multiplier,
BeaconState::Altair(_) => self.proportional_slashing_multiplier_altair, BeaconState::Altair(_) => self.proportional_slashing_multiplier_altair,
BeaconState::Merge(_) => self.proportional_slashing_multiplier_bellatrix, BeaconState::Merge(_) => self.proportional_slashing_multiplier_bellatrix,
BeaconState::Shanghai(_) => self.proportional_slashing_multiplier_bellatrix, BeaconState::Capella(_) => self.proportional_slashing_multiplier_bellatrix,
} }
} }
@ -300,7 +302,7 @@ impl ChainSpec {
BeaconState::Base(_) => self.min_slashing_penalty_quotient, BeaconState::Base(_) => self.min_slashing_penalty_quotient,
BeaconState::Altair(_) => self.min_slashing_penalty_quotient_altair, BeaconState::Altair(_) => self.min_slashing_penalty_quotient_altair,
BeaconState::Merge(_) => self.min_slashing_penalty_quotient_bellatrix, BeaconState::Merge(_) => self.min_slashing_penalty_quotient_bellatrix,
BeaconState::Shanghai(_) => self.min_slashing_penalty_quotient_bellatrix, BeaconState::Capella(_) => self.min_slashing_penalty_quotient_bellatrix,
} }
} }
@ -583,12 +585,11 @@ impl ChainSpec {
safe_slots_to_import_optimistically: 128u64, safe_slots_to_import_optimistically: 128u64,
/* /*
* Shanghai hardfork params * Capella hardfork params
*/ */
//FIXME(sean) //FIXME(sean)
shanghai_fork_version: [0x03, 0x00, 0x00, 0x00], capella_fork_version: [0x03, 0x00, 0x00, 0x00],
shanghai_fork_epoch: None, capella_fork_epoch: None,
/* /*
* Network specific * Network specific
*/ */
@ -644,10 +645,10 @@ impl ChainSpec {
// `Uint256::MAX` which is `2*256- 1`. // `Uint256::MAX` which is `2*256- 1`.
.checked_add(Uint256::one()) .checked_add(Uint256::one())
.expect("addition does not overflow"), .expect("addition does not overflow"),
// Shanghai // Capella
//FIXME(sean) //FIXME(sean)
shanghai_fork_version: [0x03, 0x00, 0x00, 0x01], capella_fork_version: [0x03, 0x00, 0x00, 0x01],
shanghai_fork_epoch: None, capella_fork_epoch: None,
// Other // Other
network_id: 2, // lighthouse testnet network id network_id: 2, // lighthouse testnet network id
deposit_chain_id: 5, deposit_chain_id: 5,
@ -804,8 +805,8 @@ impl ChainSpec {
safe_slots_to_import_optimistically: 128u64, safe_slots_to_import_optimistically: 128u64,
//FIXME(sean) //FIXME(sean)
shanghai_fork_version: [0x03, 0x00, 0x00, 0x64], capella_fork_version: [0x03, 0x00, 0x00, 0x64],
shanghai_fork_epoch: None, capella_fork_epoch: None,
/* /*
* Network specific * Network specific
@ -883,14 +884,14 @@ pub struct Config {
pub bellatrix_fork_epoch: Option<MaybeQuoted<Epoch>>, pub bellatrix_fork_epoch: Option<MaybeQuoted<Epoch>>,
// FIXME(sean): remove this default // FIXME(sean): remove this default
#[serde(default = "default_shanghai_fork_version")] #[serde(default = "default_capella_fork_version")]
#[serde(with = "eth2_serde_utils::bytes_4_hex")] #[serde(with = "eth2_serde_utils::bytes_4_hex")]
shanghai_fork_version: [u8; 4], capella_fork_version: [u8; 4],
// FIXME(sean): remove this default // FIXME(sean): remove this default
#[serde(default = "default_shanghai_fork_epoch")] #[serde(default = "default_capella_fork_epoch")]
#[serde(serialize_with = "serialize_fork_epoch")] #[serde(serialize_with = "serialize_fork_epoch")]
#[serde(deserialize_with = "deserialize_fork_epoch")] #[serde(deserialize_with = "deserialize_fork_epoch")]
pub shanghai_fork_epoch: Option<MaybeQuoted<Epoch>>, pub capella_fork_epoch: Option<MaybeQuoted<Epoch>>,
#[serde(with = "eth2_serde_utils::quoted_u64")] #[serde(with = "eth2_serde_utils::quoted_u64")]
seconds_per_slot: u64, seconds_per_slot: u64,
@ -929,7 +930,7 @@ fn default_bellatrix_fork_version() -> [u8; 4] {
[0xff, 0xff, 0xff, 0xff] [0xff, 0xff, 0xff, 0xff]
} }
fn default_shanghai_fork_version() -> [u8; 4] { fn default_capella_fork_version() -> [u8; 4] {
// This value shouldn't be used. // This value shouldn't be used.
[0xff, 0xff, 0xff, 0xff] [0xff, 0xff, 0xff, 0xff]
} }
@ -1030,9 +1031,9 @@ impl Config {
bellatrix_fork_epoch: spec bellatrix_fork_epoch: spec
.bellatrix_fork_epoch .bellatrix_fork_epoch
.map(|epoch| MaybeQuoted { value: epoch }), .map(|epoch| MaybeQuoted { value: epoch }),
shanghai_fork_version: spec.shanghai_fork_version, capella_fork_version: spec.capella_fork_version,
shanghai_fork_epoch: spec capella_fork_epoch: spec
.shanghai_fork_epoch .capella_fork_epoch
.map(|epoch| MaybeQuoted { value: epoch }), .map(|epoch| MaybeQuoted { value: epoch }),
seconds_per_slot: spec.seconds_per_slot, seconds_per_slot: spec.seconds_per_slot,
@ -1079,8 +1080,8 @@ impl Config {
altair_fork_epoch, altair_fork_epoch,
bellatrix_fork_epoch, bellatrix_fork_epoch,
bellatrix_fork_version, bellatrix_fork_version,
shanghai_fork_epoch, capella_fork_epoch,
shanghai_fork_version, capella_fork_version,
seconds_per_slot, seconds_per_slot,
seconds_per_eth1_block, seconds_per_eth1_block,
min_validator_withdrawability_delay, min_validator_withdrawability_delay,
@ -1111,8 +1112,8 @@ impl Config {
altair_fork_epoch: altair_fork_epoch.map(|q| q.value), altair_fork_epoch: altair_fork_epoch.map(|q| q.value),
bellatrix_fork_epoch: bellatrix_fork_epoch.map(|q| q.value), bellatrix_fork_epoch: bellatrix_fork_epoch.map(|q| q.value),
bellatrix_fork_version, bellatrix_fork_version,
shanghai_fork_epoch: shanghai_fork_epoch.map(|q| q.value), capella_fork_epoch: capella_fork_epoch.map(|q| q.value),
shanghai_fork_version, capella_fork_version,
seconds_per_slot, seconds_per_slot,
seconds_per_eth1_block, seconds_per_eth1_block,
min_validator_withdrawability_delay, min_validator_withdrawability_delay,

View File

@ -22,3 +22,16 @@ pub mod altair {
pub mod merge { pub mod merge {
pub const INTERVALS_PER_SLOT: u64 = 3; pub const INTERVALS_PER_SLOT: u64 = 3;
} }
pub mod cappella {
use crate::Uint256;
use lazy_static::lazy_static;
lazy_static! {
pub static ref BLS_MODULUS: Uint256 = Uint256::from_dec_str(
"52435875175126190479447740508185965837690552500527637822603658699938581184513"
)
.expect("should initialize BLS_MODULUS");
}
pub const BLOB_TX_TYPE: u8 = 5;
}

View File

@ -96,10 +96,10 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq +
type MinGasLimit: Unsigned + Clone + Sync + Send + Debug + PartialEq; type MinGasLimit: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type MaxExtraDataBytes: Unsigned + Clone + Sync + Send + Debug + PartialEq; type MaxExtraDataBytes: Unsigned + Clone + Sync + Send + Debug + PartialEq;
/* /*
* New in Shanghaisharding * New in Capella
*/ */
type MaxObjectListSize: Unsigned + Clone + Sync + Send + Debug + PartialEq; type MaxBlobsPerBlock: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type ChunksPerBlob: Unsigned + Clone + Sync + Send + Debug + PartialEq; type FieldElementsPerBlob: Unsigned + Clone + Sync + Send + Debug + PartialEq;
/* /*
* Derived values (set these CAREFULLY) * Derived values (set these CAREFULLY)
*/ */
@ -229,11 +229,11 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq +
} }
fn max_object_list_size() -> usize { fn max_object_list_size() -> usize {
Self::MaxObjectListSize::to_usize() Self::MaxBlobsPerBlock::to_usize()
} }
fn chunks_per_blob() -> usize { fn chunks_per_blob() -> usize {
Self::ChunksPerBlob::to_usize() Self::FieldElementsPerBlob::to_usize()
} }
} }
@ -275,8 +275,8 @@ impl EthSpec for MainnetEthSpec {
type GasLimitDenominator = U1024; type GasLimitDenominator = U1024;
type MinGasLimit = U5000; type MinGasLimit = U5000;
type MaxExtraDataBytes = U32; type MaxExtraDataBytes = U32;
type MaxObjectListSize = U16777216; // 2**24 type MaxBlobsPerBlock = U16777216; // 2**24
type ChunksPerBlob = U4096; type FieldElementsPerBlob = U4096;
type SyncSubcommitteeSize = U128; // 512 committee size / 4 sync committee subnet count type SyncSubcommitteeSize = U128; // 512 committee size / 4 sync committee subnet count
type MaxPendingAttestations = U4096; // 128 max attestations * 32 slots per epoch type MaxPendingAttestations = U4096; // 128 max attestations * 32 slots per epoch
type SlotsPerEth1VotingPeriod = U2048; // 64 epochs * 32 slots per epoch type SlotsPerEth1VotingPeriod = U2048; // 64 epochs * 32 slots per epoch
@ -325,8 +325,8 @@ impl EthSpec for MinimalEthSpec {
GasLimitDenominator, GasLimitDenominator,
MinGasLimit, MinGasLimit,
MaxExtraDataBytes, MaxExtraDataBytes,
MaxObjectListSize, MaxBlobsPerBlock,
ChunksPerBlob FieldElementsPerBlob
}); });
fn default_spec() -> ChainSpec { fn default_spec() -> ChainSpec {
@ -371,8 +371,8 @@ impl EthSpec for GnosisEthSpec {
type SyncSubcommitteeSize = U128; // 512 committee size / 4 sync committee subnet count type SyncSubcommitteeSize = U128; // 512 committee size / 4 sync committee subnet count
type MaxPendingAttestations = U2048; // 128 max attestations * 16 slots per epoch type MaxPendingAttestations = U2048; // 128 max attestations * 16 slots per epoch
type SlotsPerEth1VotingPeriod = U1024; // 64 epochs * 16 slots per epoch type SlotsPerEth1VotingPeriod = U1024; // 64 epochs * 16 slots per epoch
type MaxObjectListSize = U16777216; // 2**24 type MaxBlobsPerBlock = U16777216; // 2**24
type ChunksPerBlob = U4096; type FieldElementsPerBlob = U4096;
fn default_spec() -> ChainSpec { fn default_spec() -> ChainSpec {
ChainSpec::gnosis() ChainSpec::gnosis()

View File

@ -47,10 +47,10 @@ impl ForkContext {
)); ));
} }
if spec.shanghai_fork_epoch.is_some() { if spec.capella_fork_epoch.is_some() {
fork_to_digest.push(( fork_to_digest.push((
ForkName::Shanghai, ForkName::Capella,
ChainSpec::compute_fork_digest(spec.shanghai_fork_version, genesis_validators_root), ChainSpec::compute_fork_digest(spec.capella_fork_version, genesis_validators_root),
)); ));
} }

View File

@ -11,7 +11,7 @@ pub enum ForkName {
Base, Base,
Altair, Altair,
Merge, Merge,
Shanghai, Capella,
} }
impl ForkName { impl ForkName {
@ -39,9 +39,9 @@ impl ForkName {
spec.bellatrix_fork_epoch = Some(Epoch::new(0)); spec.bellatrix_fork_epoch = Some(Epoch::new(0));
spec spec
} }
ForkName::Shanghai => { ForkName::Capella => {
spec.bellatrix_fork_epoch = Some(Epoch::new(0)); spec.bellatrix_fork_epoch = Some(Epoch::new(0));
spec.shanghai_fork_epoch = Some(Epoch::new(0)); spec.capella_fork_epoch = Some(Epoch::new(0));
spec spec
} }
} }
@ -55,7 +55,7 @@ impl ForkName {
ForkName::Base => None, ForkName::Base => None,
ForkName::Altair => Some(ForkName::Base), ForkName::Altair => Some(ForkName::Base),
ForkName::Merge => Some(ForkName::Altair), ForkName::Merge => Some(ForkName::Altair),
ForkName::Shanghai => Some(ForkName::Merge), ForkName::Capella => Some(ForkName::Merge),
} }
} }
@ -66,8 +66,8 @@ impl ForkName {
match self { match self {
ForkName::Base => Some(ForkName::Altair), ForkName::Base => Some(ForkName::Altair),
ForkName::Altair => Some(ForkName::Merge), ForkName::Altair => Some(ForkName::Merge),
ForkName::Merge => Some(ForkName::Shanghai), ForkName::Merge => Some(ForkName::Capella),
ForkName::Shanghai => None, ForkName::Capella => None,
} }
} }
} }
@ -110,7 +110,7 @@ macro_rules! map_fork_name_with {
($t::Merge(value), extra_data) ($t::Merge(value), extra_data)
} }
//TODO: don't have a beacon state variant for the new fork yet //TODO: don't have a beacon state variant for the new fork yet
ForkName::Shanghai => { ForkName::Capella => {
let (value, extra_data) = $body; let (value, extra_data) = $body;
($t::Merge(value), extra_data) ($t::Merge(value), extra_data)
} }
@ -137,7 +137,7 @@ impl Display for ForkName {
ForkName::Base => "phase0".fmt(f), ForkName::Base => "phase0".fmt(f),
ForkName::Altair => "altair".fmt(f), ForkName::Altair => "altair".fmt(f),
ForkName::Merge => "bellatrix".fmt(f), ForkName::Merge => "bellatrix".fmt(f),
ForkName::Shanghai => "shanghai".fmt(f), ForkName::Capella => "capella".fmt(f),
} }
} }
} }

View File

@ -86,11 +86,13 @@ pub mod sync_subnet_id;
mod tree_hash_impls; mod tree_hash_impls;
pub mod validator_registration_data; pub mod validator_registration_data;
mod blob_wrapper; mod blobs_sidecar;
mod kzg_commitment; mod kzg_commitment;
mod signed_blobs_sidecar;
pub mod slot_data; pub mod slot_data;
#[cfg(feature = "sqlite")] #[cfg(feature = "sqlite")]
pub mod sqlite; pub mod sqlite;
pub use kzg_commitment::KZGCommitment; pub use kzg_commitment::KZGCommitment;
use ethereum_types::{H160, H256}; use ethereum_types::{H160, H256};
@ -101,17 +103,17 @@ pub use crate::attestation_data::AttestationData;
pub use crate::attestation_duty::AttestationDuty; pub use crate::attestation_duty::AttestationDuty;
pub use crate::attester_slashing::AttesterSlashing; pub use crate::attester_slashing::AttesterSlashing;
pub use crate::beacon_block::{ pub use crate::beacon_block::{
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockShanghai, BeaconBlockMerge, BeaconBlockRef, BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockCapella, BeaconBlockMerge, BeaconBlockRef,
BeaconBlockRefMut, BlindedBeaconBlock, BeaconBlockRefMut, BlindedBeaconBlock,
}; };
pub use crate::beacon_block_body::{ pub use crate::beacon_block_body::{
BeaconBlockBody, BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyMerge, BeaconBlockBody, BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyCapella,
BeaconBlockBodyRef, BeaconBlockBodyRefMut, BeaconBlockBodyShanghai, BeaconBlockBodyMerge, BeaconBlockBodyRef, BeaconBlockBodyRefMut,
}; };
pub use crate::beacon_block_header::BeaconBlockHeader; pub use crate::beacon_block_header::BeaconBlockHeader;
pub use crate::beacon_committee::{BeaconCommittee, OwnedBeaconCommittee}; pub use crate::beacon_committee::{BeaconCommittee, OwnedBeaconCommittee};
pub use crate::beacon_state::{BeaconTreeHashCache, Error as BeaconStateError, *}; pub use crate::beacon_state::{BeaconTreeHashCache, Error as BeaconStateError, *};
pub use crate::blob_wrapper::BlobWrapper; pub use crate::blobs_sidecar::BlobsSidecar;
pub use crate::chain_spec::{ChainSpec, Config, Domain}; pub use crate::chain_spec::{ChainSpec, Config, Domain};
pub use crate::checkpoint::Checkpoint; pub use crate::checkpoint::Checkpoint;
pub use crate::config_and_preset::{ pub use crate::config_and_preset::{
@ -148,7 +150,7 @@ pub use crate::shuffling_id::AttestationShufflingId;
pub use crate::signed_aggregate_and_proof::SignedAggregateAndProof; pub use crate::signed_aggregate_and_proof::SignedAggregateAndProof;
pub use crate::signed_beacon_block::{ pub use crate::signed_beacon_block::{
SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockHash, SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockHash,
SignedBeaconBlockMerge, SignedBlindedBeaconBlock,SignedBeaconBlockShanghai SignedBeaconBlockMerge, SignedBlindedBeaconBlock,SignedBeaconBlockCapella
}; };
pub use crate::signed_beacon_block_header::SignedBeaconBlockHeader; pub use crate::signed_beacon_block_header::SignedBeaconBlockHeader;
pub use crate::signed_contribution_and_proof::SignedContributionAndProof; pub use crate::signed_contribution_and_proof::SignedContributionAndProof;

View File

@ -38,7 +38,7 @@ impl From<SignedBeaconBlockHash> for Hash256 {
/// A `BeaconBlock` and a signature from its proposer. /// A `BeaconBlock` and a signature from its proposer.
#[superstruct( #[superstruct(
variants(Base, Altair, Merge, Shanghai), variants(Base, Altair, Merge, Capella),
variant_attributes( variant_attributes(
derive( derive(
Debug, Debug,
@ -72,8 +72,8 @@ pub struct SignedBeaconBlock<E: EthSpec, Payload: ExecPayload<E> = FullPayload<E
pub message: BeaconBlockAltair<E, Payload>, pub message: BeaconBlockAltair<E, Payload>,
#[superstruct(only(Merge), partial_getter(rename = "message_merge"))] #[superstruct(only(Merge), partial_getter(rename = "message_merge"))]
pub message: BeaconBlockMerge<E, Payload>, pub message: BeaconBlockMerge<E, Payload>,
#[superstruct(only(Shanghai), partial_getter(rename = "message_shanghai"))] #[superstruct(only(Capella), partial_getter(rename = "message_capella"))]
pub message: BeaconBlockShanghai<E, Payload>, pub message: BeaconBlockCapella<E, Payload>,
pub signature: Signature, pub signature: Signature,
} }
@ -131,8 +131,8 @@ impl<E: EthSpec, Payload: ExecPayload<E>> SignedBeaconBlock<E, Payload> {
BeaconBlock::Merge(message) => { BeaconBlock::Merge(message) => {
SignedBeaconBlock::Merge(SignedBeaconBlockMerge { message, signature }) SignedBeaconBlock::Merge(SignedBeaconBlockMerge { message, signature })
} }
BeaconBlock::Shanghai(message) => { BeaconBlock::Capella(message) => {
SignedBeaconBlock::Shanghai(SignedBeaconBlockShanghai { message, signature }) SignedBeaconBlock::Capella(SignedBeaconBlockCapella { message, signature })
} }
} }
} }

View File

@ -0,0 +1,15 @@
use crate::{Blob, BlobsSidecar, EthSpec, Hash256, Slot};
use bls::Signature;
use serde_derive::{Deserialize, Serialize};
use ssz::Encode;
use ssz_derive::{Decode, Encode};
use ssz_types::VariableList;
use tree_hash::TreeHash;
use tree_hash_derive::TreeHash;
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Debug, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, PartialEq)]
pub struct SignedBlobsSidecar<E: EthSpec> {
pub message: BlobsSidecar<E>,
pub signature: Signature,
}

View File

@ -78,6 +78,6 @@ pub fn previous_fork(fork_name: ForkName) -> ForkName {
ForkName::Base => ForkName::Base, ForkName::Base => ForkName::Base,
ForkName::Altair => ForkName::Base, ForkName::Altair => ForkName::Base,
ForkName::Merge => ForkName::Altair, // TODO: Check this when tests are released.. ForkName::Merge => ForkName::Altair, // TODO: Check this when tests are released..
ForkName::Shanghai => ForkName::Merge, // TODO: Check this when tests are released.. ForkName::Capella => 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); justification_and_finalization_state.apply_changes_to_state(state);
Ok(()) Ok(())
} }
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Shanghai(_) => { BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
let justification_and_finalization_state = let justification_and_finalization_state =
altair::process_justification_and_finalization( altair::process_justification_and_finalization(
state, state,
@ -118,7 +118,7 @@ impl<E: EthSpec> EpochTransition<E> for RewardsAndPenalties {
validator_statuses.process_attestations(state)?; validator_statuses.process_attestations(state)?;
base::process_rewards_and_penalties(state, &mut validator_statuses, spec) base::process_rewards_and_penalties(state, &mut validator_statuses, spec)
} }
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Shanghai(_) => { BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
altair::process_rewards_and_penalties( altair::process_rewards_and_penalties(
state, state,
&altair::ParticipationCache::new(state, spec).unwrap(), &altair::ParticipationCache::new(state, spec).unwrap(),
@ -147,7 +147,7 @@ impl<E: EthSpec> EpochTransition<E> for Slashings {
spec, spec,
)?; )?;
} }
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Shanghai(_) => { BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
process_slashings( process_slashings(
state, state,
altair::ParticipationCache::new(state, spec) 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> { fn run(state: &mut BeaconState<E>, spec: &ChainSpec) -> Result<(), EpochProcessingError> {
match state { match state {
BeaconState::Base(_) => Ok(()), BeaconState::Base(_) => Ok(()),
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Shanghai(_) => { BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
altair::process_sync_committee_updates(state, spec) 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> { fn run(state: &mut BeaconState<E>, spec: &ChainSpec) -> Result<(), EpochProcessingError> {
match state { match state {
BeaconState::Base(_) => Ok(()), BeaconState::Base(_) => Ok(()),
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Shanghai(_) => { BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
altair::process_inactivity_updates( altair::process_inactivity_updates(
state, state,
&altair::ParticipationCache::new(state, spec).unwrap(), &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> { fn run(state: &mut BeaconState<E>, _: &ChainSpec) -> Result<(), EpochProcessingError> {
match state { match state {
BeaconState::Base(_) => Ok(()), BeaconState::Base(_) => Ok(()),
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Shanghai(_) => { BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
altair::process_participation_flag_updates(state) 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. // No phase0 tests for Altair and later.
ForkName::Altair | ForkName::Merge => T::name() != "participation_record_updates", ForkName::Altair | ForkName::Merge => T::name() != "participation_record_updates",
ForkName::Shanghai => false, // TODO: revisit when tests are out ForkName::Capella => 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::Base => panic!("phase0 not supported"),
ForkName::Altair => upgrade_to_altair(&mut result_state, spec).map(|_| result_state), 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::Merge => upgrade_to_bellatrix(&mut result_state, spec).map(|_| result_state),
ForkName::Shanghai => panic!("shanghai not supported"), ForkName::Capella => panic!("capella not supported"),
}; };
compare_beacon_state_results_without_caches(&mut result, &mut expected) 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(_) => { BeaconState::Base(_) => {
base::process_attestations(state, &[self.clone()], VerifySignatures::True, spec) base::process_attestations(state, &[self.clone()], VerifySignatures::True, spec)
} }
BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Shanghai(_) => { BeaconState::Altair(_) | BeaconState::Merge(_) | BeaconState::Capella(_) => {
altair::process_attestation( altair::process_attestation(
state, state,
self, self,

View File

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

View File

@ -90,8 +90,8 @@ impl<'a, T: EthSpec, Payload: ExecPayload<T>> Web3SignerObject<'a, T, Payload> {
block: None, block: None,
block_header: Some(block.block_header()), block_header: Some(block.block_header()),
}), }),
BeaconBlock::Shanghai(_) => Ok(Web3SignerObject::BeaconBlock { BeaconBlock::Capella(_) => Ok(Web3SignerObject::BeaconBlock {
version: ForkName::Shanghai, version: ForkName::Capella,
block: None, block: None,
block_header: Some(block.block_header()), block_header: Some(block.block_header()),
}), }),