Initial merge changes

Added Execution Payload from Rayonism Fork

Updated new Containers to match Merge Spec

Updated BeaconBlockBody for Merge Spec

Completed updating BeaconState and BeaconBlockBody

Modified ExecutionPayload<T> to use Transaction<T>

Mostly Finished Changes for beacon-chain.md

Added some things for fork-choice.md

Update to match new fork-choice.md/fork.md changes

ran cargo fmt

Added Missing Pieces in eth2_libp2p for Merge

fix ef test

Various Changes to Conform Closer to Merge Spec
This commit is contained in:
Mark Mackey 2021-09-08 13:45:22 -05:00 committed by Paul Hauner
parent fe75a0a9a1
commit 5687c56d51
No known key found for this signature in database
GPG Key ID: 5E2CFF9B75FA63DF
50 changed files with 1241 additions and 133 deletions

1
Cargo.lock generated
View File

@ -1923,6 +1923,7 @@ dependencies = [
"eth2_ssz 0.4.0 (registry+https://github.com/rust-lang/crates.io-index)",
"eth2_ssz_derive 0.3.0 (registry+https://github.com/rust-lang/crates.io-index)",
"proto_array",
"state_processing",
"store",
"types",
]

View File

@ -2838,6 +2838,11 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
SyncAggregate::new()
}))
};
// Closure to fetch a sync aggregate in cases where it is required.
let get_execution_payload = || -> Result<ExecutionPayload<_>, BlockProductionError> {
// TODO: actually get the payload from eth1 node..
Ok(ExecutionPayload::default())
};
let inner_block = match state {
BeaconState::Base(_) => BeaconBlock::Base(BeaconBlockBase {
@ -2876,6 +2881,28 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
},
})
}
BeaconState::Merge(_) => {
let sync_aggregate = get_sync_aggregate()?;
let execution_payload = get_execution_payload()?;
BeaconBlock::Merge(BeaconBlockMerge {
slot,
proposer_index,
parent_root,
state_root: Hash256::zero(),
body: BeaconBlockBodyMerge {
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,
},
})
}
};
let block = SignedBeaconBlock::from_block(

View File

@ -48,7 +48,7 @@ use crate::{
BLOCK_PROCESSING_CACHE_LOCK_TIMEOUT, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT,
},
metrics, BeaconChain, BeaconChainError, BeaconChainTypes,
eth1_chain, metrics, BeaconChain, BeaconChainError, BeaconChainTypes,
};
use fork_choice::{ForkChoice, ForkChoiceStore};
use parking_lot::RwLockReadGuard;
@ -56,6 +56,7 @@ use proto_array::Block as ProtoBlock;
use slog::{debug, error, Logger};
use slot_clock::SlotClock;
use ssz::Encode;
use state_processing::per_block_processing::{is_execution_enabled, is_merge_complete};
use state_processing::{
block_signature_verifier::{BlockSignatureVerifier, Error as BlockSignatureVerifierError},
per_block_processing, per_slot_processing,
@ -68,9 +69,9 @@ use std::io::Write;
use store::{Error as DBError, HotColdDB, HotStateSummary, KeyValueStore, StoreOp};
use tree_hash::TreeHash;
use types::{
BeaconBlockRef, BeaconState, BeaconStateError, ChainSpec, CloneConfig, Epoch, EthSpec, Hash256,
InconsistentFork, PublicKey, PublicKeyBytes, RelativeEpoch, SignedBeaconBlock,
SignedBeaconBlockHeader, Slot,
BeaconBlockRef, BeaconState, BeaconStateError, ChainSpec, CloneConfig, Epoch, EthSpec,
ExecutionPayload, Hash256, InconsistentFork, PublicKey, PublicKeyBytes, RelativeEpoch,
SignedBeaconBlock, SignedBeaconBlockHeader, Slot,
};
/// Maximum block slot number. Block with slots bigger than this constant will NOT be processed.
@ -223,6 +224,66 @@ pub enum BlockError<T: EthSpec> {
///
/// The block is invalid and the peer is faulty.
InconsistentFork(InconsistentFork),
/// There was an error while validating the ExecutionPayload
///
/// ## Peer scoring
///
/// See `ExecutionPayloadError` for scoring information
ExecutionPayloadError(ExecutionPayloadError),
}
/// Returned when block validation failed due to some issue verifying
/// the execution payload.
#[derive(Debug)]
pub enum ExecutionPayloadError {
/// There's no eth1 connection (mandatory after merge)
///
/// ## Peer scoring
///
/// As this is our fault, do not penalize the peer
NoEth1Connection,
/// Error occurred during engine_executePayload
///
/// ## Peer scoring
///
/// Some issue with our configuration, do not penalize peer
Eth1VerificationError(eth1_chain::Error),
/// The execution engine returned INVALID for the payload
///
/// ## Peer scoring
///
/// The block is invalid and the peer is faulty
RejectedByExecutionEngine,
/// The execution payload is empty when is shouldn't be
///
/// ## Peer scoring
///
/// The block is invalid and the peer is faulty
PayloadEmpty,
/// The execution payload timestamp does not match the slot
///
/// ## Peer scoring
///
/// The block is invalid and the peer is faulty
InvalidPayloadTimestamp,
/// The gas used in the block exceeds the gas limit
///
/// ## Peer scoring
///
/// The block is invalid and the peer is faulty
GasUsedExceedsLimit,
/// The payload block hash equals the parent hash
///
/// ## Peer scoring
///
/// The block is invalid and the peer is faulty
BlockHashEqualsParentHash,
/// The execution payload transaction list data exceeds size limits
///
/// ## Peer scoring
///
/// The block is invalid and the peer is faulty
TransactionDataExceedsSizeLimit,
}
impl<T: EthSpec> std::fmt::Display for BlockError<T> {
@ -668,6 +729,18 @@ impl<T: BeaconChainTypes> GossipVerifiedBlock<T> {
});
}
// TODO: avoid this by adding field to fork-choice to determine if merge-block has been imported
let (parent, block) = if let Some(snapshot) = parent {
(Some(snapshot), block)
} else {
let (snapshot, block) = load_parent(block, chain)?;
(Some(snapshot), block)
};
let state = &parent.as_ref().unwrap().pre_state;
// validate the block's execution_payload
validate_execution_payload(block.message(), state)?;
Ok(Self {
block,
block_root,
@ -989,6 +1062,34 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
}
}
// This is the soonest we can run these checks as they must be called AFTER per_slot_processing
if is_execution_enabled(&state, block.message().body()) {
let eth1_chain = chain
.eth1_chain
.as_ref()
.ok_or(BlockError::ExecutionPayloadError(
ExecutionPayloadError::NoEth1Connection,
))?;
if !eth1_chain
.on_payload(block.message().body().execution_payload().ok_or(
BlockError::InconsistentFork(InconsistentFork {
fork_at_slot: eth2::types::ForkName::Merge,
object_fork: block.message().body().fork_name(),
}),
)?)
.map_err(|e| {
BlockError::ExecutionPayloadError(ExecutionPayloadError::Eth1VerificationError(
e,
))
})?
{
return Err(BlockError::ExecutionPayloadError(
ExecutionPayloadError::RejectedByExecutionEngine,
));
}
}
// If the block is sufficiently recent, notify the validator monitor.
if let Some(slot) = chain.slot_clock.now() {
let epoch = slot.epoch(T::EthSpec::slots_per_epoch());
@ -1097,6 +1198,38 @@ impl<'a, T: BeaconChainTypes> FullyVerifiedBlock<'a, T> {
}
}
/// Validate the gossip block's execution_payload according to the checks described here:
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/p2p-interface.md#beacon_block
fn validate_execution_payload<E: EthSpec>(
block: BeaconBlockRef<'_, E>,
state: &BeaconState<E>,
) -> Result<(), BlockError<E>> {
if !is_execution_enabled(state, block.body()) {
return Ok(());
}
let execution_payload = block
.body()
.execution_payload()
// TODO: this really should never error so maybe
// we should make this simpler..
.ok_or(BlockError::InconsistentFork(InconsistentFork {
fork_at_slot: eth2::types::ForkName::Merge,
object_fork: block.body().fork_name(),
}))?;
if is_merge_complete(state) {
if *execution_payload == <ExecutionPayload<E>>::default() {
return Err(BlockError::ExecutionPayloadError(
ExecutionPayloadError::PayloadEmpty,
));
}
}
// TODO: finish these
Ok(())
}
/// Check that the count of skip slots between the block and its parent does not exceed our maximum
/// value.
///

View File

