diff --git a/Cargo.lock b/Cargo.lock index 181f5ab2b0..b092faff8e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3081,21 +3081,25 @@ version = "1.3.12" dependencies = [ "alloy-consensus", "alloy-eips", + "alloy-genesis", "alloy-primitives 1.0.0", "alloy-rlp", "crunchy", "rayon", "reth-chainspec", + "reth-consensus", "reth-db", "reth-db-api", + "reth-db-common", "reth-ethereum-consensus", "reth-ethereum-primitives", + "reth-evm", "reth-evm-ethereum", "reth-primitives-traits", "reth-provider", "reth-revm", - "reth-stages", - "reth-static-file-types", + "reth-trie", + "reth-trie-db", "revm", "serde", "serde_json", diff --git a/testing/ef-tests/Cargo.toml b/testing/ef-tests/Cargo.toml index 297d9a4aa1..6021bba4c9 100644 --- a/testing/ef-tests/Cargo.toml +++ b/testing/ef-tests/Cargo.toml @@ -22,21 +22,24 @@ asm-keccak = [ reth-chainspec.workspace = true reth-ethereum-primitives.workspace = true reth-primitives-traits.workspace = true +reth-consensus.workspace = true reth-db = { workspace = true, features = ["mdbx", "test-utils", "disable-lock"] } reth-db-api.workspace = true +reth-db-common.workspace = true reth-provider = { workspace = true, features = ["test-utils"] } -reth-stages.workspace = true -reth-static-file-types.workspace = true +reth-evm.workspace = true reth-evm-ethereum.workspace = true reth-ethereum-consensus.workspace = true reth-revm = { workspace = true, features = ["std"] } - +reth-trie.workspace = true +reth-trie-db.workspace = true revm = { workspace = true, features = ["secp256k1", "blst", "c-kzg"] } alloy-rlp.workspace = true alloy-primitives.workspace = true alloy-eips.workspace = true alloy-consensus.workspace = true +alloy-genesis.workspace = true walkdir.workspace = true serde.workspace = true diff --git a/testing/ef-tests/src/cases/blockchain_test.rs b/testing/ef-tests/src/cases/blockchain_test.rs index 73eddfd320..21d2411107 100644 --- a/testing/ef-tests/src/cases/blockchain_test.rs +++ b/testing/ef-tests/src/cases/blockchain_test.rs @@ -7,14 +7,20 @@ use crate::{ use alloy_rlp::Decodable; use rayon::iter::{ParallelBridge, ParallelIterator}; use reth_chainspec::ChainSpec; -use reth_ethereum_consensus::EthBeaconConsensus; +use reth_consensus::{Consensus, HeaderValidator}; +use reth_db_common::init::{insert_genesis_hashes, insert_genesis_history, insert_genesis_state}; +use reth_ethereum_consensus::{validate_block_post_execution, EthBeaconConsensus}; use reth_ethereum_primitives::Block; -use reth_primitives_traits::SealedBlock; +use reth_evm::execute::{BlockExecutorProvider, Executor}; +use reth_evm_ethereum::execute::EthExecutorProvider; +use reth_primitives_traits::{RecoveredBlock, SealedBlock}; use reth_provider::{ - providers::StaticFileWriter, test_utils::create_test_provider_factory_with_chain_spec, - DatabaseProviderFactory, HashingWriter, StaticFileProviderFactory, StaticFileSegment, + test_utils::create_test_provider_factory_with_chain_spec, BlockWriter, DatabaseProviderFactory, + ExecutionOutcome, HistoryWriter, OriginalValuesKnown, StateWriter, StorageLocation, }; -use reth_stages::{stages::ExecutionStage, ExecInput, Stage}; +use reth_revm::database::StateProviderDatabase; +use reth_trie::{HashedPostState, KeccakKeyHasher, StateRoot}; +use reth_trie_db::DatabaseStateRoot; use std::{collections::BTreeMap, fs, path::Path, sync::Arc}; /// A handler for the blockchain test suite. @@ -45,6 +51,91 @@ pub struct BlockchainTestCase { skip: bool, } +impl BlockchainTestCase { + /// Returns `true` if the fork is not supported. + const fn excluded_fork(network: ForkSpec) -> bool { + matches!( + network, + ForkSpec::ByzantiumToConstantinopleAt5 | + ForkSpec::Constantinople | + ForkSpec::ConstantinopleFix | + ForkSpec::MergeEOF | + ForkSpec::MergeMeterInitCode | + ForkSpec::MergePush0 | + ForkSpec::Unknown + ) + } + + /// Checks if the test case is a particular test called `UncleFromSideChain` + /// + /// This fixture fails as expected, however it fails at the wrong block number. + /// Given we no longer have uncle blocks, this test case was pulled out such + /// that we ensure it still fails as expected, however we do not check the block number. + #[inline] + fn is_uncle_sidechain_case(name: &str) -> bool { + name.contains("UncleFromSideChain") + } + + /// If the test expects an exception, return the the block number + /// at which it must occur together with the original message. + /// + /// Note: There is a +1 here because the genesis block is not included + /// in the set of blocks, so the first block is actually block number 1 + /// and not block number 0. + #[inline] + fn expected_failure(case: &BlockchainTest) -> Option<(u64, String)> { + case.blocks.iter().enumerate().find_map(|(idx, blk)| { + blk.expect_exception.as_ref().map(|msg| ((idx + 1) as u64, msg.clone())) + }) + } + + /// Execute a single `BlockchainTest`, validating the outcome against the + /// expectations encoded in the JSON file. + fn run_single_case(name: &str, case: &BlockchainTest) -> Result<(), Error> { + let expectation = Self::expected_failure(case); + match run_case(case) { + // All blocks executed successfully. + Ok(()) => { + // Check if the test case specifies that it should have failed + if let Some((block, msg)) = expectation { + Err(Error::Assertion(format!( + "Test case: {name}\nExpected failure at block {block} - {msg}, but all blocks succeeded", + ))) + } else { + Ok(()) + } + } + + // A block processing failure occurred. + Err(Error::BlockProcessingFailed { block_number }) => match expectation { + // It happened on exactly the block we were told to fail on + Some((expected, _)) if block_number == expected => Ok(()), + + // Uncle side‑chain edge case, we accept as long as it failed. + // But we don't check the exact block number. + _ if Self::is_uncle_sidechain_case(name) => Ok(()), + + // Expected failure, but block number does not match + Some((expected, _)) => Err(Error::Assertion(format!( + "Test case: {name}\nExpected failure at block {expected}\nGot failure at block {block_number}", + ))), + + // No failure expected at all - bubble up original error. + None => Err(Error::BlockProcessingFailed { block_number }), + }, + + // Non‑processing error – forward as‑is. + // + // This should only happen if we get an unexpected error from processing the block. + // Since it is unexpected, we treat it as a test failure. + // + // One reason for this happening is when one forgets to wrap the error from `run_case` + // so that it produces a `Error::BlockProcessingFailed` + Err(other) => Err(other), + } + } +} + impl Case for BlockchainTestCase { fn load(path: &Path) -> Result { Ok(Self { @@ -70,33 +161,10 @@ impl Case for BlockchainTestCase { // Iterate through test cases, filtering by the network type to exclude specific forks. self.tests - .values() - .filter(|case| { - !matches!( - case.network, - ForkSpec::ByzantiumToConstantinopleAt5 | - ForkSpec::Constantinople | - ForkSpec::ConstantinopleFix | - ForkSpec::MergeEOF | - ForkSpec::MergeMeterInitCode | - ForkSpec::MergePush0 | - ForkSpec::Unknown - ) - }) + .iter() + .filter(|(_, case)| !Self::excluded_fork(case.network)) .par_bridge() - .try_for_each(|case| { - let case_result = run_case(case); - let has_failed = case_result.is_err(); - - // Check if the test should fail - let should_fail = case.blocks.iter().any(|block| block.expect_exception.is_some()); - - // A test that fails and should have failed is successful. - if has_failed && should_fail { - return Ok(()) - } - case_result - })?; + .try_for_each(|(name, case)| Self::run_single_case(name, case))?; Ok(()) } @@ -117,80 +185,147 @@ impl Case for BlockchainTestCase { fn run_case(case: &BlockchainTest) -> Result<(), Error> { // Create a new test database and initialize a provider for the test case. let chain_spec: Arc = Arc::new(case.network.into()); - let provider = create_test_provider_factory_with_chain_spec(chain_spec.clone()) - .database_provider_rw() - .unwrap(); + let factory = create_test_provider_factory_with_chain_spec(chain_spec.clone()); + let provider = factory.database_provider_rw().unwrap(); // Insert initial test state into the provider. - provider.insert_historical_block( - SealedBlock::::from_sealed_parts( - case.genesis_block_header.clone().into(), - Default::default(), - ) - .try_recover() - .unwrap(), - )?; - case.pre.write_to_db(provider.tx_ref())?; - - // Initialize receipts static file with genesis - { - let static_file_provider = provider.static_file_provider(); - let mut receipts_writer = - static_file_provider.latest_writer(StaticFileSegment::Receipts).unwrap(); - receipts_writer.increment_block(0).unwrap(); - receipts_writer.commit_without_sync_all().unwrap(); - } - - // Decode and insert blocks, creating a chain of blocks for the test case. - let last_block = case.blocks.iter().try_fold(None, |_, block| { - let decoded = SealedBlock::::decode(&mut block.rlp.as_ref())?; - provider.insert_historical_block(decoded.clone().try_recover().unwrap())?; - Ok::>, Error>(Some(decoded)) - })?; + let genesis_block = SealedBlock::::from_sealed_parts( + case.genesis_block_header.clone().into(), + Default::default(), + ) + .try_recover() + .unwrap(); provider - .static_file_provider() - .latest_writer(StaticFileSegment::Headers) - .unwrap() - .commit_without_sync_all() - .unwrap(); + .insert_block(genesis_block.clone(), StorageLocation::Database) + .map_err(|_| Error::BlockProcessingFailed { block_number: 0 })?; - // Execute the execution stage using the EVM processor factory for the test case - // network. - // - // Note: If `execute` fails, we do not check the error because the post state check - // will subsequently fail because no state is written on execution failure. - let _ = ExecutionStage::new_with_executor( - reth_evm_ethereum::execute::EthExecutorProvider::ethereum(chain_spec.clone()), - Arc::new(EthBeaconConsensus::new(chain_spec)), - ) - .execute( - &provider, - ExecInput { target: last_block.as_ref().map(|b| b.number), checkpoint: None }, - ); + let genesis_state = case.pre.clone().into_genesis_state(); + insert_genesis_state(&provider, genesis_state.iter()) + .map_err(|_| Error::BlockProcessingFailed { block_number: 0 })?; + insert_genesis_hashes(&provider, genesis_state.iter()) + .map_err(|_| Error::BlockProcessingFailed { block_number: 0 })?; + insert_genesis_history(&provider, genesis_state.iter()) + .map_err(|_| Error::BlockProcessingFailed { block_number: 0 })?; - // Validate the post-state for the test case. - match (&case.post_state, &case.post_state_hash) { - (Some(state), None) => { - // Validate accounts in the state against the provider's database. - for (&address, account) in state { - account.assert_db(address, provider.tx_ref())?; - } + // Decode blocks + let blocks = decode_blocks(&case.blocks)?; + + let executor_provider = EthExecutorProvider::ethereum(chain_spec.clone()); + let mut parent = genesis_block; + + for (block_index, block) in blocks.iter().enumerate() { + // Note: same as the comment on `decode_blocks` as to why we cannot use block.number + let block_number = (block_index + 1) as u64; + + // Insert the block into the database + provider + .insert_block(block.clone(), StorageLocation::Database) + .map_err(|_| Error::BlockProcessingFailed { block_number })?; + + // Consensus checks before block execution + pre_execution_checks(chain_spec.clone(), &parent, block) + .map_err(|_| Error::BlockProcessingFailed { block_number })?; + + // Execute the block + let state_db = StateProviderDatabase(provider.latest()); + let executor = executor_provider.executor(state_db); + let output = + executor.execute(block).map_err(|_| Error::BlockProcessingFailed { block_number })?; + + // Consensus checks after block execution + validate_block_post_execution(block, &chain_spec, &output.receipts, &output.requests) + .map_err(|_| Error::BlockProcessingFailed { block_number })?; + + // Compute and check the post state root + let hashed_state = + HashedPostState::from_bundle_state::(output.state.state()); + let (computed_state_root, _) = + StateRoot::overlay_root_with_updates(provider.tx_ref(), hashed_state.clone()) + .map_err(|_| Error::BlockProcessingFailed { block_number })?; + if computed_state_root != block.state_root { + return Err(Error::BlockProcessingFailed { block_number }) } - (None, Some(expected_state_root)) => { - // Insert state hashes into the provider based on the expected state root. - let last_block = last_block.unwrap_or_default(); - provider.insert_hashes( - 0..=last_block.number, - last_block.hash(), - *expected_state_root, - )?; - } - _ => return Err(Error::MissingPostState), + + // Commit the post state/state diff to the database + provider + .write_state( + &ExecutionOutcome::single(block.number, output), + OriginalValuesKnown::Yes, + StorageLocation::Database, + ) + .map_err(|_| Error::BlockProcessingFailed { block_number })?; + + provider + .write_hashed_state(&hashed_state.into_sorted()) + .map_err(|_| Error::BlockProcessingFailed { block_number })?; + provider + .update_history_indices(block.number..=block.number) + .map_err(|_| Error::BlockProcessingFailed { block_number })?; + + // Since there were no errors, update the parent block + parent = block.clone() } - // Drop the provider without committing to the database. - drop(provider); + // Validate the post-state for the test case. + // + // If we get here then it means that the post-state root checks + // made after we execute each block was successful. + // + // If an error occurs here, then it is: + // - Either an issue with the test setup + // - Possibly an error in the test case where the post-state root in the last block does not + // match the post-state values. + let expected_post_state = case.post_state.as_ref().ok_or(Error::MissingPostState)?; + for (&address, account) in expected_post_state { + account.assert_db(address, provider.tx_ref())?; + } + + Ok(()) +} + +fn decode_blocks( + test_case_blocks: &[crate::models::Block], +) -> Result>, Error> { + let mut blocks = Vec::with_capacity(test_case_blocks.len()); + for (block_index, block) in test_case_blocks.iter().enumerate() { + // The blocks do not include the genesis block which is why we have the plus one. + // We also cannot use block.number because for invalid blocks, this may be incorrect. + let block_number = (block_index + 1) as u64; + + let decoded = SealedBlock::::decode(&mut block.rlp.as_ref()) + .map_err(|_| Error::BlockProcessingFailed { block_number })?; + + let recovered_block = decoded + .clone() + .try_recover() + .map_err(|_| Error::BlockProcessingFailed { block_number })?; + + blocks.push(recovered_block); + } + + Ok(blocks) +} + +fn pre_execution_checks( + chain_spec: Arc, + parent: &RecoveredBlock, + block: &RecoveredBlock, +) -> Result<(), Error> { + let consensus: EthBeaconConsensus = EthBeaconConsensus::new(chain_spec); + + let sealed_header = block.sealed_header(); + let header = block.header(); + + as Consensus>::validate_body_against_header( + &consensus, + block.body(), + sealed_header, + )?; + consensus.validate_header_against_parent(sealed_header, parent.sealed_header())?; + consensus.validate_header_with_total_difficulty(header, block.difficulty)?; + consensus.validate_header(sealed_header)?; + consensus.validate_block_pre_execution(block)?; Ok(()) } diff --git a/testing/ef-tests/src/models.rs b/testing/ef-tests/src/models.rs index 7e4b6b93a7..6ef42f048a 100644 --- a/testing/ef-tests/src/models.rs +++ b/testing/ef-tests/src/models.rs @@ -3,6 +3,7 @@ use crate::{assert::assert_equal, Error}; use alloy_consensus::Header as RethHeader; use alloy_eips::eip4895::Withdrawals; +use alloy_genesis::GenesisAccount; use alloy_primitives::{keccak256, Address, Bloom, Bytes, B256, B64, U256}; use reth_chainspec::{ChainSpec, ChainSpecBuilder}; use reth_db_api::{ @@ -27,8 +28,6 @@ pub struct BlockchainTest { pub blocks: Vec, /// The expected post state. pub post_state: Option>, - /// The expected post state merkle root. - pub post_state_hash: Option, /// The test pre-state. pub pre: State, /// Hash of the best block. @@ -193,6 +192,34 @@ impl State { } Ok(()) } + + /// Return state as genesis state. + pub fn into_genesis_state(self) -> BTreeMap { + self.0 + .into_iter() + .map(|(address, account)| { + let storage = account + .storage + .iter() + .filter(|(_, v)| !v.is_zero()) + .map(|(k, v)| { + ( + B256::from_slice(&k.to_be_bytes::<32>()), + B256::from_slice(&v.to_be_bytes::<32>()), + ) + }) + .collect(); + let account = GenesisAccount { + balance: account.balance, + nonce: Some(account.nonce.try_into().unwrap()), + code: Some(account.code).filter(|c| !c.is_empty()), + storage: Some(storage), + private_key: None, + }; + (address, account) + }) + .collect::>() + } } impl Deref for State { diff --git a/testing/ef-tests/src/result.rs b/testing/ef-tests/src/result.rs index da967325a2..a1bed359b0 100644 --- a/testing/ef-tests/src/result.rs +++ b/testing/ef-tests/src/result.rs @@ -20,6 +20,14 @@ pub enum Error { /// No post state found in test #[error("no post state found for validation")] MissingPostState, + /// Block processing failed + /// Note: This includes but is not limited to execution. + /// For example, the header number could be incorrect. + #[error("block {block_number} failed to process")] + BlockProcessingFailed { + /// The block number for the block that failed + block_number: u64, + }, /// An IO error occurred #[error("an error occurred interacting with the file system at {path}: {error}")] Io { @@ -50,6 +58,9 @@ pub enum Error { /// An error occurred while decoding RLP. #[error("an error occurred deserializing RLP: {0}")] RlpDecodeError(#[from] alloy_rlp::Error), + /// A consensus error occurred. + #[error("an error occurred during consensus checks: {0}")] + ConsensusError(#[from] reth_consensus::ConsensusError), } /// The result of running a test.