mirror of
https://github.com/paradigmxyz/reth.git
synced 2026-01-08 23:08:19 -05:00
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:
8
Cargo.lock
generated
8
Cargo.lock
generated
@@ -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",
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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<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(())
|
||||
}
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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.
|
||||
|
||||
Reference in New Issue
Block a user