@ -15,8 +15,8 @@ use std::time::{SystemTime, UNIX_EPOCH};
use store::{DBColumn, Error as StoreError, StoreItem};
use task_executor::TaskExecutor;
use types::{
BeaconState, BeaconStateError, ChainSpec, Deposit, Eth1Data, EthSpec, Hash256, Slot, Unsigned,
DEPOSIT_TREE_DEPTH,
BeaconState, BeaconStateError, ChainSpec, Deposit, Eth1Data, EthSpec, ExecutionPayload,
Hash256, Slot, Unsigned, DEPOSIT_TREE_DEPTH,
};
type BlockNumber = u64;
@ -53,6 +53,8 @@ pub enum Error {
UnknownPreviousEth1BlockHash,
/// An arithmetic error occurred.
ArithError(safe_arith::ArithError),
/// Unable to execute payload
UnableToExecutePayload(String),
}
impl From<safe_arith::ArithError> for Error {
@ -274,6 +276,15 @@ where
)
}
pub fn on_payload(&self, execution_payload: &ExecutionPayload<E>) -> Result<bool, Error> {
if self.use_dummy_backend {
let dummy_backend: DummyEth1ChainBackend<E> = DummyEth1ChainBackend::default();
dummy_backend.on_payload(execution_payload)
} else {
self.backend.on_payload(execution_payload)
}
}
/// Instantiate `Eth1Chain` from a persisted `SszEth1`.
///
/// The `Eth1Chain` will have the same caches as the persisted `SszEth1`.
@ -334,6 +345,9 @@ pub trait Eth1ChainBackend<T: EthSpec>: Sized + Send + Sync {
/// an idea of how up-to-date the remote eth1 node is.
fn head_block(&self) -> Option<Eth1Block>;
/// Verifies the execution payload
fn on_payload(&self, execution_payload: &ExecutionPayload<T>) -> Result<bool, Error>;
/// Encode the `Eth1ChainBackend` instance to bytes.
fn as_bytes(&self) -> Vec<u8>;
@ -388,6 +402,10 @@ impl<T: EthSpec> Eth1ChainBackend<T> for DummyEth1ChainBackend<T> {
None
}
fn on_payload(&self, _execution_payload: &ExecutionPayload<T>) -> Result<bool, Error> {
Ok(true)
}
/// Return empty Vec<u8> for dummy backend.
fn as_bytes(&self) -> Vec<u8> {
Vec::new()
@ -556,6 +574,15 @@ impl<T: EthSpec> Eth1ChainBackend<T> for CachingEth1Backend<T> {
self.core.head_block()
}
fn on_payload(&self, execution_payload: &ExecutionPayload<T>) -> Result<bool, Error> {
futures::executor::block_on(async move {
self.core
.on_payload(execution_payload.clone())
.await
.map_err(|e| Error::UnableToExecutePayload(format!("{:?}", e)))
})
}
/// Return encoded byte representation of the block and deposit caches.
fn as_bytes(&self) -> Vec<u8> {
self.core.as_bytes()

View File

@ -44,7 +44,7 @@ pub use self::errors::{BeaconChainError, BlockProductionError};
pub use self::historical_blocks::HistoricalBlockError;
pub use attestation_verification::Error as AttestationError;
pub use beacon_fork_choice_store::{BeaconForkChoiceStore, Error as ForkChoiceStoreError};
pub use block_verification::{BlockError, GossipVerifiedBlock};
pub use block_verification::{BlockError, ExecutionPayloadError, GossipVerifiedBlock};
pub use eth1_chain::{Eth1Chain, Eth1ChainBackend};
pub use events::ServerSentEventHandler;
pub use metrics::scrape_for_metrics;

View File

@ -19,7 +19,7 @@ use std::fmt;
use std::ops::Range;
use std::str::FromStr;
use std::time::Duration;
use types::Hash256;
use types::{Hash256, PowBlock, Uint256};
/// `keccak("DepositEvent(bytes,bytes,bytes,bytes,bytes)")`
pub const DEPOSIT_EVENT_TOPIC: &str =
@ -49,6 +49,7 @@ pub enum Eth1Id {
#[derive(Clone, Copy)]
pub enum BlockQuery {
Number(u64),
Hash(Hash256),
Latest,
}
@ -135,13 +136,6 @@ pub async fn get_chain_id(endpoint: &SensitiveUrl, timeout: Duration) -> Result<
}
}
#[derive(Debug, PartialEq, Clone)]
pub struct Block {
pub hash: Hash256,
pub timestamp: u64,
pub number: u64,
}
/// Returns the current block number.
///
/// Uses HTTP JSON RPC at `endpoint`. E.g., `http://localhost:8545`.
@ -156,40 +150,74 @@ pub async fn get_block_number(endpoint: &SensitiveUrl, timeout: Duration) -> Res
.map_err(|e| format!("Failed to get block number: {}", e))
}
/// Gets a block hash by block number.
/// Gets a block by hash or block number.
///
/// Uses HTTP JSON RPC at `endpoint`. E.g., `http://localhost:8545`.
pub async fn get_block(
endpoint: &SensitiveUrl,
query: BlockQuery,
timeout: Duration,
) -> Result<Block, String> {
) -> Result<PowBlock, String> {
let query_param = match query {
BlockQuery::Number(block_number) => format!("0x{:x}", block_number),
BlockQuery::Hash(hash) => format!("{:?}", hash), // debug formatting ensures output not truncated
BlockQuery::Latest => "latest".to_string(),
};
let rpc_method = match query {
BlockQuery::Number(_) | BlockQuery::Latest => "eth_getBlockByNumber",
BlockQuery::Hash(_) => "eth_getBlockByHash",
};
let params = json!([
query_param,
false // do not return full tx objects.
]);
let response_body = send_rpc_request(endpoint, "eth_getBlockByNumber", params, timeout).await?;
let response_body = send_rpc_request(endpoint, rpc_method, params, timeout).await?;
let response = response_result_or_error(&response_body)
.map_err(|e| format!("eth_getBlockByNumber failed: {}", e))?;
.map_err(|e| format!("{} failed: {}", rpc_method, e))?;
let hash: Vec<u8> = hex_to_bytes(
let block_hash: Vec<u8> = hex_to_bytes(
response
.get("hash")
.ok_or("No hash for block")?
.as_str()
.ok_or("Block hash was not string")?,
)?;
let hash: Hash256 = if hash.len() == 32 {
Hash256::from_slice(&hash)
let block_hash: Hash256 = if block_hash.len() == 32 {
Hash256::from_slice(&block_hash)
} else {
return Err(format!("Block has was not 32 bytes: {:?}", hash));
return Err(format!("Block hash was not 32 bytes: {:?}", block_hash));
};
let parent_hash: Vec<u8> = hex_to_bytes(
response
.get("parentHash")
.ok_or("No parent hash for block")?
.as_str()
.ok_or("Parent hash was not string")?,
)?;
let parent_hash: Hash256 = if parent_hash.len() == 32 {
Hash256::from_slice(&parent_hash)
} else {
return Err(format!("parent hash was not 32 bytes: {:?}", parent_hash));
};
let total_difficulty_str = response
.get("totalDifficulty")
.ok_or("No total difficulty for block")?
.as_str()
.ok_or("Total difficulty was not a string")?;
let total_difficulty = Uint256::from_str(total_difficulty_str)
.map_err(|e| format!("total_difficulty from_str {:?}", e))?;
let difficulty_str = response
.get("difficulty")
.ok_or("No difficulty for block")?
.as_str()
.ok_or("Difficulty was not a string")?;
let difficulty =
Uint256::from_str(difficulty_str).map_err(|e| format!("difficulty from_str {:?}", e))?;
let timestamp = hex_to_u64_be(
response
.get("timestamp")
@ -198,7 +226,7 @@ pub async fn get_block(
.ok_or("Block timestamp was not string")?,
)?;
let number = hex_to_u64_be(
let block_number = hex_to_u64_be(
response
.get("number")
.ok_or("No number for block")?
@ -206,14 +234,20 @@ pub async fn get_block(
.ok_or("Block number was not string")?,
)?;
if number <= usize::max_value() as u64 {
Ok(Block {
hash,
if block_number <= usize::max_value() as u64 {
Ok(PowBlock {
block_hash,
parent_hash,
total_difficulty,
difficulty,
timestamp,
number,
block_number,
})
} else {
Err(format!("Block number {} is larger than a usize", number))
Err(format!(
"Block number {} is larger than a usize",
block_number
))
}
.map_err(|e| format!("Failed to get block number: {}", e))
}

View File

@ -21,7 +21,7 @@ use std::sync::Arc;
use std::time::{SystemTime, UNIX_EPOCH};
use tokio::sync::RwLock as TRwLock;
use tokio::time::{interval_at, Duration, Instant};
use types::{ChainSpec, EthSpec, Unsigned};
use types::{ChainSpec, EthSpec, ExecutionPayload, Unsigned};
/// Indicates the default eth1 network id we use for the deposit contract.
pub const DEFAULT_NETWORK_ID: Eth1Id = Eth1Id::Goerli;
@ -331,6 +331,8 @@ pub enum SingleEndpointError {
GetDepositCountFailed(String),
/// Failed to read the deposit contract root from the eth1 node.
GetDepositLogsFailed(String),
/// Failed to run engine_ExecutePayload
EngineExecutePayloadFailed,
}
#[derive(Debug, PartialEq)]
@ -669,6 +671,21 @@ impl Service {
}
}
/// This is were we call out to engine_executePayload to determine if payload is valid
pub async fn on_payload<T: EthSpec>(
&self,
_execution_payload: ExecutionPayload<T>,
) -> Result<bool, Error> {
let endpoints = self.init_endpoints();
// TODO: call engine_executePayload and figure out how backup endpoint works..
endpoints
.first_success(|_e| async move { Ok(true) })
.await
.map(|(res, _)| res)
.map_err(Error::FallbackError)
}
/// Update the deposit and block cache, returning an error if either fail.
///
/// ## Returns
@ -1242,7 +1259,7 @@ async fn download_eth1_block(
});
// Performs a `get_blockByNumber` call to an eth1 node.
let http_block = get_block(
let pow_block = get_block(
endpoint,
block_number_opt
.map(BlockQuery::Number)
@ -1253,9 +1270,9 @@ async fn download_eth1_block(
.await?;
Ok(Eth1Block {
hash: http_block.hash,
number: http_block.number,
timestamp: http_block.timestamp,
hash: pow_block.block_hash,
number: pow_block.block_number,
timestamp: pow_block.timestamp,
deposit_root,
deposit_count,
})

View File

@ -1,6 +1,6 @@
#![cfg(test)]
use environment::{Environment, EnvironmentBuilder};
use eth1::http::{get_deposit_count, get_deposit_logs_in_range, get_deposit_root, Block, Log};
use eth1::http::{get_deposit_count, get_deposit_logs_in_range, get_deposit_root, Log};
use eth1::{Config, Service};
use eth1::{DepositCache, DEFAULT_CHAIN_ID, DEFAULT_NETWORK_ID};
use eth1_test_rig::GanacheEth1Instance;
@ -571,8 +571,9 @@ mod deposit_tree {
mod http {
use super::*;
use eth1::http::BlockQuery;
use types::PowBlock;
async fn get_block(eth1: &GanacheEth1Instance, block_number: u64) -> Block {
async fn get_block(eth1: &GanacheEth1Instance, block_number: u64) -> PowBlock {
eth1::http::get_block(
&SensitiveUrl::parse(eth1.endpoint().as_str()).unwrap(),
BlockQuery::Number(block_number),
@ -639,7 +640,7 @@ mod http {
// Check the block hash.
let new_block = get_block(&eth1, block_number).await;
assert_ne!(
new_block.hash, old_block.hash,
new_block.block_hash, old_block.block_hash,
"block hash should change with each deposit"
);
@ -661,7 +662,7 @@ mod http {
// Check to ensure the block root is changing
assert_ne!(
new_root,
Some(new_block.hash),
Some(new_block.block_hash),
"the deposit root should be different to the block hash"
);
}

View File

@ -209,7 +209,9 @@ pub fn gossipsub_config(fork_context: Arc<ForkContext>) -> GossipsubConfig {
) -> Vec<u8> {
let topic_bytes = message.topic.as_str().as_bytes();
match fork_context.current_fork() {
ForkName::Altair => {
// 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
ForkName::Altair | ForkName::Merge => {
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

@ -17,7 +17,7 @@ use std::sync::Arc;
use tokio_util::codec::{Decoder, Encoder};
use types::{
EthSpec, ForkContext, ForkName, SignedBeaconBlock, SignedBeaconBlockAltair,
SignedBeaconBlockBase,
SignedBeaconBlockBase, SignedBeaconBlockMerge,
};
use unsigned_varint::codec::Uvi;
@ -375,7 +375,7 @@ fn handle_error<T>(
}
/// Returns `Some(context_bytes)` for encoding RPC responses that require context bytes.
/// Returns `None` when context bytes are not required.
/// Returns `None` when context bytes are not required.
fn context_bytes<T: EthSpec>(
protocol: &ProtocolId,
fork_context: &ForkContext,
@ -383,23 +383,25 @@ fn context_bytes<T: EthSpec>(
) -> Option<[u8; CONTEXT_BYTES_LEN]> {
// Add the context bytes if required
if protocol.has_context_bytes() {
if let RPCCodedResponse::Success(RPCResponse::BlocksByRange(res)) = resp {
if let SignedBeaconBlock::Altair { .. } = **res {
// Altair context being `None` implies that "altair never happened".
// This code should be unreachable if altair is disabled since only Version::V1 would be valid in that case.
return fork_context.to_context_bytes(ForkName::Altair);
} else if let SignedBeaconBlock::Base { .. } = **res {
return Some(fork_context.genesis_context_bytes());
}
}
if let RPCCodedResponse::Success(RPCResponse::BlocksByRoot(res)) = resp {
if let SignedBeaconBlock::Altair { .. } = **res {
// Altair context being `None` implies that "altair never happened".
// This code should be unreachable if altair is disabled since only Version::V1 would be valid in that case.
return fork_context.to_context_bytes(ForkName::Altair);
} else if let SignedBeaconBlock::Base { .. } = **res {
return Some(fork_context.genesis_context_bytes());
if let RPCCodedResponse::Success(rpc_variant) = resp {
if let RPCResponse::BlocksByRange(ref_box_block)
| RPCResponse::BlocksByRoot(ref_box_block) = rpc_variant
{
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::Merge { .. } => {
// TODO: check this
// Merge context being `None` implies that "merge never happened".
fork_context.to_context_bytes(ForkName::Merge)
}
SignedBeaconBlock::Altair { .. } => {
// Altair context being `None` implies that "altair never happened".
// This code should be unreachable if altair is disabled since only Version::V1 would be valid in that case.
fork_context.to_context_bytes(ForkName::Altair)
}
SignedBeaconBlock::Base { .. } => Some(fork_context.genesis_context_bytes()),
};
}
}
}
@ -559,6 +561,12 @@ fn handle_v2_response<T: EthSpec>(
ForkName::Base => Ok(Some(RPCResponse::BlocksByRange(Box::new(
SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?),
)))),
// TODO: check this (though it seems okay)
ForkName::Merge => Ok(Some(RPCResponse::BlocksByRange(Box::new(
SignedBeaconBlock::Merge(SignedBeaconBlockMerge::from_ssz_bytes(
decoded_buffer,
)?),
)))),
},
Protocol::BlocksByRoot => match fork_name {
ForkName::Altair => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
@ -569,6 +577,12 @@ fn handle_v2_response<T: EthSpec>(
ForkName::Base => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
SignedBeaconBlock::Base(SignedBeaconBlockBase::from_ssz_bytes(decoded_buffer)?),
)))),
// TODO: check this (though it seems right)
ForkName::Merge => Ok(Some(RPCResponse::BlocksByRoot(Box::new(
SignedBeaconBlock::Merge(SignedBeaconBlockMerge::from_ssz_bytes(
decoded_buffer,
)?),
)))),
},
_ => Err(RPCError::ErrorResponse(
RPCResponseErrorCode::InvalidRequest,

View File

@ -21,8 +21,8 @@ use tokio_util::{
compat::{Compat, FuturesAsyncReadCompatExt},
};
use types::{
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, EthSpec, ForkContext, Hash256, MainnetEthSpec,
Signature, SignedBeaconBlock,
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, EthSpec, ForkContext,
Hash256, MainnetEthSpec, Signature, SignedBeaconBlock,
};
lazy_static! {
@ -53,6 +53,20 @@ lazy_static! {
)
.as_ssz_bytes()
.len();
pub static ref SIGNED_BEACON_BLOCK_MERGE_MIN: usize = SignedBeaconBlock::<MainnetEthSpec>::from_block(
BeaconBlock::Merge(BeaconBlockMerge::<MainnetEthSpec>::empty(&MainnetEthSpec::default_spec())),
Signature::empty(),
)
.as_ssz_bytes()
.len();
pub static ref SIGNED_BEACON_BLOCK_MERGE_MAX: usize = SignedBeaconBlock::<MainnetEthSpec>::from_block(
BeaconBlock::Merge(BeaconBlockMerge::full(&MainnetEthSpec::default_spec())),
Signature::empty(),
)
.as_ssz_bytes()
.len();
pub static ref BLOCKS_BY_ROOT_REQUEST_MIN: usize =
VariableList::<Hash256, MaxRequestBlocks>::from(Vec::<Hash256>::new())
.as_ssz_bytes()
@ -253,12 +267,18 @@ impl ProtocolId {
Protocol::Goodbye => RpcLimits::new(0, 0), // Goodbye request has no response
Protocol::BlocksByRange => RpcLimits::new(
std::cmp::min(
*SIGNED_BEACON_BLOCK_ALTAIR_MIN,
*SIGNED_BEACON_BLOCK_BASE_MIN,
std::cmp::min(
*SIGNED_BEACON_BLOCK_ALTAIR_MIN,
*SIGNED_BEACON_BLOCK_BASE_MIN,
),
*SIGNED_BEACON_BLOCK_MERGE_MIN,
),
std::cmp::max(
*SIGNED_BEACON_BLOCK_ALTAIR_MAX,
*SIGNED_BEACON_BLOCK_BASE_MAX,
std::cmp::max(
*SIGNED_BEACON_BLOCK_ALTAIR_MAX,
*SIGNED_BEACON_BLOCK_BASE_MAX,
),
*SIGNED_BEACON_BLOCK_MERGE_MAX,
),
),
Protocol::BlocksByRoot => RpcLimits::new(

View File

@ -10,7 +10,8 @@ use std::io::{Error, ErrorKind};
use types::{
Attestation, AttesterSlashing, EthSpec, ForkContext, ForkName, ProposerSlashing,
SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase,
SignedContributionAndProof, SignedVoluntaryExit, SubnetId, SyncCommitteeMessage, SyncSubnetId,
SignedBeaconBlockMerge, SignedContributionAndProof, SignedVoluntaryExit, SubnetId,
SyncCommitteeMessage, SyncSubnetId,
};
#[derive(Debug, Clone, PartialEq)]
@ -161,6 +162,10 @@ impl<T: EthSpec> PubsubMessage<T> {
SignedBeaconBlockAltair::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?,
),
Some(ForkName::Merge) => SignedBeaconBlock::<T>::Merge(
SignedBeaconBlockMerge::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?,
),
None => {
return Err(format!(
"Unknown gossipsub fork digest: {:?}",

View File

@ -5,7 +5,8 @@ use beacon_chain::{
observed_operations::ObservationOutcome,
sync_committee_verification::Error as SyncCommitteeError,
validator_monitor::get_block_delay_ms,
BeaconChainError, BeaconChainTypes, BlockError, ForkChoiceError, GossipVerifiedBlock,
BeaconChainError, BeaconChainTypes, BlockError, ExecutionPayloadError, ForkChoiceError,
GossipVerifiedBlock,
};
use lighthouse_network::{Client, MessageAcceptance, MessageId, PeerAction, PeerId, ReportSource};
use slog::{crit, debug, error, info, trace, warn};
@ -746,6 +747,16 @@ impl<T: BeaconChainTypes> Worker<T> {
self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Ignore);
return None;
}
// TODO: check that this is what we're supposed to do when we don't want to
// penalize a peer for our configuration issue
// in the verification process BUT is this the proper way to handle it?
Err(e @BlockError::ExecutionPayloadError(ExecutionPayloadError::Eth1VerificationError(_)))
| Err(e @BlockError::ExecutionPayloadError(ExecutionPayloadError::NoEth1Connection)) => {
debug!(self.log, "Could not verify block for gossip, ignoring the block";
"error" => %e);
self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Ignore);
return;
}
Err(e @ BlockError::StateRootMismatch { .. })
| Err(e @ BlockError::IncorrectBlockProposer { .. })
| Err(e @ BlockError::BlockSlotLimitReached)
@ -759,6 +770,8 @@ impl<T: BeaconChainTypes> Worker<T> {
| Err(e @ BlockError::TooManySkippedSlots { .. })
| Err(e @ BlockError::WeakSubjectivityConflict)
| Err(e @ BlockError::InconsistentFork(_))
// TODO: is this what we should be doing when block verification fails?
| Err(e @BlockError::ExecutionPayloadError(_))
| Err(e @ BlockError::GenesisBlock) => {
warn!(self.log, "Could not verify block for gossip, rejecting the block";
"error" => %e);

View File

@ -883,6 +883,10 @@ impl<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> HotColdDB<E, Hot, Cold>
&mut block.message.state_root,
&mut block.message.parent_root,
),
SignedBeaconBlock::Merge(block) => (
&mut block.message.state_root,
&mut block.message.parent_root,
),
};
*state_root = Hash256::zero();

View File

@ -14,8 +14,8 @@ use types::*;
///
/// Utilises lazy-loading from separate storage for its vector fields.
#[superstruct(
variants(Base, Altair),
variant_attributes(derive(Debug, PartialEq, Clone, Encode, Decode),)
variants(Base, Altair, Merge),
variant_attributes(derive(Debug, PartialEq, Clone, Encode, Decode))
)]
#[derive(Debug, PartialEq, Clone, Encode)]
#[ssz(enum_behaviour = "transparent")]
@ -66,9 +66,9 @@ where
pub current_epoch_attestations: VariableList<PendingAttestation<T>, T::MaxPendingAttestations>,
// Participation (Altair and later)
#[superstruct(only(Altair))]
#[superstruct(only(Altair, Merge))]
pub previous_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
#[superstruct(only(Altair))]
#[superstruct(only(Altair, Merge))]
pub current_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
// Finality
@ -78,14 +78,18 @@ where
pub finalized_checkpoint: Checkpoint,
// Inactivity
#[superstruct(only(Altair))]
#[superstruct(only(Altair, Merge))]
pub inactivity_scores: VariableList<u64, T::ValidatorRegistryLimit>,
// Light-client sync committees
#[superstruct(only(Altair))]
#[superstruct(only(Altair, Merge))]
pub current_sync_committee: Arc<SyncCommittee<T>>,
#[superstruct(only(Altair))]
#[superstruct(only(Altair, Merge))]
pub next_sync_committee: Arc<SyncCommittee<T>>,
// Execution
#[superstruct(only(Merge))]
pub latest_execution_payload_header: ExecutionPayloadHeader<T>,
}
/// Implement the conversion function from BeaconState -> PartialBeaconState.
@ -160,6 +164,20 @@ impl<T: EthSpec> PartialBeaconState<T> {
inactivity_scores
]
),
BeaconState::Merge(s) => impl_from_state_forgetful!(
s,
outer,
Merge,
PartialBeaconStateMerge,
[
previous_epoch_participation,
current_epoch_participation,
current_sync_committee,
next_sync_committee,
inactivity_scores,
latest_execution_payload_header
]
),
}
}
@ -334,6 +352,19 @@ impl<E: EthSpec> TryInto<BeaconState<E>> for PartialBeaconState<E> {
inactivity_scores
]
),
PartialBeaconState::Merge(inner) => impl_try_into_beacon_state!(
inner,
Merge,
BeaconStateMerge,
[
previous_epoch_participation,
current_epoch_participation,
current_sync_committee,
next_sync_committee,
inactivity_scores,
latest_execution_payload_header
]
),
};
Ok(state)
}

View File

@ -8,6 +8,7 @@ edition = "2018"
[dependencies]
types = { path = "../types" }
state_processing = { path = "../state_processing" }
proto_array = { path = "../proto_array" }
eth2_ssz = "0.4.0"
eth2_ssz_derive = "0.3.0"

View File

@ -2,9 +2,11 @@ use std::marker::PhantomData;
use proto_array::{Block as ProtoBlock, ProtoArrayForkChoice};
use ssz_derive::{Decode, Encode};
use state_processing::per_block_processing::is_merge_block;
use types::{
AttestationShufflingId, BeaconBlock, BeaconState, BeaconStateError, ChainSpec, Checkpoint,
Epoch, EthSpec, Hash256, IndexedAttestation, RelativeEpoch, SignedBeaconBlock, Slot,
Epoch, EthSpec, Hash256, IndexedAttestation, PowBlock, RelativeEpoch, SignedBeaconBlock, Slot,
Uint256,
};
use crate::ForkChoiceStore;
@ -60,6 +62,10 @@ pub enum InvalidBlock {
finalized_root: Hash256,
block_ancestor: Option<Hash256>,
},
InvalidTerminalPowBlock {
block_total_difficulty: Uint256,
parent_total_difficulty: Uint256,
},
}
#[derive(Debug)]
@ -231,6 +237,14 @@ where
}
}
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/fork-choice.md#is_valid_terminal_pow_block
fn is_valid_terminal_pow_block(block: &PowBlock, parent: &PowBlock, spec: &ChainSpec) -> bool {
let is_total_difficulty_reached = block.total_difficulty >= spec.terminal_total_difficulty;
let is_parent_total_difficulty_valid = parent.total_difficulty < spec.terminal_total_difficulty;
is_total_difficulty_reached && is_parent_total_difficulty_valid
}
impl<T, E> ForkChoice<T, E>
where
T: ForkChoiceStore<E>,
@ -489,6 +503,19 @@ where
}));
}
// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/fork-choice.md#on_block
if is_merge_block(state, block.body()) {
// TODO: get POW blocks from eth1 chain here as indicated in the merge spec link ^
let pow_block = PowBlock::default();
let pow_parent = PowBlock::default();
if !is_valid_terminal_pow_block(&pow_block, &pow_parent, spec) {
return Err(Error::InvalidBlock(InvalidBlock::InvalidTerminalPowBlock {
block_total_difficulty: pow_block.total_difficulty,
parent_total_difficulty: pow_parent.total_difficulty,
}));
}
}
// Update justified checkpoint.
if state.current_justified_checkpoint().epoch > self.fc_store.justified_checkpoint().epoch {
if state.current_justified_checkpoint().epoch

View File

@ -1,6 +1,6 @@
use super::*;
use core::num::NonZeroUsize;
use ethereum_types::{H256, U128, U256};
use ethereum_types::{H160, H256, U128, U256};
use smallvec::SmallVec;
use std::sync::Arc;
@ -256,6 +256,27 @@ impl<T: Decode> Decode for Arc<T> {
}
}
impl Decode for H160 {
fn is_ssz_fixed_len() -> bool {
true
}
fn ssz_fixed_len() -> usize {
20
}
fn from_ssz_bytes(bytes: &[u8]) -> Result<Self, DecodeError> {
let len = bytes.len();
let expected = <Self as Decode>::ssz_fixed_len();
if len != expected {
Err(DecodeError::InvalidByteLength { len, expected })
} else {
Ok(Self::from_slice(bytes))
}
}
}
impl Decode for H256 {
fn is_ssz_fixed_len() -> bool {
true

View File

@ -1,6 +1,6 @@
use super::*;
use core::num::NonZeroUsize;
use ethereum_types::{H256, U128, U256};
use ethereum_types::{H160, H256, U128, U256};
use smallvec::SmallVec;
use std::sync::Arc;
@ -305,6 +305,24 @@ impl Encode for NonZeroUsize {
}
}
impl Encode for H160 {
fn is_ssz_fixed_len() -> bool {
true
}
fn ssz_fixed_len() -> usize {
20
}
fn ssz_bytes_len(&self) -> usize {
20
}
fn ssz_append(&self, buf: &mut Vec<u8>) {
buf.extend_from_slice(self.as_bytes());
}
}
impl Encode for H256 {
fn is_ssz_fixed_len() -> bool {
true

View File

@ -33,7 +33,7 @@ pub fn slash_validator<T: EthSpec>(
let min_slashing_penalty_quotient = match state {
BeaconState::Base(_) => spec.min_slashing_penalty_quotient,
BeaconState::Altair(_) => spec.min_slashing_penalty_quotient_altair,
BeaconState::Altair(_) | BeaconState::Merge(_) => spec.min_slashing_penalty_quotient_altair,
};
decrease_balance(
state,
@ -48,7 +48,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(_) => whistleblower_reward
BeaconState::Altair(_) | BeaconState::Merge(_) => whistleblower_reward
.safe_mul(PROPOSER_WEIGHT)?
.safe_div(WEIGHT_DENOMINATOR)?,
};

View File

@ -52,6 +52,8 @@ pub fn initialize_beacon_state_from_eth1<T: EthSpec>(
state.fork_mut().previous_version = spec.altair_fork_version;
}
// TODO: handle upgrade_to_merge() here
// Now that we have our validators, initialize the caches (including the committees)
state.build_all_caches(spec)?;

View File

@ -149,6 +149,10 @@ pub fn per_block_processing<T: EthSpec>(
)?;
}
if is_execution_enabled(state, block.body()) {
process_execution_payload(state, block.body().execution_payload().unwrap(), spec)?
}
Ok(())
}
@ -283,3 +287,135 @@ pub fn get_new_eth1_data<T: EthSpec>(
Ok(None)
}
}
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/beacon-chain.md#is_valid_gas_limit
pub fn is_valid_gas_limit<T: EthSpec>(
payload: &ExecutionPayload<T>,
parent: &ExecutionPayloadHeader<T>,
) -> bool {
// check if payload used too much gas
if payload.gas_used > payload.gas_limit {
return false;
}
// check if payload changed the gas limit too much
if payload.gas_limit >= parent.gas_limit + parent.gas_limit / T::gas_limit_denominator() {
return false;
}
if payload.gas_limit <= parent.gas_limit - parent.gas_limit / T::gas_limit_denominator() {
return false;
}
// check if the gas limit is at least the minimum gas limit
if payload.gas_limit < T::min_gas_limit() {
return false;
}
return true;
}
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/beacon-chain.md#process_execution_payload
pub fn process_execution_payload<T: EthSpec>(
state: &mut BeaconState<T>,
payload: &ExecutionPayload<T>,
spec: &ChainSpec,
) -> Result<(), BlockProcessingError> {
if is_merge_complete(state) {
block_verify!(
payload.parent_hash == state.latest_execution_payload_header()?.block_hash,
BlockProcessingError::ExecutionHashChainIncontiguous {
expected: state.latest_execution_payload_header()?.block_hash,
found: payload.parent_hash,
}
);
block_verify!(
payload.block_number
== state
.latest_execution_payload_header()?
.block_number
.safe_add(1)?,
BlockProcessingError::ExecutionBlockNumberIncontiguous {
expected: state
.latest_execution_payload_header()?
.block_number
.safe_add(1)?,
found: payload.block_number,
}
);
block_verify!(
payload.random == *state.get_randao_mix(state.current_epoch())?,
BlockProcessingError::ExecutionRandaoMismatch {
expected: *state.get_randao_mix(state.current_epoch())?,
found: payload.random,
}
);
block_verify!(
is_valid_gas_limit(payload, state.latest_execution_payload_header()?),
BlockProcessingError::ExecutionInvalidGasLimit {
used: payload.gas_used,
limit: payload.gas_limit,
}
);
}
let timestamp = compute_timestamp_at_slot(state, spec)?;
block_verify!(
payload.timestamp == timestamp,
BlockProcessingError::ExecutionInvalidTimestamp {
expected: timestamp,
found: payload.timestamp,
}
);
*state.latest_execution_payload_header_mut()? = ExecutionPayloadHeader {
parent_hash: payload.parent_hash,
coinbase: payload.coinbase,
state_root: payload.state_root,
receipt_root: payload.receipt_root,
logs_bloom: payload.logs_bloom.clone(),
random: payload.random,
block_number: payload.block_number,
gas_limit: payload.gas_limit,
gas_used: payload.gas_used,
timestamp: payload.timestamp,
base_fee_per_gas: payload.base_fee_per_gas,
block_hash: payload.block_hash,
transactions_root: payload.transactions.tree_hash_root(),
};
Ok(())
}
/// These functions will definitely be called before the merge. Their entire purpose is to check if
/// the merge has happened or if we're on the transition block. Thus we don't want to propagate
/// errors from the `BeaconState` being an earlier variant than `BeaconStateMerge` as we'd have to
/// repeaetedly write code to treat these errors as false.
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/beacon-chain.md#is_merge_complete
pub fn is_merge_complete<T: EthSpec>(state: &BeaconState<T>) -> bool {
state
.latest_execution_payload_header()
.map(|header| *header != <ExecutionPayloadHeader<T>>::default())
.unwrap_or(false)
}
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/beacon-chain.md#is_merge_block
pub fn is_merge_block<T: EthSpec>(state: &BeaconState<T>, body: BeaconBlockBodyRef<T>) -> bool {
body.execution_payload()
.map(|payload| !is_merge_complete(state) && *payload != <ExecutionPayload<T>>::default())
.unwrap_or(false)
}
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/beacon-chain.md#is_execution_enabled
pub fn is_execution_enabled<T: EthSpec>(
state: &BeaconState<T>,
body: BeaconBlockBodyRef<T>,
) -> bool {
is_merge_block(state, body) || is_merge_complete(state)
}
/// https://github.com/ethereum/consensus-specs/blob/dev/specs/merge/beacon-chain.md#compute_timestamp_at_slot
pub fn compute_timestamp_at_slot<T: EthSpec>(
state: &BeaconState<T>,
spec: &ChainSpec,
) -> Result<u64, ArithError> {
let slots_since_genesis = state.slot().as_u64().safe_sub(spec.genesis_slot.as_u64())?;
slots_since_genesis
.safe_mul(spec.seconds_per_slot)
.and_then(|since_genesis| state.genesis_time().safe_add(since_genesis))
}

View File

@ -57,6 +57,26 @@ pub enum BlockProcessingError {
ArithError(ArithError),
InconsistentBlockFork(InconsistentFork),
InconsistentStateFork(InconsistentFork),
ExecutionHashChainIncontiguous {
expected: Hash256,
found: Hash256,
},
ExecutionBlockNumberIncontiguous {
expected: u64,
found: u64,
},
ExecutionRandaoMismatch {
expected: Hash256,
found: Hash256,
},
ExecutionInvalidGasLimit {
used: u64,
limit: u64,
},
ExecutionInvalidTimestamp {
expected: u64,
found: u64,
},
}
impl From<BeaconStateError> for BlockProcessingError {

View File

@ -228,7 +228,7 @@ pub fn process_attestations<'a, T: EthSpec>(
BeaconBlockBodyRef::Base(_) => {
base::process_attestations(state, block_body.attestations(), verify_signatures, spec)?;
}
BeaconBlockBodyRef::Altair(_) => {
BeaconBlockBodyRef::Altair(_) | BeaconBlockBodyRef::Merge(_) => {
altair::process_attestations(
state,
block_body.attestations(),

View File

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

View File

@ -1,4 +1,4 @@
use crate::upgrade::upgrade_to_altair;
use crate::upgrade::{upgrade_to_altair, upgrade_to_merge};
use crate::{per_epoch_processing::EpochProcessingSummary, *};
use safe_arith::{ArithError, SafeArith};
use types::*;
@ -44,11 +44,17 @@ pub fn per_slot_processing<T: EthSpec>(
state.slot_mut().safe_add_assign(1)?;
// If the Altair fork epoch is reached, perform an irregular state upgrade.
if state.slot().safe_rem(T::slots_per_epoch())? == 0
&& spec.altair_fork_epoch == Some(state.current_epoch())
{
upgrade_to_altair(state, spec)?;
// Process fork upgrades here. Note that multiple upgrades can potentially run
// in sequence if they are scheduled in the same Epoch (common in testnets)
if state.slot().safe_rem(T::slots_per_epoch())? == 0 {
// If the Altair fork epoch is reached, perform an irregular state upgrade.
if spec.altair_fork_epoch == Some(state.current_epoch()) {
upgrade_to_altair(state, spec)?;
}
// If the Merge fork epoch is reached, perform an irregular state upgrade.
if spec.merge_fork_epoch == Some(state.current_epoch()) {
upgrade_to_merge(state, spec)?;
}
}
Ok(summary)

View File

@ -1,3 +1,5 @@
pub mod altair;
pub mod merge;
pub use altair::upgrade_to_altair;
pub use merge::upgrade_to_merge;

View File

@ -0,0 +1,72 @@
use std::mem;
use types::{
BeaconState, BeaconStateError as Error, BeaconStateMerge, ChainSpec, EthSpec,
ExecutionPayloadHeader, Fork,
};
/// Transform a `Altair` state into an `Merge` state.
pub fn upgrade_to_merge<E: EthSpec>(
pre_state: &mut BeaconState<E>,
spec: &ChainSpec,
) -> Result<(), Error> {
let epoch = pre_state.current_epoch();
let pre = pre_state.as_altair_mut()?;
// Where possible, use something like `mem::take` to move fields from behind the &mut
// reference. For other fields that don't have a good default value, use `clone`.
//
// Fixed size vectors get cloned because replacing them would require the same size
// allocation as cloning.
let post = BeaconState::Merge(BeaconStateMerge {
// Versioning
genesis_time: pre.genesis_time,
genesis_validators_root: pre.genesis_validators_root,
slot: pre.slot,
fork: Fork {
previous_version: pre.fork.current_version,
current_version: spec.merge_fork_version,
epoch,
},
// History
latest_block_header: pre.latest_block_header.clone(),
block_roots: pre.block_roots.clone(),
state_roots: pre.state_roots.clone(),
historical_roots: mem::take(&mut pre.historical_roots),
// Eth1
eth1_data: pre.eth1_data.clone(),
eth1_data_votes: mem::take(&mut pre.eth1_data_votes),
eth1_deposit_index: pre.eth1_deposit_index,
// Registry
validators: mem::take(&mut pre.validators),
balances: mem::take(&mut pre.balances),
// Randomness
randao_mixes: pre.randao_mixes.clone(),
// Slashings
slashings: pre.slashings.clone(),
// `Participation
previous_epoch_participation: mem::take(&mut pre.previous_epoch_participation),
current_epoch_participation: mem::take(&mut pre.current_epoch_participation),
// Finality
justification_bits: pre.justification_bits.clone(),
previous_justified_checkpoint: pre.previous_justified_checkpoint,
current_justified_checkpoint: pre.current_justified_checkpoint,
finalized_checkpoint: pre.finalized_checkpoint,
// Inactivity
inactivity_scores: mem::take(&mut pre.inactivity_scores),
// Sync committees
current_sync_committee: pre.current_sync_committee.clone(),
next_sync_committee: pre.next_sync_committee.clone(),
// Execution
latest_execution_payload_header: <ExecutionPayloadHeader<E>>::default(),
// Caches
total_active_balance: pre.total_active_balance,
committee_caches: mem::take(&mut pre.committee_caches),
pubkey_cache: mem::take(&mut pre.pubkey_cache),
exit_cache: mem::take(&mut pre.exit_cache),
tree_hash_cache: mem::take(&mut pre.tree_hash_cache),
});
*pre_state = post;
Ok(())
}

View File

@ -1,5 +1,5 @@
use super::*;
use ethereum_types::{H256, U128, U256};
use ethereum_types::{H160, H256, U128, U256};
fn int_to_hash256(int: u64) -> Hash256 {
let mut bytes = [0; HASHSIZE];
@ -126,6 +126,28 @@ impl TreeHash for U256 {
}
}
impl TreeHash for H160 {
fn tree_hash_type() -> TreeHashType {
TreeHashType::Vector
}
fn tree_hash_packed_encoding(&self) -> Vec<u8> {
let mut result = vec![0; 32];
result[0..20].copy_from_slice(self.as_bytes());
result
}
fn tree_hash_packing_factor() -> usize {
1
}
fn tree_hash_root(&self) -> Hash256 {
let mut result = [0; 32];
result[0..20].copy_from_slice(self.as_bytes());
Hash256::from_slice(&result)
}
}
impl TreeHash for H256 {
fn tree_hash_type() -> TreeHashType {
TreeHashType::Vector

View File

@ -1,5 +1,6 @@
use crate::beacon_block_body::{
BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyRef, BeaconBlockBodyRefMut,
BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyMerge, BeaconBlockBodyRef,
BeaconBlockBodyRefMut,
};
use crate::test_utils::TestRandom;
use crate::*;
@ -14,7 +15,7 @@ use tree_hash_derive::TreeHash;
/// A block of the `BeaconChain`.
#[superstruct(
variants(Base, Altair),
variants(Base, Altair, Merge),
variant_attributes(
derive(
Debug,
@ -55,6 +56,8 @@ pub struct BeaconBlock<T: EthSpec> {
pub body: BeaconBlockBodyBase<T>,
#[superstruct(only(Altair), partial_getter(rename = "body_altair"))]
pub body: BeaconBlockBodyAltair<T>,
#[superstruct(only(Merge), partial_getter(rename = "body_merge"))]
pub body: BeaconBlockBodyMerge<T>,
}
impl<T: EthSpec> SignedRoot for BeaconBlock<T> {}
@ -63,7 +66,9 @@ impl<'a, T: EthSpec> SignedRoot for BeaconBlockRef<'a, T> {}
impl<T: EthSpec> BeaconBlock<T> {
/// Returns an empty block to be used during genesis.
pub fn empty(spec: &ChainSpec) -> Self {
if spec.altair_fork_epoch == Some(T::genesis_epoch()) {
if spec.merge_fork_epoch == Some(T::genesis_epoch()) {
Self::Merge(BeaconBlockMerge::empty(spec))
} else if spec.altair_fork_epoch == Some(T::genesis_epoch()) {
Self::Altair(BeaconBlockAltair::empty(spec))
} else {
Self::Base(BeaconBlockBase::empty(spec))
@ -171,6 +176,7 @@ impl<'a, T: EthSpec> BeaconBlockRef<'a, T> {
let object_fork = match self {
BeaconBlockRef::Base { .. } => ForkName::Base,
BeaconBlockRef::Altair { .. } => ForkName::Altair,
BeaconBlockRef::Merge { .. } => ForkName::Merge,
};
if fork_at_slot == object_fork {
@ -188,6 +194,7 @@ impl<'a, T: EthSpec> BeaconBlockRef<'a, T> {
match self {
BeaconBlockRef::Base(block) => BeaconBlockBodyRef::Base(&block.body),
BeaconBlockRef::Altair(block) => BeaconBlockBodyRef::Altair(&block.body),
BeaconBlockRef::Merge(block) => BeaconBlockBodyRef::Merge(&block.body),
}
}
@ -196,6 +203,7 @@ impl<'a, T: EthSpec> BeaconBlockRef<'a, T> {
match self {
BeaconBlockRef::Base(block) => block.body.tree_hash_root(),
BeaconBlockRef::Altair(block) => block.body.tree_hash_root(),
BeaconBlockRef::Merge(block) => block.body.tree_hash_root(),
}
}
@ -230,6 +238,7 @@ impl<'a, T: EthSpec> BeaconBlockRefMut<'a, T> {
match self {
BeaconBlockRefMut::Base(block) => BeaconBlockBodyRefMut::Base(&mut block.body),
BeaconBlockRefMut::Altair(block) => BeaconBlockBodyRefMut::Altair(&mut block.body),
BeaconBlockRefMut::Merge(block) => BeaconBlockBodyRefMut::Merge(&mut block.body),
}
}
}
@ -411,6 +420,61 @@ impl<T: EthSpec> BeaconBlockAltair<T> {
}
}
impl<T: EthSpec> BeaconBlockMerge<T> {
/// Returns an empty Merge block to be used during genesis.
pub fn empty(spec: &ChainSpec) -> Self {
BeaconBlockMerge {
slot: spec.genesis_slot,
proposer_index: 0,
parent_root: Hash256::zero(),
state_root: Hash256::zero(),
body: BeaconBlockBodyMerge {
randao_reveal: Signature::empty(),
eth1_data: Eth1Data {
deposit_root: Hash256::zero(),
block_hash: Hash256::zero(),
deposit_count: 0,
},
graffiti: Graffiti::default(),
proposer_slashings: VariableList::empty(),
attester_slashings: VariableList::empty(),
attestations: VariableList::empty(),
deposits: VariableList::empty(),
voluntary_exits: VariableList::empty(),
sync_aggregate: SyncAggregate::empty(),
execution_payload: ExecutionPayload::empty(),
},
}
}
/// Return an Merge block where the block has maximum size.
pub fn full(spec: &ChainSpec) -> Self {
let altair_block = BeaconBlockAltair::full(spec);
BeaconBlockMerge {
slot: spec.genesis_slot,
proposer_index: 0,
parent_root: Hash256::zero(),
state_root: Hash256::zero(),
body: BeaconBlockBodyMerge {
proposer_slashings: altair_block.body.proposer_slashings,
attester_slashings: altair_block.body.attester_slashings,
attestations: altair_block.body.attestations,
deposits: altair_block.body.deposits,
voluntary_exits: altair_block.body.voluntary_exits,
sync_aggregate: altair_block.body.sync_aggregate,
randao_reveal: Signature::empty(),
eth1_data: Eth1Data {
deposit_root: Hash256::zero(),
block_hash: Hash256::zero(),
deposit_count: 0,
},
graffiti: Graffiti::default(),
execution_payload: ExecutionPayload::default(),
},
}
}
}
#[cfg(test)]
mod tests {
use super::*;

View File

@ -11,7 +11,7 @@ use tree_hash_derive::TreeHash;
///
/// This *superstruct* abstracts over the hard-fork.
#[superstruct(
variants(Base, Altair),
variants(Base, Altair, Merge),
variant_attributes(
derive(
Debug,
@ -26,7 +26,9 @@ use tree_hash_derive::TreeHash;
),
serde(bound = "T: EthSpec", deny_unknown_fields),
cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))
)
),
cast_error(ty = "Error", expr = "Error::IncorrectStateVariant"),
partial_getter_error(ty = "Error", expr = "Error::IncorrectStateVariant")
)]
#[derive(Debug, PartialEq, Clone, Serialize, Deserialize)]
#[serde(untagged)]
@ -41,8 +43,10 @@ pub struct BeaconBlockBody<T: EthSpec> {
pub attestations: VariableList<Attestation<T>, T::MaxAttestations>,
pub deposits: VariableList<Deposit, T::MaxDeposits>,
pub voluntary_exits: VariableList<SignedVoluntaryExit, T::MaxVoluntaryExits>,
#[superstruct(only(Altair))]
#[superstruct(only(Altair, Merge))]
pub sync_aggregate: SyncAggregate<T>,
#[superstruct(only(Merge))]
pub execution_payload: ExecutionPayload<T>,
}
impl<'a, T: EthSpec> BeaconBlockBodyRef<'a, T> {
@ -51,6 +55,25 @@ impl<'a, T: EthSpec> BeaconBlockBodyRef<'a, T> {
match self {
BeaconBlockBodyRef::Base(_) => None,
BeaconBlockBodyRef::Altair(inner) => Some(&inner.sync_aggregate),
BeaconBlockBodyRef::Merge(inner) => Some(&inner.sync_aggregate),
}
}
/// Access the execution payload from the block's body, if one exists.
pub fn execution_payload(self) -> Option<&'a ExecutionPayload<T>> {
match self {
BeaconBlockBodyRef::Base(_) => None,
BeaconBlockBodyRef::Altair(_) => None,
BeaconBlockBodyRef::Merge(inner) => Some(&inner.execution_payload),
}
}
/// Get the fork_name of this object
pub fn fork_name(self) -> ForkName {
match self {
BeaconBlockBodyRef::Base { .. } => ForkName::Base,
BeaconBlockBodyRef::Altair { .. } => ForkName::Altair,
BeaconBlockBodyRef::Merge { .. } => ForkName::Merge,
}
}
}

View File

@ -172,7 +172,7 @@ impl From<BeaconStateHash> for Hash256 {
/// The state of the `BeaconChain` at some slot.
#[superstruct(
variants(Base, Altair),
variants(Base, Altair, Merge),
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))]
#[superstruct(only(Altair, Merge))]
pub previous_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
#[superstruct(only(Altair))]
#[superstruct(only(Altair, Merge))]
pub current_epoch_participation: VariableList<ParticipationFlags, T::ValidatorRegistryLimit>,
// Finality
@ -267,15 +267,19 @@ where
// Inactivity
#[serde(with = "ssz_types::serde_utils::quoted_u64_var_list")]
#[superstruct(only(Altair))]
#[superstruct(only(Altair, Merge))]
pub inactivity_scores: VariableList<u64, T::ValidatorRegistryLimit>,
// Light-client sync committees
#[superstruct(only(Altair))]
#[superstruct(only(Altair, Merge))]
pub current_sync_committee: Arc<SyncCommittee<T>>,
#[superstruct(only(Altair))]
#[superstruct(only(Altair, Merge))]
pub next_sync_committee: Arc<SyncCommittee<T>>,
// Execution
#[superstruct(only(Merge))]
pub latest_execution_payload_header: ExecutionPayloadHeader<T>,
// Caching (not in the spec)
#[serde(skip_serializing, skip_deserializing)]
#[ssz(skip_serializing, skip_deserializing)]
@ -384,6 +388,7 @@ impl<T: EthSpec> BeaconState<T> {
let object_fork = match self {
BeaconState::Base { .. } => ForkName::Base,
BeaconState::Altair { .. } => ForkName::Altair,
BeaconState::Merge { .. } => ForkName::Merge,
};
if fork_at_slot == object_fork {
@ -1089,6 +1094,7 @@ impl<T: EthSpec> BeaconState<T> {
match self {
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),
}
}
@ -1284,11 +1290,13 @@ impl<T: EthSpec> BeaconState<T> {
match self {
BeaconState::Base(_) => Err(BeaconStateError::IncorrectStateVariant),
BeaconState::Altair(state) => Ok(&mut state.current_epoch_participation),
BeaconState::Merge(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),
}
} else {
Err(BeaconStateError::EpochOutOfBounds)
@ -1572,6 +1580,7 @@ impl<T: EthSpec> BeaconState<T> {
let mut res = match self {
BeaconState::Base(inner) => BeaconState::Base(inner.clone()),
BeaconState::Altair(inner) => BeaconState::Altair(inner.clone()),
BeaconState::Merge(inner) => BeaconState::Merge(inner.clone()),
};
if config.committee_caches {
*res.committee_caches_mut() = self.committee_caches().clone();

View File

@ -127,6 +127,10 @@ pub struct ChainSpec {
pub altair_fork_version: [u8; 4],
/// The Altair fork epoch is optional, with `None` representing "Altair never happens".
pub altair_fork_epoch: Option<Epoch>,
pub merge_fork_version: [u8; 4],
/// The Merge fork epoch is optional, with `None` representing "Merge never happens".
pub merge_fork_epoch: Option<Epoch>,
pub terminal_total_difficulty: Uint256,
/*
* Networking
@ -156,7 +160,7 @@ impl ChainSpec {
) -> EnrForkId {
EnrForkId {
fork_digest: self.fork_digest::<T>(slot, genesis_validators_root),
next_fork_version: self.next_fork_version(),
next_fork_version: self.next_fork_version::<T>(slot),
next_fork_epoch: self
.next_fork_epoch::<T>(slot)
.map(|(_, e)| e)
@ -178,10 +182,12 @@ impl ChainSpec {
/// Returns the `next_fork_version`.
///
/// Since `next_fork_version = current_fork_version` if no future fork is planned,
/// this function returns `altair_fork_version` until the next fork is planned.
pub fn next_fork_version(&self) -> [u8; 4] {
self.altair_fork_version
/// `next_fork_version = current_fork_version` if no future fork is planned,
pub fn next_fork_version<E: EthSpec>(&self, slot: Slot) -> [u8; 4] {
match self.next_fork_epoch::<E>(slot) {
Some((fork, _)) => self.fork_version_for_name(fork),
None => self.fork_version_for_name(self.fork_name_at_slot::<E>(slot)),
}
}
/// Returns the epoch of the next scheduled fork along with its corresponding `ForkName`.
@ -201,9 +207,12 @@ 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.altair_fork_epoch {
Some(fork_epoch) if epoch >= fork_epoch => ForkName::Altair,
_ => ForkName::Base,
match self.merge_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,
},
}
}
@ -212,6 +221,7 @@ impl ChainSpec {
match fork_name {
ForkName::Base => self.genesis_fork_version,
ForkName::Altair => self.altair_fork_version,
ForkName::Merge => self.merge_fork_version,
}
}
@ -220,6 +230,7 @@ impl ChainSpec {
match fork_name {
ForkName::Base => Some(Epoch::new(0)),
ForkName::Altair => self.altair_fork_epoch,
ForkName::Merge => self.merge_fork_epoch,
}
}
@ -467,6 +478,9 @@ impl ChainSpec {
domain_contribution_and_proof: 9,
altair_fork_version: [0x01, 0x00, 0x00, 0x00],
altair_fork_epoch: Some(Epoch::new(74240)),
merge_fork_version: [0x02, 0x00, 0x00, 0x00],
merge_fork_epoch: None,
terminal_total_difficulty: Uint256::MAX,
/*
* Network specific

View File

@ -19,3 +19,13 @@ pub mod altair {
pub const NUM_FLAG_INDICES: usize = 3;
}
pub mod merge_testing {
use ethereum_types::H256;
pub const GENESIS_GAS_LIMIT: u64 = 30_000_000;
pub const GENESIS_BASE_FEE_PER_GAS: H256 = H256([
0x00, 0xca, 0x9a, 0x3b, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
0x00, 0x00,
]);
}

View File

@ -9,6 +9,9 @@ use ssz_types::typenum::{
use std::fmt::{self, Debug};
use std::str::FromStr;
use ssz_types::typenum::{bit::B0, UInt, U1048576, U16384, U256, U625};
pub type U5000 = UInt<UInt<UInt<U625, B0>, B0>, B0>; // 625 * 8 = 5000
const MAINNET: &str = "mainnet";
const MINIMAL: &str = "minimal";
@ -80,6 +83,14 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq +
type SyncCommitteeSize: Unsigned + Clone + Sync + Send + Debug + PartialEq;
/// The number of `sync_committee` subnets.
type SyncCommitteeSubnetCount: Unsigned + Clone + Sync + Send + Debug + PartialEq;
/*
* New in Merge
*/
type MaxBytesPerOpaqueTransaction: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type MaxTransactionsPerPayload: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type BytesPerLogsBloom: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type GasLimitDenominator: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type MinGasLimit: Unsigned + Clone + Sync + Send + Debug + PartialEq;
/*
* Derived values (set these CAREFULLY)
*/
@ -187,6 +198,31 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq +
fn sync_subcommittee_size() -> usize {
Self::SyncSubcommitteeSize::to_usize()
}
/// Returns the `MAX_BYTES_PER_OPAQUE_TRANSACTION` constant for this specification.
fn max_bytes_per_opaque_transaction() -> usize {
Self::MaxBytesPerOpaqueTransaction::to_usize()
}
/// Returns the `MAX_TRANSACTIONS_PER_PAYLOAD` constant for this specification.
fn max_transactions_per_payload() -> usize {
Self::MaxTransactionsPerPayload::to_usize()
}
/// Returns the `BYTES_PER_LOGS_BLOOM` constant for this specification.
fn bytes_per_logs_bloom() -> usize {
Self::BytesPerLogsBloom::to_usize()
}
/// Returns the `GAS_LIMIT_DENOMINATOR` constant for this specification.
fn gas_limit_denominator() -> u64 {
Self::GasLimitDenominator::to_u64()
}
/// Returns the `MIN_GAS_LIMIT` constant for this specification.
fn min_gas_limit() -> u64 {
Self::MinGasLimit::to_u64()
}
}
/// Macro to inherit some type values from another EthSpec.
@ -221,6 +257,11 @@ impl EthSpec for MainnetEthSpec {
type MaxVoluntaryExits = U16;
type SyncCommitteeSize = U512;
type SyncCommitteeSubnetCount = U4;
type MaxBytesPerOpaqueTransaction = U1048576;
type MaxTransactionsPerPayload = U16384;
type BytesPerLogsBloom = U256;
type GasLimitDenominator = U1024;
type MinGasLimit = U5000;
type SyncSubcommitteeSize = U128; // 512 committee size / 4 sync committee subnet count
type MaxPendingAttestations = U4096; // 128 max attestations * 32 slots per epoch
type SlotsPerEth1VotingPeriod = U2048; // 64 epochs * 32 slots per epoch
@ -262,7 +303,12 @@ impl EthSpec for MinimalEthSpec {
MaxAttesterSlashings,
MaxAttestations,
MaxDeposits,
MaxVoluntaryExits
MaxVoluntaryExits,
MaxBytesPerOpaqueTransaction,
MaxTransactionsPerPayload,
BytesPerLogsBloom,
GasLimitDenominator,
MinGasLimit
});
fn default_spec() -> ChainSpec {

View File

@ -0,0 +1,174 @@
use crate::{test_utils::TestRandom, *};
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use std::{ops::Index, slice::SliceIndex};
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TreeHash)]
#[ssz(enum_behaviour = "union")]
#[tree_hash(enum_behaviour = "union")]
pub enum Transaction<T: EthSpec> {
OpaqueTransaction(VariableList<u8, T::MaxBytesPerOpaqueTransaction>),
}
impl<T: EthSpec, I: SliceIndex<[u8]>> Index<I> for Transaction<T> {
type Output = I::Output;
#[inline]
fn index(&self, index: I) -> &Self::Output {
match self {
Self::OpaqueTransaction(v) => Index::index(v, index),
}
}
}
impl<T: EthSpec> From<VariableList<u8, T::MaxBytesPerOpaqueTransaction>> for Transaction<T> {
fn from(list: VariableList<u8, <T as EthSpec>::MaxBytesPerOpaqueTransaction>) -> Self {
Self::OpaqueTransaction(list)
}
}
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(
Default, Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom,
)]
pub struct ExecutionPayload<T: EthSpec> {
pub parent_hash: Hash256,
pub coinbase: Address,
pub state_root: Hash256,
pub receipt_root: Hash256,
#[serde(with = "serde_logs_bloom")]
pub logs_bloom: FixedVector<u8, T::BytesPerLogsBloom>,
pub random: Hash256,
#[serde(with = "eth2_serde_utils::quoted_u64")]
pub block_number: u64,
#[serde(with = "eth2_serde_utils::quoted_u64")]
pub gas_limit: u64,
#[serde(with = "eth2_serde_utils::quoted_u64")]
pub gas_used: u64,
#[serde(with = "eth2_serde_utils::quoted_u64")]
pub timestamp: u64,
pub base_fee_per_gas: Hash256,
pub block_hash: Hash256,
#[serde(with = "serde_transactions")]
#[test_random(default)]
pub transactions: VariableList<Transaction<T>, T::MaxTransactionsPerPayload>,
}
impl<T: EthSpec> ExecutionPayload<T> {
// TODO: check this whole thing later
pub fn empty() -> Self {
Self {
parent_hash: Hash256::zero(),
coinbase: Address::default(),
state_root: Hash256::zero(),
receipt_root: Hash256::zero(),
logs_bloom: FixedVector::default(),
random: Hash256::zero(),
block_number: 0,
gas_limit: 0,
gas_used: 0,
timestamp: 0,
base_fee_per_gas: Hash256::zero(),
block_hash: Hash256::zero(),
transactions: VariableList::empty(),
}
}
}
/// Serializes the `logs_bloom` field.
pub mod serde_logs_bloom {
use super::*;
use eth2_serde_utils::hex::PrefixedHexVisitor;
use serde::{Deserializer, Serializer};
pub fn serialize<S, U>(bytes: &FixedVector<u8, U>, serializer: S) -> Result<S::Ok, S::Error>
where
S: Serializer,
U: Unsigned,
{
let mut hex_string: String = "0x".to_string();
hex_string.push_str(&hex::encode(&bytes[..]));
serializer.serialize_str(&hex_string)
}
pub fn deserialize<'de, D, U>(deserializer: D) -> Result<FixedVector<u8, U>, D::Error>
where
D: Deserializer<'de>,
U: Unsigned,
{
let vec = deserializer.deserialize_string(PrefixedHexVisitor)?;
FixedVector::new(vec)
.map_err(|e| serde::de::Error::custom(format!("invalid logs bloom: {:?}", e)))
}
}
/// Serializes the `transactions` field.
pub mod serde_transactions {
use super::*;
use eth2_serde_utils::hex;
use serde::ser::SerializeSeq;
use serde::{de, Deserializer, Serializer};
use std::marker::PhantomData;
pub struct ListOfBytesListVisitor<T: EthSpec> {
_t: PhantomData<T>,
}
impl<'a, T> serde::de::Visitor<'a> for ListOfBytesListVisitor<T>
where
T: EthSpec,
{
type Value = VariableList<Transaction<T>, T::MaxTransactionsPerPayload>;
fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result {
write!(formatter, "a list of 0x-prefixed byte lists")
}
fn visit_seq<A>(self, mut seq: A) -> Result<Self::Value, A::Error>
where
A: serde::de::SeqAccess<'a>,
{
let mut outer = VariableList::default();
while let Some(val) = seq.next_element::<String>()? {
let inner_vec = hex::decode(&val).map_err(de::Error::custom)?;
let inner = VariableList::new(inner_vec).map_err(|e| {
serde::de::Error::custom(format!("invalid transaction: {:?}", e))
})?;
outer.push(inner.into()).map_err(|e| {
serde::de::Error::custom(format!("too many transactions: {:?}", e))
})?;
}
Ok(outer)
}
}
pub fn serialize<S, T>(
value: &VariableList<Transaction<T>, T::MaxTransactionsPerPayload>,
serializer: S,
) -> Result<S::Ok, S::Error>
where
S: Serializer,
T: EthSpec,
{
let mut seq = serializer.serialize_seq(Some(value.len()))?;
for val in value {
seq.serialize_element(&hex::encode(&val[..]))?;
}
seq.end()
}
pub fn deserialize<'de, D, T>(
deserializer: D,
) -> Result<VariableList<Transaction<T>, T::MaxTransactionsPerPayload>, D::Error>
where
D: Deserializer<'de>,
T: EthSpec,
{
deserializer.deserialize_any(ListOfBytesListVisitor { _t: PhantomData })
}
}

View File

@ -0,0 +1,37 @@
use crate::{execution_payload::serde_logs_bloom, test_utils::TestRandom, *};
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(
Default, Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom,
)]
pub struct ExecutionPayloadHeader<T: EthSpec> {
pub parent_hash: Hash256,
pub coinbase: Address,
pub state_root: Hash256,
pub receipt_root: Hash256,
#[serde(with = "serde_logs_bloom")]
pub logs_bloom: FixedVector<u8, T::BytesPerLogsBloom>,
pub random: Hash256,
#[serde(with = "eth2_serde_utils::quoted_u64")]
pub block_number: u64,
#[serde(with = "eth2_serde_utils::quoted_u64")]
pub gas_limit: u64,
#[serde(with = "eth2_serde_utils::quoted_u64")]
pub gas_used: u64,
#[serde(with = "eth2_serde_utils::quoted_u64")]
pub timestamp: u64,
pub base_fee_per_gas: Hash256,
pub block_hash: Hash256,
pub transactions_root: Hash256,
}
impl<T: EthSpec> ExecutionPayloadHeader<T> {
// TODO: check this whole thing later
pub fn empty() -> Self {
Self::default()
}
}

View File

@ -35,6 +35,15 @@ impl ForkContext {
));
}
// Only add Merge to list of forks if it's enabled
// Note: `merge_fork_epoch == None` implies merge hasn't been activated yet on the config.
if spec.merge_fork_epoch.is_some() {
fork_to_digest.push((
ForkName::Merge,
ChainSpec::compute_fork_digest(spec.merge_fork_version, genesis_validators_root),
));
}
let fork_to_digest: HashMap<ForkName, [u8; 4]> = fork_to_digest.into_iter().collect();
let digest_to_fork = fork_to_digest

View File

@ -10,6 +10,7 @@ use std::str::FromStr;
pub enum ForkName {
Base,
Altair,
Merge,
}
impl ForkName {
@ -24,10 +25,17 @@ impl ForkName {
match self {
ForkName::Base => {
spec.altair_fork_epoch = None;
spec.merge_fork_epoch = None;
spec
}
ForkName::Altair => {
spec.altair_fork_epoch = Some(Epoch::new(0));
spec.merge_fork_epoch = None;
spec
}
ForkName::Merge => {
spec.altair_fork_epoch = None;
spec.merge_fork_epoch = Some(Epoch::new(0));
spec
}
}
@ -40,6 +48,7 @@ impl ForkName {
match self {
ForkName::Base => None,
ForkName::Altair => Some(ForkName::Base),
ForkName::Merge => Some(ForkName::Altair),
}
}
@ -49,7 +58,8 @@ impl ForkName {
pub fn next_fork(self) -> Option<ForkName> {
match self {
ForkName::Base => Some(ForkName::Altair),
ForkName::Altair => None,
ForkName::Altair => Some(ForkName::Merge),
ForkName::Merge => None,
}
}
}
@ -98,6 +108,7 @@ impl FromStr for ForkName {
Ok(match fork_name.to_lowercase().as_ref() {
"phase0" | "base" => ForkName::Base,
"altair" => ForkName::Altair,
"merge" => ForkName::Merge,
_ => return Err(()),
})
}
@ -108,6 +119,7 @@ impl Display for ForkName {
match self {
ForkName::Base => "phase0".fmt(f),
ForkName::Altair => "altair".fmt(f),
ForkName::Merge => "merge".fmt(f),
}
}
}
@ -139,7 +151,7 @@ mod test {
#[test]
fn previous_and_next_fork_consistent() {
assert_eq!(ForkName::Altair.next_fork(), None);
assert_eq!(ForkName::Merge.next_fork(), None);
assert_eq!(ForkName::Base.previous_fork(), None);
for (prev_fork, fork) in ForkName::list_all().into_iter().tuple_windows() {

View File

@ -37,6 +37,8 @@ pub mod deposit_message;
pub mod enr_fork_id;
pub mod eth1_data;
pub mod eth_spec;
pub mod execution_payload;
pub mod execution_payload_header;
pub mod fork;
pub mod fork_data;
pub mod fork_name;
@ -45,6 +47,7 @@ pub mod graffiti;
pub mod historical_batch;
pub mod indexed_attestation;
pub mod pending_attestation;
pub mod pow_block;
pub mod proposer_slashing;
pub mod relative_epoch;
pub mod selection_proof;
@ -90,11 +93,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, BeaconBlockRef, BeaconBlockRefMut,
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, BeaconBlockRef,
BeaconBlockRefMut,
};
pub use crate::beacon_block_body::{
BeaconBlockBody, BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyRef,
BeaconBlockBodyRefMut,
BeaconBlockBody, BeaconBlockBodyAltair, BeaconBlockBodyBase, BeaconBlockBodyMerge,
BeaconBlockBodyRef, BeaconBlockBodyRefMut,
};
pub use crate::beacon_block_header::BeaconBlockHeader;
pub use crate::beacon_committee::{BeaconCommittee, OwnedBeaconCommittee};
@ -109,6 +113,8 @@ pub use crate::deposit_message::DepositMessage;
pub use crate::enr_fork_id::EnrForkId;
pub use crate::eth1_data::Eth1Data;
pub use crate::eth_spec::EthSpecId;
pub use crate::execution_payload::ExecutionPayload;
pub use crate::execution_payload_header::ExecutionPayloadHeader;
pub use crate::fork::Fork;
pub use crate::fork_context::ForkContext;
pub use crate::fork_data::ForkData;
@ -120,6 +126,7 @@ pub use crate::indexed_attestation::IndexedAttestation;
pub use crate::participation_flags::ParticipationFlags;
pub use crate::participation_list::ParticipationList;
pub use crate::pending_attestation::PendingAttestation;
pub use crate::pow_block::PowBlock;
pub use crate::preset::{AltairPreset, BasePreset};
pub use crate::proposer_slashing::ProposerSlashing;
pub use crate::relative_epoch::{Error as RelativeEpochError, RelativeEpoch};
@ -128,6 +135,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,
};
pub use crate::signed_beacon_block_header::SignedBeaconBlockHeader;
pub use crate::signed_contribution_and_proof::SignedContributionAndProof;
@ -150,6 +158,7 @@ pub use crate::voluntary_exit::VoluntaryExit;
pub type CommitteeIndex = u64;
pub type Hash256 = H256;
pub type Uint256 = ethereum_types::U256;
pub type Address = H160;
pub type ForkVersion = [u8; 4];

View File

@ -0,0 +1,13 @@
use crate::*;
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Default, Debug, PartialEq, Clone)]
pub struct PowBlock {
pub block_hash: Hash256,
pub parent_hash: Hash256,
pub total_difficulty: Uint256,
pub difficulty: Uint256,
// needed to unify with other parts of codebase
pub timestamp: u64,
pub block_number: u64,
}

View File

@ -37,7 +37,7 @@ impl From<SignedBeaconBlockHash> for Hash256 {
/// A `BeaconBlock` and a signature from its proposer.
#[superstruct(
variants(Base, Altair),
variants(Base, Altair, Merge),
variant_attributes(
derive(
Debug,
@ -64,6 +64,8 @@ pub struct SignedBeaconBlock<E: EthSpec> {
pub message: BeaconBlockBase<E>,
#[superstruct(only(Altair), partial_getter(rename = "message_altair"))]
pub message: BeaconBlockAltair<E>,
#[superstruct(only(Merge), partial_getter(rename = "message_merge"))]
pub message: BeaconBlockMerge<E>,
pub signature: Signature,
}
@ -116,6 +118,9 @@ impl<E: EthSpec> SignedBeaconBlock<E> {
BeaconBlock::Altair(message) => {
SignedBeaconBlock::Altair(SignedBeaconBlockAltair { message, signature })
}
BeaconBlock::Merge(message) => {
SignedBeaconBlock::Merge(SignedBeaconBlockMerge { message, signature })
}
}
}
@ -129,6 +134,7 @@ impl<E: EthSpec> SignedBeaconBlock<E> {
SignedBeaconBlock::Altair(block) => {
(BeaconBlock::Altair(block.message), block.signature)
}
SignedBeaconBlock::Merge(block) => (BeaconBlock::Merge(block.message), block.signature),
}
}
@ -137,6 +143,7 @@ impl<E: EthSpec> SignedBeaconBlock<E> {
match self {
SignedBeaconBlock::Base(inner) => BeaconBlockRef::Base(&inner.message),
SignedBeaconBlock::Altair(inner) => BeaconBlockRef::Altair(&inner.message),
SignedBeaconBlock::Merge(inner) => BeaconBlockRef::Merge(&inner.message),
}
}
@ -145,6 +152,7 @@ impl<E: EthSpec> SignedBeaconBlock<E> {
match self {
SignedBeaconBlock::Base(inner) => BeaconBlockRefMut::Base(&mut inner.message),
SignedBeaconBlock::Altair(inner) => BeaconBlockRefMut::Altair(&mut inner.message),
SignedBeaconBlock::Merge(inner) => BeaconBlockRefMut::Merge(&mut inner.message),
}
}

View File

@ -14,6 +14,7 @@ mod public_key_bytes;
mod secret_key;
mod signature;
mod signature_bytes;
mod uint256;
pub fn test_random_instance<T: TestRandom>() -> T {
let mut rng = XorShiftRng::from_seed([0x42; 16]);

View File

@ -1,10 +1,10 @@
use super::*;
use crate::Hash256;
use crate::Uint256;
impl TestRandom for Hash256 {
impl TestRandom for Uint256 {
fn random_for_test(rng: &mut impl RngCore) -> Self {
let mut key_bytes = vec![0; 32];
let mut key_bytes = [0; 32];
rng.fill_bytes(&mut key_bytes);
Hash256::from_slice(&key_bytes[..])
Self::from_little_endian(&key_bytes[..])
}
}

View File

@ -0,0 +1,10 @@
use super::*;
use crate::Hash256;
impl TestRandom for Hash256 {
fn random_for_test(rng: &mut impl RngCore) -> Self {
let mut key_bytes = vec![0; 32];
rng.fill_bytes(&mut key_bytes);
Hash256::from_slice(&key_bytes[..])
}
}

View File

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

View File

@ -94,10 +94,12 @@ impl<E: EthSpec> EpochTransition<E> for JustificationAndFinalization {
spec,
)
}
BeaconState::Altair(_) => altair::process_justification_and_finalization(
state,
&altair::ParticipationCache::new(state, spec).unwrap(),
),
BeaconState::Altair(_) | BeaconState::Merge(_) => {
altair::process_justification_and_finalization(
state,
&altair::ParticipationCache::new(state, spec).unwrap(),
)
}
}
}
}
@ -110,11 +112,13 @@ impl<E: EthSpec> EpochTransition<E> for RewardsAndPenalties {
validator_statuses.process_attestations(state)?;
base::process_rewards_and_penalties(state, &mut validator_statuses, spec)
}
BeaconState::Altair(_) => altair::process_rewards_and_penalties(
state,
&altair::ParticipationCache::new(state, spec).unwrap(),
spec,
),
BeaconState::Altair(_) | BeaconState::Merge(_) => {
altair::process_rewards_and_penalties(
state,
&altair::ParticipationCache::new(state, spec).unwrap(),
spec,
)
}
}
}
}
@ -138,7 +142,7 @@ impl<E: EthSpec> EpochTransition<E> for Slashings {
spec,
)?;
}
BeaconState::Altair(_) => {
BeaconState::Altair(_) | BeaconState::Merge(_) => {
process_slashings(
state,
altair::ParticipationCache::new(state, spec)
@ -197,7 +201,9 @@ impl<E: EthSpec> EpochTransition<E> for SyncCommitteeUpdates {
fn run(state: &mut BeaconState<E>, spec: &ChainSpec) -> Result<(), EpochProcessingError> {
match state {
BeaconState::Base(_) => Ok(()),
BeaconState::Altair(_) => altair::process_sync_committee_updates(state, spec),
BeaconState::Altair(_) | BeaconState::Merge(_) => {
altair::process_sync_committee_updates(state, spec)
}
}
}
}
@ -206,7 +212,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(_) => altair::process_inactivity_updates(
BeaconState::Altair(_) | BeaconState::Merge(_) => altair::process_inactivity_updates(
state,
&altair::ParticipationCache::new(state, spec).unwrap(),
spec,
@ -219,7 +225,9 @@ impl<E: EthSpec> EpochTransition<E> for ParticipationFlagUpdates {
fn run(state: &mut BeaconState<E>, _: &ChainSpec) -> Result<(), EpochProcessingError> {
match state {
BeaconState::Base(_) => Ok(()),
BeaconState::Altair(_) => altair::process_participation_flag_updates(state),
BeaconState::Altair(_) | BeaconState::Merge(_) => {
altair::process_participation_flag_updates(state)
}
}
}
}
@ -267,7 +275,7 @@ impl<E: EthSpec, T: EpochTransition<E>> Case for EpochProcessing<E, T> {
&& T::name() != "inactivity_updates"
&& T::name() != "participation_flag_updates"
}
ForkName::Altair => true,
ForkName::Altair | ForkName::Merge => true, // TODO: revisit when tests are out
}
}

View File

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

View File

@ -37,6 +37,9 @@ impl<E: EthSpec> LoadCase for TransitionTest<E> {
ForkName::Altair => {
spec.altair_fork_epoch = Some(metadata.fork_epoch);
}
ForkName::Merge => {
spec.merge_fork_epoch = Some(metadata.fork_epoch);
}
}
// Load blocks

View File

@ -34,6 +34,7 @@ pub trait Handler {
let fork_name_str = match fork_name {
ForkName::Base => "phase0",
ForkName::Altair => "altair",
ForkName::Merge => "merge", // TODO: check this
};
let handler_path = PathBuf::from(env!("CARGO_MANIFEST_DIR"))