reth_stateless/
validation.rs

1use crate::{witness_db::WitnessDatabase, ExecutionWitness};
2use alloc::{
3    boxed::Box,
4    collections::BTreeMap,
5    fmt::Debug,
6    string::{String, ToString},
7    sync::Arc,
8    vec::Vec,
9};
10use alloy_consensus::{BlockHeader, Header};
11use alloy_primitives::{keccak256, map::B256Map, B256};
12use alloy_rlp::Decodable;
13use reth_chainspec::{EthChainSpec, EthereumHardforks};
14use reth_consensus::{Consensus, HeaderValidator};
15use reth_errors::ConsensusError;
16use reth_ethereum_consensus::{validate_block_post_execution, EthBeaconConsensus};
17use reth_ethereum_primitives::{Block, EthPrimitives};
18use reth_evm::{execute::Executor, ConfigureEvm};
19use reth_primitives_traits::{block::error::BlockRecoveryError, Block as _, RecoveredBlock};
20use reth_revm::state::Bytecode;
21use reth_trie_common::{HashedPostState, KeccakKeyHasher};
22use reth_trie_sparse::{blinded::DefaultBlindedProviderFactory, SparseStateTrie};
23
24/// Errors that can occur during stateless validation.
25#[derive(Debug, thiserror::Error)]
26pub enum StatelessValidationError {
27    /// Error when the number of ancestor headers exceeds the limit.
28    #[error("ancestor header count ({count}) exceeds limit ({limit})")]
29    AncestorHeaderLimitExceeded {
30        /// The number of headers provided.
31        count: usize,
32        /// The limit.
33        limit: usize,
34    },
35
36    /// Error when the ancestor headers do not form a contiguous chain.
37    #[error("invalid ancestor chain")]
38    InvalidAncestorChain,
39
40    /// Error when revealing the witness data failed.
41    #[error("failed to reveal witness data for pre-state root {pre_state_root}")]
42    WitnessRevealFailed {
43        /// The pre-state root used for verification.
44        pre_state_root: B256,
45    },
46
47    /// Error during stateless block execution.
48    #[error("stateless block execution failed")]
49    StatelessExecutionFailed(String),
50
51    /// Error during consensus validation of the block.
52    #[error("consensus validation failed: {0}")]
53    ConsensusValidationFailed(#[from] ConsensusError),
54
55    /// Error during stateless state root calculation.
56    #[error("stateless state root calculation failed")]
57    StatelessStateRootCalculationFailed,
58
59    /// Error calculating the pre-state root from the witness data.
60    #[error("stateless pre-state root calculation failed")]
61    StatelessPreStateRootCalculationFailed,
62
63    /// Error when required ancestor headers are missing (e.g., parent header for pre-state root).
64    #[error("missing required ancestor headers")]
65    MissingAncestorHeader,
66
67    /// Error when deserializing ancestor headers
68    #[error("could not deserialize ancestor headers")]
69    HeaderDeserializationFailed,
70
71    /// Error when the computed state root does not match the one in the block header.
72    #[error("mismatched post- state root: {got}\n {expected}")]
73    PostStateRootMismatch {
74        /// The computed post-state root
75        got: B256,
76        /// The expected post-state root; in the block header
77        expected: B256,
78    },
79
80    /// Error when the computed pre-state root does not match the expected one.
81    #[error("mismatched pre-state root: {got} \n {expected}")]
82    PreStateRootMismatch {
83        /// The computed pre-state root
84        got: B256,
85        /// The expected pre-state root from the previous block
86        expected: B256,
87    },
88
89    /// Error when recovering signers
90    #[error("error recovering the signers in the block")]
91    SignerRecovery(#[from] Box<BlockRecoveryError<Block>>),
92}
93
94/// Performs stateless validation of a block using the provided witness data.
95///
96/// This function attempts to fully validate a given `current_block` statelessly, ie without access
97/// to a persistent database.
98/// It relies entirely on the `witness` data and `ancestor_headers`
99/// provided alongside the block.
100///
101/// The witness data is validated in the following way:
102///
103/// 1. **Ancestor Header Verification:** Checks if the `ancestor_headers` are present, form a
104///    contiguous chain back from `current_block`'s parent, and do not exceed the `BLOCKHASH` opcode
105///    limit using `compute_ancestor_hashes`. We must have at least one ancestor header, even if the
106///    `BLOCKHASH` opcode is not used because we need the state root of the previous block to verify
107///    the pre state reads.
108///
109/// 2. **Pre-State Verification:** Retrieves the expected `pre_state_root` from the parent header
110///    from `ancestor_headers`. Verifies the provided [`ExecutionWitness`] against this root using
111///    [`verify_execution_witness`].
112///
113/// 3. **Chain Verification:** The code currently does not verify the [`EthChainSpec`] and expects a
114///    higher level function to assert that this is correct by, for example, asserting that it is
115///    equal to the Ethereum Mainnet `ChainSpec` or asserting against the genesis hash that this
116///    `ChainSpec` defines.
117///
118/// High Level Overview of functionality:
119///
120/// - Verify all state accesses against a trusted pre-state root
121/// - Put all state accesses into an in-memory database
122/// - Use the in-memory database to execute the block
123/// - Validate the output of block execution (e.g. receipts, logs, requests)
124/// - Compute the post-state root using the state-diff from block execution
125/// - Check that the post-state root is the state root in the block.
126///
127/// If all steps succeed the function returns `Some` containing the hash of the validated
128/// `current_block`.
129pub fn stateless_validation<ChainSpec, E>(
130    current_block: Block,
131    witness: ExecutionWitness,
132    chain_spec: Arc<ChainSpec>,
133    evm_config: E,
134) -> Result<B256, StatelessValidationError>
135where
136    ChainSpec: Send + Sync + EthChainSpec + EthereumHardforks + Debug,
137    E: ConfigureEvm<Primitives = EthPrimitives> + Clone + 'static,
138{
139    let current_block = current_block
140        .try_into_recovered()
141        .map_err(|err| StatelessValidationError::SignerRecovery(Box::new(err)))?;
142
143    let mut ancestor_headers: Vec<Header> = witness
144        .headers
145        .iter()
146        .map(|serialized_header| {
147            let bytes = serialized_header.as_ref();
148            Header::decode(&mut &bytes[..])
149                .map_err(|_| StatelessValidationError::HeaderDeserializationFailed)
150        })
151        .collect::<Result<_, _>>()?;
152    // Sort the headers by their block number to ensure that they are in
153    // ascending order.
154    ancestor_headers.sort_by_key(|header| header.number());
155
156    // Validate block against pre-execution consensus rules
157    validate_block_consensus(chain_spec.clone(), &current_block)?;
158
159    // Check that the ancestor headers form a contiguous chain and are not just random headers.
160    let ancestor_hashes = compute_ancestor_hashes(&current_block, &ancestor_headers)?;
161
162    // Get the last ancestor header and retrieve its state root.
163    //
164    // There should be at least one ancestor header, this is because we need the parent header to
165    // retrieve the previous state root.
166    // The edge case here would be the genesis block, but we do not create proofs for the genesis
167    // block.
168    let pre_state_root = match ancestor_headers.last() {
169        Some(prev_header) => prev_header.state_root,
170        None => return Err(StatelessValidationError::MissingAncestorHeader),
171    };
172
173    // First verify that the pre-state reads are correct
174    let (mut sparse_trie, bytecode) = verify_execution_witness(&witness, pre_state_root)?;
175
176    // Create an in-memory database that will use the reads to validate the block
177    let db = WitnessDatabase::new(&sparse_trie, bytecode, ancestor_hashes);
178
179    // Execute the block
180    let executor = evm_config.executor(db);
181    let output = executor
182        .execute(&current_block)
183        .map_err(|e| StatelessValidationError::StatelessExecutionFailed(e.to_string()))?;
184
185    // Post validation checks
186    validate_block_post_execution(&current_block, &chain_spec, &output.receipts, &output.requests)
187        .map_err(StatelessValidationError::ConsensusValidationFailed)?;
188
189    // Compute and check the post state root
190    let hashed_state = HashedPostState::from_bundle_state::<KeccakKeyHasher>(&output.state.state);
191    let state_root = crate::root::calculate_state_root(&mut sparse_trie, hashed_state)
192        .map_err(|_e| StatelessValidationError::StatelessStateRootCalculationFailed)?;
193    if state_root != current_block.state_root {
194        return Err(StatelessValidationError::PostStateRootMismatch {
195            got: state_root,
196            expected: current_block.state_root,
197        });
198    }
199
200    // Return block hash
201    Ok(current_block.hash_slow())
202}
203
204/// Performs consensus validation checks on a block without execution or state validation.
205///
206/// This function validates a block against Ethereum consensus rules by:
207///
208/// 1. **Difficulty Validation:** Validates the header with total difficulty to verify proof-of-work
209///    (pre-merge) or to enforce post-merge requirements.
210///
211/// 2. **Header Validation:** Validates the sealed header against protocol specifications,
212///    including:
213///    - Gas limit checks
214///    - Base fee validation for EIP-1559
215///    - Withdrawals root validation for Shanghai fork
216///    - Blob-related fields validation for Cancun fork
217///
218/// 3. **Pre-Execution Validation:** Validates block structure, transaction format, signature
219///    validity, and other pre-execution requirements.
220///
221/// This function acts as a preliminary validation before executing and validating the state
222/// transition function.
223fn validate_block_consensus<ChainSpec>(
224    chain_spec: Arc<ChainSpec>,
225    block: &RecoveredBlock<Block>,
226) -> Result<(), StatelessValidationError>
227where
228    ChainSpec: Send + Sync + EthChainSpec + EthereumHardforks + Debug,
229{
230    let consensus = EthBeaconConsensus::new(chain_spec);
231
232    consensus.validate_header(block.sealed_header())?;
233
234    consensus.validate_block_pre_execution(block)?;
235
236    Ok(())
237}
238
239/// Verifies execution witness [`ExecutionWitness`] against an expected pre-state root.
240///
241/// This function takes the RLP-encoded values provided in [`ExecutionWitness`]
242/// (which includes state trie nodes, storage trie nodes, and contract bytecode)
243/// and uses it to populate a new [`SparseStateTrie`].
244///
245/// If the computed root hash matches the `pre_state_root`, it signifies that the
246/// provided execution witness is consistent with that pre-state root. In this case, the function
247/// returns the populated [`SparseStateTrie`] and a [`B256Map`] containing the
248/// contract bytecode (mapping code hash to [`Bytecode`]).
249///
250/// The bytecode has a separate mapping because the [`SparseStateTrie`] does not store the
251/// contract bytecode, only the hash of it (code hash).
252///
253/// If the roots do not match, it returns `None`, indicating the witness is invalid
254/// for the given `pre_state_root`.
255// Note: This approach might be inefficient for ZKVMs requiring minimal memory operations, which
256// would explain why they have for the most part re-implemented this function.
257pub fn verify_execution_witness(
258    witness: &ExecutionWitness,
259    pre_state_root: B256,
260) -> Result<(SparseStateTrie, B256Map<Bytecode>), StatelessValidationError> {
261    let mut trie = SparseStateTrie::new(DefaultBlindedProviderFactory);
262    let mut state_witness = B256Map::default();
263    let mut bytecode = B256Map::default();
264
265    for rlp_encoded in &witness.state {
266        let hash = keccak256(rlp_encoded);
267        state_witness.insert(hash, rlp_encoded.clone());
268    }
269    for rlp_encoded in &witness.codes {
270        let hash = keccak256(rlp_encoded);
271        bytecode.insert(hash, Bytecode::new_raw(rlp_encoded.clone()));
272    }
273
274    // Reveal the witness with our state root
275    // This method builds a trie using the sparse trie using the state_witness with
276    // the root being the pre_state_root.
277    // Here are some things to note:
278    // - You can pass in more witnesses than is needed for the block execution.
279    // - If you try to get an account and it has not been seen. This means that the account
280    // was not inserted into the Trie. It does not mean that the account does not exist.
281    // In order to determine an account not existing, we must do an exclusion proof.
282    trie.reveal_witness(pre_state_root, &state_witness)
283        .map_err(|_e| StatelessValidationError::WitnessRevealFailed { pre_state_root })?;
284
285    // Calculate the root
286    let computed_root = trie
287        .root()
288        .map_err(|_e| StatelessValidationError::StatelessPreStateRootCalculationFailed)?;
289
290    if computed_root == pre_state_root {
291        Ok((trie, bytecode))
292    } else {
293        Err(StatelessValidationError::PreStateRootMismatch {
294            got: computed_root,
295            expected: pre_state_root,
296        })
297    }
298}
299
300/// Verifies the contiguity, number of ancestor headers and extracts their hashes.
301///
302/// This function is used to prepare the data required for the `BLOCKHASH`
303/// opcode in a stateless execution context.
304///
305/// It verifies that the provided `ancestor_headers` form a valid, unbroken chain leading back from
306///    the parent of the `current_block`.
307///
308/// Note: This function becomes obsolete if EIP-2935 is implemented.
309/// Note: The headers are assumed to be in ascending order.
310///
311/// If both checks pass, it returns a [`BTreeMap`] mapping the block number of each
312/// ancestor header to its corresponding block hash.
313fn compute_ancestor_hashes(
314    current_block: &RecoveredBlock<Block>,
315    ancestor_headers: &[Header],
316) -> Result<BTreeMap<u64, B256>, StatelessValidationError> {
317    let mut ancestor_hashes = BTreeMap::new();
318
319    let mut child_header = current_block.header();
320
321    // Next verify that headers supplied are contiguous
322    for parent_header in ancestor_headers.iter().rev() {
323        let parent_hash = child_header.parent_hash();
324        ancestor_hashes.insert(parent_header.number, parent_hash);
325
326        if parent_hash != parent_header.hash_slow() {
327            return Err(StatelessValidationError::InvalidAncestorChain); // Blocks must be contiguous
328        }
329
330        if parent_header.number + 1 != child_header.number {
331            return Err(StatelessValidationError::InvalidAncestorChain); // Header number should be
332                                                                        // contiguous
333        }
334
335        child_header = parent_header
336    }
337
338    Ok(ancestor_hashes)
339}