chore: Explicitly check for failure on expected blocks in ef-tests (#15803)

Co-authored-by: Roman Krasiuk <rokrassyuk@gmail.com>
This commit is contained in:
kevaundray
2025-04-23 12:18:18 +01:00
committed by GitHub
parent 57f77ac438
commit 9d3509c8ac
5 changed files with 283 additions and 103 deletions

View File

@@ -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

View File

@@ -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 sidechain 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 }),
},
// Nonprocessing error forward asis.
//
// 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<Self, Error> {
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<ChainSpec> = 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::<Block>::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::<Block>::decode(&mut block.rlp.as_ref())?;
provider.insert_historical_block(decoded.clone().try_recover().unwrap())?;
Ok::<Option<SealedBlock<Block>>, Error>(Some(decoded))
})?;
let genesis_block = SealedBlock::<Block>::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::<KeccakKeyHasher>(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<Vec<RecoveredBlock<Block>>, 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::<Block>::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<ChainSpec>,
parent: &RecoveredBlock<Block>,
block: &RecoveredBlock<Block>,
) -> Result<(), Error> {
let consensus: EthBeaconConsensus<ChainSpec> = EthBeaconConsensus::new(chain_spec);
let sealed_header = block.sealed_header();
let header = block.header();
<EthBeaconConsensus<ChainSpec> as Consensus<Block>>::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(())
}

View File

@@ -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<Block>,
/// The expected post state.
pub post_state: Option<BTreeMap<Address, Account>>,
/// The expected post state merkle root.
pub post_state_hash: Option<B256>,
/// 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<Address, GenesisAccount> {
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::<BTreeMap<_, _>>()
}
}
impl Deref for State {

View File

@@ -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.