From 5687c56d5191318bceafc46cff19fac32f32583c Mon Sep 17 00:00:00 2001 From: Mark Mackey Date: Wed, 8 Sep 2021 13:45:22 -0500 Subject: [PATCH] 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 to use Transaction 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 --- Cargo.lock | 1 + beacon_node/beacon_chain/src/beacon_chain.rs | 27 +++ .../beacon_chain/src/block_verification.rs | 141 +++++++++++++- beacon_node/beacon_chain/src/eth1_chain.rs | 31 +++- beacon_node/beacon_chain/src/lib.rs | 2 +- beacon_node/eth1/src/http.rs | 78 +++++--- beacon_node/eth1/src/service.rs | 27 ++- beacon_node/eth1/tests/test.rs | 9 +- beacon_node/lighthouse_network/src/config.rs | 4 +- .../src/rpc/codec/ssz_snappy.rs | 52 ++++-- .../lighthouse_network/src/rpc/protocol.rs | 32 +++- .../lighthouse_network/src/types/pubsub.rs | 7 +- .../beacon_processor/worker/gossip_methods.rs | 15 +- beacon_node/store/src/hot_cold_store.rs | 4 + beacon_node/store/src/partial_beacon_state.rs | 45 ++++- consensus/fork_choice/Cargo.toml | 1 + consensus/fork_choice/src/fork_choice.rs | 29 ++- consensus/ssz/src/decode/impls.rs | 23 ++- consensus/ssz/src/encode/impls.rs | 20 +- .../src/common/slash_validator.rs | 4 +- consensus/state_processing/src/genesis.rs | 2 + .../src/per_block_processing.rs | 136 ++++++++++++++ .../src/per_block_processing/errors.rs | 20 ++ .../process_operations.rs | 2 +- .../src/per_epoch_processing.rs | 2 +- .../src/per_slot_processing.rs | 18 +- consensus/state_processing/src/upgrade.rs | 2 + .../state_processing/src/upgrade/merge.rs | 72 ++++++++ consensus/tree_hash/src/impls.rs | 24 ++- consensus/types/src/beacon_block.rs | 70 ++++++- consensus/types/src/beacon_block_body.rs | 29 ++- consensus/types/src/beacon_state.rs | 21 ++- consensus/types/src/chain_spec.rs | 30 ++- consensus/types/src/consts.rs | 10 + consensus/types/src/eth_spec.rs | 48 ++++- consensus/types/src/execution_payload.rs | 174 ++++++++++++++++++ .../types/src/execution_payload_header.rs | 37 ++++ consensus/types/src/fork_context.rs | 9 + consensus/types/src/fork_name.rs | 16 +- consensus/types/src/lib.rs | 15 +- consensus/types/src/pow_block.rs | 13 ++ consensus/types/src/signed_beacon_block.rs | 10 +- consensus/types/src/test_utils/test_random.rs | 1 + .../src/test_utils/test_random/hash256.rs | 8 +- .../src/test_utils/test_random/uint256.rs | 10 + testing/ef_tests/src/cases/common.rs | 1 + .../ef_tests/src/cases/epoch_processing.rs | 36 ++-- testing/ef_tests/src/cases/operations.rs | 2 +- testing/ef_tests/src/cases/transition.rs | 3 + testing/ef_tests/src/handler.rs | 1 + 50 files changed, 1241 insertions(+), 133 deletions(-) create mode 100644 consensus/state_processing/src/upgrade/merge.rs create mode 100644 consensus/types/src/execution_payload.rs create mode 100644 consensus/types/src/execution_payload_header.rs create mode 100644 consensus/types/src/pow_block.rs create mode 100644 consensus/types/src/test_utils/test_random/uint256.rs diff --git a/Cargo.lock b/Cargo.lock index 602bfc261..2503176fe 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -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", ] diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 5f8b70bf4..7a253e4e8 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -2838,6 +2838,11 @@ impl BeaconChain { SyncAggregate::new() })) }; + // Closure to fetch a sync aggregate in cases where it is required. + let get_execution_payload = || -> Result, 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 BeaconChain { }, }) } + 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( diff --git a/beacon_node/beacon_chain/src/block_verification.rs b/beacon_node/beacon_chain/src/block_verification.rs index 5ddeafa45..bf03cf979 100644 --- a/beacon_node/beacon_chain/src/block_verification.rs +++ b/beacon_node/beacon_chain/src/block_verification.rs @@ -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 { /// /// 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 std::fmt::Display for BlockError { @@ -668,6 +729,18 @@ impl GossipVerifiedBlock { }); } + // 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( + block: BeaconBlockRef<'_, E>, + state: &BeaconState, +) -> Result<(), BlockError> { + 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 == >::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. /// diff --git a/beacon_node/beacon_chain/src/eth1_chain.rs b/beacon_node/beacon_chain/src/eth1_chain.rs index aa6978b79..ec046b685 100644 --- a/beacon_node/beacon_chain/src/eth1_chain.rs +++ b/beacon_node/beacon_chain/src/eth1_chain.rs @@ -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 for Error { @@ -274,6 +276,15 @@ where ) } + pub fn on_payload(&self, execution_payload: &ExecutionPayload) -> Result { + if self.use_dummy_backend { + let dummy_backend: DummyEth1ChainBackend = 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: Sized + Send + Sync { /// an idea of how up-to-date the remote eth1 node is. fn head_block(&self) -> Option; + /// Verifies the execution payload + fn on_payload(&self, execution_payload: &ExecutionPayload) -> Result; + /// Encode the `Eth1ChainBackend` instance to bytes. fn as_bytes(&self) -> Vec; @@ -388,6 +402,10 @@ impl Eth1ChainBackend for DummyEth1ChainBackend { None } + fn on_payload(&self, _execution_payload: &ExecutionPayload) -> Result { + Ok(true) + } + /// Return empty Vec for dummy backend. fn as_bytes(&self) -> Vec { Vec::new() @@ -556,6 +574,15 @@ impl Eth1ChainBackend for CachingEth1Backend { self.core.head_block() } + fn on_payload(&self, execution_payload: &ExecutionPayload) -> Result { + 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 { self.core.as_bytes() diff --git a/beacon_node/beacon_chain/src/lib.rs b/beacon_node/beacon_chain/src/lib.rs index 2cd636f23..19c366572 100644 --- a/beacon_node/beacon_chain/src/lib.rs +++ b/beacon_node/beacon_chain/src/lib.rs @@ -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; diff --git a/beacon_node/eth1/src/http.rs b/beacon_node/eth1/src/http.rs index 9e3465f0f..489142377 100644 --- a/beacon_node/eth1/src/http.rs +++ b/beacon_node/eth1/src/http.rs @@ -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 { +) -> Result { 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 = hex_to_bytes( + let block_hash: Vec = 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 = 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)) } diff --git a/beacon_node/eth1/src/service.rs b/beacon_node/eth1/src/service.rs index 460f53e73..ca6e0c588 100644 --- a/beacon_node/eth1/src/service.rs +++ b/beacon_node/eth1/src/service.rs @@ -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( + &self, + _execution_payload: ExecutionPayload, + ) -> Result { + 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, }) diff --git a/beacon_node/eth1/tests/test.rs b/beacon_node/eth1/tests/test.rs index bb00ebaab..4141f8b78 100644 --- a/beacon_node/eth1/tests/test.rs +++ b/beacon_node/eth1/tests/test.rs @@ -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(ð1, 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" ); } diff --git a/beacon_node/lighthouse_network/src/config.rs b/beacon_node/lighthouse_network/src/config.rs index e18fd00ae..4ea3fa4b6 100644 --- a/beacon_node/lighthouse_network/src/config.rs +++ b/beacon_node/lighthouse_network/src/config.rs @@ -209,7 +209,9 @@ pub fn gossipsub_config(fork_context: Arc) -> GossipsubConfig { ) -> Vec { 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(), diff --git a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs index 6c6f0b9bc..c9db51406 100644 --- a/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs +++ b/beacon_node/lighthouse_network/src/rpc/codec/ssz_snappy.rs @@ -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( } /// 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( protocol: &ProtocolId, fork_context: &ForkContext, @@ -383,23 +383,25 @@ fn context_bytes( ) -> 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( 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( 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, diff --git a/beacon_node/lighthouse_network/src/rpc/protocol.rs b/beacon_node/lighthouse_network/src/rpc/protocol.rs index c00b9c049..0a711257b 100644 --- a/beacon_node/lighthouse_network/src/rpc/protocol.rs +++ b/beacon_node/lighthouse_network/src/rpc/protocol.rs @@ -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::::from_block( + BeaconBlock::Merge(BeaconBlockMerge::::empty(&MainnetEthSpec::default_spec())), + Signature::empty(), + ) + .as_ssz_bytes() + .len(); + pub static ref SIGNED_BEACON_BLOCK_MERGE_MAX: usize = SignedBeaconBlock::::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::::from(Vec::::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( diff --git a/beacon_node/lighthouse_network/src/types/pubsub.rs b/beacon_node/lighthouse_network/src/types/pubsub.rs index 75ef6e8ab..af2656a27 100644 --- a/beacon_node/lighthouse_network/src/types/pubsub.rs +++ b/beacon_node/lighthouse_network/src/types/pubsub.rs @@ -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 PubsubMessage { SignedBeaconBlockAltair::from_ssz_bytes(data) .map_err(|e| format!("{:?}", e))?, ), + Some(ForkName::Merge) => SignedBeaconBlock::::Merge( + SignedBeaconBlockMerge::from_ssz_bytes(data) + .map_err(|e| format!("{:?}", e))?, + ), None => { return Err(format!( "Unknown gossipsub fork digest: {:?}", diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index 2e5ee5160..dd42531cf 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -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 Worker { 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 Worker { | 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); diff --git a/beacon_node/store/src/hot_cold_store.rs b/beacon_node/store/src/hot_cold_store.rs index 0194544c8..cfa49847d 100644 --- a/beacon_node/store/src/hot_cold_store.rs +++ b/beacon_node/store/src/hot_cold_store.rs @@ -883,6 +883,10 @@ impl, Cold: ItemStore> HotColdDB &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(); diff --git a/beacon_node/store/src/partial_beacon_state.rs b/beacon_node/store/src/partial_beacon_state.rs index 9c8fcc4b7..010796afd 100644 --- a/beacon_node/store/src/partial_beacon_state.rs +++ b/beacon_node/store/src/partial_beacon_state.rs @@ -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, T::MaxPendingAttestations>, // Participation (Altair and later) - #[superstruct(only(Altair))] + #[superstruct(only(Altair, Merge))] pub previous_epoch_participation: VariableList, - #[superstruct(only(Altair))] + #[superstruct(only(Altair, Merge))] pub current_epoch_participation: VariableList, // Finality @@ -78,14 +78,18 @@ where pub finalized_checkpoint: Checkpoint, // Inactivity - #[superstruct(only(Altair))] + #[superstruct(only(Altair, Merge))] pub inactivity_scores: VariableList, // Light-client sync committees - #[superstruct(only(Altair))] + #[superstruct(only(Altair, Merge))] pub current_sync_committee: Arc>, - #[superstruct(only(Altair))] + #[superstruct(only(Altair, Merge))] pub next_sync_committee: Arc>, + + // Execution + #[superstruct(only(Merge))] + pub latest_execution_payload_header: ExecutionPayloadHeader, } /// Implement the conversion function from BeaconState -> PartialBeaconState. @@ -160,6 +164,20 @@ impl PartialBeaconState { 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 TryInto> for PartialBeaconState { 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) } diff --git a/consensus/fork_choice/Cargo.toml b/consensus/fork_choice/Cargo.toml index f708045df..2bfe3f137 100644 --- a/consensus/fork_choice/Cargo.toml +++ b/consensus/fork_choice/Cargo.toml @@ -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" diff --git a/consensus/fork_choice/src/fork_choice.rs b/consensus/fork_choice/src/fork_choice.rs index d0aa8abc1..f1b9a6999 100644 --- a/consensus/fork_choice/src/fork_choice.rs +++ b/consensus/fork_choice/src/fork_choice.rs @@ -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, }, + 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 ForkChoice where T: ForkChoiceStore, @@ -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 diff --git a/consensus/ssz/src/decode/impls.rs b/consensus/ssz/src/decode/impls.rs index 29b2aec8e..0e6b39083 100644 --- a/consensus/ssz/src/decode/impls.rs +++ b/consensus/ssz/src/decode/impls.rs @@ -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 Decode for Arc { } } +impl Decode for H160 { + fn is_ssz_fixed_len() -> bool { + true + } + + fn ssz_fixed_len() -> usize { + 20 + } + + fn from_ssz_bytes(bytes: &[u8]) -> Result { + let len = bytes.len(); + let expected = ::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 diff --git a/consensus/ssz/src/encode/impls.rs b/consensus/ssz/src/encode/impls.rs index 00d3e0a3a..5728685d0 100644 --- a/consensus/ssz/src/encode/impls.rs +++ b/consensus/ssz/src/encode/impls.rs @@ -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) { + buf.extend_from_slice(self.as_bytes()); + } +} + impl Encode for H256 { fn is_ssz_fixed_len() -> bool { true diff --git a/consensus/state_processing/src/common/slash_validator.rs b/consensus/state_processing/src/common/slash_validator.rs index 7643043ba..237905a30 100644 --- a/consensus/state_processing/src/common/slash_validator.rs +++ b/consensus/state_processing/src/common/slash_validator.rs @@ -33,7 +33,7 @@ pub fn slash_validator( 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( 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)?, }; diff --git a/consensus/state_processing/src/genesis.rs b/consensus/state_processing/src/genesis.rs index 7834c6daf..b9f3c781a 100644 --- a/consensus/state_processing/src/genesis.rs +++ b/consensus/state_processing/src/genesis.rs @@ -52,6 +52,8 @@ pub fn initialize_beacon_state_from_eth1( 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)?; diff --git a/consensus/state_processing/src/per_block_processing.rs b/consensus/state_processing/src/per_block_processing.rs index 845aee747..29e5a7c46 100644 --- a/consensus/state_processing/src/per_block_processing.rs +++ b/consensus/state_processing/src/per_block_processing.rs @@ -149,6 +149,10 @@ pub fn per_block_processing( )?; } + 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( 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( + payload: &ExecutionPayload, + parent: &ExecutionPayloadHeader, +) -> 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( + state: &mut BeaconState, + payload: &ExecutionPayload, + 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(state: &BeaconState) -> bool { + state + .latest_execution_payload_header() + .map(|header| *header != >::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(state: &BeaconState, body: BeaconBlockBodyRef) -> bool { + body.execution_payload() + .map(|payload| !is_merge_complete(state) && *payload != >::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( + state: &BeaconState, + body: BeaconBlockBodyRef, +) -> 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( + state: &BeaconState, + spec: &ChainSpec, +) -> Result { + 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)) +} diff --git a/consensus/state_processing/src/per_block_processing/errors.rs b/consensus/state_processing/src/per_block_processing/errors.rs index 2ba9ea78c..b6fa363e0 100644 --- a/consensus/state_processing/src/per_block_processing/errors.rs +++ b/consensus/state_processing/src/per_block_processing/errors.rs @@ -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 for BlockProcessingError { diff --git a/consensus/state_processing/src/per_block_processing/process_operations.rs b/consensus/state_processing/src/per_block_processing/process_operations.rs index f2cef47d6..a4a0738eb 100644 --- a/consensus/state_processing/src/per_block_processing/process_operations.rs +++ b/consensus/state_processing/src/per_block_processing/process_operations.rs @@ -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(), diff --git a/consensus/state_processing/src/per_epoch_processing.rs b/consensus/state_processing/src/per_epoch_processing.rs index 245876b86..d813dc42f 100644 --- a/consensus/state_processing/src/per_epoch_processing.rs +++ b/consensus/state_processing/src/per_epoch_processing.rs @@ -35,7 +35,7 @@ pub fn process_epoch( 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), } } diff --git a/consensus/state_processing/src/per_slot_processing.rs b/consensus/state_processing/src/per_slot_processing.rs index 43eaa89c1..454cee5ff 100644 --- a/consensus/state_processing/src/per_slot_processing.rs +++ b/consensus/state_processing/src/per_slot_processing.rs @@ -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( 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) diff --git a/consensus/state_processing/src/upgrade.rs b/consensus/state_processing/src/upgrade.rs index ca8e51596..fda1a714a 100644 --- a/consensus/state_processing/src/upgrade.rs +++ b/consensus/state_processing/src/upgrade.rs @@ -1,3 +1,5 @@ pub mod altair; +pub mod merge; pub use altair::upgrade_to_altair; +pub use merge::upgrade_to_merge; diff --git a/consensus/state_processing/src/upgrade/merge.rs b/consensus/state_processing/src/upgrade/merge.rs new file mode 100644 index 000000000..c41987609 --- /dev/null +++ b/consensus/state_processing/src/upgrade/merge.rs @@ -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( + pre_state: &mut BeaconState, + 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: >::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(()) +} diff --git a/consensus/tree_hash/src/impls.rs b/consensus/tree_hash/src/impls.rs index 7fdcfceb7..00fed489c 100644 --- a/consensus/tree_hash/src/impls.rs +++ b/consensus/tree_hash/src/impls.rs @@ -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 { + 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 diff --git a/consensus/types/src/beacon_block.rs b/consensus/types/src/beacon_block.rs index f11b92148..cc706224c 100644 --- a/consensus/types/src/beacon_block.rs +++ b/consensus/types/src/beacon_block.rs @@ -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 { pub body: BeaconBlockBodyBase, #[superstruct(only(Altair), partial_getter(rename = "body_altair"))] pub body: BeaconBlockBodyAltair, + #[superstruct(only(Merge), partial_getter(rename = "body_merge"))] + pub body: BeaconBlockBodyMerge, } impl SignedRoot for BeaconBlock {} @@ -63,7 +66,9 @@ impl<'a, T: EthSpec> SignedRoot for BeaconBlockRef<'a, T> {} impl BeaconBlock { /// 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 BeaconBlockAltair { } } +impl BeaconBlockMerge { + /// 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::*; diff --git a/consensus/types/src/beacon_block_body.rs b/consensus/types/src/beacon_block_body.rs index ceb90fef9..3b417f5d0 100644 --- a/consensus/types/src/beacon_block_body.rs +++ b/consensus/types/src/beacon_block_body.rs @@ -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 { pub attestations: VariableList, T::MaxAttestations>, pub deposits: VariableList, pub voluntary_exits: VariableList, - #[superstruct(only(Altair))] + #[superstruct(only(Altair, Merge))] pub sync_aggregate: SyncAggregate, + #[superstruct(only(Merge))] + pub execution_payload: ExecutionPayload, } 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> { + 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, } } } diff --git a/consensus/types/src/beacon_state.rs b/consensus/types/src/beacon_state.rs index a12f35143..2c1c2a651 100644 --- a/consensus/types/src/beacon_state.rs +++ b/consensus/types/src/beacon_state.rs @@ -172,7 +172,7 @@ impl From 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, T::MaxPendingAttestations>, // Participation (Altair and later) - #[superstruct(only(Altair))] + #[superstruct(only(Altair, Merge))] pub previous_epoch_participation: VariableList, - #[superstruct(only(Altair))] + #[superstruct(only(Altair, Merge))] pub current_epoch_participation: VariableList, // 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, // Light-client sync committees - #[superstruct(only(Altair))] + #[superstruct(only(Altair, Merge))] pub current_sync_committee: Arc>, - #[superstruct(only(Altair))] + #[superstruct(only(Altair, Merge))] pub next_sync_committee: Arc>, + // Execution + #[superstruct(only(Merge))] + pub latest_execution_payload_header: ExecutionPayloadHeader, + // Caching (not in the spec) #[serde(skip_serializing, skip_deserializing)] #[ssz(skip_serializing, skip_deserializing)] @@ -384,6 +388,7 @@ impl BeaconState { 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 BeaconState { 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 BeaconState { 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 BeaconState { 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(); diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index ee213f1f8..812c998be 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -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, + pub merge_fork_version: [u8; 4], + /// The Merge fork epoch is optional, with `None` representing "Merge never happens". + pub merge_fork_epoch: Option, + pub terminal_total_difficulty: Uint256, /* * Networking @@ -156,7 +160,7 @@ impl ChainSpec { ) -> EnrForkId { EnrForkId { fork_digest: self.fork_digest::(slot, genesis_validators_root), - next_fork_version: self.next_fork_version(), + next_fork_version: self.next_fork_version::(slot), next_fork_epoch: self .next_fork_epoch::(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(&self, slot: Slot) -> [u8; 4] { + match self.next_fork_epoch::(slot) { + Some((fork, _)) => self.fork_version_for_name(fork), + None => self.fork_version_for_name(self.fork_name_at_slot::(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 diff --git a/consensus/types/src/consts.rs b/consensus/types/src/consts.rs index 04e8e60ee..6088086ca 100644 --- a/consensus/types/src/consts.rs +++ b/consensus/types/src/consts.rs @@ -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, + ]); +} diff --git a/consensus/types/src/eth_spec.rs b/consensus/types/src/eth_spec.rs index 6e21edf9f..c45ecf8f7 100644 --- a/consensus/types/src/eth_spec.rs +++ b/consensus/types/src/eth_spec.rs @@ -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, 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 { diff --git a/consensus/types/src/execution_payload.rs b/consensus/types/src/execution_payload.rs new file mode 100644 index 000000000..bb0915098 --- /dev/null +++ b/consensus/types/src/execution_payload.rs @@ -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 { + OpaqueTransaction(VariableList), +} + +impl> Index for Transaction { + type Output = I::Output; + + #[inline] + fn index(&self, index: I) -> &Self::Output { + match self { + Self::OpaqueTransaction(v) => Index::index(v, index), + } + } +} + +impl From> for Transaction { + fn from(list: VariableList::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 { + pub parent_hash: Hash256, + pub coinbase: Address, + pub state_root: Hash256, + pub receipt_root: Hash256, + #[serde(with = "serde_logs_bloom")] + pub logs_bloom: FixedVector, + 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, T::MaxTransactionsPerPayload>, +} + +impl ExecutionPayload { + // 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(bytes: &FixedVector, serializer: S) -> Result + 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, 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: PhantomData, + } + impl<'a, T> serde::de::Visitor<'a> for ListOfBytesListVisitor + where + T: EthSpec, + { + type Value = VariableList, 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(self, mut seq: A) -> Result + where + A: serde::de::SeqAccess<'a>, + { + let mut outer = VariableList::default(); + + while let Some(val) = seq.next_element::()? { + 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( + value: &VariableList, T::MaxTransactionsPerPayload>, + serializer: S, + ) -> Result + 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, T::MaxTransactionsPerPayload>, D::Error> + where + D: Deserializer<'de>, + T: EthSpec, + { + deserializer.deserialize_any(ListOfBytesListVisitor { _t: PhantomData }) + } +} diff --git a/consensus/types/src/execution_payload_header.rs b/consensus/types/src/execution_payload_header.rs new file mode 100644 index 000000000..f0340eff6 --- /dev/null +++ b/consensus/types/src/execution_payload_header.rs @@ -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 { + pub parent_hash: Hash256, + pub coinbase: Address, + pub state_root: Hash256, + pub receipt_root: Hash256, + #[serde(with = "serde_logs_bloom")] + pub logs_bloom: FixedVector, + 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 ExecutionPayloadHeader { + // TODO: check this whole thing later + pub fn empty() -> Self { + Self::default() + } +} diff --git a/consensus/types/src/fork_context.rs b/consensus/types/src/fork_context.rs index 1d488f769..88a2f3126 100644 --- a/consensus/types/src/fork_context.rs +++ b/consensus/types/src/fork_context.rs @@ -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 = fork_to_digest.into_iter().collect(); let digest_to_fork = fork_to_digest diff --git a/consensus/types/src/fork_name.rs b/consensus/types/src/fork_name.rs index 85ba35e39..faf6c04de 100644 --- a/consensus/types/src/fork_name.rs +++ b/consensus/types/src/fork_name.rs @@ -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 { 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() { diff --git a/consensus/types/src/lib.rs b/consensus/types/src/lib.rs index 403544f00..9ccd52f7b 100644 --- a/consensus/types/src/lib.rs +++ b/consensus/types/src/lib.rs @@ -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]; diff --git a/consensus/types/src/pow_block.rs b/consensus/types/src/pow_block.rs new file mode 100644 index 000000000..056459af1 --- /dev/null +++ b/consensus/types/src/pow_block.rs @@ -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, +} diff --git a/consensus/types/src/signed_beacon_block.rs b/consensus/types/src/signed_beacon_block.rs index a9d6f1d98..383805f97 100644 --- a/consensus/types/src/signed_beacon_block.rs +++ b/consensus/types/src/signed_beacon_block.rs @@ -37,7 +37,7 @@ impl From 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 { pub message: BeaconBlockBase, #[superstruct(only(Altair), partial_getter(rename = "message_altair"))] pub message: BeaconBlockAltair, + #[superstruct(only(Merge), partial_getter(rename = "message_merge"))] + pub message: BeaconBlockMerge, pub signature: Signature, } @@ -116,6 +118,9 @@ impl SignedBeaconBlock { BeaconBlock::Altair(message) => { SignedBeaconBlock::Altair(SignedBeaconBlockAltair { message, signature }) } + BeaconBlock::Merge(message) => { + SignedBeaconBlock::Merge(SignedBeaconBlockMerge { message, signature }) + } } } @@ -129,6 +134,7 @@ impl SignedBeaconBlock { SignedBeaconBlock::Altair(block) => { (BeaconBlock::Altair(block.message), block.signature) } + SignedBeaconBlock::Merge(block) => (BeaconBlock::Merge(block.message), block.signature), } } @@ -137,6 +143,7 @@ impl SignedBeaconBlock { 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 SignedBeaconBlock { 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), } } diff --git a/consensus/types/src/test_utils/test_random.rs b/consensus/types/src/test_utils/test_random.rs index bafbdca5f..064b57f42 100644 --- a/consensus/types/src/test_utils/test_random.rs +++ b/consensus/types/src/test_utils/test_random.rs @@ -14,6 +14,7 @@ mod public_key_bytes; mod secret_key; mod signature; mod signature_bytes; +mod uint256; pub fn test_random_instance() -> T { let mut rng = XorShiftRng::from_seed([0x42; 16]); diff --git a/consensus/types/src/test_utils/test_random/hash256.rs b/consensus/types/src/test_utils/test_random/hash256.rs index 8733f7de2..a74cc6b3d 100644 --- a/consensus/types/src/test_utils/test_random/hash256.rs +++ b/consensus/types/src/test_utils/test_random/hash256.rs @@ -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[..]) } } diff --git a/consensus/types/src/test_utils/test_random/uint256.rs b/consensus/types/src/test_utils/test_random/uint256.rs new file mode 100644 index 000000000..8733f7de2 --- /dev/null +++ b/consensus/types/src/test_utils/test_random/uint256.rs @@ -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[..]) + } +} diff --git a/testing/ef_tests/src/cases/common.rs b/testing/ef_tests/src/cases/common.rs index 175ad113b..ade8711cd 100644 --- a/testing/ef_tests/src/cases/common.rs +++ b/testing/ef_tests/src/cases/common.rs @@ -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.. } } diff --git a/testing/ef_tests/src/cases/epoch_processing.rs b/testing/ef_tests/src/cases/epoch_processing.rs index 56e6c9b7b..fa27a94ce 100644 --- a/testing/ef_tests/src/cases/epoch_processing.rs +++ b/testing/ef_tests/src/cases/epoch_processing.rs @@ -94,10 +94,12 @@ impl EpochTransition 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 EpochTransition 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 EpochTransition for Slashings { spec, )?; } - BeaconState::Altair(_) => { + BeaconState::Altair(_) | BeaconState::Merge(_) => { process_slashings( state, altair::ParticipationCache::new(state, spec) @@ -197,7 +201,9 @@ impl EpochTransition for SyncCommitteeUpdates { fn run(state: &mut BeaconState, 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 EpochTransition for InactivityUpdates { fn run(state: &mut BeaconState, 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 EpochTransition for ParticipationFlagUpdates { fn run(state: &mut BeaconState, _: &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> Case for EpochProcessing { && T::name() != "inactivity_updates" && T::name() != "participation_flag_updates" } - ForkName::Altair => true, + ForkName::Altair | ForkName::Merge => true, // TODO: revisit when tests are out } } diff --git a/testing/ef_tests/src/cases/operations.rs b/testing/ef_tests/src/cases/operations.rs index 293195662..360abbb67 100644 --- a/testing/ef_tests/src/cases/operations.rs +++ b/testing/ef_tests/src/cases/operations.rs @@ -72,7 +72,7 @@ impl Operation for Attestation { 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, diff --git a/testing/ef_tests/src/cases/transition.rs b/testing/ef_tests/src/cases/transition.rs index d41a52d52..861e65d3d 100644 --- a/testing/ef_tests/src/cases/transition.rs +++ b/testing/ef_tests/src/cases/transition.rs @@ -37,6 +37,9 @@ impl LoadCase for TransitionTest { ForkName::Altair => { spec.altair_fork_epoch = Some(metadata.fork_epoch); } + ForkName::Merge => { + spec.merge_fork_epoch = Some(metadata.fork_epoch); + } } // Load blocks diff --git a/testing/ef_tests/src/handler.rs b/testing/ef_tests/src/handler.rs index 11bda8f9f..e42098342 100644 --- a/testing/ef_tests/src/handler.rs +++ b/testing/ef_tests/src/handler.rs @@ -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"))