feat: add stateless crate to expose stateless validation (#15591)

Co-authored-by: Roman Krasiuk <rokrassyuk@gmail.com>
Co-authored-by: Alexey Shekhirin <5773434+shekhirin@users.noreply.github.com>
This commit is contained in:
kevaundray
2025-05-06 13:38:07 +01:00
committed by GitHub
parent 1e33d4cfe2
commit 3e5c230f4d
13 changed files with 846 additions and 20 deletions

View File

@@ -28,6 +28,7 @@ crates_to_check=(
reth-ethereum-forks
reth-ethereum-primitives
reth-ethereum-consensus
reth-stateless
## optimism
reth-optimism-chainspec

42
Cargo.lock generated
View File

@@ -137,9 +137,9 @@ dependencies = [
[[package]]
name = "alloy-consensus-any"
version = "0.15.8"
version = "0.15.9"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "8c13988a8d23dfd00a49dc6702bc704000d34853951f23b9c125a342ee537443"
checksum = "c5eb741241cd0a6c59eb2d75221ae1530de821781d9a32d532ed010c1dec277a"
dependencies = [
"alloy-consensus",
"alloy-eips 0.15.9",
@@ -599,9 +599,9 @@ dependencies = [
[[package]]
name = "alloy-rpc-types-any"
version = "0.15.8"
version = "0.15.9"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "0d1a67b833618d34929f343c764cd84b88fb76af48fe3f4581ac6919e453822f"
checksum = "a148344212e34b213a9e3516ace482df74f4146b43bd898f2e8c655707851ec3"
dependencies = [
"alloy-consensus-any",
"alloy-rpc-types-eth",
@@ -1861,9 +1861,9 @@ dependencies = [
[[package]]
name = "brotli"
version = "8.0.0"
version = "8.0.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "cf19e729cdbd51af9a397fb9ef8ac8378007b797f8273cfbfdf45dcaa316167b"
checksum = "9991eea70ea4f293524138648e41ee89b0b2b12ddef3b255effa43c8056e0e0d"
dependencies = [
"alloc-no-stdlib",
"alloc-stdlib",
@@ -3063,12 +3063,15 @@ dependencies = [
"reth-primitives-traits",
"reth-provider",
"reth-revm",
"reth-stateless",
"reth-tracing",
"reth-trie",
"reth-trie-db",
"revm",
"serde",
"serde_json",
"thiserror 2.0.12",
"tracing",
"walkdir",
]
@@ -5136,9 +5139,9 @@ dependencies = [
[[package]]
name = "libm"
version = "0.2.13"
version = "0.2.14"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "c9627da5196e5d8ed0b0495e61e518847578da83483c37288316d9b2e03a7f72"
checksum = "a25169bd5913a4b437588a7e3d127cd6e90127b60e0ffbd834a38f1599e016b8"
[[package]]
name = "libp2p-identity"
@@ -9884,6 +9887,29 @@ dependencies = [
"test-fuzz",
]
[[package]]
name = "reth-stateless"
version = "1.3.12"
dependencies = [
"alloy-consensus",
"alloy-primitives",
"alloy-rlp",
"alloy-trie",
"itertools 0.14.0",
"reth-chainspec",
"reth-consensus",
"reth-errors",
"reth-ethereum-consensus",
"reth-evm",
"reth-evm-ethereum",
"reth-primitives",
"reth-revm",
"reth-trie-common",
"reth-trie-sparse",
"serde",
"thiserror 2.0.12",
]
[[package]]
name = "reth-static-file"
version = "1.3.12"

View File

@@ -109,6 +109,7 @@ members = [
"crates/stages/api/",
"crates/stages/stages/",
"crates/stages/types/",
"crates/stateless",
"crates/static-file/static-file",
"crates/static-file/types/",
"crates/storage/codecs/",
@@ -423,6 +424,7 @@ reth-rpc-types-compat = { path = "crates/rpc/rpc-types-compat" }
reth-stages = { path = "crates/stages/stages" }
reth-stages-api = { path = "crates/stages/api" }
reth-stages-types = { path = "crates/stages/types", default-features = false }
reth-stateless = { path = "crates/stateless" }
reth-static-file = { path = "crates/static-file/static-file" }
reth-static-file-types = { path = "crates/static-file/types", default-features = false }
reth-storage-api = { path = "crates/storage/storage-api", default-features = false }

View File

@@ -0,0 +1,36 @@
[package]
name = "reth-stateless"
version.workspace = true
edition.workspace = true
rust-version.workspace = true
license.workspace = true
homepage.workspace = true
repository.workspace = true
exclude.workspace = true
[lints]
workspace = true
[dependencies]
# alloy
alloy-primitives.workspace = true
alloy-rlp.workspace = true
alloy-trie.workspace = true
alloy-consensus.workspace = true
# reth
reth-ethereum-consensus.workspace = true
reth-primitives.workspace = true
reth-errors.workspace = true
reth-evm.workspace = true
reth-evm-ethereum.workspace = true
reth-revm.workspace = true
reth-trie-common.workspace = true
reth-trie-sparse.workspace = true
reth-chainspec.workspace = true
reth-consensus.workspace = true
# misc
thiserror.workspace = true
itertools.workspace = true
serde.workspace = true # Needed because we copied `ExecutionWitness` into this crate

View File

@@ -0,0 +1,78 @@
/*
This file has been copied from alloy-rpc-types-debug.
The reason we have it is because alloy-rpc-types-debug is not no_std compatible.
Checking the Cargo.toml file shows that it enables "std" for alloy-primitives.
Currently it does not seem to need "std", so one option is to disable it.
One could alternatively argue that the type may not belong in rpc-debug-types and
may belong somewhere like alloy-primitives and re-exported in rpc-types-debug.
Reason being, with all variants of statelessness (when shipped),
there is an `ExecutionWitness` that comes with the block, so its
not just used for debugging.
*/
use alloc::vec::Vec;
use alloy_primitives::Bytes;
use serde::{Deserialize, Serialize};
/// Represents the execution witness of a block. Contains an optional map of state preimages.
#[derive(Clone, Debug, Default, PartialEq, Eq, Serialize, Deserialize)]
pub struct ExecutionWitness {
/// List of all hashed trie nodes preimages that were required during the execution of
/// the block, including during state root recomputation.
pub state: Vec<Bytes>,
/// List of all contract codes (created / accessed) preimages that were required during
/// the execution of the block, including during state root recomputation.
pub codes: Vec<Bytes>,
/// List of all hashed account and storage keys (addresses and slots) preimages
/// (unhashed account addresses and storage slots, respectively) that were required during
/// the execution of the block.
pub keys: Vec<Bytes>,
/// Block headers required for proving correctness of stateless execution.
///
/// This collection stores ancestor(parent) block headers needed to verify:
/// - State reads are correct (ie the code and accounts are correct wrt the pre-state root)
/// - BLOCKHASH opcode execution results are correct
///
/// ## Why this field will be empty in the future
///
/// This field is expected to be empty in the future because:
/// - EIP-2935 (Prague) will include block hashes directly in the state
/// - Verkle/Delayed execution will change the block structure to contain the pre-state root
/// instead of the post-state root.
///
/// Once both of these upgrades have been implemented, this field will be empty
/// moving forward because the data that this was proving will either be in the
/// current block or in the state.
///
/// ## State Read Verification
///
/// To verify state reads are correct, we need the pre-state root of the current block,
/// which is (currently) equal to the post-state root of the previous block. We therefore
/// need the previous block's header in order to prove that the state reads are correct.
///
/// Note: While the pre-state root is located in the previous block, this field
/// will always have one or more items.
///
/// ## BLOCKHASH Opcode Verification
///
/// The BLOCKHASH opcode returns the block hash for a given block number, but it
/// only works for the 256 most recent blocks, not including the current block.
/// To verify that a block hash is indeed correct wrt the BLOCKHASH opcode
/// and not an arbitrary set of block hashes, we need a contiguous set of
/// block headers starting from the current block.
///
/// ### Example
///
/// Consider a blockchain at block 200, and inside of block 200, a transaction
/// calls BLOCKHASH(100):
/// - This is valid because block 100 is within the 256-block lookback window
/// - To verify this, we need all of the headers from block 100 through block 200
/// - These headers form a chain proving the correctness of block 100's hash.
///
/// The naive way to construct the headers would be to unconditionally include the last
/// 256 block headers. However note, we may not need all 256, like in the example above.
pub headers: Vec<Bytes>,
}

View File

@@ -0,0 +1,44 @@
//! Provides types and functions for stateless execution and validation of Ethereum blocks.
//!
//! This crate enables the verification of block execution without requiring access to a
//! full node's persistent database. Instead, it relies on pre-generated "witness" data
//! that proves the specific state accessed during the block's execution.
//!
//! # Key Components
//!
//! * `WitnessDatabase`: An implementation of [`reth_revm::Database`] that uses a
//! [`reth_trie_sparse::SparseStateTrie`] populated from witness data, along with provided
//! bytecode and ancestor block hashes, to serve state reads during execution.
//! * `stateless_validation`: The core function that orchestrates the stateless validation process.
//! It takes a block, its execution witness, ancestor headers, and chain specification, then
//! performs:
//! 1. Witness verification against the parent block's state root.
//! 2. Block execution using the `WitnessDatabase`.
//! 3. Post-execution consensus checks.
//! 4. Post-state root calculation and comparison against the block header.
//!
//! # Usage
//!
//! The primary entry point is typically the `validation::stateless_validation` function. Callers
//! need to provide the block to be validated along with accurately generated `ExecutionWitness`
//! data corresponding to that block's execution trace and the necessary Headers of ancestor
//! blocks.
#![doc(
html_logo_url = "https://raw.githubusercontent.com/paradigmxyz/reth/main/assets/reth-docs.png",
html_favicon_url = "https://avatars0.githubusercontent.com/u/97369466?s=256",
issue_tracker_base_url = "https://github.com/paradigmxyz/reth/issues/"
)]
#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))]
#![cfg_attr(not(test), warn(unused_crate_dependencies))]
#![no_std]
extern crate alloc;
pub(crate) mod root;
/// Implementation of stateless validation
pub mod validation;
pub(crate) mod witness_db;
mod execution_witness;
pub use execution_witness::ExecutionWitness;

View File

@@ -0,0 +1,96 @@
// Copied and modified from ress: https://github.com/paradigmxyz/ress/blob/06bf2c4788e45b8fcbd640e38b6243e6f87c4d0e/crates/engine/src/tree/root.rs
use alloc::vec::Vec;
use alloy_primitives::B256;
use alloy_rlp::{Decodable, Encodable};
use itertools::Itertools;
use reth_trie_common::{
HashedPostState, Nibbles, TrieAccount, EMPTY_ROOT_HASH, TRIE_ACCOUNT_RLP_MAX_SIZE,
};
use reth_trie_sparse::{errors::SparseStateTrieResult, SparseStateTrie, SparseTrie};
/// Calculates the post-execution state root by applying state changes to a sparse trie.
///
/// This function takes a [`SparseStateTrie`] with the pre-state and a [`HashedPostState`]
/// containing account and storage changes resulting from block execution (state diff).
///
/// It modifies the input `trie` in place to reflect these changes and then calculates the
/// final post-execution state root.
pub(crate) fn calculate_state_root(
trie: &mut SparseStateTrie,
state: HashedPostState,
) -> SparseStateTrieResult<B256> {
// 1. Apply storageslot updates and compute each contracts storage root
//
//
// We walk over every (address, storage) pair in deterministic order
// and update the corresponding peraccount storage trie inplace.
// When were done we collect (address, updated_storage_trie) in a `Vec`
// so that we can insert them back into the outer state trie afterwards ― this avoids
// borrowing issues.
let mut storage_results = Vec::with_capacity(state.storages.len());
for (address, storage) in state.storages.into_iter().sorted_unstable_by_key(|(addr, _)| *addr) {
// Take the existing storage trie (or create an empty, “revealed” one)
let mut storage_trie =
trie.take_storage_trie(&address).unwrap_or_else(SparseTrie::revealed_empty);
if storage.wiped {
storage_trie.wipe()?;
}
// Apply slotlevel changes
for (hashed_slot, value) in
storage.storage.into_iter().sorted_unstable_by_key(|(slot, _)| *slot)
{
let nibbles = Nibbles::unpack(hashed_slot);
if value.is_zero() {
storage_trie.remove_leaf(&nibbles)?;
} else {
storage_trie.update_leaf(nibbles, alloy_rlp::encode_fixed_size(&value).to_vec())?;
}
}
// Finalise the storagetrie root before pushing the result
storage_trie.root();
storage_results.push((address, storage_trie));
}
// Insert every updated storage trie back into the outer state trie
for (address, storage_trie) in storage_results {
trie.insert_storage_trie(address, storage_trie);
}
// 2. Apply accountlevel updates and (re)encode the account nodes
// Update accounts with new values
// TODO: upstream changes into reth so that `SparseStateTrie::update_account` handles this
let mut account_rlp_buf = Vec::with_capacity(TRIE_ACCOUNT_RLP_MAX_SIZE);
for (hashed_address, account) in
state.accounts.into_iter().sorted_unstable_by_key(|(addr, _)| *addr)
{
let nibbles = Nibbles::unpack(hashed_address);
let account = account.unwrap_or_default();
// Determine which storage root should be used for this account
let storage_root = if let Some(storage_trie) = trie.storage_trie_mut(&hashed_address) {
storage_trie.root()
} else if let Some(value) = trie.get_account_value(&hashed_address) {
TrieAccount::decode(&mut &value[..])?.storage_root
} else {
EMPTY_ROOT_HASH
};
// Decide whether to remove or update the account leaf
if account.is_empty() && storage_root == EMPTY_ROOT_HASH {
trie.remove_account_leaf(&nibbles)?;
} else {
account_rlp_buf.clear();
account.into_trie_account(storage_root).encode(&mut account_rlp_buf);
trie.update_account_leaf(nibbles, account_rlp_buf.clone())?;
}
}
// Return new state root
trie.root()
}

View File

@@ -0,0 +1,322 @@
use crate::{witness_db::WitnessDatabase, ExecutionWitness};
use alloc::{
collections::BTreeMap,
string::{String, ToString},
sync::Arc,
vec::Vec,
};
use alloy_consensus::{Block, BlockHeader, Header};
use alloy_primitives::{keccak256, map::B256Map, B256};
use alloy_rlp::Decodable;
use reth_chainspec::ChainSpec;
use reth_consensus::{Consensus, HeaderValidator};
use reth_errors::ConsensusError;
use reth_ethereum_consensus::{validate_block_post_execution, EthBeaconConsensus};
use reth_evm::{execute::Executor, ConfigureEvm};
use reth_evm_ethereum::execute::EthExecutorProvider;
use reth_primitives::{RecoveredBlock, TransactionSigned};
use reth_revm::state::Bytecode;
use reth_trie_common::{HashedPostState, KeccakKeyHasher};
use reth_trie_sparse::{blinded::DefaultBlindedProviderFactory, SparseStateTrie};
/// Errors that can occur during stateless validation.
#[derive(Debug, thiserror::Error)]
pub enum StatelessValidationError {
/// Error when the number of ancestor headers exceeds the limit.
#[error("ancestor header count ({count}) exceeds limit ({limit})")]
AncestorHeaderLimitExceeded {
/// The number of headers provided.
count: usize,
/// The limit.
limit: usize,
},
/// Error when the ancestor headers do not form a contiguous chain.
#[error("invalid ancestor chain")]
InvalidAncestorChain,
/// Error when revealing the witness data failed.
#[error("failed to reveal witness data for pre-state root {pre_state_root}")]
WitnessRevealFailed {
/// The pre-state root used for verification.
pre_state_root: B256,
},
/// Error during stateless block execution.
#[error("stateless block execution failed")]
StatelessExecutionFailed(String),
/// Error during consensus validation of the block.
#[error("consensus validation failed: {0}")]
ConsensusValidationFailed(#[from] ConsensusError),
/// Error during stateless state root calculation.
#[error("stateless state root calculation failed")]
StatelessStateRootCalculationFailed,
/// Error calculating the pre-state root from the witness data.
#[error("stateless pre-state root calculation failed")]
StatelessPreStateRootCalculationFailed,
/// Error when required ancestor headers are missing (e.g., parent header for pre-state root).
#[error("missing required ancestor headers")]
MissingAncestorHeader,
/// Error when deserializing ancestor headers
#[error("could not deserialize ancestor headers")]
HeaderDeserializationFailed,
/// Error when the computed state root does not match the one in the block header.
#[error("mismatched post- state root: {got}\n {expected}")]
PostStateRootMismatch {
/// The computed post-state root
got: B256,
/// The expected post-state root; in the block header
expected: B256,
},
/// Error when the computed pre-state root does not match the expected one.
#[error("mismatched pre-state root: {got} \n {expected}")]
PreStateRootMismatch {
/// The computed pre-state root
got: B256,
/// The expected pre-state root from the previous block
expected: B256,
},
}
/// Performs stateless validation of a block using the provided witness data.
///
/// This function attempts to fully validate a given `current_block` statelessly, ie without access
/// to a persistent database.
/// It relies entirely on the `witness` data and `ancestor_headers`
/// provided alongside the block.
///
/// The witness data is validated in the following way:
///
/// 1. **Ancestor Header Verification:** Checks if the `ancestor_headers` are present, form a
/// contiguous chain back from `current_block`'s parent, and do not exceed the `BLOCKHASH` opcode
/// limit using `compute_ancestor_hashes`. We must have at least one ancestor header, even if the
/// `BLOCKHASH` opcode is not used because we need the state root of the previous block to verify
/// the pre state reads.
///
/// 2. **Pre-State Verification:** Retrieves the expected `pre_state_root` from the parent header
/// from `ancestor_headers`. Verifies the provided [`ExecutionWitness`] against this root using
/// [`verify_execution_witness`].
///
/// 3. **Chain Verification:** The code currently does not verify the [`ChainSpec`] and expects a
/// higher level function to assert that this is correct by, for example, asserting that it is
/// equal to the Ethereum Mainnet `ChainSpec` or asserting against the genesis hash that this
/// `ChainSpec` defines.
///
/// High Level Overview of functionality:
///
/// - Verify all state accesses against a trusted pre-state root
/// - Put all state accesses into an in-memory database
/// - Use the in-memory database to execute the block
/// - Validate the output of block execution (e.g. receipts, logs, requests)
/// - Compute the post-state root using the state-diff from block execution
/// - Check that the post-state root is the state root in the block.
///
/// If all steps succeed the function returns `Some` containing the hash of the validated
/// `current_block`.
pub fn stateless_validation(
current_block: RecoveredBlock<Block<TransactionSigned>>,
witness: ExecutionWitness,
chain_spec: Arc<ChainSpec>,
) -> Result<B256, StatelessValidationError> {
let mut ancestor_headers: Vec<Header> = witness
.headers
.iter()
.map(|serialized_header| {
let bytes = serialized_header.as_ref();
Header::decode(&mut &bytes[..])
.map_err(|_| StatelessValidationError::HeaderDeserializationFailed)
})
.collect::<Result<_, _>>()?;
// Sort the headers by their block number to ensure that they are in
// ascending order.
ancestor_headers.sort_by_key(|header| header.number());
// Validate block against pre-execution consensus rules
validate_block_consensus(chain_spec.clone(), &current_block)?;
// Check that the ancestor headers form a contiguous chain and are not just random headers.
let ancestor_hashes = compute_ancestor_hashes(&current_block, &ancestor_headers)?;
// Get the last ancestor header and retrieve its state root.
//
// There should be at least one ancestor header, this is because we need the parent header to
// retrieve the previous state root.
// The edge case here would be the genesis block, but we do not create proofs for the genesis
// block.
let pre_state_root = match ancestor_headers.last() {
Some(prev_header) => prev_header.state_root,
None => return Err(StatelessValidationError::MissingAncestorHeader),
};
// First verify that the pre-state reads are correct
let (mut sparse_trie, bytecode) = verify_execution_witness(&witness, pre_state_root)?;
// Create an in-memory database that will use the reads to validate the block
let db = WitnessDatabase::new(&sparse_trie, bytecode, ancestor_hashes);
// Execute the block
let basic_block_executor = EthExecutorProvider::ethereum(chain_spec.clone());
let executor = basic_block_executor.batch_executor(db);
let output = executor
.execute(&current_block)
.map_err(|e| StatelessValidationError::StatelessExecutionFailed(e.to_string()))?;
// Post validation checks
validate_block_post_execution(&current_block, &chain_spec, &output.receipts, &output.requests)
.map_err(StatelessValidationError::ConsensusValidationFailed)?;
// Compute and check the post state root
let hashed_state = HashedPostState::from_bundle_state::<KeccakKeyHasher>(&output.state.state);
let state_root = crate::root::calculate_state_root(&mut sparse_trie, hashed_state)
.map_err(|_e| StatelessValidationError::StatelessStateRootCalculationFailed)?;
if state_root != current_block.state_root {
return Err(StatelessValidationError::PostStateRootMismatch {
got: state_root,
expected: current_block.state_root,
});
}
// Return block hash
Ok(current_block.hash_slow())
}
/// Performs consensus validation checks on a block without execution or state validation.
///
/// This function validates a block against Ethereum consensus rules by:
///
/// 1. **Difficulty Validation:** Validates the header with total difficulty to verify proof-of-work
/// (pre-merge) or to enforce post-merge requirements.
///
/// 2. **Header Validation:** Validates the sealed header against protocol specifications,
/// including:
/// - Gas limit checks
/// - Base fee validation for EIP-1559
/// - Withdrawals root validation for Shanghai fork
/// - Blob-related fields validation for Cancun fork
///
/// 3. **Pre-Execution Validation:** Validates block structure, transaction format, signature
/// validity, and other pre-execution requirements.
///
/// This function acts as a preliminary validation before executing and validating the state
/// transition function.
fn validate_block_consensus(
chain_spec: Arc<ChainSpec>,
block: &RecoveredBlock<Block<TransactionSigned>>,
) -> Result<(), StatelessValidationError> {
let consensus = EthBeaconConsensus::new(chain_spec);
consensus.validate_header(block.sealed_header())?;
consensus.validate_block_pre_execution(block)?;
Ok(())
}
/// Verifies execution witness [`ExecutionWitness`] against an expected pre-state root.
///
/// This function takes the RLP-encoded values provided in [`ExecutionWitness`]
/// (which includes state trie nodes, storage trie nodes, and contract bytecode)
/// and uses it to populate a new [`SparseStateTrie`].
///
/// If the computed root hash matches the `pre_state_root`, it signifies that the
/// provided execution witness is consistent with that pre-state root. In this case, the function
/// returns the populated [`SparseStateTrie`] and a [`B256Map`] containing the
/// contract bytecode (mapping code hash to [`Bytecode`]).
///
/// The bytecode has a separate mapping because the [`SparseStateTrie`] does not store the
/// contract bytecode, only the hash of it (code hash).
///
/// If the roots do not match, it returns `None`, indicating the witness is invalid
/// for the given `pre_state_root`.
// Note: This approach might be inefficient for ZKVMs requiring minimal memory operations, which
// would explain why they have for the most part re-implemented this function.
pub fn verify_execution_witness(
witness: &ExecutionWitness,
pre_state_root: B256,
) -> Result<(SparseStateTrie, B256Map<Bytecode>), StatelessValidationError> {
let mut trie = SparseStateTrie::new(DefaultBlindedProviderFactory);
let mut state_witness = B256Map::default();
let mut bytecode = B256Map::default();
for rlp_encoded in &witness.state {
let hash = keccak256(rlp_encoded);
state_witness.insert(hash, rlp_encoded.clone());
}
for rlp_encoded in &witness.codes {
let hash = keccak256(rlp_encoded);
bytecode.insert(hash, Bytecode::new_raw(rlp_encoded.clone()));
}
// Reveal the witness with our state root
// This method builds a trie using the sparse trie using the state_witness with
// the root being the pre_state_root.
// Here are some things to note:
// - You can pass in more witnesses than is needed for the block execution.
// - If you try to get an account and it has not been seen. This means that the account
// was not inserted into the Trie. It does not mean that the account does not exist.
// In order to determine an account not existing, we must do an exclusion proof.
trie.reveal_witness(pre_state_root, &state_witness)
.map_err(|_e| StatelessValidationError::WitnessRevealFailed { pre_state_root })?;
// Calculate the root
let computed_root = trie
.root()
.map_err(|_e| StatelessValidationError::StatelessPreStateRootCalculationFailed)?;
if computed_root == pre_state_root {
Ok((trie, bytecode))
} else {
Err(StatelessValidationError::PreStateRootMismatch {
got: computed_root,
expected: pre_state_root,
})
}
}
/// Verifies the contiguity, number of ancestor headers and extracts their hashes.
///
/// This function is used to prepare the data required for the `BLOCKHASH`
/// opcode in a stateless execution context.
///
/// It verifies that the provided `ancestor_headers` form a valid, unbroken chain leading back from
/// the parent of the `current_block`.
///
/// Note: This function becomes obsolete if EIP-2935 is implemented.
/// Note: The headers are assumed to be in ascending order.
///
/// If both checks pass, it returns a [`BTreeMap`] mapping the block number of each
/// ancestor header to its corresponding block hash.
fn compute_ancestor_hashes(
current_block: &RecoveredBlock<Block<TransactionSigned>>,
ancestor_headers: &[Header],
) -> Result<BTreeMap<u64, B256>, StatelessValidationError> {
let mut ancestor_hashes = BTreeMap::new();
let mut child_header = current_block.header();
// Next verify that headers supplied are contiguous
for parent_header in ancestor_headers.iter().rev() {
let parent_hash = child_header.parent_hash();
ancestor_hashes.insert(parent_header.number, parent_hash);
if parent_hash != parent_header.hash_slow() {
return Err(StatelessValidationError::InvalidAncestorChain); // Blocks must be contiguous
}
if parent_header.number + 1 != child_header.number {
return Err(StatelessValidationError::InvalidAncestorChain); // Header number should be
// contiguous
}
child_header = parent_header
}
Ok(ancestor_hashes)
}

View File

@@ -0,0 +1,148 @@
//! Provides the [`WitnessDatabase`] type, an implementation of [`reth_revm::Database`]
//! specifically designed for stateless execution environments.
use alloc::{collections::btree_map::BTreeMap, format};
use alloy_primitives::{keccak256, map::B256Map, Address, B256, U256};
use alloy_rlp::Decodable;
use alloy_trie::{TrieAccount, EMPTY_ROOT_HASH};
use reth_errors::ProviderError;
use reth_revm::{bytecode::Bytecode, state::AccountInfo, Database};
use reth_trie_sparse::SparseStateTrie;
/// An EVM database implementation backed by witness data.
///
/// This struct implements the [`reth_revm::Database`] trait, allowing the EVM to execute
/// transactions using:
/// - Account and storage slot data provided by a [`reth_trie_sparse::SparseStateTrie`].
/// - Bytecode and ancestor block hashes provided by in-memory maps.
///
/// This is designed for stateless execution scenarios where direct access to a full node's
/// database is not available or desired.
#[derive(Debug)]
pub(crate) struct WitnessDatabase<'a> {
/// Map of block numbers to block hashes.
/// This is used to service the `BLOCKHASH` opcode.
// TODO: use Vec instead -- ancestors should be contiguous
// TODO: so we can use the current_block_number and an offset to
// TODO: get the block number of a particular ancestor
block_hashes_by_block_number: BTreeMap<u64, B256>,
/// Map of code hashes to bytecode.
/// Used to fetch contract code needed during execution.
bytecode: B256Map<Bytecode>,
/// The sparse Merkle Patricia Trie containing account and storage state.
/// This is used to provide account/storage values during EVM execution.
/// TODO: Ideally we do not have this trie and instead a simple map.
/// TODO: Then as a corollary we can avoid unnecessary hashing in `Database::storage`
/// TODO: and `Database::basic` without needing to cache the hashed Addresses and Keys
trie: &'a SparseStateTrie,
}
impl<'a> WitnessDatabase<'a> {
/// Creates a new [`WitnessDatabase`] instance.
///
/// # Assumptions
///
/// This function assumes:
/// 1. The provided `trie` has been populated with state data consistent with a known state root
/// (e.g., using witness data and verifying against a parent block's state root).
/// 2. The `bytecode` map contains all bytecode corresponding to code hashes present in the
/// account data within the `trie`.
/// 3. The `ancestor_hashes` map contains the block hashes for the relevant ancestor blocks (up
/// to 256 including the current block number). It assumes these hashes correspond to a
/// contiguous chain of blocks. The caller is responsible for verifying the contiguity and
/// the block limit.
pub(crate) const fn new(
trie: &'a SparseStateTrie,
bytecode: B256Map<Bytecode>,
ancestor_hashes: BTreeMap<u64, B256>,
) -> Self {
Self { trie, block_hashes_by_block_number: ancestor_hashes, bytecode }
}
}
impl Database for WitnessDatabase<'_> {
/// The database error type.
type Error = ProviderError;
/// Get basic account information by hashing the address and looking up the account RLP
/// in the underlying [`SparseStateTrie`].
///
/// Returns `Ok(None)` if the account is not found in the trie.
fn basic(&mut self, address: Address) -> Result<Option<AccountInfo>, Self::Error> {
let hashed_address = keccak256(address);
if let Some(bytes) = self.trie.get_account_value(&hashed_address) {
let account = TrieAccount::decode(&mut bytes.as_slice())?;
return Ok(Some(AccountInfo {
balance: account.balance,
nonce: account.nonce,
code_hash: account.code_hash,
code: None,
}));
}
if !self.trie.check_valid_account_witness(hashed_address) {
return Err(ProviderError::TrieWitnessError(format!(
"incomplete account witness for {hashed_address:?}"
)));
}
Ok(None)
}
/// Get storage value of an account at a specific slot.
///
/// Returns `U256::ZERO` if the slot is not found in the trie.
fn storage(&mut self, address: Address, slot: U256) -> Result<U256, Self::Error> {
let hashed_address = keccak256(address);
let hashed_slot = keccak256(B256::from(slot));
if let Some(raw) = self.trie.get_storage_slot_value(&hashed_address, &hashed_slot) {
return Ok(U256::decode(&mut raw.as_slice())?)
}
// Storage slot value is not present in the trie, validate that the witness is complete.
// If the account exists in the trie...
if let Some(bytes) = self.trie.get_account_value(&hashed_address) {
// ...check that its storage is either empty or the storage trie was sufficiently
// revealed...
let account = TrieAccount::decode(&mut bytes.as_slice())?;
if account.storage_root != EMPTY_ROOT_HASH &&
!self.trie.check_valid_storage_witness(hashed_address, hashed_slot)
{
return Err(ProviderError::TrieWitnessError(format!(
"incomplete storage witness: prover must supply exclusion proof for slot {hashed_slot:?} in account {hashed_address:?}"
)));
}
} else if !self.trie.check_valid_account_witness(hashed_address) {
// ...else if account is missing, validate that the account trie was sufficiently
// revealed.
return Err(ProviderError::TrieWitnessError(format!(
"incomplete account witness for {hashed_address:?}"
)));
}
Ok(U256::ZERO)
}
/// Get account code by its hash from the provided bytecode map.
///
/// Returns an error if the bytecode for the given hash is not found in the map.
fn code_by_hash(&mut self, code_hash: B256) -> Result<Bytecode, Self::Error> {
let bytecode = self.bytecode.get(&code_hash).ok_or_else(|| {
ProviderError::TrieWitnessError(format!("bytecode for {code_hash} not found"))
})?;
Ok(bytecode.clone())
}
/// Get block hash by block number from the provided ancestor hashes map.
///
/// Returns an error if the hash for the given block number is not found in the map.
fn block_hash(&mut self, block_number: u64) -> Result<B256, Self::Error> {
self.block_hashes_by_block_number
.get(&block_number)
.copied()
.ok_or(ProviderError::StateForNumberNotFound(block_number))
}
}

View File

@@ -1,6 +1,6 @@
use crate::{
blinded::{BlindedProvider, BlindedProviderFactory, DefaultBlindedProviderFactory},
RevealedSparseTrie, SparseTrie, TrieMasks,
LeafLookup, RevealedSparseTrie, SparseTrie, TrieMasks,
};
use alloc::{collections::VecDeque, vec::Vec};
use alloy_primitives::{
@@ -106,6 +106,34 @@ impl<F: BlindedProviderFactory> SparseStateTrie<F> {
self.revealed_account_paths.contains(&Nibbles::unpack(account))
}
/// Was the account witness for `address` complete?
pub fn check_valid_account_witness(&self, address: B256) -> bool {
let path = Nibbles::unpack(address);
let trie = match self.state_trie_ref() {
Some(t) => t,
None => return false,
};
matches!(
trie.find_leaf(&path, None),
Ok(LeafLookup::Exists | LeafLookup::NonExistent { .. })
)
}
/// Was the storage-slot witness for (`address`,`slot`) complete?
pub fn check_valid_storage_witness(&self, address: B256, slot: B256) -> bool {
let path = Nibbles::unpack(slot);
let trie = match self.storage_trie_ref(&address) {
Some(t) => t,
None => return false,
};
matches!(
trie.find_leaf(&path, None),
Ok(LeafLookup::Exists | LeafLookup::NonExistent { .. })
)
}
/// Returns `true` if storage slot for account was already revealed.
pub fn is_storage_slot_revealed(&self, account: B256, slot: B256) -> bool {
self.revealed_storage_paths

View File

@@ -13,10 +13,7 @@ workspace = true
[features]
ef-tests = []
asm-keccak = [
"alloy-primitives/asm-keccak",
"revm/asm-keccak",
]
asm-keccak = ["alloy-primitives/asm-keccak", "revm/asm-keccak"]
[dependencies]
reth-chainspec.workspace = true
@@ -30,7 +27,9 @@ reth-provider = { workspace = true, features = ["test-utils"] }
reth-evm.workspace = true
reth-evm-ethereum.workspace = true
reth-ethereum-consensus.workspace = true
reth-revm = { workspace = true, features = ["std"] }
reth-revm = { workspace = true, features = ["std", "witness"] }
reth-stateless = { workspace = true }
reth-tracing.workspace = true
reth-trie.workspace = true
reth-trie-db.workspace = true
revm = { workspace = true, features = ["secp256k1", "blst", "c-kzg"] }
@@ -46,6 +45,7 @@ serde.workspace = true
serde_json.workspace = true
thiserror.workspace = true
rayon.workspace = true
tracing.workspace = true
# TODO: When we build for a windows target on an ubuntu runner, crunchy tries to
# get the wrong path, update this when the workflow has been updated
#

View File

@@ -4,7 +4,7 @@ use crate::{
models::{BlockchainTest, ForkSpec},
Case, Error, Suite,
};
use alloy_rlp::Decodable;
use alloy_rlp::{Decodable, Encodable};
use rayon::iter::{ParallelBridge, ParallelIterator};
use reth_chainspec::ChainSpec;
use reth_consensus::{Consensus, HeaderValidator};
@@ -16,9 +16,11 @@ use reth_evm_ethereum::execute::EthExecutorProvider;
use reth_primitives_traits::{RecoveredBlock, SealedBlock};
use reth_provider::{
test_utils::create_test_provider_factory_with_chain_spec, BlockWriter, DatabaseProviderFactory,
ExecutionOutcome, HistoryWriter, OriginalValuesKnown, StateWriter, StorageLocation,
ExecutionOutcome, HeaderProvider, HistoryWriter, OriginalValuesKnown, StateProofProvider,
StateWriter, StorageLocation,
};
use reth_revm::database::StateProviderDatabase;
use reth_revm::{database::StateProviderDatabase, witness::ExecutionWitnessRecord, State};
use reth_stateless::{validation::stateless_validation, ExecutionWitness};
use reth_trie::{HashedPostState, KeccakKeyHasher, StateRoot};
use reth_trie_db::DatabaseStateRoot;
use std::{collections::BTreeMap, fs, path::Path, sync::Arc};
@@ -212,6 +214,7 @@ fn run_case(case: &BlockchainTest) -> Result<(), Error> {
let executor_provider = EthExecutorProvider::ethereum(chain_spec.clone());
let mut parent = genesis_block;
let mut program_inputs = Vec::new();
for (block_index, block) in blocks.iter().enumerate() {
// Note: same as the comment on `decode_blocks` as to why we cannot use block.number
@@ -226,16 +229,50 @@ fn run_case(case: &BlockchainTest) -> Result<(), Error> {
pre_execution_checks(chain_spec.clone(), &parent, block)
.map_err(|_| Error::BlockProcessingFailed { block_number })?;
let mut witness_record = ExecutionWitnessRecord::default();
// Execute the block
let state_db = StateProviderDatabase(provider.latest());
let state_provider = provider.latest();
let state_db = StateProviderDatabase(&state_provider);
let executor = executor_provider.batch_executor(state_db);
let output =
executor.execute(block).map_err(|_| Error::BlockProcessingFailed { block_number })?;
let output = executor
.execute_with_state_closure(&(*block).clone(), |statedb: &State<_>| {
witness_record.record_executed_state(statedb);
})
.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 })?;
// Generate the stateless witness
// TODO: Most of this code is copy-pasted from debug_executionWitness
let ExecutionWitnessRecord { hashed_state, codes, keys, lowest_block_number } =
witness_record;
let state = state_provider.witness(Default::default(), hashed_state)?;
let mut exec_witness = ExecutionWitness { state, codes, keys, headers: Default::default() };
let smallest = lowest_block_number.unwrap_or_else(|| {
// Return only the parent header, if there were no calls to the
// BLOCKHASH opcode.
block_number.saturating_sub(1)
});
let range = smallest..block_number;
exec_witness.headers = provider
.headers_range(range)?
.into_iter()
.map(|header| {
let mut serialized_header = Vec::new();
header.encode(&mut serialized_header);
serialized_header.into()
})
.collect();
program_inputs.push((block.clone(), exec_witness));
// Compute and check the post state root
let hashed_state =
HashedPostState::from_bundle_state::<KeccakKeyHasher>(output.state.state());
@@ -280,6 +317,12 @@ fn run_case(case: &BlockchainTest) -> Result<(), Error> {
account.assert_db(address, provider.tx_ref())?;
}
// Now validate using the stateless client if everything else passes
for (block, execution_witness) in program_inputs {
stateless_validation(block, execution_witness, chain_spec.clone())
.expect("stateless validation failed");
}
Ok(())
}

View File

@@ -7,6 +7,7 @@ macro_rules! general_state_test {
($test_name:ident, $dir:ident) => {
#[test]
fn $test_name() {
reth_tracing::init_test_tracing();
BlockchainTests::new(format!("GeneralStateTests/{}", stringify!($dir))).run();
}
};
@@ -81,6 +82,7 @@ macro_rules! blockchain_test {
($test_name:ident, $dir:ident) => {
#[test]
fn $test_name() {
reth_tracing::init_test_tracing();
BlockchainTests::new(format!("{}", stringify!($dir))).run();
}
};