diff --git a/Cargo.lock b/Cargo.lock index 282ee81105..87ac06b240 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -9769,7 +9769,9 @@ dependencies = [ "reth-node-types", "reth-primitives-traits", "reth-prune-types", + "reth-revm", "reth-stages-types", + "reth-stateless", "reth-static-file-types", "reth-storage-api", "reth-storage-errors", @@ -9781,6 +9783,7 @@ dependencies = [ "revm-state", "strum 0.27.2", "tempfile", + "thiserror 2.0.16", "tokio", "tracing", ] @@ -9894,6 +9897,7 @@ dependencies = [ "alloy-consensus", "alloy-primitives", "reth-ethereum-forks", + "reth-execution-types", "reth-primitives-traits", "reth-storage-api", "reth-storage-errors", @@ -9965,6 +9969,7 @@ dependencies = [ "reth-rpc-eth-api", "reth-rpc-eth-types", "reth-rpc-server-types", + "reth-stateless", "reth-storage-api", "reth-tasks", "reth-testing-utils", @@ -10008,6 +10013,7 @@ dependencies = [ "reth-engine-primitives", "reth-network-peers", "reth-rpc-eth-api", + "reth-stateless", "reth-trie-common", ] @@ -10410,6 +10416,7 @@ dependencies = [ "reth-ethereum-consensus", "reth-ethereum-primitives", "reth-evm", + "reth-execution-types", "reth-primitives-traits", "reth-revm", "reth-trie-common", diff --git a/crates/chain-state/src/in_memory.rs b/crates/chain-state/src/in_memory.rs index a6c8553810..46dde75706 100644 --- a/crates/chain-state/src/in_memory.rs +++ b/crates/chain-state/src/in_memory.rs @@ -318,7 +318,7 @@ impl CanonicalInMemoryState { { if self.inner.in_memory_state.blocks.read().get(&persisted_num_hash.hash).is_none() { // do nothing - return + return; } } @@ -542,7 +542,7 @@ impl CanonicalInMemoryState { if let Some(tx) = block_state.block_ref().recovered_block().body().transaction_by_hash(&hash) { - return Some(tx.clone()) + return Some(tx.clone()); } } None @@ -869,6 +869,7 @@ mod tests { use rand::Rng; use reth_errors::ProviderResult; use reth_ethereum_primitives::{EthPrimitives, Receipt}; + use reth_execution_types::{FlatPreState, FlatWitnessRecord}; use reth_primitives_traits::{Account, Bytecode}; use reth_storage_api::{ AccountReader, BlockHashReader, BytecodeReader, HashedPostStateProvider, @@ -1031,6 +1032,10 @@ mod tests { ) -> ProviderResult> { Ok(Vec::default()) } + + fn flat_witness(&self, _record: FlatWitnessRecord) -> ProviderResult { + Ok(FlatPreState::default()) + } } #[test] diff --git a/crates/chain-state/src/memory_overlay.rs b/crates/chain-state/src/memory_overlay.rs index 254edb248b..a622c198d3 100644 --- a/crates/chain-state/src/memory_overlay.rs +++ b/crates/chain-state/src/memory_overlay.rs @@ -2,6 +2,7 @@ use super::ExecutedBlock; use alloy_consensus::BlockHeader; use alloy_primitives::{keccak256, Address, BlockNumber, Bytes, StorageKey, StorageValue, B256}; use reth_errors::ProviderResult; +use reth_execution_types::{FlatPreState, FlatWitnessRecord}; use reth_primitives_traits::{Account, Bytecode, NodePrimitives}; use reth_storage_api::{ AccountReader, BlockHashReader, BytecodeReader, HashedPostStateProvider, StateProofProvider, @@ -203,6 +204,10 @@ impl StateProofProvider for MemoryOverlayStateProviderRef<'_, input.prepend_self(self.trie_input().clone()); self.historical.witness(input, target) } + + fn flat_witness(&self, record: FlatWitnessRecord) -> ProviderResult { + self.historical.flat_witness(record) + } } impl HashedPostStateProvider for MemoryOverlayStateProviderRef<'_, N> { diff --git a/crates/engine/tree/src/tree/cached_state.rs b/crates/engine/tree/src/tree/cached_state.rs index 3e9cda38f1..90b81ed4a3 100644 --- a/crates/engine/tree/src/tree/cached_state.rs +++ b/crates/engine/tree/src/tree/cached_state.rs @@ -9,8 +9,9 @@ use reth_errors::ProviderResult; use reth_metrics::Metrics; use reth_primitives_traits::{Account, Bytecode}; use reth_provider::{ - AccountReader, BlockHashReader, BytecodeReader, HashedPostStateProvider, StateProofProvider, - StateProvider, StateRootProvider, StorageRootProvider, + AccountReader, BlockHashReader, BytecodeReader, FlatPreState, FlatWitnessRecord, + HashedPostStateProvider, StateProofProvider, StateProvider, StateRootProvider, + StorageRootProvider, }; use reth_revm::db::BundleState; use reth_trie::{ @@ -119,7 +120,7 @@ impl AccountReader for CachedStateProvider { fn basic_account(&self, address: &Address) -> ProviderResult> { if let Some(res) = self.caches.account_cache.get(address) { self.metrics.account_cache_hits.increment(1); - return Ok(res) + return Ok(res); } self.metrics.account_cache_misses.increment(1); @@ -170,7 +171,7 @@ impl BytecodeReader for CachedStateProvider { fn bytecode_by_hash(&self, code_hash: &B256) -> ProviderResult> { if let Some(res) = self.caches.code_cache.get(code_hash) { self.metrics.code_cache_hits.increment(1); - return Ok(res) + return Ok(res); } self.metrics.code_cache_misses.increment(1); @@ -230,6 +231,10 @@ impl StateProofProvider for CachedStateProvider { ) -> ProviderResult> { self.state_provider.witness(input, target) } + + fn flat_witness(&self, record: FlatWitnessRecord) -> ProviderResult { + self.state_provider.flat_witness(record) + } } impl StorageRootProvider for CachedStateProvider { @@ -410,7 +415,7 @@ impl ExecutionCache { // If the account was not modified, as in not changed and not destroyed, then we have // nothing to do w.r.t. this particular account and can move on if account.status.is_not_modified() { - continue + continue; } // If the account was destroyed, invalidate from the account / storage caches @@ -419,7 +424,7 @@ impl ExecutionCache { self.account_cache.invalidate(addr); invalidated_accounts.insert(addr); - continue + continue; } // If we have an account that was modified, but it has a `None` account info, some wild @@ -427,7 +432,7 @@ impl ExecutionCache { // `None` current info, should be destroyed. let Some(ref account_info) = account.info else { trace!(target: "engine::caching", ?account, "Account with None account info found in state updates"); - return Err(()) + return Err(()); }; // Now we iterate over all storage and make updates to the cached storage values diff --git a/crates/engine/tree/src/tree/instrumented_state.rs b/crates/engine/tree/src/tree/instrumented_state.rs index 9d96aca3a2..e0a9df484a 100644 --- a/crates/engine/tree/src/tree/instrumented_state.rs +++ b/crates/engine/tree/src/tree/instrumented_state.rs @@ -5,8 +5,9 @@ use reth_errors::ProviderResult; use reth_metrics::Metrics; use reth_primitives_traits::{Account, Bytecode}; use reth_provider::{ - AccountReader, BlockHashReader, BytecodeReader, HashedPostStateProvider, StateProofProvider, - StateProvider, StateRootProvider, StorageRootProvider, + AccountReader, BlockHashReader, BytecodeReader, FlatPreState, FlatWitnessRecord, + HashedPostStateProvider, StateProofProvider, StateProvider, StateRootProvider, + StorageRootProvider, }; use reth_trie::{ updates::TrieUpdates, AccountProof, HashedPostState, HashedStorage, MultiProof, @@ -251,6 +252,10 @@ impl StateProofProvider for InstrumentedStateProvider ) -> ProviderResult> { self.state_provider.witness(input, target) } + + fn flat_witness(&self, record: FlatWitnessRecord) -> ProviderResult { + self.state_provider.flat_witness(record) + } } impl StorageRootProvider for InstrumentedStateProvider { diff --git a/crates/evm/execution-types/src/lib.rs b/crates/evm/execution-types/src/lib.rs index 8b795981fb..926a047617 100644 --- a/crates/evm/execution-types/src/lib.rs +++ b/crates/evm/execution-types/src/lib.rs @@ -20,6 +20,9 @@ pub use execute::*; mod execution_outcome; pub use execution_outcome::*; +pub mod witness; +pub use witness::*; + /// Bincode-compatible serde implementations for commonly used types for (EVM) block execution. /// /// `bincode` crate doesn't work with optionally serializable serde fields, but some of the diff --git a/crates/evm/execution-types/src/witness.rs b/crates/evm/execution-types/src/witness.rs new file mode 100644 index 0000000000..5b6594f6b4 --- /dev/null +++ b/crates/evm/execution-types/src/witness.rs @@ -0,0 +1,63 @@ +//! Witness recording types for EVM execution. + +use alloy_primitives::{keccak256, Address, B256, U256}; +use revm::{ + database::{AccountStatus, DbAccount, State}, + primitives::{HashMap, HashSet}, + state::Bytecode, +}; + +/// Records pre-state data for witness generation. +#[derive(Debug, Clone, Default)] +pub struct FlatPreState { + /// Accounts accessed during execution. + pub accounts: HashMap, + /// Bytecode accessed during execution. + pub contracts: HashMap, + /// The set of addresses that have been self-destructed in the execution. + pub destructed_addresses: HashSet
, +} + +/// Records pre-state accesses that occurred during execution. +#[derive(Debug, Clone, Default)] +pub struct FlatWitnessRecord { + /// Accounts accessed during execution. + pub accounts: HashMap, + /// Bytecode accessed during execution. + pub contracts: HashMap>, +} + +/// Represents an accessed account during execution. +#[derive(Debug, Clone)] +pub enum AccessedAccount { + /// Indicates if the account was destroyed during execution. + Destroyed, + /// Storage keys accessed during execution. + StorageKeys(HashSet), +} + +impl FlatWitnessRecord { + /// Records the accessed state after execution. + pub fn record_executed_state(&mut self, statedb: &State) { + self.contracts = statedb + .cache + .contracts + .values() + .map(|code| (keccak256(code.original_bytes()), Some(code.clone()))) + .collect(); + + for (address, account) in &statedb.cache.accounts { + let account = match account.status { + AccountStatus::Destroyed => AccessedAccount::Destroyed, + _ => { + let storage_keys = account + .account + .as_ref() + .map_or_else(HashSet::default, |a| a.storage.keys().copied().collect()); + AccessedAccount::StorageKeys(storage_keys) + } + }; + self.accounts.insert(*address, account); + } + } +} diff --git a/crates/revm/Cargo.toml b/crates/revm/Cargo.toml index 488a685b38..1720d14502 100644 --- a/crates/revm/Cargo.toml +++ b/crates/revm/Cargo.toml @@ -17,6 +17,7 @@ reth-primitives-traits.workspace = true reth-storage-errors.workspace = true reth-storage-api.workspace = true reth-trie = { workspace = true, optional = true } +reth-execution-types.workspace = true # alloy alloy-primitives.workspace = true diff --git a/crates/revm/src/test_utils.rs b/crates/revm/src/test_utils.rs index e0d4007087..2539f97864 100644 --- a/crates/revm/src/test_utils.rs +++ b/crates/revm/src/test_utils.rs @@ -2,6 +2,7 @@ use alloc::vec::Vec; use alloy_primitives::{ keccak256, map::HashMap, Address, BlockNumber, Bytes, StorageKey, B256, U256, }; +use reth_execution_types::{FlatPreState, FlatWitnessRecord}; use reth_primitives_traits::{Account, Bytecode}; use reth_storage_api::{ AccountReader, BlockHashReader, BytecodeReader, HashedPostStateProvider, StateProofProvider, @@ -142,6 +143,10 @@ impl StateProofProvider for StateProviderTest { fn witness(&self, _input: TrieInput, _target: HashedPostState) -> ProviderResult> { unimplemented!("witness generation is not supported") } + + fn flat_witness(&self, record: FlatWitnessRecord) -> ProviderResult { + unimplemented!("flat witness generation is not supported") + } } impl HashedPostStateProvider for StateProviderTest { diff --git a/crates/rpc/rpc-api/Cargo.toml b/crates/rpc/rpc-api/Cargo.toml index 7d170d342f..919ae3e3a0 100644 --- a/crates/rpc/rpc-api/Cargo.toml +++ b/crates/rpc/rpc-api/Cargo.toml @@ -18,6 +18,7 @@ reth-engine-primitives.workspace = true reth-network-peers.workspace = true reth-trie-common.workspace = true reth-chain-state.workspace = true +reth-stateless.workspace = true # ethereum alloy-eips.workspace = true diff --git a/crates/rpc/rpc-api/src/debug.rs b/crates/rpc/rpc-api/src/debug.rs index 5dd7401782..d25727b4de 100644 --- a/crates/rpc/rpc-api/src/debug.rs +++ b/crates/rpc/rpc-api/src/debug.rs @@ -8,6 +8,7 @@ use alloy_rpc_types_trace::geth::{ BlockTraceResult, GethDebugTracingCallOptions, GethDebugTracingOptions, GethTrace, TraceResult, }; use jsonrpsee::{core::RpcResult, proc_macros::rpc}; +use reth_stateless::flat_witness::FlatExecutionWitness; use reth_trie_common::{updates::TrieUpdates, HashedPostState}; /// Debug rpc interface. @@ -144,6 +145,16 @@ pub trait DebugApi { async fn debug_execution_witness(&self, block: BlockNumberOrTag) -> RpcResult; + /// The `debug_flatExecutionWitness` method allows for re-execution of a block with the purpose of + /// generating an execution-only witness. + /// + /// The first argument is the block number or tag. + #[method(name = "flatExecutionWitness")] + async fn debug_flat_execution_witness( + &self, + block: BlockNumberOrTag, + ) -> RpcResult; + /// The `debug_executionWitnessByBlockHash` method allows for re-execution of a block with the /// purpose of generating an execution witness. The witness comprises of a map of all hashed /// trie nodes to their preimages that were required during the execution of the block, @@ -156,6 +167,16 @@ pub trait DebugApi { hash: B256, ) -> RpcResult; + /// The `debug_executionWitnessByBlockHash` method allows for re-execution of a block with the + /// purpose of generating an execution-only witness. + /// + /// The first argument is the block hash. + #[method(name = "flatExecutionWitnessByBlockHash")] + async fn debug_flat_execution_witness_by_block_hash( + &self, + hash: B256, + ) -> RpcResult; + /// Sets the logging backtrace location. When a backtrace location is set and a log message is /// emitted at that location, the stack of the goroutine executing the log statement will /// be printed to stderr. diff --git a/crates/rpc/rpc-eth-types/src/cache/db.rs b/crates/rpc/rpc-eth-types/src/cache/db.rs index abb8983485..4078d1e632 100644 --- a/crates/rpc/rpc-eth-types/src/cache/db.rs +++ b/crates/rpc/rpc-eth-types/src/cache/db.rs @@ -4,6 +4,7 @@ use alloy_primitives::{Address, B256, U256}; use reth_errors::ProviderResult; +use reth_execution_types::{FlatPreState, FlatWitnessRecord}; use reth_revm::{database::StateProviderDatabase, DatabaseRef}; use reth_storage_api::{BytecodeReader, HashedPostStateProvider, StateProvider}; use reth_trie::{HashedStorage, MultiProofTargets}; @@ -105,6 +106,10 @@ impl reth_storage_api::StateProofProvider for StateProviderTraitObjWrapper<'_> { ) -> reth_errors::ProviderResult> { self.0.witness(input, target) } + + fn flat_witness(&self, record: FlatWitnessRecord) -> ProviderResult { + self.0.flat_witness(record) + } } impl reth_storage_api::AccountReader for StateProviderTraitObjWrapper<'_> { diff --git a/crates/rpc/rpc/Cargo.toml b/crates/rpc/rpc/Cargo.toml index e028e47448..b91fc730c0 100644 --- a/crates/rpc/rpc/Cargo.toml +++ b/crates/rpc/rpc/Cargo.toml @@ -40,6 +40,7 @@ reth-consensus.workspace = true reth-consensus-common.workspace = true reth-node-api.workspace = true reth-trie-common.workspace = true +reth-stateless.workspace = true # ethereum alloy-evm = { workspace = true, features = ["overrides"] } diff --git a/crates/rpc/rpc/src/debug.rs b/crates/rpc/rpc/src/debug.rs index 62aa625b9f..5f68196394 100644 --- a/crates/rpc/rpc/src/debug.rs +++ b/crates/rpc/rpc/src/debug.rs @@ -21,6 +21,7 @@ use jsonrpsee::core::RpcResult; use reth_chainspec::{ChainSpecProvider, EthChainSpec, EthereumHardforks}; use reth_errors::RethError; use reth_evm::{execute::Executor, ConfigureEvm, EvmEnvFor, TxEnvFor}; +use reth_execution_types::FlatWitnessRecord; use reth_primitives_traits::{Block as _, BlockBody, ReceiptWithBloom, RecoveredBlock}; use reth_revm::{ database::StateProviderDatabase, @@ -35,6 +36,7 @@ use reth_rpc_eth_api::{ }; use reth_rpc_eth_types::{EthApiError, StateCacheDb}; use reth_rpc_server_types::{result::internal_rpc_err, ToRpcResult}; +use reth_stateless::flat_witness::FlatExecutionWitness; use reth_storage_api::{ BlockIdReader, BlockReaderIdExt, HeaderProvider, ProviderBlock, ReceiptProviderIdExt, StateProofProvider, StateProviderFactory, StateRootProvider, TransactionVariant, @@ -45,6 +47,7 @@ use revm::{context::Block, context_interface::Transaction, state::EvmState, Data use revm_inspectors::tracing::{ FourByteInspector, MuxInspector, TracingInspector, TracingInspectorConfig, TransactionContext, }; +use revm_primitives::{keccak256, U256}; use std::sync::Arc; use tokio::sync::{AcquireError, OwnedSemaphorePermit}; @@ -465,7 +468,7 @@ where // additional tracers Err(EthApiError::Unsupported("unsupported tracer").into()) } - } + }; } // default structlog tracer @@ -501,7 +504,7 @@ where opts: Option, ) -> Result>, Eth::Error> { if bundles.is_empty() { - return Err(EthApiError::InvalidParams(String::from("bundles are empty.")).into()) + return Err(EthApiError::InvalidParams(String::from("bundles are empty.")).into()); } let StateContext { transaction_index, block_number } = state_context.unwrap_or_default(); @@ -621,6 +624,38 @@ where self.debug_execution_witness_for_block(block).await } + /// The `debug_flatExecutionWitnessByBlockHash` method allows for re-execution of a block with the purpose of + /// generating an execution-only witness. + pub async fn debug_flat_execution_witness_by_block_hash( + &self, + hash: B256, + ) -> Result { + let this = self.clone(); + let block = this + .eth_api() + .recovered_block(hash.into()) + .await? + .ok_or(EthApiError::HeaderNotFound(hash.into()))?; + + self.debug_flat_execution_witness_for_block(block).await + } + + /// The `debug_flatExecutionWitness` method allows for re-execution of a block with the purpose of + /// generating an execution-only witness. + pub async fn debug_flat_execution_witness( + &self, + block_id: BlockNumberOrTag, + ) -> Result { + let this = self.clone(); + let block = this + .eth_api() + .recovered_block(block_id.into()) + .await? + .ok_or(EthApiError::HeaderNotFound(block_id.into()))?; + + self.debug_flat_execution_witness_for_block(block).await + } + /// The `debug_executionWitness` method allows for re-execution of a block with the purpose of /// generating an execution witness. The witness comprises of a map of all hashed trie nodes to /// their preimages that were required during the execution of the block, including during state @@ -699,6 +734,72 @@ where Ok(exec_witness) } + /// Generates a flat execution witness, using the given recovered block. + pub async fn debug_flat_execution_witness_for_block( + &self, + block: Arc>>, + ) -> Result { + let this = self.clone(); + let block_number = block.header().number(); + + let (pre_state, lowest_block_number) = self + .eth_api() + .spawn_with_state_at_block(block.parent_hash().into(), move |state_provider| { + let db = StateProviderDatabase::new(&state_provider); + let block_executor = this.eth_api().evm_config().executor(db); + + let mut witness_record = FlatWitnessRecord::default(); + let mut lowest_block_number = Default::default(); + + let _ = block_executor + .execute_with_state_closure(&block, |statedb: &State<_>| { + witness_record.record_executed_state(statedb); + lowest_block_number = statedb.block_hashes.keys().next().copied() + }) + .map_err(|err| EthApiError::Internal(err.into()))?; + + let pre_state = + state_provider.flat_witness(witness_record).map_err(EthApiError::from)?; + + Ok((pre_state, lowest_block_number)) + }) + .await?; + + let smallest = match lowest_block_number { + Some(smallest) => smallest, + None => { + // Return only the parent header, if there were no calls to the + // BLOCKHASH opcode. + block_number.saturating_sub(1) + } + }; + + let range = smallest..block_number; + + let headers: Vec = self + .provider() + .headers_range(range.clone()) + .map_err(EthApiError::from)? + .into_iter() + .map(|header| { + let mut serialized_header = Vec::new(); + header.encode(&mut serialized_header); + serialized_header.into() + }) + .collect(); + + let block_hashes = headers + .iter() + .zip(range) + .map(|(bytes, num)| (U256::from(num), keccak256(bytes))) + .collect(); + + let parent = headers.last().unwrap().clone(); + let exec_witness = FlatExecutionWitness::new(pre_state, block_hashes, parent); + + Ok(exec_witness) + } + /// Returns the code associated with a given hash at the specified block ID. If no code is /// found, it returns None. If no block ID is provided, it defaults to the latest block. pub async fn debug_code_by_hash( @@ -758,7 +859,7 @@ where GethDebugBuiltInTracerType::FourByteTracer => { let mut inspector = FourByteInspector::default(); let res = self.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?; - return Ok((FourByteFrame::from(&inspector).into(), res.state)) + return Ok((FourByteFrame::from(&inspector).into(), res.state)); } GethDebugBuiltInTracerType::CallTracer => { let call_config = tracer_config @@ -781,7 +882,7 @@ where .geth_builder() .geth_call_traces(call_config, res.result.gas_used()); - return Ok((frame.into(), res.state)) + return Ok((frame.into(), res.state)); } GethDebugBuiltInTracerType::PreStateTracer => { let prestate_config = tracer_config @@ -804,7 +905,7 @@ where .geth_prestate_traces(&res, &prestate_config, db) .map_err(Eth::Error::from_eth_err)?; - return Ok((frame.into(), res.state)) + return Ok((frame.into(), res.state)); } GethDebugBuiltInTracerType::NoopTracer => { Ok((NoopFrame::default().into(), Default::default())) @@ -823,7 +924,7 @@ where let frame = inspector .try_into_mux_frame(&res, db, tx_info) .map_err(Eth::Error::from_eth_err)?; - return Ok((frame.into(), res.state)) + return Ok((frame.into(), res.state)); } GethDebugBuiltInTracerType::FlatCallTracer => { let flat_call_config = tracer_config @@ -882,7 +983,7 @@ where // additional tracers Err(EthApiError::Unsupported("unsupported tracer").into()) } - } + }; } // default structlog tracer @@ -1087,6 +1188,15 @@ where Self::debug_execution_witness(self, block).await.map_err(Into::into) } + /// Handler for `debug_flatExecutionWitness` + async fn debug_flat_execution_witness( + &self, + block: BlockNumberOrTag, + ) -> RpcResult { + let _permit = self.acquire_trace_permit().await; + Self::debug_flat_execution_witness(self, block).await.map_err(Into::into) + } + /// Handler for `debug_executionWitnessByBlockHash` async fn debug_execution_witness_by_block_hash( &self, @@ -1096,6 +1206,15 @@ where Self::debug_execution_witness_by_block_hash(self, hash).await.map_err(Into::into) } + /// Handler for `debug_flatExecutionWitnessByBlockHash` + async fn debug_flat_execution_witness_by_block_hash( + &self, + hash: B256, + ) -> RpcResult { + let _permit = self.acquire_trace_permit().await; + Self::debug_flat_execution_witness_by_block_hash(self, hash).await.map_err(Into::into) + } + async fn debug_backtrace_at(&self, _location: &str) -> RpcResult<()> { Ok(()) } diff --git a/crates/stateless/Cargo.toml b/crates/stateless/Cargo.toml index 13508b1285..1bdd39c7d4 100644 --- a/crates/stateless/Cargo.toml +++ b/crates/stateless/Cargo.toml @@ -29,11 +29,12 @@ reth-ethereum-primitives = { workspace = true, features = [ ] } reth-errors.workspace = true reth-evm.workspace = true -reth-revm.workspace = true +reth-revm = { workspace = true, features = ["serde"] } reth-trie-common.workspace = true reth-trie-sparse.workspace = true reth-chainspec.workspace = true reth-consensus.workspace = true +reth-execution-types.workspace = true # misc thiserror.workspace = true diff --git a/crates/stateless/src/flat_witness/bincode.rs b/crates/stateless/src/flat_witness/bincode.rs new file mode 100644 index 0000000000..8f777b811e --- /dev/null +++ b/crates/stateless/src/flat_witness/bincode.rs @@ -0,0 +1,50 @@ +#![allow(missing_docs)] + +use alloy_primitives::{map::HashMap, Address, Bytes, B256, U256}; +use reth_revm::{ + db::{Cache, DbAccount}, + state::Bytecode, +}; +use serde_with::{DeserializeAs, SerializeAs}; + +#[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] +pub struct CacheBincode { + pub accounts: HashMap, + pub contracts: HashMap, + pub block_hashes: HashMap, +} + +impl SerializeAs for CacheBincode { + fn serialize_as(source: &Cache, serializer: S) -> Result + where + S: serde::Serializer, + { + use serde::Serialize; + let cache_bincode = Self { + accounts: source.accounts.clone(), + contracts: source.contracts.iter().map(|(k, v)| (*k, v.original_bytes())).collect(), + block_hashes: source.block_hashes.clone(), + }; + cache_bincode.serialize(serializer) + } +} + +impl<'de> DeserializeAs<'de, Cache> for CacheBincode { + fn deserialize_as(deserializer: D) -> Result + where + D: serde::Deserializer<'de>, + { + use serde::Deserialize; + let cache_bincode = Self::deserialize(deserializer)?; + Ok(Cache { + accounts: cache_bincode.accounts, + contracts: cache_bincode + .contracts + .into_iter() + .map(|(k, v)| (k, Bytecode::new_raw(v))) + .collect(), + logs: Default::default(), + block_hashes: cache_bincode.block_hashes, + }) + } +} diff --git a/crates/stateless/src/flat_witness/mod.rs b/crates/stateless/src/flat_witness/mod.rs new file mode 100644 index 0000000000..e8269366c2 --- /dev/null +++ b/crates/stateless/src/flat_witness/mod.rs @@ -0,0 +1,136 @@ +//! Flat execution witness for stateless block validation. +//! +//! This module provides a simplified witness structure containing the minimal state +//! data required for stateless execution of a block. The "flat" representation stores +//! state directly in a cache rather than as a Merkle proof, optimizing for execution +//! speed. This is useful if the flat execution witness is later cryptographically proven +//! correct in an independent proof. + +use core::error; + +pub mod bincode; + +use alloc::fmt; +use alloy_primitives::{ + map::{HashMap, HashSet}, + Address, Bytes, StorageValue, B256, U256, +}; +use reth_execution_types::FlatPreState; +use reth_revm::{ + db::{Cache, CacheDB, DBErrorMarker}, + primitives::StorageKey, + state::{AccountInfo, Bytecode}, + DatabaseRef, +}; +use serde_with::serde_as; + +/// A flat execution witness containing the state and context needed for stateless block execution. +#[serde_with::serde_as] +#[derive(Debug, Clone, Default, serde::Serialize, serde::Deserialize)] +pub struct FlatExecutionWitness { + /// The state required for executing the block. + #[serde_as(as = "bincode::CacheBincode")] + pub state: Cache, + /// The parent block header required for pre-execution validations. + pub parent_header: Bytes, + /// The set of addresses that have been self-destructed in the execution. + pub destructed_addresses: HashSet
, +} + +impl FlatExecutionWitness { + /// Creates a new flat execution witness from state components. + pub fn new( + pre_state: FlatPreState, + block_hashes: HashMap, + parent_header: Bytes, + ) -> Self { + Self { + state: Cache { + accounts: pre_state.accounts, + contracts: pre_state.contracts, + block_hashes, + logs: Default::default(), + }, + destructed_addresses: pre_state.destructed_addresses, + parent_header, + } + } + + /// Creates a cached database from the witness state. + /// + /// Returns a `CacheDB` backed by `FailingDB`, which ensures all state must come from the + /// cache. Any cache miss results in an error, enforcing stateless execution constraints. + // pub fn create_db(self) -> CacheDB { + pub fn create_db(self) -> CacheDB { + CacheDB { + cache: self.state, + db: SelfDestructCompatibleFailingDB::new(self.destructed_addresses), + } + } +} + +/// Database backend that fails on all accesses except storage reads from self-destructed accounts. +/// +/// This enforces that all state accesses during execution must be present in the cache. Cache +/// misses indicate missing witness data and must fail, since only cached accesses are +/// cryptographically verified against the trie witness proof. Returning default values would bypass +/// verification. +/// +/// **Exception: Self-destructed account storage** +/// +/// Storage reads from self-destructed accounts return zero (`Default::default()`) on cache miss. +/// This is safe because the trie witness includes complete storage tries for self-destructed +/// accounts, not just accessed slots—allowing any non-existent slot to be proven as zero. This +/// behavior matches the witness generation in `TrieWitness::get_proof_targets` and compensates for +/// `StateDB` not tracking individual storage accesses of self-destructed accounts. +#[derive(Debug, Clone)] +pub struct SelfDestructCompatibleFailingDB { + destructed_addresses: HashSet
, +} + +impl SelfDestructCompatibleFailingDB { + /// Creates a new instance with the given set of self-destructed addresses. + pub const fn new(destructed_addresses: HashSet
) -> Self { + Self { destructed_addresses } + } +} + +/// Error indicating that a database access was attempted outside the captured state. +#[derive(Debug)] +pub struct NonCapturedStateError; + +impl fmt::Display for NonCapturedStateError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "Database access not allowed in stateless execution context") + } +} + +impl error::Error for NonCapturedStateError {} +impl DBErrorMarker for NonCapturedStateError {} + +impl DatabaseRef for SelfDestructCompatibleFailingDB { + type Error = NonCapturedStateError; + + fn basic_ref(&self, _address: Address) -> Result, Self::Error> { + Err(NonCapturedStateError) + } + + fn code_by_hash_ref(&self, _code_hash: B256) -> Result { + Err(NonCapturedStateError) + } + + fn storage_ref( + &self, + _address: Address, + _index: StorageKey, + ) -> Result { + if self.destructed_addresses.contains(&_address) { + return Ok(Default::default()); + } + Err(NonCapturedStateError) + } + + fn block_hash_ref(&self, _number: u64) -> Result { + Err(NonCapturedStateError) + } +} diff --git a/crates/stateless/src/lib.rs b/crates/stateless/src/lib.rs index ac9e3480fc..cae1cc828f 100644 --- a/crates/stateless/src/lib.rs +++ b/crates/stateless/src/lib.rs @@ -39,6 +39,8 @@ mod recover_block; /// Sparse trie implementation for stateless validation pub mod trie; +pub mod flat_witness; + use alloy_genesis::ChainConfig; #[doc(inline)] pub use recover_block::UncompressedPublicKey; @@ -58,18 +60,27 @@ pub use alloy_rpc_types_debug::ExecutionWitness; use reth_ethereum_primitives::Block; +use crate::flat_witness::FlatExecutionWitness; + /// `StatelessInput` is a convenience structure for serializing the input needed /// for the stateless validation function. +pub type StatelessInput = GenericStatelessInput; + +/// `StatelessExecutionInput` is a convenience structure for serializing the input needed +/// for an execution-only stateless validation function. +pub type StatelessExecutionInput = GenericStatelessInput; + +/// Generic structure for stateless validation input. #[serde_with::serde_as] #[derive(Clone, Debug, Default, serde::Serialize, serde::Deserialize)] -pub struct StatelessInput { +pub struct GenericStatelessInput { /// The block being executed in the stateless validation function #[serde_as( as = "reth_primitives_traits::serde_bincode_compat::Block" )] pub block: Block, - /// `ExecutionWitness` for the stateless validation function - pub witness: ExecutionWitness, + /// `FlatExecutionWitness` for the stateless validation function + pub witness: Witness, /// Chain configuration for the stateless validation function #[serde_as(as = "alloy_genesis::serde_bincode_compat::ChainConfig<'_>")] pub chain_config: ChainConfig, diff --git a/crates/stateless/src/validation.rs b/crates/stateless/src/validation.rs index 54fa76949b..5e44d87972 100644 --- a/crates/stateless/src/validation.rs +++ b/crates/stateless/src/validation.rs @@ -1,4 +1,5 @@ use crate::{ + flat_witness::FlatExecutionWitness, recover_block::{recover_block_with_public_keys, UncompressedPublicKey}, track_cycles, trie::{StatelessSparseTrie, StatelessTrie}, @@ -13,14 +14,18 @@ use alloc::{ vec::Vec, }; use alloy_consensus::{BlockHeader, Header}; -use alloy_primitives::{keccak256, B256}; +use alloy_primitives::{keccak256, Bytes, B256}; use reth_chainspec::{EthChainSpec, EthereumHardforks}; use reth_consensus::{Consensus, HeaderValidator}; use reth_errors::ConsensusError; use reth_ethereum_consensus::{validate_block_post_execution, EthBeaconConsensus}; use reth_ethereum_primitives::{Block, EthPrimitives}; use reth_evm::{execute::Executor, ConfigureEvm}; -use reth_primitives_traits::{RecoveredBlock, SealedHeader}; +use reth_primitives_traits::{RecoveredBlock, SealedBlock, SealedHeader}; +use reth_revm::{ + db::{AccountState, Cache}, + Database, +}; use reth_trie_common::{HashedPostState, KeccakKeyHasher}; /// BLOCKHASH ancestor lookup window limit per EVM (number of most recent blocks accessible). @@ -102,6 +107,54 @@ pub enum StatelessValidationError { /// Custom error. #[error("{0}")] Custom(&'static str), + + /// Error validating the flatdb against the sparse trie witness. + #[error("flatdb and sparse trie validation mismatch")] + FlatdbSparseTrieMismatch, + + /// Error when a block hash in flatdb does not match the expected ancestor hash. + #[error("flatdb ancestor hash not found in witness for block {block_number}: got {got}, expected {expected:?}")] + FlatdbAncestorHashNotFound { + /// The block number being checked. + block_number: u64, + /// The block hash from flatdb. + got: B256, + /// The expected block hash from the witness. + expected: Option, + }, + + /// Error when flatdb bytecode hash does not match the computed hash. + #[error("flatdb bytecode hash mismatch: computed {got}, expected {expected}")] + FlatdbBytecodeHashMismatch { + /// The computed hash from the bytecode. + got: B256, + /// The expected code hash from flatdb. + expected: B256, + }, + + /// Error when flatdb account state does not match the trie witness. + #[error("flatdb account state mismatch with trie witness for address {address}")] + FlatdbAccountStateMismatch { + /// The address of the account with mismatched state. + address: alloy_primitives::Address, + }, + + /// Error when flatdb storage slot state does not match the trie witness. + #[error("flatdb storage slot state mismatch with trie witness for address {address}")] + FlatdbStorageSlotStateMismatch { + /// The address of the account with mismatched state. + address: alloy_primitives::Address, + /// The storage slot with mismatched state. + slot: alloy_primitives::U256, + }, + + /// Error getting account from the witness database. + #[error("getting account from witness database")] + GetAccountFromWitnessDatabase, + + /// Error getting storage slot from the witness database. + #[error("getting storage slot from witness database")] + GetStorageSlotFromWitnessDatabase, } /// Performs stateless validation of a block using the provided witness data. @@ -150,13 +203,14 @@ where ChainSpec: Send + Sync + EthChainSpec
+ EthereumHardforks + Debug, E: ConfigureEvm + Clone + 'static, { - stateless_validation_with_trie::( + let (hash, _) = stateless_validation_with_trie::( current_block, public_keys, witness, chain_spec, evm_config, - ) + )?; + Ok(hash) } /// Performs stateless validation of a block using a custom `StatelessTrie` implementation. @@ -171,16 +225,252 @@ pub fn stateless_validation_with_trie( witness: ExecutionWitness, chain_spec: Arc, evm_config: E, -) -> Result +) -> Result<(B256, HashedPostState), StatelessValidationError> where T: StatelessTrie, ChainSpec: Send + Sync + EthChainSpec
+ EthereumHardforks + Debug, E: ConfigureEvm + Clone + 'static, { - let current_block = recover_block_with_public_keys(current_block, public_keys, &*chain_spec)?; + let ExecutionContext { block, parent, ancestor_hashes } = prepare_execution_context( + current_block, + public_keys, + &witness.headers, + chain_spec.clone(), + )?; + + // Verify that the pre-state reads are correct + let (mut trie, bytecode) = + track_cycles!("verify_witness", T::new(&witness, parent.state_root)?); + let db = WitnessDatabase::new(&trie, bytecode, ancestor_hashes); + + let hashed_post_state = + stateless_validation_execution(&block, &parent, chain_spec, evm_config, db)?; + + // Compute and check the post state root + track_cycles!("post_state_compute", { + let state_root = trie.calculate_state_root(hashed_post_state.clone())?; + if state_root != block.state_root { + return Err(StatelessValidationError::PostStateRootMismatch { + got: state_root, + expected: block.state_root, + }); + } + }); + + // Return block hash + Ok((block.hash_slow(), hashed_post_state)) +} + +/// Performs stateless validation of a block using a flatdb execution witness. +pub fn stateless_validation_with_flatdb( + current_block: Block, + public_keys: Vec, + witness: FlatExecutionWitness, + chain_spec: Arc, + evm_config: E, +) -> Result<(B256, HashedPostState), StatelessValidationError> +where + ChainSpec: Send + Sync + EthChainSpec
+ EthereumHardforks + Debug, + E: ConfigureEvm + Clone + 'static, +{ + let recovered_block = recover_block_with_public_keys(current_block, public_keys, &*chain_spec)?; + + let hash = keccak256(&witness.parent_header); + let parent_header = alloy_rlp::decode_exact::
(&witness.parent_header) + .map(|h| SealedHeader::new(h, hash)) + .map_err(|_| StatelessValidationError::HeaderDeserializationFailed)?; + let db = witness.create_db(); + + let hashed_post_state = stateless_validation_execution( + &recovered_block, + &parent_header, + chain_spec, + evm_config, + db, + )?; + + Ok((recovered_block.hash_slow(), hashed_post_state)) +} + +/// Executes the block statelessly with a provided `Database` and performs post-execution +/// validation. +fn stateless_validation_execution( + current_block: &RecoveredBlock, + parent: &SealedHeader, + chain_spec: Arc, + evm_config: E, + db: DB, +) -> Result +where + ChainSpec: Send + Sync + EthChainSpec
+ EthereumHardforks + Debug, + E: ConfigureEvm + Clone + 'static, +{ + // Validate block against pre-execution consensus rules + validate_block_consensus(chain_spec.clone(), current_block, parent)?; + + // Execute the block + let executor = evm_config.executor(db); + let output = track_cycles!( + "block_execution", + 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)?; - let mut ancestor_headers: Vec<_> = witness - .headers + Ok(HashedPostState::from_bundle_state::(&output.state.state)) +} + +/// Cross-validates flatdb state against a cryptographically verified sparse trie witness. +/// +/// Flatdb provides optimized flat storage for EVM execution, but lacks cryptographic +/// verification. This function ensures flatdb was correctly constructed by checking that: +/// - Block hashes match between flatdb and the verified trie witness +/// - Contract bytecode hashes are correct +/// - Account state matches the sparse trie (witness may contain extra accounts; that's ok) +/// - Post-state root computed from flatdb matches the block header +/// +/// Returns block hash if all validations pass. +pub fn stateless_validation_flatdb_storage_check( + current_block: Block, + trie_witness: ExecutionWitness, + flatdb_pre_state: Cache, + flatdb_post_state: HashedPostState, +) -> Result +where + T: StatelessTrie, +{ + let sealed_block = reth_primitives_traits::Block::seal_slow(current_block); + let (parent, ancestor_hashes) = + parse_and_validate_ancestor_chain(&sealed_block, &trie_witness.headers)?; + + // Verify that the flatdb block hashes map was correctly constructed. + track_cycles!("verify_flatdb_block_hashes", { + for (block_num, block_hash) in flatdb_pre_state.block_hashes { + let block_num = block_num.try_into().unwrap(); + match ancestor_hashes.get(&block_num) { + Some(expected_hash) if *expected_hash == block_hash => {} + _ => { + return Err(StatelessValidationError::FlatdbAncestorHashNotFound { + block_number: block_num, + got: block_hash, + expected: ancestor_hashes.get(&block_num).copied(), + }) + } + } + } + }); + + // Verify that the contract code hashed map was correctly constructed. + track_cycles!("verify_flatdb_bytecodes", { + for (expected_codehash, flatdb_code) in &flatdb_pre_state.contracts { + let got_codehash = keccak256(flatdb_code.original_bytes()); + if got_codehash != *expected_codehash { + return Err(StatelessValidationError::FlatdbBytecodeHashMismatch { + got: got_codehash, + expected: *expected_codehash, + }); + } + } + }); + + // Verify the trie witness so we have a cryptographically verified state to compare against. + let (mut trie, _) = + track_cycles!("verify_trie_witness", T::new(&trie_witness, parent.state_root)?); + + // Verify that all accounts in flatdb pre-state match those in the trie witness. + track_cycles!("verify_flatdb_state_against_trie", { + // For bytecodes we pass Default::default() since the WitnessDatabase won't be used for + // checking bytecode. We already did it above by checking that the flatdb bytecodes + // hashes match. If the flatdb accounts contains an account with a code_hash not + // present in the bytecode map, then the execution would have failed since + // the code would be missing. + let mut db = WitnessDatabase::new(&trie, Default::default(), ancestor_hashes); + for (address, flatdb_account) in flatdb_pre_state.accounts { + let trie_account = db + .basic(address) + .map_err(|_| StatelessValidationError::GetAccountFromWitnessDatabase)?; + let flatdb_account_info = (flatdb_account.account_state != AccountState::NotExisting) + .then_some(flatdb_account.info); + + if trie_account != flatdb_account_info { + return Err(StatelessValidationError::FlatdbAccountStateMismatch { address }); + } + for (slot, value) in flatdb_account.storage { + let trie_value = db + .storage(address, slot) + .map_err(|_| StatelessValidationError::GetStorageSlotFromWitnessDatabase)?; + if trie_value != value { + return Err(StatelessValidationError::FlatdbStorageSlotStateMismatch { + address, + slot, + }); + } + } + } + }); + + // Compute and check the post state root using the provided post-state from the execution using + // flatdb state. + track_cycles!("post_state_compute", { + let state_root = trie.calculate_state_root(flatdb_post_state)?; + if state_root != sealed_block.state_root { + return Err(StatelessValidationError::PostStateRootMismatch { + got: state_root, + expected: sealed_block.state_root, + }); + } + }); + + Ok(sealed_block.hash()) +} + +/// Execution context prepared from witness data for stateless block validation. +#[derive(Debug, Clone)] +pub struct ExecutionContext { + /// The current block being validated, with transaction signers recovered from signatures. + pub block: RecoveredBlock, + /// The immediate parent header, containing the pre-state root required for witness + /// verification. + pub parent: SealedHeader, + /// Map of ancestor block numbers to their hashes, used for BLOCKHASH opcode lookups. + /// Limited to 256 most recent blocks per EVM specification. + pub ancestor_hashes: BTreeMap, +} + +/// Extracts and validates the core witness components required for stateless execution. +/// +/// This function prepares three essential components from raw witness data: +/// 1. **Recovered Block**: Validates and recovers transaction signers using provided public keys +/// 2. **Parent Header**: The immediate ancestor header containing the pre-state root +/// 3. **Ancestor Hashes**: Block hash map for BLOCKHASH opcode support (up to 256 blocks) +/// +/// The ancestor headers must form a contiguous chain leading to `current_block`'s parent. +/// At least one ancestor header (the parent) is required for pre-state root verification. +pub fn prepare_execution_context( + current_block: Block, + public_keys: Vec, + headers: &[Bytes], + chain_spec: Arc, +) -> Result +where + ChainSpec: Send + Sync + EthChainSpec
+ EthereumHardforks + Debug, +{ + let block = recover_block_with_public_keys(current_block, public_keys, &*chain_spec)?; + let (parent, ancestor_hashes) = parse_and_validate_ancestor_chain(&block, headers)?; + + Ok(ExecutionContext { block, parent, ancestor_hashes }) +} + +/// Deserializes, validates, and extracts ancestor headers from raw witness data. +pub fn parse_and_validate_ancestor_chain( + current_block: &SealedBlock, + headers: &[Bytes], +) -> Result<(SealedHeader, BTreeMap), StatelessValidationError> { + let mut ancestor_headers: Vec<_> = headers .iter() .map(|bytes| { let hash = keccak256(bytes); @@ -203,7 +493,7 @@ where } // Check that the ancestor headers form a contiguous chain and are not just random headers. - let ancestor_hashes = compute_ancestor_hashes(¤t_block, &ancestor_headers)?; + let ancestor_hashes = compute_ancestor_hashes(current_block, &ancestor_headers)?; // There should be at least one ancestor header. // The edge case here would be the genesis block, but we do not create proofs for the genesis @@ -213,44 +503,7 @@ where None => return Err(StatelessValidationError::MissingAncestorHeader), }; - // Validate block against pre-execution consensus rules - validate_block_consensus(chain_spec.clone(), ¤t_block, parent)?; - - // First verify that the pre-state reads are correct - let (mut trie, bytecode) = - track_cycles!("verify_witness", T::new(&witness, parent.state_root)?); - - // Create an in-memory database that will use the reads to validate the block - let db = WitnessDatabase::new(&trie, bytecode, ancestor_hashes); - - // Execute the block - let executor = evm_config.executor(db); - let output = track_cycles!( - "block_execution", - executor - .execute(¤t_block) - .map_err(|e| StatelessValidationError::StatelessExecutionFailed(e.to_string()))? - ); - - // Post validation checks - validate_block_post_execution(¤t_block, &chain_spec, &output.receipts, &output.requests) - .map_err(StatelessValidationError::ConsensusValidationFailed)?; - - // Compute and check the post state root - track_cycles!("post_state_compute", { - let hashed_state = - HashedPostState::from_bundle_state::(&output.state.state); - let state_root = trie.calculate_state_root(hashed_state)?; - 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()) + Ok((parent.clone(), ancestor_hashes)) } /// Performs consensus validation checks on a block without execution or state validation. @@ -301,7 +554,7 @@ where /// 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, + current_block: &SealedBlock, ancestor_headers: &[SealedHeader], ) -> Result, StatelessValidationError> { let mut ancestor_hashes = BTreeMap::new(); diff --git a/crates/storage/provider/Cargo.toml b/crates/storage/provider/Cargo.toml index e8599a8970..26bd0f726b 100644 --- a/crates/storage/provider/Cargo.toml +++ b/crates/storage/provider/Cargo.toml @@ -16,7 +16,10 @@ workspace = true reth-chainspec.workspace = true reth-execution-types.workspace = true reth-ethereum-primitives = { workspace = true, features = ["reth-codec"] } -reth-primitives-traits = { workspace = true, features = ["reth-codec", "secp256k1"] } +reth-primitives-traits = { workspace = true, features = [ + "reth-codec", + "secp256k1", +] } reth-fs-util.workspace = true reth-errors.workspace = true reth-storage-errors.workspace = true @@ -32,6 +35,9 @@ reth-codecs.workspace = true reth-chain-state.workspace = true reth-node-types.workspace = true reth-static-file-types.workspace = true +reth-revm = { workspace = true, features = ["witness"] } +reth-stateless.workspace = true + # ethereum alloy-eips.workspace = true alloy-primitives.workspace = true @@ -49,11 +55,14 @@ metrics.workspace = true # misc itertools.workspace = true -notify = { workspace = true, default-features = false, features = ["macos_fsevent"] } +notify = { workspace = true, default-features = false, features = [ + "macos_fsevent", +] } parking_lot.workspace = true dashmap = { workspace = true, features = ["inline"] } strum.workspace = true eyre.workspace = true +thiserror.workspace = true # test-utils reth-ethereum-engine-primitives = { workspace = true, optional = true } @@ -64,7 +73,10 @@ rayon.workspace = true [dev-dependencies] reth-db = { workspace = true, features = ["test-utils"] } -reth-primitives-traits = { workspace = true, features = ["arbitrary", "test-utils"] } +reth-primitives-traits = { workspace = true, features = [ + "arbitrary", + "test-utils", +] } reth-chain-state = { workspace = true, features = ["test-utils"] } reth-trie = { workspace = true, features = ["test-utils"] } reth-testing-utils.workspace = true diff --git a/crates/storage/provider/src/lib.rs b/crates/storage/provider/src/lib.rs index 70822c604b..5e91081921 100644 --- a/crates/storage/provider/src/lib.rs +++ b/crates/storage/provider/src/lib.rs @@ -43,6 +43,9 @@ pub use revm_database::states::OriginalValuesKnown; /// Writer standalone type. pub mod writer; +/// Witness recording utilities. +pub mod witness; + pub use reth_chain_state::{ CanonStateNotification, CanonStateNotificationSender, CanonStateNotificationStream, CanonStateNotifications, CanonStateSubscriptions, diff --git a/crates/storage/provider/src/providers/state/historical.rs b/crates/storage/provider/src/providers/state/historical.rs index f3e69bf7d9..1c22109625 100644 --- a/crates/storage/provider/src/providers/state/historical.rs +++ b/crates/storage/provider/src/providers/state/historical.rs @@ -3,7 +3,7 @@ use crate::{ ChangeSetReader, HashedPostStateProvider, ProviderError, StateProvider, StateRootProvider, }; use alloy_eips::merge::EPOCH_SLOTS; -use alloy_primitives::{Address, BlockNumber, Bytes, StorageKey, StorageValue, B256}; +use alloy_primitives::{Address, BlockNumber, Bytes, StorageKey, StorageValue, B256, U256}; use reth_db_api::{ cursor::{DbCursorRO, DbDupCursorRO}, models::{storage_sharded_key::StorageShardedKey, ShardedKey}, @@ -12,6 +12,7 @@ use reth_db_api::{ transaction::DbTx, BlockNumberList, }; +use reth_execution_types::{AccessedAccount, FlatPreState, FlatWitnessRecord}; use reth_primitives_traits::{Account, Bytecode}; use reth_storage_api::{ BlockNumReader, BytecodeReader, DBProvider, StateProofProvider, StorageRootProvider, @@ -28,6 +29,7 @@ use reth_trie_db::{ DatabaseHashedPostState, DatabaseHashedStorage, DatabaseProof, DatabaseStateRoot, DatabaseStorageProof, DatabaseStorageRoot, DatabaseTrieWitness, }; +use revm_database::DbAccount; use std::fmt::Debug; @@ -79,7 +81,7 @@ impl<'b, Provider: DBProvider + BlockNumReader> HistoricalStateProviderRef<'b, P /// Lookup an account in the `AccountsHistory` table pub fn account_history_lookup(&self, address: Address) -> ProviderResult { if !self.lowest_available_blocks.is_account_history_available(self.block_number) { - return Err(ProviderError::StateAtBlockPruned(self.block_number)) + return Err(ProviderError::StateAtBlockPruned(self.block_number)); } // history key to search IntegerList of block number changesets. @@ -98,7 +100,7 @@ impl<'b, Provider: DBProvider + BlockNumReader> HistoricalStateProviderRef<'b, P storage_key: StorageKey, ) -> ProviderResult { if !self.lowest_available_blocks.is_storage_history_available(self.block_number) { - return Err(ProviderError::StateAtBlockPruned(self.block_number)) + return Err(ProviderError::StateAtBlockPruned(self.block_number)); } // history key to search IntegerList of block number changesets. @@ -122,7 +124,7 @@ impl<'b, Provider: DBProvider + BlockNumReader> HistoricalStateProviderRef<'b, P if !self.lowest_available_blocks.is_account_history_available(self.block_number) || !self.lowest_available_blocks.is_storage_history_available(self.block_number) { - return Err(ProviderError::StateAtBlockPruned(self.block_number)) + return Err(ProviderError::StateAtBlockPruned(self.block_number)); } if self.check_distance_against_limit(EPOCH_SLOTS)? { @@ -139,7 +141,7 @@ impl<'b, Provider: DBProvider + BlockNumReader> HistoricalStateProviderRef<'b, P /// Retrieve revert hashed storage for this history provider and target address. fn revert_storage(&self, address: Address) -> ProviderResult { if !self.lowest_available_blocks.is_storage_history_available(self.block_number) { - return Err(ProviderError::StateAtBlockPruned(self.block_number)) + return Err(ProviderError::StateAtBlockPruned(self.block_number)); } if self.check_distance_against_limit(EPOCH_SLOTS * 10)? { @@ -357,7 +359,7 @@ impl StorageRootProvider } } -impl StateProofProvider +impl StateProofProvider for HistoricalStateProviderRef<'_, Provider> { /// Get account and storage proofs. @@ -386,6 +388,86 @@ impl StateProofProvider .map_err(ProviderError::from) .map(|hm| hm.into_values().collect()) } + + fn flat_witness(&self, record: FlatWitnessRecord) -> ProviderResult { + let mut prestate = FlatPreState::default(); + for (code_hash, code) in &record.contracts { + match code { + Some(bytecode) => { + prestate.contracts.insert(*code_hash, bytecode.clone()); + } + None => { + // Fetch code from provider if not present in record + if let Some(code) = self.bytecode_by_hash(code_hash)? { + prestate.contracts.insert(*code_hash, code.0); + } + } + } + } + + for (address, account) in &record.accounts { + let provider_account = self.basic_account(address)?; + + let mut db_account = match provider_account { + Some(account) => DbAccount { info: account.into(), ..Default::default() }, + None => DbAccount::new_not_existing(), + }; + + match account { + AccessedAccount::Destroyed => { + // When an account is destroyed, statedb discards all storage slots, losing + // track of which slots were accessed pre-destruction. We + // fetch all storage slots for destroyed accounts (mirroring + // TrieWitness::get_proof_targets in trie/trie/src/witness.rs), + // requiring a lower-level db cursor. Only relevant pre-Cancun where + // SELFDESTRUCT clears storage. + + let tx = self.tx(); + + // Get the original storage slot values from reverts + let storage_overlay = + reth_trie_db::from_reverts(tx, *address, self.block_number)?; + // Since we have to include all the existing storage slots, we can already + // include them. Note we filter out zero values, as that + // means those storage slots weren't in the trie at + // this point. + for (key, value) in + storage_overlay.into_iter().filter(|(_, val)| *val != U256::ZERO) + { + db_account.storage.insert(key.into(), value); + } + + // The rest of storage slots present at this block number are in the plain + // storage table. We only add the ones not already present + // from reverts, since those contain newer values. + let mut storage_cursor = tx.cursor_dup_read::()?; + if let Some((_, first_entry)) = storage_cursor.seek_exact(*address)? { + db_account + .storage + .entry(U256::from_be_bytes(first_entry.key.0)) + .or_insert(first_entry.value); + + while let Some((_, entry)) = storage_cursor.next_dup()? { + db_account + .storage + .entry(U256::from_be_bytes(entry.key.0)) + .or_insert(entry.value); + } + } + prestate.destructed_addresses.insert(*address); + } + AccessedAccount::StorageKeys(storage_keys) => { + // Fetch pre-state storage for accessed slots + for slot in storage_keys { + let val = self.storage(*address, (*slot).into())?.unwrap_or_default(); + db_account.storage.insert(*slot, val); + } + } + } + prestate.accounts.insert(*address, db_account); + } + Ok(prestate) + } } impl HashedPostStateProvider for HistoricalStateProviderRef<'_, Provider> { diff --git a/crates/storage/provider/src/providers/state/latest.rs b/crates/storage/provider/src/providers/state/latest.rs index de8eef2cc9..f44cf820df 100644 --- a/crates/storage/provider/src/providers/state/latest.rs +++ b/crates/storage/provider/src/providers/state/latest.rs @@ -2,8 +2,10 @@ use crate::{ providers::state::macros::delegate_provider_impls, AccountReader, BlockHashReader, HashedPostStateProvider, StateProvider, StateRootProvider, }; -use alloy_primitives::{Address, BlockNumber, Bytes, StorageKey, StorageValue, B256}; +use alloy_primitives::{Address, BlockNumber, Bytes, StorageKey, StorageValue, B256, U256}; +use reth_db::cursor::DbCursorRO; use reth_db_api::{cursor::DbDupCursorRO, tables, transaction::DbTx}; +use reth_execution_types::{AccessedAccount, FlatPreState, FlatWitnessRecord}; use reth_primitives_traits::{Account, Bytecode}; use reth_storage_api::{BytecodeReader, DBProvider, StateProofProvider, StorageRootProvider}; use reth_storage_errors::provider::{ProviderError, ProviderResult}; @@ -18,6 +20,7 @@ use reth_trie_db::{ DatabaseProof, DatabaseStateRoot, DatabaseStorageProof, DatabaseStorageRoot, DatabaseTrieWitness, }; +use revm_database::DbAccount; /// State provider over latest state that takes tx reference. /// @@ -117,7 +120,9 @@ impl StorageRootProvider for LatestStateProviderRef } } -impl StateProofProvider for LatestStateProviderRef<'_, Provider> { +impl StateProofProvider + for LatestStateProviderRef<'_, Provider> +{ fn proof( &self, input: TrieInput, @@ -140,6 +145,67 @@ impl StateProofProvider for LatestStateProviderRef< .map_err(ProviderError::from) .map(|hm| hm.into_values().collect()) } + + fn flat_witness(&self, record: FlatWitnessRecord) -> ProviderResult { + let mut prestate = FlatPreState::default(); + for (code_hash, code) in &record.contracts { + match code { + Some(bytecode) => { + prestate.contracts.insert(*code_hash, bytecode.clone()); + } + None => { + // Fetch code from provider if not present in record + if let Some(code) = self.bytecode_by_hash(code_hash)? { + prestate.contracts.insert(*code_hash, code.0); + } + } + } + } + + for (address, account) in &record.accounts { + let provider_account = self.basic_account(address)?; + + let mut db_account = match provider_account { + Some(account) => DbAccount { info: account.into(), ..Default::default() }, + None => DbAccount::new_not_existing(), + }; + + match account { + AccessedAccount::Destroyed => { + // When an account is destroyed, statedb discards all storage slots, losing + // track of which slots were accessed pre-destruction. We + // fetch all storage slots for destroyed accounts (mirroring + // TrieWitness::get_proof_targets in trie/trie/src/witness.rs), + // requiring a lower-level db cursor. Only relevant pre-Cancun where + // SELFDESTRUCT clears storage. + let tx = self.tx(); + let mut storage_cursor = tx.cursor_dup_read::()?; + if let Some((_, first_entry)) = storage_cursor.seek_exact(*address)? { + db_account + .storage + .insert(U256::from_be_bytes(first_entry.key.0), first_entry.value); + + while let Some((_, entry)) = storage_cursor.next_dup()? { + db_account + .storage + .insert(U256::from_be_bytes(entry.key.0), entry.value); + } + } + prestate.destructed_addresses.insert(*address); + } + AccessedAccount::StorageKeys(storage_keys) => { + // Fetch pre-state storage for accessed slots + for slot in storage_keys { + let val = self.storage(*address, (*slot).into())?.unwrap_or_default(); + db_account.storage.insert(*slot, val); + } + } + } + prestate.accounts.insert(*address, db_account); + } + + Ok(prestate) + } } impl HashedPostStateProvider for LatestStateProviderRef<'_, Provider> { @@ -161,7 +227,7 @@ impl StateProvider if let Some(entry) = cursor.seek_by_key_subkey(account, storage_key)? && entry.key == storage_key { - return Ok(Some(entry.value)) + return Ok(Some(entry.value)); } Ok(None) } diff --git a/crates/storage/provider/src/providers/state/macros.rs b/crates/storage/provider/src/providers/state/macros.rs index 74bb371819..38880afb4d 100644 --- a/crates/storage/provider/src/providers/state/macros.rs +++ b/crates/storage/provider/src/providers/state/macros.rs @@ -58,6 +58,7 @@ macro_rules! delegate_provider_impls { fn proof(&self, input: reth_trie::TrieInput, address: alloy_primitives::Address, slots: &[alloy_primitives::B256]) -> reth_storage_errors::provider::ProviderResult; fn multiproof(&self, input: reth_trie::TrieInput, targets: reth_trie::MultiProofTargets) -> reth_storage_errors::provider::ProviderResult; fn witness(&self, input: reth_trie::TrieInput, target: reth_trie::HashedPostState) -> reth_storage_errors::provider::ProviderResult>; + fn flat_witness(&self, input: reth_execution_types::FlatWitnessRecord) -> ProviderResult; } HashedPostStateProvider $(where [$($generics)*])? { fn hashed_post_state(&self, bundle_state: &revm_database::BundleState) -> reth_trie::HashedPostState; diff --git a/crates/storage/provider/src/test_utils/mock.rs b/crates/storage/provider/src/test_utils/mock.rs index 3e33e2b050..013911dd07 100644 --- a/crates/storage/provider/src/test_utils/mock.rs +++ b/crates/storage/provider/src/test_utils/mock.rs @@ -24,7 +24,7 @@ use reth_db_api::{ models::{AccountBeforeTx, StoredBlockBodyIndices}, }; use reth_ethereum_primitives::EthPrimitives; -use reth_execution_types::ExecutionOutcome; +use reth_execution_types::{ExecutionOutcome, FlatPreState, FlatWitnessRecord}; use reth_primitives_traits::{ Account, Block, BlockBody, Bytecode, GotExpected, NodePrimitives, RecoveredBlock, SealedHeader, SignerRecoverable, @@ -414,7 +414,7 @@ impl TransactionsProvider excess_blob_gas: block.header().excess_blob_gas(), timestamp: block.header().timestamp(), }; - return Ok(Some((tx.clone(), meta))) + return Ok(Some((tx.clone(), meta))); } } } @@ -426,7 +426,7 @@ impl TransactionsProvider let mut current_tx_number: TxNumber = 0; for block in lock.values() { if current_tx_number + (block.body().transaction_count() as TxNumber) > id { - return Ok(Some(block.header().number())) + return Ok(Some(block.header().number())); } current_tx_number += block.body().transaction_count() as TxNumber; } @@ -861,6 +861,10 @@ where fn witness(&self, _input: TrieInput, _target: HashedPostState) -> ProviderResult> { Ok(Vec::default()) } + + fn flat_witness(&self, _record: FlatWitnessRecord) -> ProviderResult { + Ok(Default::default()) + } } impl HashedPostStateProvider diff --git a/crates/storage/provider/src/witness.rs b/crates/storage/provider/src/witness.rs new file mode 100644 index 0000000000..0015c3de95 --- /dev/null +++ b/crates/storage/provider/src/witness.rs @@ -0,0 +1,26 @@ +//! Witness recording utilities for state providers. + +use reth_execution_types::{FlatPreState, FlatWitnessRecord}; + +/// Errors that can occur during flat witness record generation. +#[derive(Debug, thiserror::Error)] +pub enum FlatWitnessRecordError { + /// Failed to retrieve account from state provider + #[error("Failed to get prestate account: {0}")] + GetAccountError(#[from] crate::ProviderError), + /// Failed to access database cursor + #[error("Failed to create database cursor: {0}")] + DatabaseCursorError(#[from] reth_db::DatabaseError), +} + +/// Extension trait for recording flat witness data from state providers. +pub trait RecordFlatWitness { + /// Records pre-state from database for all accounts touched during execution. + /// + /// The provided `statedb` contains post-execution state, so this method uses the provider + /// to pull the pre-state of the accessed accounts, storage slots, and codes. + fn flat_witness( + &self, + witness: FlatWitnessRecord, + ) -> Result; +} diff --git a/crates/storage/rpc-provider/src/lib.rs b/crates/storage/rpc-provider/src/lib.rs index ed6e49eefb..fb3a1f96e8 100644 --- a/crates/storage/rpc-provider/src/lib.rs +++ b/crates/storage/rpc-provider/src/lib.rs @@ -48,9 +48,9 @@ use reth_provider::{ AccountReader, BlockHashReader, BlockIdReader, BlockNumReader, BlockReader, BytecodeReader, CanonChainTracker, CanonStateNotification, CanonStateNotifications, CanonStateSubscriptions, ChainStateBlockReader, ChainStateBlockWriter, ChangeSetReader, DatabaseProviderFactory, - HeaderProvider, PruneCheckpointReader, ReceiptProvider, StageCheckpointReader, StateProvider, - StateProviderBox, StateProviderFactory, StateReader, StateRootProvider, StorageReader, - TransactionVariant, TransactionsProvider, + FlatPreState, FlatWitnessRecord, HeaderProvider, PruneCheckpointReader, ReceiptProvider, + StageCheckpointReader, StateProvider, StateProviderBox, StateProviderFactory, StateReader, + StateRootProvider, StorageReader, TransactionVariant, TransactionsProvider, }; use reth_prune_types::{PruneCheckpoint, PruneSegment}; use reth_rpc_convert::{TryFromBlockResponse, TryFromReceiptResponse, TryFromTransactionResponse}; @@ -1314,6 +1314,10 @@ where ) -> Result, ProviderError> { Err(ProviderError::UnsupportedProvider) } + + fn flat_witness(&self, _record: FlatWitnessRecord) -> ProviderResult { + Err(ProviderError::UnsupportedProvider) + } } impl reth_storage_api::HashedPostStateProvider diff --git a/crates/storage/storage-api/src/noop.rs b/crates/storage/storage-api/src/noop.rs index 6b70a5260a..16dcd67f98 100644 --- a/crates/storage/storage-api/src/noop.rs +++ b/crates/storage/storage-api/src/noop.rs @@ -27,7 +27,7 @@ use reth_chainspec::{ChainInfo, ChainSpecProvider, EthChainSpec, MAINNET}; use reth_db_api::mock::{DatabaseMock, TxMock}; use reth_db_models::{AccountBeforeTx, StoredBlockBodyIndices}; use reth_ethereum_primitives::EthPrimitives; -use reth_execution_types::ExecutionOutcome; +use reth_execution_types::{ExecutionOutcome, FlatPreState, FlatWitnessRecord}; use reth_primitives_traits::{Account, Bytecode, NodePrimitives, RecoveredBlock, SealedHeader}; #[cfg(feature = "db-api")] use reth_prune_types::PruneModes; @@ -483,6 +483,10 @@ impl StateProofProvider for NoopProvider ProviderResult> { Ok(Vec::default()) } + + fn flat_witness(&self, _record: FlatWitnessRecord) -> ProviderResult { + Ok(Default::default()) + } } impl HashedPostStateProvider for NoopProvider { diff --git a/crates/storage/storage-api/src/trie.rs b/crates/storage/storage-api/src/trie.rs index 9ff02c106e..de2d56e792 100644 --- a/crates/storage/storage-api/src/trie.rs +++ b/crates/storage/storage-api/src/trie.rs @@ -1,5 +1,6 @@ use alloc::vec::Vec; use alloy_primitives::{Address, BlockNumber, Bytes, B256}; +use reth_execution_types::{FlatPreState, FlatWitnessRecord}; use reth_storage_errors::provider::ProviderResult; use reth_trie_common::{ updates::{StorageTrieUpdatesSorted, TrieUpdates, TrieUpdatesSorted}, @@ -87,6 +88,9 @@ pub trait StateProofProvider: Send + Sync { /// Get trie witness for provided state. fn witness(&self, input: TrieInput, target: HashedPostState) -> ProviderResult>; + + /// Get pre-state for state touched during execution. + fn flat_witness(&self, record: FlatWitnessRecord) -> ProviderResult; } /// Trie Reader diff --git a/crates/trie/db/src/lib.rs b/crates/trie/db/src/lib.rs index 5417e5bd1e..325e28f210 100644 --- a/crates/trie/db/src/lib.rs +++ b/crates/trie/db/src/lib.rs @@ -16,7 +16,7 @@ pub use hashed_cursor::{ pub use prefix_set::PrefixSetLoader; pub use proof::{DatabaseProof, DatabaseStorageProof}; pub use state::{DatabaseHashedPostState, DatabaseStateRoot}; -pub use storage::{DatabaseHashedStorage, DatabaseStorageRoot}; +pub use storage::{from_reverts, DatabaseHashedStorage, DatabaseStorageRoot}; pub use trie_cursor::{ DatabaseAccountTrieCursor, DatabaseStorageTrieCursor, DatabaseTrieCursorFactory, }; diff --git a/crates/trie/db/src/storage.rs b/crates/trie/db/src/storage.rs index 42d0d464c7..bead2a8911 100644 --- a/crates/trie/db/src/storage.rs +++ b/crates/trie/db/src/storage.rs @@ -1,5 +1,9 @@ use crate::{DatabaseHashedCursorFactory, DatabaseTrieCursorFactory}; -use alloy_primitives::{keccak256, map::hash_map, Address, BlockNumber, B256}; +use alloy_primitives::{ + keccak256, + map::{hash_map, B256Map}, + Address, BlockNumber, B256, U256, +}; use reth_db_api::{ cursor::DbCursorRO, models::BlockNumberAddress, tables, transaction::DbTx, DatabaseError, }; @@ -82,16 +86,28 @@ impl<'a, TX: DbTx> DatabaseStorageRoot<'a, TX> impl DatabaseHashedStorage for HashedStorage { fn from_reverts(tx: &TX, address: Address, from: BlockNumber) -> Result { let mut storage = Self::new(false); - let mut storage_changesets_cursor = tx.cursor_read::()?; - for entry in storage_changesets_cursor.walk_range(BlockNumberAddress((from, address))..)? { - let (BlockNumberAddress((_, storage_address)), storage_change) = entry?; - if storage_address == address { - let hashed_slot = keccak256(storage_change.key); - if let hash_map::Entry::Vacant(entry) = storage.storage.entry(hashed_slot) { - entry.insert(storage_change.value); - } - } + for (slot, value) in from_reverts(tx, address, from)? { + storage.storage.insert(keccak256(slot), value); } Ok(storage) } } + +/// Retrieves the original storage values from changed slots since the specified block and address. +pub fn from_reverts( + tx: &TX, + address: Address, + from: BlockNumber, +) -> Result, DatabaseError> { + let mut storage: B256Map = Default::default(); + let mut storage_changesets_cursor = tx.cursor_read::()?; + for entry in storage_changesets_cursor.walk_range(BlockNumberAddress((from, address))..)? { + let (BlockNumberAddress((_, storage_address)), storage_change) = entry?; + if storage_address == address { + if let hash_map::Entry::Vacant(entry) = storage.entry(storage_change.key) { + entry.insert(storage_change.value); + } + } + } + Ok(storage) +} diff --git a/testing/ef-tests/src/cases/blockchain_test.rs b/testing/ef-tests/src/cases/blockchain_test.rs index c54ef2ad7b..7bf2970dcd 100644 --- a/testing/ef-tests/src/cases/blockchain_test.rs +++ b/testing/ef-tests/src/cases/blockchain_test.rs @@ -4,6 +4,8 @@ use crate::{ models::{BlockchainTest, ForkSpec}, Case, Error, Suite, }; + +use alloy_primitives::{keccak256, map::HashMap, B256, U256}; use alloy_rlp::{Decodable, Encodable}; use rayon::iter::{ParallelBridge, ParallelIterator}; use reth_chainspec::ChainSpec; @@ -16,13 +18,19 @@ use reth_evm_ethereum::EthEvmConfig; use reth_primitives_traits::{Block as BlockTrait, RecoveredBlock, SealedBlock}; use reth_provider::{ test_utils::create_test_provider_factory_with_chain_spec, BlockWriter, DatabaseProviderFactory, - ExecutionOutcome, HeaderProvider, HistoryWriter, OriginalValuesKnown, StateProofProvider, - StateWriter, StaticFileProviderFactory, StaticFileSegment, StaticFileWriter, + ExecutionOutcome, FlatWitnessRecord, HeaderProvider, HistoryWriter, OriginalValuesKnown, + StateProofProvider, StateWriter, StaticFileProviderFactory, StaticFileSegment, + StaticFileWriter, }; use reth_revm::{database::StateProviderDatabase, witness::ExecutionWitnessRecord, State}; use reth_stateless::{ - trie::StatelessSparseTrie, validation::stateless_validation_with_trie, ExecutionWitness, - UncompressedPublicKey, + flat_witness::FlatExecutionWitness, + trie::StatelessSparseTrie, + validation::{ + stateless_validation_flatdb_storage_check, stateless_validation_with_flatdb, + stateless_validation_with_trie, + }, + ExecutionWitness, UncompressedPublicKey, }; use reth_trie::{HashedPostState, KeccakKeyHasher, StateRoot}; use reth_trie_db::DatabaseStateRoot; @@ -106,7 +114,7 @@ impl BlockchainTestCase { pub fn run_single_case( name: &str, case: &BlockchainTest, - ) -> Result, ExecutionWitness)>, Error> { + ) -> Result, ExecutionWitnesses)>, Error> { let expectation = Self::expected_failure(case); match run_case(case) { // All blocks executed successfully. @@ -187,6 +195,15 @@ impl Case for BlockchainTestCase { } } +/// Execution witnessses for both trie and flatdb stateless validation. +#[derive(Debug, Clone)] +pub struct ExecutionWitnesses { + /// Trie-based execution witness. + pub trie: ExecutionWitness, + /// Flatdb-based execution witness. + pub flatdb: FlatExecutionWitness, +} + /// Executes a single `BlockchainTest` returning an error as soon as any block has a consensus /// validation failure. /// @@ -203,7 +220,7 @@ impl Case for BlockchainTestCase { /// witness if the error is of variant `BlockProcessingFailed`. fn run_case( case: &BlockchainTest, -) -> Result, ExecutionWitness)>, Error> { +) -> Result, ExecutionWitnesses)>, Error> { // Create a new test database and initialize a provider for the test case. let chain_spec: Arc = Arc::new(case.network.into()); let factory = create_test_provider_factory_with_chain_spec(chain_spec.clone()); @@ -241,7 +258,7 @@ fn run_case( let executor_provider = EthEvmConfig::ethereum(chain_spec.clone()); let mut parent = genesis_block; - let mut program_inputs = Vec::new(); + let mut program_inputs: Vec<(RecoveredBlock, ExecutionWitnesses)> = 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 @@ -264,6 +281,7 @@ fn run_case( })?; let mut witness_record = ExecutionWitnessRecord::default(); + let mut flat_witness_record = FlatWitnessRecord::default(); // Execute the block let state_provider = provider.latest(); @@ -273,6 +291,7 @@ fn run_case( let output = executor .execute_with_state_closure_always(&(*block).clone(), |statedb: &State<_>| { witness_record.record_executed_state(statedb); + flat_witness_record.record_executed_state(statedb); }) .map_err(|err| Error::block_failed(block_number, program_inputs.clone(), err))?; @@ -296,7 +315,7 @@ fn run_case( let range = smallest..block_number; exec_witness.headers = provider - .headers_range(range)? + .headers_range(range.clone())? .into_iter() .map(|header| { let mut serialized_header = Vec::new(); @@ -305,7 +324,18 @@ fn run_case( }) .collect(); - program_inputs.push((block.clone(), exec_witness)); + let flat_prestate = state_provider.flat_witness(flat_witness_record).unwrap(); + let block_hashes: HashMap = exec_witness + .headers + .iter() + .zip(range) + .map(|(bytes, num)| (U256::from(num), keccak256(bytes))) + .collect(); + let parent_encoded = exec_witness.headers.last().cloned().unwrap(); + let flat_witness = FlatExecutionWitness::new(flat_prestate, block_hashes, parent_encoded); + + program_inputs + .push((block.clone(), ExecutionWitnesses { trie: exec_witness, flatdb: flat_witness })); // Compute and check the post state root let hashed_state = @@ -359,21 +389,49 @@ fn run_case( } // Now validate using the stateless client if everything else passes - for (recovered_block, execution_witness) in &program_inputs { + for (recovered_block, execution_witnesses) in &program_inputs { let block = recovered_block.clone().into_block(); // Recover the actual public keys from the transaction signatures let public_keys = recover_signers(block.body().transactions()) .expect("Failed to recover public keys from transaction signatures"); - stateless_validation_with_trie::( - block, - public_keys, - execution_witness.clone(), + // Validate stateless execution using a sparse trie for the storage access. + let (_, trie_post_state) = stateless_validation_with_trie::( + block.clone(), + public_keys.clone(), + execution_witnesses.trie.clone(), chain_spec.clone(), EthEvmConfig::new(chain_spec.clone()), ) .expect("stateless validation failed"); + + // Validate stateless execution using a flatdb for the storage access. + let (_, flatdb_post_state) = stateless_validation_with_flatdb::<_, _>( + block.clone(), + public_keys.clone(), + execution_witnesses.flatdb.clone(), + chain_spec.clone(), + EthEvmConfig::new(chain_spec.clone()), + ) + .expect("stateless validation with flatdb failed"); + + if trie_post_state != flatdb_post_state { + return Err(Error::Assertion( + "Post state mismatch between trie and flatdb implementations".to_string(), + )); + } + + // Validate that the flatdb used as pre-state can be proven using a sparse trie (i.e., in a + // different proof). Also checks that the post-state diff generated during flatdb + // execution results in the expected post-state root. + stateless_validation_flatdb_storage_check::( + block, + execution_witnesses.trie.clone(), + execution_witnesses.flatdb.state.clone(), + flatdb_post_state, + ) + .expect("stateless flatdb state check failed"); } Ok(program_inputs) @@ -510,8 +568,15 @@ fn path_contains(path_str: &str, rhs: &[&str]) -> bool { path_str.contains(&rhs) } -fn execution_witness_with_parent(parent: &RecoveredBlock) -> ExecutionWitness { +fn execution_witness_with_parent(parent: &RecoveredBlock) -> ExecutionWitnesses { let mut serialized_header = Vec::new(); parent.header().encode(&mut serialized_header); - ExecutionWitness { headers: vec![serialized_header.into()], ..Default::default() } + let trie_witness = + ExecutionWitness { headers: vec![serialized_header.clone().into()], ..Default::default() }; + let flatdb_witness = FlatExecutionWitness::new( + Default::default(), + HashMap::from_iter([(U256::from(parent.number), parent.hash())]), + serialized_header.into(), + ); + ExecutionWitnesses { trie: trie_witness, flatdb: flatdb_witness } } diff --git a/testing/ef-tests/src/result.rs b/testing/ef-tests/src/result.rs index 481d1fe770..59441eb5fe 100644 --- a/testing/ef-tests/src/result.rs +++ b/testing/ef-tests/src/result.rs @@ -1,11 +1,10 @@ //! Test results and errors -use crate::Case; +use crate::{cases::blockchain_test::ExecutionWitnesses, Case}; use reth_db::DatabaseError; use reth_ethereum_primitives::Block; use reth_primitives_traits::RecoveredBlock; use reth_provider::ProviderError; -use reth_stateless::ExecutionWitness; use std::path::{Path, PathBuf}; use thiserror::Error; @@ -29,7 +28,7 @@ pub enum Error { block_number: u64, /// Contains the inputs necessary for the block stateless validation guest program used in /// zkVMs to prove the block is invalid. - partial_program_inputs: Vec<(RecoveredBlock, ExecutionWitness)>, + partial_program_inputs: Vec<(RecoveredBlock, ExecutionWitnesses)>, /// The specific error #[source] err: Box, @@ -73,7 +72,7 @@ impl Error { /// Create a new [`Error::BlockProcessingFailed`] error. pub fn block_failed( block_number: u64, - partial_program_inputs: Vec<(RecoveredBlock, ExecutionWitness)>, + partial_program_inputs: Vec<(RecoveredBlock, ExecutionWitnesses)>, err: impl std::error::Error + Send + Sync + 'static, ) -> Self { Self::BlockProcessingFailed { block_number, partial_program_inputs, err: Box::new(err) }