reth_evm/
execute.rs

1//! Traits for execution.
2
3use crate::{ConfigureEvm, Database, OnStateHook, TxEnvFor};
4use alloc::{boxed::Box, vec::Vec};
5use alloy_consensus::{BlockHeader, Header};
6use alloy_eips::eip2718::WithEncoded;
7pub use alloy_evm::block::{BlockExecutor, BlockExecutorFactory};
8use alloy_evm::{
9    block::{CommitChanges, ExecutableTx},
10    Evm, EvmEnv, EvmFactory, RecoveredTx, ToTxEnv,
11};
12use alloy_primitives::{Address, B256};
13pub use reth_execution_errors::{
14    BlockExecutionError, BlockValidationError, InternalBlockExecutionError,
15};
16use reth_execution_types::BlockExecutionResult;
17pub use reth_execution_types::{BlockExecutionOutput, ExecutionOutcome};
18use reth_primitives_traits::{
19    Block, HeaderTy, NodePrimitives, ReceiptTy, Recovered, RecoveredBlock, SealedHeader, TxTy,
20};
21use reth_storage_api::StateProvider;
22pub use reth_storage_errors::provider::ProviderError;
23use reth_trie_common::{updates::TrieUpdates, HashedPostState};
24use revm::{
25    context::result::ExecutionResult,
26    database::{states::bundle_state::BundleRetention, BundleState, State},
27};
28
29/// A type that knows how to execute a block. It is assumed to operate on a
30/// [`crate::Evm`] internally and use [`State`] as database.
31pub trait Executor<DB: Database>: Sized {
32    /// The primitive types used by the executor.
33    type Primitives: NodePrimitives;
34    /// The error type returned by the executor.
35    type Error;
36
37    /// Executes a single block and returns [`BlockExecutionResult`], without the state changes.
38    fn execute_one(
39        &mut self,
40        block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
41    ) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>;
42
43    /// Executes the EVM with the given input and accepts a state hook closure that is invoked with
44    /// the EVM state after execution.
45    fn execute_one_with_state_hook<F>(
46        &mut self,
47        block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
48        state_hook: F,
49    ) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
50    where
51        F: OnStateHook + 'static;
52
53    /// Consumes the type and executes the block.
54    ///
55    /// # Note
56    /// Execution happens without any validation of the output.
57    ///
58    /// # Returns
59    /// The output of the block execution.
60    fn execute(
61        mut self,
62        block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
63    ) -> Result<BlockExecutionOutput<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
64    {
65        let result = self.execute_one(block)?;
66        let mut state = self.into_state();
67        Ok(BlockExecutionOutput { state: state.take_bundle(), result })
68    }
69
70    /// Executes multiple inputs in the batch, and returns an aggregated [`ExecutionOutcome`].
71    fn execute_batch<'a, I>(
72        mut self,
73        blocks: I,
74    ) -> Result<ExecutionOutcome<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
75    where
76        I: IntoIterator<Item = &'a RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>>,
77    {
78        let mut results = Vec::new();
79        let mut first_block = None;
80        for block in blocks {
81            if first_block.is_none() {
82                first_block = Some(block.header().number());
83            }
84            results.push(self.execute_one(block)?);
85        }
86
87        Ok(ExecutionOutcome::from_blocks(
88            first_block.unwrap_or_default(),
89            self.into_state().take_bundle(),
90            results,
91        ))
92    }
93
94    /// Executes the EVM with the given input and accepts a state closure that is invoked with
95    /// the EVM state after execution.
96    fn execute_with_state_closure<F>(
97        mut self,
98        block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
99        mut f: F,
100    ) -> Result<BlockExecutionOutput<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
101    where
102        F: FnMut(&State<DB>),
103    {
104        let result = self.execute_one(block)?;
105        let mut state = self.into_state();
106        f(&state);
107        Ok(BlockExecutionOutput { state: state.take_bundle(), result })
108    }
109
110    /// Executes the EVM with the given input and accepts a state hook closure that is invoked with
111    /// the EVM state after execution.
112    fn execute_with_state_hook<F>(
113        mut self,
114        block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
115        state_hook: F,
116    ) -> Result<BlockExecutionOutput<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
117    where
118        F: OnStateHook + 'static,
119    {
120        let result = self.execute_one_with_state_hook(block, state_hook)?;
121        let mut state = self.into_state();
122        Ok(BlockExecutionOutput { state: state.take_bundle(), result })
123    }
124
125    /// Consumes the executor and returns the [`State`] containing all state changes.
126    fn into_state(self) -> State<DB>;
127
128    /// The size hint of the batch's tracked state size.
129    ///
130    /// This is used to optimize DB commits depending on the size of the state.
131    fn size_hint(&self) -> usize;
132}
133
134/// Helper type for the output of executing a block.
135#[derive(Debug, Clone)]
136pub struct ExecuteOutput<R> {
137    /// Receipts obtained after executing a block.
138    pub receipts: Vec<R>,
139    /// Cumulative gas used in the block execution.
140    pub gas_used: u64,
141}
142
143/// Input for block building. Consumed by [`BlockAssembler`].
144///
145/// This struct contains all the data needed by the [`BlockAssembler`] to create
146/// a complete block after transaction execution.
147///
148/// # Fields Overview
149///
150/// - `evm_env`: The EVM configuration used during execution (spec ID, block env, etc.)
151/// - `execution_ctx`: Additional context like withdrawals and ommers
152/// - `parent`: The parent block header this block builds on
153/// - `transactions`: All transactions that were successfully executed
154/// - `output`: Execution results including receipts and gas used
155/// - `bundle_state`: Accumulated state changes from all transactions
156/// - `state_provider`: Access to the current state for additional lookups
157/// - `state_root`: The calculated state root after all changes
158///
159/// # Usage
160///
161/// This is typically created internally by [`BlockBuilder::finish`] after all
162/// transactions have been executed:
163///
164/// ```rust,ignore
165/// let input = BlockAssemblerInput {
166///     evm_env: builder.evm_env(),
167///     execution_ctx: builder.context(),
168///     parent: &parent_header,
169///     transactions: executed_transactions,
170///     output: &execution_result,
171///     bundle_state: &state_changes,
172///     state_provider: &state,
173///     state_root: calculated_root,
174/// };
175///
176/// let block = assembler.assemble_block(input)?;
177/// ```
178#[derive(derive_more::Debug)]
179#[non_exhaustive]
180pub struct BlockAssemblerInput<'a, 'b, F: BlockExecutorFactory, H = Header> {
181    /// Configuration of EVM used when executing the block.
182    ///
183    /// Contains context relevant to EVM such as [`revm::context::BlockEnv`].
184    pub evm_env: EvmEnv<<F::EvmFactory as EvmFactory>::Spec>,
185    /// [`BlockExecutorFactory::ExecutionCtx`] used to execute the block.
186    pub execution_ctx: F::ExecutionCtx<'a>,
187    /// Parent block header.
188    pub parent: &'a SealedHeader<H>,
189    /// Transactions that were executed in this block.
190    pub transactions: Vec<F::Transaction>,
191    /// Output of block execution.
192    pub output: &'b BlockExecutionResult<F::Receipt>,
193    /// [`BundleState`] after the block execution.
194    pub bundle_state: &'a BundleState,
195    /// Provider with access to state.
196    #[debug(skip)]
197    pub state_provider: &'b dyn StateProvider,
198    /// State root for this block.
199    pub state_root: B256,
200}
201
202/// A type that knows how to assemble a block from execution results.
203///
204/// The [`BlockAssembler`] is the final step in block production. After transactions
205/// have been executed by the [`BlockExecutor`], the assembler takes all the execution
206/// outputs and creates a properly formatted block.
207///
208/// # Responsibilities
209///
210/// The assembler is responsible for:
211/// - Setting the correct block header fields (gas used, receipts root, logs bloom, etc.)
212/// - Including the executed transactions in the correct order
213/// - Setting the state root from the post-execution state
214/// - Applying any chain-specific rules or adjustments
215///
216/// # Example Flow
217///
218/// ```rust,ignore
219/// // 1. Execute transactions and get results
220/// let execution_result = block_executor.finish()?;
221///
222/// // 2. Calculate state root from changes
223/// let state_root = state_provider.state_root(&bundle_state)?;
224///
225/// // 3. Assemble the final block
226/// let block = assembler.assemble_block(BlockAssemblerInput {
227///     evm_env,           // Environment used during execution
228///     execution_ctx,     // Context like withdrawals, ommers
229///     parent,            // Parent block header
230///     transactions,      // Executed transactions
231///     output,            // Execution results (receipts, gas)
232///     bundle_state,      // All state changes
233///     state_provider,    // For additional lookups if needed
234///     state_root,        // Computed state root
235/// })?;
236/// ```
237///
238/// # Relationship with Block Building
239///
240/// The assembler works together with:
241/// - `NextBlockEnvAttributes`: Provides the configuration for the new block
242/// - [`BlockExecutor`]: Executes transactions and produces results
243/// - [`BlockBuilder`]: Orchestrates the entire process and calls the assembler
244#[auto_impl::auto_impl(&, Arc)]
245pub trait BlockAssembler<F: BlockExecutorFactory> {
246    /// The block type produced by the assembler.
247    type Block: Block;
248
249    /// Builds a block. see [`BlockAssemblerInput`] documentation for more details.
250    fn assemble_block(
251        &self,
252        input: BlockAssemblerInput<'_, '_, F, <Self::Block as Block>::Header>,
253    ) -> Result<Self::Block, BlockExecutionError>;
254}
255
256/// Output of block building.
257#[derive(Debug, Clone)]
258pub struct BlockBuilderOutcome<N: NodePrimitives> {
259    /// Result of block execution.
260    pub execution_result: BlockExecutionResult<N::Receipt>,
261    /// Hashed state after execution.
262    pub hashed_state: HashedPostState,
263    /// Trie updates collected during state root calculation.
264    pub trie_updates: TrieUpdates,
265    /// The built block.
266    pub block: RecoveredBlock<N::Block>,
267}
268
269/// A type that knows how to execute and build a block.
270///
271/// It wraps an inner [`BlockExecutor`] and provides a way to execute transactions and
272/// construct a block.
273///
274/// This is a helper to erase `BasicBlockBuilder` type.
275pub trait BlockBuilder {
276    /// The primitive types used by the inner [`BlockExecutor`].
277    type Primitives: NodePrimitives;
278    /// Inner [`BlockExecutor`].
279    type Executor: BlockExecutor<
280        Transaction = TxTy<Self::Primitives>,
281        Receipt = ReceiptTy<Self::Primitives>,
282    >;
283
284    /// Invokes [`BlockExecutor::apply_pre_execution_changes`].
285    fn apply_pre_execution_changes(&mut self) -> Result<(), BlockExecutionError>;
286
287    /// Invokes [`BlockExecutor::execute_transaction_with_commit_condition`] and saves the
288    /// transaction in internal state only if the transaction was committed.
289    fn execute_transaction_with_commit_condition(
290        &mut self,
291        tx: impl ExecutorTx<Self::Executor>,
292        f: impl FnOnce(
293            &ExecutionResult<<<Self::Executor as BlockExecutor>::Evm as Evm>::HaltReason>,
294        ) -> CommitChanges,
295    ) -> Result<Option<u64>, BlockExecutionError>;
296
297    /// Invokes [`BlockExecutor::execute_transaction_with_result_closure`] and saves the
298    /// transaction in internal state.
299    fn execute_transaction_with_result_closure(
300        &mut self,
301        tx: impl ExecutorTx<Self::Executor>,
302        f: impl FnOnce(&ExecutionResult<<<Self::Executor as BlockExecutor>::Evm as Evm>::HaltReason>),
303    ) -> Result<u64, BlockExecutionError> {
304        self.execute_transaction_with_commit_condition(tx, |res| {
305            f(res);
306            CommitChanges::Yes
307        })
308        .map(Option::unwrap_or_default)
309    }
310
311    /// Invokes [`BlockExecutor::execute_transaction`] and saves the transaction in
312    /// internal state.
313    fn execute_transaction(
314        &mut self,
315        tx: impl ExecutorTx<Self::Executor>,
316    ) -> Result<u64, BlockExecutionError> {
317        self.execute_transaction_with_result_closure(tx, |_| ())
318    }
319
320    /// Completes the block building process and returns the [`BlockBuilderOutcome`].
321    fn finish(
322        self,
323        state_provider: impl StateProvider,
324    ) -> Result<BlockBuilderOutcome<Self::Primitives>, BlockExecutionError>;
325
326    /// Provides mutable access to the inner [`BlockExecutor`].
327    fn executor_mut(&mut self) -> &mut Self::Executor;
328
329    /// Provides access to the inner [`BlockExecutor`].
330    fn executor(&self) -> &Self::Executor;
331
332    /// Helper to access inner [`BlockExecutor::Evm`] mutably.
333    fn evm_mut(&mut self) -> &mut <Self::Executor as BlockExecutor>::Evm {
334        self.executor_mut().evm_mut()
335    }
336
337    /// Helper to access inner [`BlockExecutor::Evm`].
338    fn evm(&self) -> &<Self::Executor as BlockExecutor>::Evm {
339        self.executor().evm()
340    }
341
342    /// Consumes the type and returns the underlying [`BlockExecutor`].
343    fn into_executor(self) -> Self::Executor;
344}
345
346/// A type that constructs a block from transactions and execution results.
347#[derive(Debug)]
348pub struct BasicBlockBuilder<'a, F, Executor, Builder, N: NodePrimitives>
349where
350    F: BlockExecutorFactory,
351{
352    /// The block executor used to execute transactions.
353    pub executor: Executor,
354    /// The transactions executed in this block.
355    pub transactions: Vec<Recovered<TxTy<N>>>,
356    /// The parent block execution context.
357    pub ctx: F::ExecutionCtx<'a>,
358    /// The sealed parent block header.
359    pub parent: &'a SealedHeader<HeaderTy<N>>,
360    /// The assembler used to build the block.
361    pub assembler: Builder,
362}
363
364/// Conversions for executable transactions.
365pub trait ExecutorTx<Executor: BlockExecutor> {
366    /// Converts the transaction into [`ExecutableTx`].
367    fn as_executable(&self) -> impl ExecutableTx<Executor>;
368
369    /// Converts the transaction into [`Recovered`].
370    fn into_recovered(self) -> Recovered<Executor::Transaction>;
371}
372
373impl<Executor: BlockExecutor> ExecutorTx<Executor>
374    for WithEncoded<Recovered<Executor::Transaction>>
375{
376    fn as_executable(&self) -> impl ExecutableTx<Executor> {
377        self
378    }
379
380    fn into_recovered(self) -> Recovered<Executor::Transaction> {
381        self.1
382    }
383}
384
385impl<Executor: BlockExecutor> ExecutorTx<Executor> for Recovered<Executor::Transaction> {
386    fn as_executable(&self) -> impl ExecutableTx<Executor> {
387        self
388    }
389
390    fn into_recovered(self) -> Self {
391        self
392    }
393}
394
395impl<'a, F, DB, Executor, Builder, N> BlockBuilder
396    for BasicBlockBuilder<'a, F, Executor, Builder, N>
397where
398    F: BlockExecutorFactory<Transaction = N::SignedTx, Receipt = N::Receipt>,
399    Executor: BlockExecutor<
400        Evm: Evm<
401            Spec = <F::EvmFactory as EvmFactory>::Spec,
402            HaltReason = <F::EvmFactory as EvmFactory>::HaltReason,
403            DB = &'a mut State<DB>,
404        >,
405        Transaction = N::SignedTx,
406        Receipt = N::Receipt,
407    >,
408    DB: Database + 'a,
409    Builder: BlockAssembler<F, Block = N::Block>,
410    N: NodePrimitives,
411{
412    type Primitives = N;
413    type Executor = Executor;
414
415    fn apply_pre_execution_changes(&mut self) -> Result<(), BlockExecutionError> {
416        self.executor.apply_pre_execution_changes()
417    }
418
419    fn execute_transaction_with_commit_condition(
420        &mut self,
421        tx: impl ExecutorTx<Self::Executor>,
422        f: impl FnOnce(
423            &ExecutionResult<<<Self::Executor as BlockExecutor>::Evm as Evm>::HaltReason>,
424        ) -> CommitChanges,
425    ) -> Result<Option<u64>, BlockExecutionError> {
426        if let Some(gas_used) =
427            self.executor.execute_transaction_with_commit_condition(tx.as_executable(), f)?
428        {
429            self.transactions.push(tx.into_recovered());
430            Ok(Some(gas_used))
431        } else {
432            Ok(None)
433        }
434    }
435
436    fn finish(
437        self,
438        state: impl StateProvider,
439    ) -> Result<BlockBuilderOutcome<N>, BlockExecutionError> {
440        let (evm, result) = self.executor.finish()?;
441        let (db, evm_env) = evm.finish();
442
443        // merge all transitions into bundle state
444        db.merge_transitions(BundleRetention::Reverts);
445
446        // calculate the state root
447        let hashed_state = state.hashed_post_state(&db.bundle_state);
448        let (state_root, trie_updates) = state
449            .state_root_with_updates(hashed_state.clone())
450            .map_err(BlockExecutionError::other)?;
451
452        let (transactions, senders) =
453            self.transactions.into_iter().map(|tx| tx.into_parts()).unzip();
454
455        let block = self.assembler.assemble_block(BlockAssemblerInput {
456            evm_env,
457            execution_ctx: self.ctx,
458            parent: self.parent,
459            transactions,
460            output: &result,
461            bundle_state: &db.bundle_state,
462            state_provider: &state,
463            state_root,
464        })?;
465
466        let block = RecoveredBlock::new_unhashed(block, senders);
467
468        Ok(BlockBuilderOutcome { execution_result: result, hashed_state, trie_updates, block })
469    }
470
471    fn executor_mut(&mut self) -> &mut Self::Executor {
472        &mut self.executor
473    }
474
475    fn executor(&self) -> &Self::Executor {
476        &self.executor
477    }
478
479    fn into_executor(self) -> Self::Executor {
480        self.executor
481    }
482}
483
484/// A generic block executor that uses a [`BlockExecutor`] to
485/// execute blocks.
486#[expect(missing_debug_implementations)]
487pub struct BasicBlockExecutor<F, DB> {
488    /// Block execution strategy.
489    pub(crate) strategy_factory: F,
490    /// Database.
491    pub(crate) db: State<DB>,
492}
493
494impl<F, DB: Database> BasicBlockExecutor<F, DB> {
495    /// Creates a new `BasicBlockExecutor` with the given strategy.
496    pub fn new(strategy_factory: F, db: DB) -> Self {
497        let db =
498            State::builder().with_database(db).with_bundle_update().without_state_clear().build();
499        Self { strategy_factory, db }
500    }
501}
502
503impl<F, DB> Executor<DB> for BasicBlockExecutor<F, DB>
504where
505    F: ConfigureEvm,
506    DB: Database,
507{
508    type Primitives = F::Primitives;
509    type Error = BlockExecutionError;
510
511    fn execute_one(
512        &mut self,
513        block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
514    ) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
515    {
516        let result = self
517            .strategy_factory
518            .executor_for_block(&mut self.db, block)
519            .execute_block(block.transactions_recovered())?;
520
521        self.db.merge_transitions(BundleRetention::Reverts);
522
523        Ok(result)
524    }
525
526    fn execute_one_with_state_hook<H>(
527        &mut self,
528        block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
529        state_hook: H,
530    ) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
531    where
532        H: OnStateHook + 'static,
533    {
534        let result = self
535            .strategy_factory
536            .executor_for_block(&mut self.db, block)
537            .with_state_hook(Some(Box::new(state_hook)))
538            .execute_block(block.transactions_recovered())?;
539
540        self.db.merge_transitions(BundleRetention::Reverts);
541
542        Ok(result)
543    }
544
545    fn into_state(self) -> State<DB> {
546        self.db
547    }
548
549    fn size_hint(&self) -> usize {
550        self.db.bundle_state.size_hint()
551    }
552}
553
554/// A helper trait marking a 'static type that can be converted into an [`ExecutableTx`] for block
555/// executor.
556pub trait ExecutableTxFor<Evm: ConfigureEvm>:
557    ToTxEnv<TxEnvFor<Evm>> + RecoveredTx<TxTy<Evm::Primitives>>
558{
559}
560
561impl<T, Evm: ConfigureEvm> ExecutableTxFor<Evm> for T where
562    T: ToTxEnv<TxEnvFor<Evm>> + RecoveredTx<TxTy<Evm::Primitives>>
563{
564}
565
566/// A container for a transaction and a transaction environment.
567#[derive(Debug, Clone)]
568pub struct WithTxEnv<TxEnv, T> {
569    /// The transaction environment for EVM.
570    pub tx_env: TxEnv,
571    /// The recovered transaction.
572    pub tx: T,
573}
574
575impl<TxEnv, Tx, T: RecoveredTx<Tx>> RecoveredTx<Tx> for WithTxEnv<TxEnv, T> {
576    fn tx(&self) -> &Tx {
577        self.tx.tx()
578    }
579
580    fn signer(&self) -> &Address {
581        self.tx.signer()
582    }
583}
584
585impl<TxEnv: Clone, T> ToTxEnv<TxEnv> for WithTxEnv<TxEnv, T> {
586    fn to_tx_env(&self) -> TxEnv {
587        self.tx_env.clone()
588    }
589}
590
591#[cfg(test)]
592mod tests {
593    use super::*;
594    use crate::Address;
595    use alloy_consensus::constants::KECCAK_EMPTY;
596    use alloy_evm::block::state_changes::balance_increment_state;
597    use alloy_primitives::{address, map::HashMap, U256};
598    use core::marker::PhantomData;
599    use reth_ethereum_primitives::EthPrimitives;
600    use revm::{
601        database::{CacheDB, EmptyDB},
602        state::AccountInfo,
603    };
604
605    #[derive(Clone, Debug, Default)]
606    struct TestExecutorProvider;
607
608    impl TestExecutorProvider {
609        fn executor<DB>(&self, _db: DB) -> TestExecutor<DB>
610        where
611            DB: Database,
612        {
613            TestExecutor(PhantomData)
614        }
615    }
616
617    struct TestExecutor<DB>(PhantomData<DB>);
618
619    impl<DB: Database> Executor<DB> for TestExecutor<DB> {
620        type Primitives = EthPrimitives;
621        type Error = BlockExecutionError;
622
623        fn execute_one(
624            &mut self,
625            _block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
626        ) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
627        {
628            Err(BlockExecutionError::msg("execution unavailable for tests"))
629        }
630
631        fn execute_one_with_state_hook<F>(
632            &mut self,
633            _block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
634            _state_hook: F,
635        ) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
636        where
637            F: OnStateHook + 'static,
638        {
639            Err(BlockExecutionError::msg("execution unavailable for tests"))
640        }
641
642        fn into_state(self) -> State<DB> {
643            unreachable!()
644        }
645
646        fn size_hint(&self) -> usize {
647            0
648        }
649    }
650
651    #[test]
652    fn test_provider() {
653        let provider = TestExecutorProvider;
654        let db = CacheDB::<EmptyDB>::default();
655        let executor = provider.executor(db);
656        let _ = executor.execute(&Default::default());
657    }
658
659    fn setup_state_with_account(
660        addr: Address,
661        balance: u128,
662        nonce: u64,
663    ) -> State<CacheDB<EmptyDB>> {
664        let db = CacheDB::<EmptyDB>::default();
665        let mut state = State::builder().with_database(db).with_bundle_update().build();
666
667        let account_info = AccountInfo {
668            balance: U256::from(balance),
669            nonce,
670            code_hash: KECCAK_EMPTY,
671            code: None,
672        };
673        state.insert_account(addr, account_info);
674        state
675    }
676
677    #[test]
678    fn test_balance_increment_state_zero() {
679        let addr = address!("0x1000000000000000000000000000000000000000");
680        let mut state = setup_state_with_account(addr, 100, 1);
681
682        let mut increments = HashMap::default();
683        increments.insert(addr, 0);
684
685        let result = balance_increment_state(&increments, &mut state).unwrap();
686        assert!(result.is_empty(), "Zero increments should be ignored");
687    }
688
689    #[test]
690    fn test_balance_increment_state_empty_increments_map() {
691        let mut state = State::builder()
692            .with_database(CacheDB::<EmptyDB>::default())
693            .with_bundle_update()
694            .build();
695
696        let increments = HashMap::default();
697        let result = balance_increment_state(&increments, &mut state).unwrap();
698        assert!(result.is_empty(), "Empty increments map should return empty state");
699    }
700
701    #[test]
702    fn test_balance_increment_state_multiple_valid_increments() {
703        let addr1 = address!("0x1000000000000000000000000000000000000000");
704        let addr2 = address!("0x2000000000000000000000000000000000000000");
705
706        let mut state = setup_state_with_account(addr1, 100, 1);
707
708        let account2 =
709            AccountInfo { balance: U256::from(200), nonce: 1, code_hash: KECCAK_EMPTY, code: None };
710        state.insert_account(addr2, account2);
711
712        let mut increments = HashMap::default();
713        increments.insert(addr1, 50);
714        increments.insert(addr2, 100);
715
716        let result = balance_increment_state(&increments, &mut state).unwrap();
717
718        assert_eq!(result.len(), 2);
719        assert_eq!(result.get(&addr1).unwrap().info.balance, U256::from(100));
720        assert_eq!(result.get(&addr2).unwrap().info.balance, U256::from(200));
721    }
722
723    #[test]
724    fn test_balance_increment_state_mixed_zero_and_nonzero_increments() {
725        let addr1 = address!("0x1000000000000000000000000000000000000000");
726        let addr2 = address!("0x2000000000000000000000000000000000000000");
727
728        let mut state = setup_state_with_account(addr1, 100, 1);
729
730        let account2 =
731            AccountInfo { balance: U256::from(200), nonce: 1, code_hash: KECCAK_EMPTY, code: None };
732        state.insert_account(addr2, account2);
733
734        let mut increments = HashMap::default();
735        increments.insert(addr1, 0);
736        increments.insert(addr2, 100);
737
738        let result = balance_increment_state(&increments, &mut state).unwrap();
739
740        assert_eq!(result.len(), 1, "Only non-zero increments should be included");
741        assert!(!result.contains_key(&addr1), "Zero increment account should not be included");
742        assert_eq!(result.get(&addr2).unwrap().info.balance, U256::from(200));
743    }
744}