anvil/eth/backend/mem/
mod.rs

1//! In-memory blockchain backend.
2
3use self::state::trie_storage;
4use super::executor::new_evm_with_inspector_ref;
5use crate::{
6    ForkChoice, NodeConfig, PrecompileFactory,
7    config::PruneStateHistoryConfig,
8    eth::{
9        backend::{
10            cheats::{CheatEcrecover, CheatsManager},
11            db::{Db, MaybeFullDatabase, SerializableState},
12            env::Env,
13            executor::{ExecutedTransactions, TransactionExecutor},
14            fork::ClientFork,
15            genesis::GenesisConfig,
16            mem::{
17                state::{storage_root, trie_accounts},
18                storage::MinedTransactionReceipt,
19            },
20            notifications::{NewBlockNotification, NewBlockNotifications},
21            time::{TimeManager, utc_from_secs},
22            validate::TransactionValidator,
23        },
24        error::{BlockchainError, ErrDetail, InvalidTransactionError},
25        fees::{FeeDetails, FeeManager, MIN_SUGGESTED_PRIORITY_FEE},
26        macros::node_info,
27        pool::transactions::PoolTransaction,
28        sign::build_typed_transaction,
29    },
30    inject_custom_precompiles,
31    mem::{
32        inspector::AnvilInspector,
33        storage::{BlockchainStorage, InMemoryBlockStates, MinedBlockOutcome},
34    },
35};
36use alloy_chains::NamedChain;
37use alloy_consensus::{
38    Account, Blob, BlockHeader, EnvKzgSettings, Header, Receipt, ReceiptWithBloom, Signed,
39    Transaction as TransactionTrait, TxEnvelope,
40    proofs::{calculate_receipt_root, calculate_transaction_root},
41    transaction::Recovered,
42};
43use alloy_eip5792::{Capabilities, DelegationCapability};
44use alloy_eips::{
45    eip1559::BaseFeeParams, eip4844::kzg_to_versioned_hash, eip7840::BlobParams,
46    eip7910::SystemContract,
47};
48use alloy_evm::{
49    Database, Evm,
50    eth::EthEvmContext,
51    overrides::{OverrideBlockHashes, apply_state_overrides},
52    precompiles::{DynPrecompile, Precompile, PrecompilesMap},
53};
54use alloy_network::{
55    AnyHeader, AnyRpcBlock, AnyRpcHeader, AnyRpcTransaction, AnyTxEnvelope, AnyTxType,
56    EthereumWallet, UnknownTxEnvelope, UnknownTypedTransaction,
57};
58use alloy_primitives::{
59    Address, B256, Bytes, TxHash, TxKind, U64, U256, address, hex, keccak256, logs_bloom,
60    map::HashMap,
61};
62use alloy_rpc_types::{
63    AccessList, Block as AlloyBlock, BlockId, BlockNumberOrTag as BlockNumber, BlockTransactions,
64    EIP1186AccountProofResponse as AccountProof, EIP1186StorageProof as StorageProof, Filter,
65    Header as AlloyHeader, Index, Log, Transaction, TransactionReceipt,
66    anvil::Forking,
67    request::TransactionRequest,
68    serde_helpers::JsonStorageKey,
69    simulate::{SimBlock, SimCallResult, SimulatePayload, SimulatedBlock},
70    state::EvmOverrides,
71    trace::{
72        filter::TraceFilter,
73        geth::{
74            GethDebugBuiltInTracerType, GethDebugTracerType, GethDebugTracingCallOptions,
75            GethDebugTracingOptions, GethTrace, NoopFrame,
76        },
77        parity::LocalizedTransactionTrace,
78    },
79};
80use alloy_serde::{OtherFields, WithOtherFields};
81use alloy_signer::Signature;
82use alloy_signer_local::PrivateKeySigner;
83use alloy_trie::{HashBuilder, Nibbles, proof::ProofRetainer};
84use anvil_core::eth::{
85    block::{Block, BlockInfo},
86    transaction::{
87        DepositReceipt, MaybeImpersonatedTransaction, PendingTransaction, ReceiptResponse,
88        TransactionInfo, TypedReceipt, TypedTransaction, has_optimism_fields,
89        transaction_request_to_typed,
90    },
91    wallet::WalletCapabilities,
92};
93use anvil_rpc::error::RpcError;
94use chrono::Datelike;
95use eyre::{Context, Result};
96use flate2::{Compression, read::GzDecoder, write::GzEncoder};
97use foundry_evm::{
98    backend::{DatabaseError, DatabaseResult, RevertStateSnapshotAction},
99    constants::DEFAULT_CREATE2_DEPLOYER_RUNTIME_CODE,
100    decode::RevertDecoder,
101    inspectors::AccessListInspector,
102    traces::{CallTraceDecoder, TracingInspectorConfig},
103    utils::{get_blob_base_fee_update_fraction, get_blob_base_fee_update_fraction_by_spec_id},
104};
105use foundry_evm_core::{either_evm::EitherEvm, precompiles::EC_RECOVER};
106use futures::channel::mpsc::{UnboundedSender, unbounded};
107use op_alloy_consensus::DEPOSIT_TX_TYPE_ID;
108use op_revm::{
109    OpContext, OpHaltReason, OpTransaction, transaction::deposit::DepositTransactionParts,
110};
111use parking_lot::{Mutex, RwLock};
112use revm::{
113    DatabaseCommit, Inspector,
114    context::{Block as RevmBlock, BlockEnv, Cfg, TxEnv},
115    context_interface::{
116        block::BlobExcessGasAndPrice,
117        result::{ExecutionResult, Output, ResultAndState},
118    },
119    database::{CacheDB, WrapDatabaseRef},
120    interpreter::InstructionResult,
121    precompile::{PrecompileSpecId, Precompiles},
122    primitives::{KECCAK_EMPTY, hardfork::SpecId},
123    state::AccountInfo,
124};
125use std::{
126    collections::BTreeMap,
127    fmt::Debug,
128    io::{Read, Write},
129    ops::Not,
130    path::PathBuf,
131    sync::Arc,
132    time::Duration,
133};
134use storage::{Blockchain, DEFAULT_HISTORY_LIMIT, MinedTransaction};
135use tokio::sync::RwLock as AsyncRwLock;
136
137pub mod cache;
138pub mod fork_db;
139pub mod in_memory_db;
140pub mod inspector;
141pub mod state;
142pub mod storage;
143
144/// Helper trait that combines revm::DatabaseRef with Debug.
145/// This is needed because alloy-evm requires Debug on Database implementations.
146/// With trait upcasting now stable, we can now upcast from this trait to revm::DatabaseRef.
147pub trait DatabaseRef: revm::DatabaseRef<Error = DatabaseError> + Debug {}
148impl<T> DatabaseRef for T where T: revm::DatabaseRef<Error = DatabaseError> + Debug {}
149impl DatabaseRef for dyn crate::eth::backend::db::Db {}
150
151// Gas per transaction not creating a contract.
152pub const MIN_TRANSACTION_GAS: u128 = 21000;
153// Gas per transaction creating a contract.
154pub const MIN_CREATE_GAS: u128 = 53000;
155// Executor
156pub const EXECUTOR: Address = address!("0x6634F723546eCc92277e8a2F93d4f248bf1189ea");
157pub const EXECUTOR_PK: &str = "0x502d47e1421cb9abef497096728e69f07543232b93ef24de4998e18b5fd9ba0f";
158// Experimental ERC20
159pub const EXP_ERC20_CONTRACT: Address = address!("0x238c8CD93ee9F8c7Edf395548eF60c0d2e46665E");
160// Runtime code of the experimental ERC20 contract
161pub const EXP_ERC20_RUNTIME_CODE: &[u8] = &hex!(
162    "60806040526004361015610010575b005b5f3560e01c806306fdde03146106f7578063095ea7b31461068c57806318160ddd1461066757806323b872dd146105a15780632bb7c5951461050e578063313ce567146104f35780633644e5151461045557806340c10f191461043057806370a08231146103fe5780637ecebe00146103cc57806395d89b4114610366578063a9059cbb146102ea578063ad0c8fdd146102ad578063d505accf146100fb5763dd62ed3e0361000e57346100f75760403660031901126100f7576100d261075c565b6100da610772565b602052637f5e9f20600c525f5260206034600c2054604051908152f35b5f80fd5b346100f75760e03660031901126100f75761011461075c565b61011c610772565b6084359160643560443560ff851685036100f757610138610788565b60208101906e04578706572696d656e74455243323608c1b8252519020908242116102a0576040519360018060a01b03169460018060a01b03169565383775081901600e52855f5260c06020600c20958654957f8b73c3c69bb8fe3d512ecc4cf759cc79239f7b179b0ffacaa9a75d522b39400f8252602082019586528660408301967fc89efdaa54c0f20c7adf612882df0950f5a951637e0307cdcb4c672f298b8bc688528b6060850198468a528c608087019330855260a08820602e527f6e71edae12b1b97f4d1f60370fef10105fa2faae0126114a169c64845d6126c9885252528688525260a082015220604e526042602c205f5260ff1660205260a43560405260c43560605260208060805f60015afa93853d5103610293577f8c5be1e5ebec7d5bd14f71427d1e84f3dd0314c0f7b2291e5b200ac8c7c3b92594602094019055856303faf4f960a51b176040526034602c2055a3005b63ddafbaef5f526004601cfd5b631a15a3cc5f526004601cfd5b5f3660031901126100f7576103e834023481046103e814341517156102d65761000e90336107ac565b634e487b7160e01b5f52601160045260245ffd5b346100f75760403660031901126100f75761030361075c565b602435906387a211a2600c52335f526020600c2080548084116103595783900390555f526020600c20818154019055602052600c5160601c335f51602061080d5f395f51905f52602080a3602060405160018152f35b63f4d678b85f526004601cfd5b346100f7575f3660031901126100f757604051604081019080821067ffffffffffffffff8311176103b8576103b491604052600381526204558560ec1b602082015260405191829182610732565b0390f35b634e487b7160e01b5f52604160045260245ffd5b346100f75760203660031901126100f7576103e561075c565b6338377508600c525f52602080600c2054604051908152f35b346100f75760203660031901126100f75761041761075c565b6387a211a2600c525f52602080600c2054604051908152f35b346100f75760403660031901126100f75761000e61044c61075c565b602435906107ac565b346100f7575f3660031901126100f757602060a0610471610788565b828101906e04578706572696d656e74455243323608c1b8252519020604051907f8b73c3c69bb8fe3d512ecc4cf759cc79239f7b179b0ffacaa9a75d522b39400f8252838201527fc89efdaa54c0f20c7adf612882df0950f5a951637e0307cdcb4c672f298b8bc6604082015246606082015230608082015220604051908152f35b346100f7575f3660031901126100f757602060405160128152f35b346100f75760203660031901126100f7576004356387a211a2600c52335f526020600c2090815490818111610359575f80806103e88487839688039055806805345cdf77eb68f44c54036805345cdf77eb68f44c5580835282335f51602061080d5f395f51905f52602083a304818115610598575b3390f11561058d57005b6040513d5f823e3d90fd5b506108fc610583565b346100f75760603660031901126100f7576105ba61075c565b6105c2610772565b604435908260601b33602052637f5e9f208117600c526034600c20908154918219610643575b506387a211a2915017600c526020600c2080548084116103595783900390555f526020600c20818154019055602052600c5160601c9060018060a01b03165f51602061080d5f395f51905f52602080a3602060405160018152f35b82851161065a57846387a211a293039055856105e8565b6313be252b5f526004601cfd5b346100f7575f3660031901126100f75760206805345cdf77eb68f44c54604051908152f35b346100f75760403660031901126100f7576106a561075c565b60243590602052637f5e9f20600c52335f52806034600c20555f52602c5160601c337f8c5be1e5ebec7d5bd14f71427d1e84f3dd0314c0f7b2291e5b200ac8c7c3b92560205fa3602060405160018152f35b346100f7575f3660031901126100f7576103b4610712610788565b6e04578706572696d656e74455243323608c1b6020820152604051918291825b602060409281835280519182918282860152018484015e5f828201840152601f01601f1916010190565b600435906001600160a01b03821682036100f757565b602435906001600160a01b03821682036100f757565b604051906040820182811067ffffffffffffffff8211176103b857604052600f8252565b6805345cdf77eb68f44c548281019081106107ff576805345cdf77eb68f44c556387a211a2600c525f526020600c20818154019055602052600c5160601c5f5f51602061080d5f395f51905f52602080a3565b63e5cfe9575f526004601cfdfeddf252ad1be2c89b69c2b068fc378daa952ba7f163c4a11628f55a4df523b3efa2646970667358221220fbe302881d9891005ba1448ba48547cc1cb17dea1a5c4011dfcb035de325bb1d64736f6c634300081b0033"
163);
164
165pub type State = foundry_evm::utils::StateChangeset;
166
167/// A block request, which includes the Pool Transactions if it's Pending
168#[derive(Debug)]
169pub enum BlockRequest {
170    Pending(Vec<Arc<PoolTransaction>>),
171    Number(u64),
172}
173
174impl BlockRequest {
175    pub fn block_number(&self) -> BlockNumber {
176        match *self {
177            Self::Pending(_) => BlockNumber::Pending,
178            Self::Number(n) => BlockNumber::Number(n),
179        }
180    }
181}
182
183/// Gives access to the [revm::Database]
184#[derive(Clone, Debug)]
185pub struct Backend {
186    /// Access to [`revm::Database`] abstraction.
187    ///
188    /// This will be used in combination with [`alloy_evm::Evm`] and is responsible for feeding
189    /// data to the evm during its execution.
190    ///
191    /// At time of writing, there are two different types of `Db`:
192    ///   - [`MemDb`](crate::mem::in_memory_db::MemDb): everything is stored in memory
193    ///   - [`ForkDb`](crate::mem::fork_db::ForkedDatabase): forks off a remote client, missing
194    ///     data is retrieved via RPC-calls
195    ///
196    /// In order to commit changes to the [`revm::Database`], the [`alloy_evm::Evm`] requires
197    /// mutable access, which requires a write-lock from this `db`. In forking mode, the time
198    /// during which the write-lock is active depends on whether the `ForkDb` can provide all
199    /// requested data from memory or whether it has to retrieve it via RPC calls first. This
200    /// means that it potentially blocks for some time, even taking into account the rate
201    /// limits of RPC endpoints. Therefore the `Db` is guarded by a `tokio::sync::RwLock` here
202    /// so calls that need to read from it, while it's currently written to, don't block. E.g.
203    /// a new block is currently mined and a new [`Self::set_storage_at()`] request is being
204    /// executed.
205    db: Arc<AsyncRwLock<Box<dyn Db>>>,
206    /// stores all block related data in memory.
207    blockchain: Blockchain,
208    /// Historic states of previous blocks.
209    states: Arc<RwLock<InMemoryBlockStates>>,
210    /// Env data of the chain
211    env: Arc<RwLock<Env>>,
212    /// This is set if this is currently forked off another client.
213    fork: Arc<RwLock<Option<ClientFork>>>,
214    /// Provides time related info, like timestamp.
215    time: TimeManager,
216    /// Contains state of custom overrides.
217    cheats: CheatsManager,
218    /// Contains fee data.
219    fees: FeeManager,
220    /// Initialised genesis.
221    genesis: GenesisConfig,
222    /// Listeners for new blocks that get notified when a new block was imported.
223    new_block_listeners: Arc<Mutex<Vec<UnboundedSender<NewBlockNotification>>>>,
224    /// Keeps track of active state snapshots at a specific block.
225    active_state_snapshots: Arc<Mutex<HashMap<U256, (u64, B256)>>>,
226    enable_steps_tracing: bool,
227    print_logs: bool,
228    print_traces: bool,
229    /// Recorder used for decoding traces, used together with print_traces
230    call_trace_decoder: Arc<CallTraceDecoder>,
231    /// How to keep history state
232    prune_state_history_config: PruneStateHistoryConfig,
233    /// max number of blocks with transactions in memory
234    transaction_block_keeper: Option<usize>,
235    node_config: Arc<AsyncRwLock<NodeConfig>>,
236    /// Slots in an epoch
237    slots_in_an_epoch: u64,
238    /// Precompiles to inject to the EVM.
239    precompile_factory: Option<Arc<dyn PrecompileFactory>>,
240    /// Prevent race conditions during mining
241    mining: Arc<tokio::sync::Mutex<()>>,
242    // === wallet === //
243    capabilities: Arc<RwLock<WalletCapabilities>>,
244    executor_wallet: Arc<RwLock<Option<EthereumWallet>>>,
245    /// Disable pool balance checks
246    disable_pool_balance_checks: bool,
247}
248
249impl Backend {
250    /// Initialises the balance of the given accounts
251    #[expect(clippy::too_many_arguments)]
252    pub async fn with_genesis(
253        db: Arc<AsyncRwLock<Box<dyn Db>>>,
254        env: Arc<RwLock<Env>>,
255        genesis: GenesisConfig,
256        fees: FeeManager,
257        fork: Arc<RwLock<Option<ClientFork>>>,
258        enable_steps_tracing: bool,
259        print_logs: bool,
260        print_traces: bool,
261        call_trace_decoder: Arc<CallTraceDecoder>,
262        prune_state_history_config: PruneStateHistoryConfig,
263        max_persisted_states: Option<usize>,
264        transaction_block_keeper: Option<usize>,
265        automine_block_time: Option<Duration>,
266        cache_path: Option<PathBuf>,
267        node_config: Arc<AsyncRwLock<NodeConfig>>,
268    ) -> Result<Self> {
269        // if this is a fork then adjust the blockchain storage
270        let blockchain = if let Some(fork) = fork.read().as_ref() {
271            trace!(target: "backend", "using forked blockchain at {}", fork.block_number());
272            Blockchain::forked(fork.block_number(), fork.block_hash(), fork.total_difficulty())
273        } else {
274            let env = env.read();
275            Blockchain::new(
276                &env,
277                env.evm_env.cfg_env.spec,
278                fees.is_eip1559().then(|| fees.base_fee()),
279                genesis.timestamp,
280                genesis.number,
281            )
282        };
283
284        let start_timestamp = if let Some(fork) = fork.read().as_ref() {
285            fork.timestamp()
286        } else {
287            genesis.timestamp
288        };
289
290        let mut states = if prune_state_history_config.is_config_enabled() {
291            // if prune state history is enabled, configure the state cache only for memory
292            prune_state_history_config
293                .max_memory_history
294                .map(|limit| InMemoryBlockStates::new(limit, 0))
295                .unwrap_or_default()
296                .memory_only()
297        } else if max_persisted_states.is_some() {
298            max_persisted_states
299                .map(|limit| InMemoryBlockStates::new(DEFAULT_HISTORY_LIMIT, limit))
300                .unwrap_or_default()
301        } else {
302            Default::default()
303        };
304
305        if let Some(cache_path) = cache_path {
306            states = states.disk_path(cache_path);
307        }
308
309        let (slots_in_an_epoch, precompile_factory, disable_pool_balance_checks) = {
310            let cfg = node_config.read().await;
311            (cfg.slots_in_an_epoch, cfg.precompile_factory.clone(), cfg.disable_pool_balance_checks)
312        };
313
314        let backend = Self {
315            db,
316            blockchain,
317            states: Arc::new(RwLock::new(states)),
318            env,
319            fork,
320            time: TimeManager::new(start_timestamp),
321            cheats: Default::default(),
322            new_block_listeners: Default::default(),
323            fees,
324            genesis,
325            active_state_snapshots: Arc::new(Mutex::new(Default::default())),
326            enable_steps_tracing,
327            print_logs,
328            print_traces,
329            call_trace_decoder,
330            prune_state_history_config,
331            transaction_block_keeper,
332            node_config,
333            slots_in_an_epoch,
334            precompile_factory,
335            mining: Arc::new(tokio::sync::Mutex::new(())),
336            capabilities: Arc::new(RwLock::new(WalletCapabilities(Default::default()))),
337            executor_wallet: Arc::new(RwLock::new(None)),
338            disable_pool_balance_checks,
339        };
340
341        if let Some(interval_block_time) = automine_block_time {
342            backend.update_interval_mine_block_time(interval_block_time);
343        }
344
345        // Note: this can only fail in forking mode, in which case we can't recover
346        backend.apply_genesis().await.wrap_err("failed to create genesis")?;
347        Ok(backend)
348    }
349
350    /// Writes the CREATE2 deployer code directly to the database at the address provided.
351    pub async fn set_create2_deployer(&self, address: Address) -> DatabaseResult<()> {
352        self.set_code(address, Bytes::from_static(DEFAULT_CREATE2_DEPLOYER_RUNTIME_CODE)).await?;
353
354        Ok(())
355    }
356
357    /// Get the capabilities of the wallet.
358    ///
359    /// Currently the only capability is delegation.
360    ///
361    /// See `anvil_core::eth::wallet::Capabilities` for construction helpers.
362    pub(crate) fn get_capabilities(&self) -> WalletCapabilities {
363        self.capabilities.read().clone()
364    }
365
366    /// Updates memory limits that should be more strict when auto-mine is enabled
367    pub(crate) fn update_interval_mine_block_time(&self, block_time: Duration) {
368        self.states.write().update_interval_mine_block_time(block_time)
369    }
370
371    /// Adds an address to the wallet's delegation capability.
372    pub(crate) fn add_capability(&self, address: Address) {
373        let chain_id = self.env.read().evm_env.cfg_env.chain_id;
374        let mut capabilities = self.capabilities.write();
375        let mut capability = capabilities
376            .get(chain_id)
377            .cloned()
378            .unwrap_or(Capabilities { delegation: DelegationCapability { addresses: vec![] } });
379        capability.delegation.addresses.push(address);
380        capabilities.0.insert(chain_id, capability);
381    }
382
383    pub(crate) fn set_executor(&self, executor_pk: String) -> Result<Address, BlockchainError> {
384        let signer: PrivateKeySigner =
385            executor_pk.parse().map_err(|_| RpcError::invalid_params("Invalid private key"))?;
386
387        let executor = signer.address();
388        let wallet = EthereumWallet::new(signer);
389
390        *self.executor_wallet.write() = Some(wallet);
391
392        Ok(executor)
393    }
394
395    /// Applies the configured genesis settings
396    ///
397    /// This will fund, create the genesis accounts
398    async fn apply_genesis(&self) -> Result<(), DatabaseError> {
399        trace!(target: "backend", "setting genesis balances");
400
401        if self.fork.read().is_some() {
402            // fetch all account first
403            let mut genesis_accounts_futures = Vec::with_capacity(self.genesis.accounts.len());
404            for address in self.genesis.accounts.iter().copied() {
405                let db = Arc::clone(&self.db);
406
407                // The forking Database backend can handle concurrent requests, we can fetch all dev
408                // accounts concurrently by spawning the job to a new task
409                genesis_accounts_futures.push(tokio::task::spawn(async move {
410                    let db = db.read().await;
411                    let info = db.basic_ref(address)?.unwrap_or_default();
412                    Ok::<_, DatabaseError>((address, info))
413                }));
414            }
415
416            let genesis_accounts = futures::future::join_all(genesis_accounts_futures).await;
417
418            let mut db = self.db.write().await;
419
420            for res in genesis_accounts {
421                let (address, mut info) = res.unwrap()?;
422                info.balance = self.genesis.balance;
423                db.insert_account(address, info.clone());
424            }
425        } else {
426            let mut db = self.db.write().await;
427            for (account, info) in self.genesis.account_infos() {
428                db.insert_account(account, info);
429            }
430
431            // insert the new genesis hash to the database so it's available for the next block in
432            // the evm
433            db.insert_block_hash(U256::from(self.best_number()), self.best_hash());
434        }
435
436        let db = self.db.write().await;
437        // apply the genesis.json alloc
438        self.genesis.apply_genesis_json_alloc(db)?;
439
440        trace!(target: "backend", "set genesis balances");
441
442        Ok(())
443    }
444
445    /// Sets the account to impersonate
446    ///
447    /// Returns `true` if the account is already impersonated
448    pub fn impersonate(&self, addr: Address) -> bool {
449        if self.cheats.impersonated_accounts().contains(&addr) {
450            return true;
451        }
452        // Ensure EIP-3607 is disabled
453        let mut env = self.env.write();
454        env.evm_env.cfg_env.disable_eip3607 = true;
455        self.cheats.impersonate(addr)
456    }
457
458    /// Removes the account that from the impersonated set
459    ///
460    /// If the impersonated `addr` is a contract then we also reset the code here
461    pub fn stop_impersonating(&self, addr: Address) {
462        self.cheats.stop_impersonating(&addr);
463    }
464
465    /// If set to true will make every account impersonated
466    pub fn auto_impersonate_account(&self, enabled: bool) {
467        self.cheats.set_auto_impersonate_account(enabled);
468    }
469
470    /// Returns the configured fork, if any
471    pub fn get_fork(&self) -> Option<ClientFork> {
472        self.fork.read().clone()
473    }
474
475    /// Returns the database
476    pub fn get_db(&self) -> &Arc<AsyncRwLock<Box<dyn Db>>> {
477        &self.db
478    }
479
480    /// Returns the `AccountInfo` from the database
481    pub async fn get_account(&self, address: Address) -> DatabaseResult<AccountInfo> {
482        Ok(self.db.read().await.basic_ref(address)?.unwrap_or_default())
483    }
484
485    /// Whether we're forked off some remote client
486    pub fn is_fork(&self) -> bool {
487        self.fork.read().is_some()
488    }
489
490    /// Resets the fork to a fresh state
491    pub async fn reset_fork(&self, forking: Forking) -> Result<(), BlockchainError> {
492        if !self.is_fork() {
493            if let Some(eth_rpc_url) = forking.clone().json_rpc_url {
494                let mut env = self.env.read().clone();
495
496                let (db, config) = {
497                    let mut node_config = self.node_config.write().await;
498
499                    // we want to force the correct base fee for the next block during
500                    // `setup_fork_db_config`
501                    node_config.base_fee.take();
502
503                    node_config.setup_fork_db_config(eth_rpc_url, &mut env, &self.fees).await?
504                };
505
506                *self.db.write().await = Box::new(db);
507
508                let fork = ClientFork::new(config, Arc::clone(&self.db));
509
510                *self.env.write() = env;
511                *self.fork.write() = Some(fork);
512            } else {
513                return Err(RpcError::invalid_params(
514                    "Forking not enabled and RPC URL not provided to start forking",
515                )
516                .into());
517            }
518        }
519
520        if let Some(fork) = self.get_fork() {
521            let block_number =
522                forking.block_number.map(BlockNumber::from).unwrap_or(BlockNumber::Latest);
523            // reset the fork entirely and reapply the genesis config
524            fork.reset(forking.json_rpc_url.clone(), block_number).await?;
525            let fork_block_number = fork.block_number();
526            let fork_block = fork
527                .block_by_number(fork_block_number)
528                .await?
529                .ok_or(BlockchainError::BlockNotFound)?;
530            // update all settings related to the forked block
531            {
532                if let Some(fork_url) = forking.json_rpc_url {
533                    self.reset_block_number(fork_url, fork_block_number).await?;
534                } else {
535                    // If rpc url is unspecified, then update the fork with the new block number and
536                    // existing rpc url, this updates the cache path
537                    {
538                        let maybe_fork_url = { self.node_config.read().await.eth_rpc_url.clone() };
539                        if let Some(fork_url) = maybe_fork_url {
540                            self.reset_block_number(fork_url, fork_block_number).await?;
541                        }
542                    }
543
544                    let gas_limit = self.node_config.read().await.fork_gas_limit(&fork_block);
545                    let mut env = self.env.write();
546
547                    env.evm_env.cfg_env.chain_id = fork.chain_id();
548                    env.evm_env.block_env = BlockEnv {
549                        number: U256::from(fork_block_number),
550                        timestamp: U256::from(fork_block.header.timestamp),
551                        gas_limit,
552                        difficulty: fork_block.header.difficulty,
553                        prevrandao: Some(fork_block.header.mix_hash.unwrap_or_default()),
554                        // Keep previous `beneficiary` and `basefee` value
555                        beneficiary: env.evm_env.block_env.beneficiary,
556                        basefee: env.evm_env.block_env.basefee,
557                        ..env.evm_env.block_env.clone()
558                    };
559
560                    // this is the base fee of the current block, but we need the base fee of
561                    // the next block
562                    let next_block_base_fee = self.fees.get_next_block_base_fee_per_gas(
563                        fork_block.header.gas_used,
564                        gas_limit,
565                        fork_block.header.base_fee_per_gas.unwrap_or_default(),
566                    );
567
568                    self.fees.set_base_fee(next_block_base_fee);
569                }
570
571                // reset the time to the timestamp of the forked block
572                self.time.reset(fork_block.header.timestamp);
573
574                // also reset the total difficulty
575                self.blockchain.storage.write().total_difficulty = fork.total_difficulty();
576            }
577            // reset storage
578            *self.blockchain.storage.write() = BlockchainStorage::forked(
579                fork.block_number(),
580                fork.block_hash(),
581                fork.total_difficulty(),
582            );
583            self.states.write().clear();
584            self.db.write().await.clear();
585
586            self.apply_genesis().await?;
587
588            trace!(target: "backend", "reset fork");
589
590            Ok(())
591        } else {
592            Err(RpcError::invalid_params("Forking not enabled").into())
593        }
594    }
595
596    /// Resets the backend to a fresh in-memory state, clearing all existing data
597    pub async fn reset_to_in_mem(&self) -> Result<(), BlockchainError> {
598        // Clear the fork if any exists
599        *self.fork.write() = None;
600
601        // Get environment and genesis config
602        let env = self.env.read().clone();
603        let genesis_timestamp = self.genesis.timestamp;
604        let genesis_number = self.genesis.number;
605        let spec_id = self.spec_id();
606
607        // Reset environment to genesis state
608        {
609            let mut env = self.env.write();
610            env.evm_env.block_env.number = U256::from(genesis_number);
611            env.evm_env.block_env.timestamp = U256::from(genesis_timestamp);
612            // Reset other block env fields to their defaults
613            env.evm_env.block_env.basefee = self.fees.base_fee();
614            env.evm_env.block_env.prevrandao = Some(B256::ZERO);
615        }
616
617        // Clear all storage and reinitialize with genesis
618        let base_fee = if self.fees.is_eip1559() { Some(self.fees.base_fee()) } else { None };
619        *self.blockchain.storage.write() =
620            BlockchainStorage::new(&env, spec_id, base_fee, genesis_timestamp, genesis_number);
621        self.states.write().clear();
622
623        // Clear the database
624        self.db.write().await.clear();
625
626        // Reset time manager
627        self.time.reset(genesis_timestamp);
628
629        // Reset fees to initial state
630        if self.fees.is_eip1559() {
631            self.fees.set_base_fee(crate::eth::fees::INITIAL_BASE_FEE);
632        }
633
634        self.fees.set_gas_price(crate::eth::fees::INITIAL_GAS_PRICE);
635
636        // Reapply genesis configuration
637        self.apply_genesis().await?;
638
639        trace!(target: "backend", "reset to fresh in-memory state");
640
641        Ok(())
642    }
643
644    async fn reset_block_number(
645        &self,
646        fork_url: String,
647        fork_block_number: u64,
648    ) -> Result<(), BlockchainError> {
649        let mut node_config = self.node_config.write().await;
650        node_config.fork_choice = Some(ForkChoice::Block(fork_block_number as i128));
651
652        let mut env = self.env.read().clone();
653        let (forked_db, client_fork_config) =
654            node_config.setup_fork_db_config(fork_url, &mut env, &self.fees).await?;
655
656        *self.db.write().await = Box::new(forked_db);
657        let fork = ClientFork::new(client_fork_config, Arc::clone(&self.db));
658        *self.fork.write() = Some(fork);
659        *self.env.write() = env;
660
661        Ok(())
662    }
663
664    /// Returns the `TimeManager` responsible for timestamps
665    pub fn time(&self) -> &TimeManager {
666        &self.time
667    }
668
669    /// Returns the `CheatsManager` responsible for executing cheatcodes
670    pub fn cheats(&self) -> &CheatsManager {
671        &self.cheats
672    }
673
674    /// Whether to skip blob validation
675    pub fn skip_blob_validation(&self, impersonator: Option<Address>) -> bool {
676        self.cheats().auto_impersonate_accounts()
677            || impersonator
678                .is_some_and(|addr| self.cheats().impersonated_accounts().contains(&addr))
679    }
680
681    /// Returns the `FeeManager` that manages fee/pricings
682    pub fn fees(&self) -> &FeeManager {
683        &self.fees
684    }
685
686    /// The env data of the blockchain
687    pub fn env(&self) -> &Arc<RwLock<Env>> {
688        &self.env
689    }
690
691    /// Returns the current best hash of the chain
692    pub fn best_hash(&self) -> B256 {
693        self.blockchain.storage.read().best_hash
694    }
695
696    /// Returns the current best number of the chain
697    pub fn best_number(&self) -> u64 {
698        self.blockchain.storage.read().best_number
699    }
700
701    /// Sets the block number
702    pub fn set_block_number(&self, number: u64) {
703        let mut env = self.env.write();
704        env.evm_env.block_env.number = U256::from(number);
705    }
706
707    /// Returns the client coinbase address.
708    pub fn coinbase(&self) -> Address {
709        self.env.read().evm_env.block_env.beneficiary
710    }
711
712    /// Returns the client coinbase address.
713    pub fn chain_id(&self) -> U256 {
714        U256::from(self.env.read().evm_env.cfg_env.chain_id)
715    }
716
717    pub fn set_chain_id(&self, chain_id: u64) {
718        self.env.write().evm_env.cfg_env.chain_id = chain_id;
719    }
720
721    /// Returns balance of the given account.
722    pub async fn current_balance(&self, address: Address) -> DatabaseResult<U256> {
723        Ok(self.get_account(address).await?.balance)
724    }
725
726    /// Returns balance of the given account.
727    pub async fn current_nonce(&self, address: Address) -> DatabaseResult<u64> {
728        Ok(self.get_account(address).await?.nonce)
729    }
730
731    /// Sets the coinbase address
732    pub fn set_coinbase(&self, address: Address) {
733        self.env.write().evm_env.block_env.beneficiary = address;
734    }
735
736    /// Sets the nonce of the given address
737    pub async fn set_nonce(&self, address: Address, nonce: U256) -> DatabaseResult<()> {
738        self.db.write().await.set_nonce(address, nonce.try_into().unwrap_or(u64::MAX))
739    }
740
741    /// Sets the balance of the given address
742    pub async fn set_balance(&self, address: Address, balance: U256) -> DatabaseResult<()> {
743        self.db.write().await.set_balance(address, balance)
744    }
745
746    /// Sets the code of the given address
747    pub async fn set_code(&self, address: Address, code: Bytes) -> DatabaseResult<()> {
748        self.db.write().await.set_code(address, code.0.into())
749    }
750
751    /// Sets the value for the given slot of the given address
752    pub async fn set_storage_at(
753        &self,
754        address: Address,
755        slot: U256,
756        val: B256,
757    ) -> DatabaseResult<()> {
758        self.db.write().await.set_storage_at(address, slot.into(), val)
759    }
760
761    /// Returns the configured specid
762    pub fn spec_id(&self) -> SpecId {
763        self.env.read().evm_env.cfg_env.spec
764    }
765
766    /// Returns true for post London
767    pub fn is_eip1559(&self) -> bool {
768        (self.spec_id() as u8) >= (SpecId::LONDON as u8)
769    }
770
771    /// Returns true for post Merge
772    pub fn is_eip3675(&self) -> bool {
773        (self.spec_id() as u8) >= (SpecId::MERGE as u8)
774    }
775
776    /// Returns true for post Berlin
777    pub fn is_eip2930(&self) -> bool {
778        (self.spec_id() as u8) >= (SpecId::BERLIN as u8)
779    }
780
781    /// Returns true for post Cancun
782    pub fn is_eip4844(&self) -> bool {
783        (self.spec_id() as u8) >= (SpecId::CANCUN as u8)
784    }
785
786    /// Returns true for post Prague
787    pub fn is_eip7702(&self) -> bool {
788        (self.spec_id() as u8) >= (SpecId::PRAGUE as u8)
789    }
790
791    /// Returns true if op-stack deposits are active
792    pub fn is_optimism(&self) -> bool {
793        self.env.read().networks.optimism
794    }
795
796    /// Returns the precompiles for the current spec.
797    pub fn precompiles(&self) -> BTreeMap<String, Address> {
798        let spec_id = self.env.read().evm_env.cfg_env.spec;
799        let precompiles = Precompiles::new(PrecompileSpecId::from_spec_id(spec_id));
800
801        let mut precompiles_map = BTreeMap::<String, Address>::default();
802        for (address, precompile) in precompiles.inner() {
803            precompiles_map.insert(precompile.id().name().to_string(), *address);
804        }
805
806        // Extend with configured network precompiles.
807        precompiles_map.extend(self.env.read().networks.precompiles());
808
809        if let Some(factory) = &self.precompile_factory {
810            for (precompile, _) in &factory.precompiles() {
811                precompiles_map.insert(precompile.id().name().to_string(), *precompile.address());
812            }
813        }
814
815        precompiles_map
816    }
817
818    /// Returns the system contracts for the current spec.
819    pub fn system_contracts(&self) -> BTreeMap<SystemContract, Address> {
820        let mut system_contracts = BTreeMap::<SystemContract, Address>::default();
821
822        let spec_id = self.env.read().evm_env.cfg_env.spec;
823
824        if spec_id >= SpecId::CANCUN {
825            system_contracts.extend(SystemContract::cancun());
826        }
827
828        if spec_id >= SpecId::PRAGUE {
829            system_contracts.extend(SystemContract::prague(None));
830        }
831
832        system_contracts
833    }
834
835    /// Returns [`BlobParams`] corresponding to the current spec.
836    pub fn blob_params(&self) -> BlobParams {
837        let spec_id = self.env.read().evm_env.cfg_env.spec;
838
839        if spec_id >= SpecId::OSAKA {
840            return BlobParams::osaka();
841        }
842
843        if spec_id >= SpecId::PRAGUE {
844            return BlobParams::prague();
845        }
846
847        BlobParams::cancun()
848    }
849
850    /// Returns an error if EIP1559 is not active (pre Berlin)
851    pub fn ensure_eip1559_active(&self) -> Result<(), BlockchainError> {
852        if self.is_eip1559() {
853            return Ok(());
854        }
855        Err(BlockchainError::EIP1559TransactionUnsupportedAtHardfork)
856    }
857
858    /// Returns an error if EIP1559 is not active (pre muirGlacier)
859    pub fn ensure_eip2930_active(&self) -> Result<(), BlockchainError> {
860        if self.is_eip2930() {
861            return Ok(());
862        }
863        Err(BlockchainError::EIP2930TransactionUnsupportedAtHardfork)
864    }
865
866    pub fn ensure_eip4844_active(&self) -> Result<(), BlockchainError> {
867        if self.is_eip4844() {
868            return Ok(());
869        }
870        Err(BlockchainError::EIP4844TransactionUnsupportedAtHardfork)
871    }
872
873    pub fn ensure_eip7702_active(&self) -> Result<(), BlockchainError> {
874        if self.is_eip7702() {
875            return Ok(());
876        }
877        Err(BlockchainError::EIP7702TransactionUnsupportedAtHardfork)
878    }
879
880    /// Returns an error if op-stack deposits are not active
881    pub fn ensure_op_deposits_active(&self) -> Result<(), BlockchainError> {
882        if self.is_optimism() {
883            return Ok(());
884        }
885        Err(BlockchainError::DepositTransactionUnsupported)
886    }
887
888    /// Returns the block gas limit
889    pub fn gas_limit(&self) -> u64 {
890        self.env.read().evm_env.block_env.gas_limit
891    }
892
893    /// Sets the block gas limit
894    pub fn set_gas_limit(&self, gas_limit: u64) {
895        self.env.write().evm_env.block_env.gas_limit = gas_limit;
896    }
897
898    /// Returns the current base fee
899    pub fn base_fee(&self) -> u64 {
900        self.fees.base_fee()
901    }
902
903    /// Returns whether the minimum suggested priority fee is enforced
904    pub fn is_min_priority_fee_enforced(&self) -> bool {
905        self.fees.is_min_priority_fee_enforced()
906    }
907
908    pub fn excess_blob_gas_and_price(&self) -> Option<BlobExcessGasAndPrice> {
909        self.fees.excess_blob_gas_and_price()
910    }
911
912    /// Sets the current basefee
913    pub fn set_base_fee(&self, basefee: u64) {
914        self.fees.set_base_fee(basefee)
915    }
916
917    /// Sets the gas price
918    pub fn set_gas_price(&self, price: u128) {
919        self.fees.set_gas_price(price)
920    }
921
922    pub fn elasticity(&self) -> f64 {
923        self.fees.elasticity()
924    }
925
926    /// Returns the total difficulty of the chain until this block
927    ///
928    /// Note: this will always be `0` in memory mode
929    /// In forking mode this will always be the total difficulty of the forked block
930    pub fn total_difficulty(&self) -> U256 {
931        self.blockchain.storage.read().total_difficulty
932    }
933
934    /// Creates a new `evm_snapshot` at the current height.
935    ///
936    /// Returns the id of the snapshot created.
937    pub async fn create_state_snapshot(&self) -> U256 {
938        let num = self.best_number();
939        let hash = self.best_hash();
940        let id = self.db.write().await.snapshot_state();
941        trace!(target: "backend", "creating snapshot {} at {}", id, num);
942        self.active_state_snapshots.lock().insert(id, (num, hash));
943        id
944    }
945
946    /// Reverts the state to the state snapshot identified by the given `id`.
947    pub async fn revert_state_snapshot(&self, id: U256) -> Result<bool, BlockchainError> {
948        let block = { self.active_state_snapshots.lock().remove(&id) };
949        if let Some((num, hash)) = block {
950            let best_block_hash = {
951                // revert the storage that's newer than the snapshot
952                let current_height = self.best_number();
953                let mut storage = self.blockchain.storage.write();
954
955                for n in ((num + 1)..=current_height).rev() {
956                    trace!(target: "backend", "reverting block {}", n);
957                    if let Some(hash) = storage.hashes.remove(&n)
958                        && let Some(block) = storage.blocks.remove(&hash)
959                    {
960                        for tx in block.transactions {
961                            let _ = storage.transactions.remove(&tx.hash());
962                        }
963                    }
964                }
965
966                storage.best_number = num;
967                storage.best_hash = hash;
968                hash
969            };
970            let block =
971                self.block_by_hash(best_block_hash).await?.ok_or(BlockchainError::BlockNotFound)?;
972
973            let reset_time = block.header.timestamp;
974            self.time.reset(reset_time);
975
976            let mut env = self.env.write();
977            env.evm_env.block_env = BlockEnv {
978                number: U256::from(num),
979                timestamp: U256::from(block.header.timestamp),
980                difficulty: block.header.difficulty,
981                // ensures prevrandao is set
982                prevrandao: Some(block.header.mix_hash.unwrap_or_default()),
983                gas_limit: block.header.gas_limit,
984                // Keep previous `beneficiary` and `basefee` value
985                beneficiary: env.evm_env.block_env.beneficiary,
986                basefee: env.evm_env.block_env.basefee,
987                ..Default::default()
988            }
989        }
990        Ok(self.db.write().await.revert_state(id, RevertStateSnapshotAction::RevertRemove))
991    }
992
993    pub fn list_state_snapshots(&self) -> BTreeMap<U256, (u64, B256)> {
994        self.active_state_snapshots.lock().clone().into_iter().collect()
995    }
996
997    /// Get the current state.
998    pub async fn serialized_state(
999        &self,
1000        preserve_historical_states: bool,
1001    ) -> Result<SerializableState, BlockchainError> {
1002        let at = self.env.read().evm_env.block_env.clone();
1003        let best_number = self.blockchain.storage.read().best_number;
1004        let blocks = self.blockchain.storage.read().serialized_blocks();
1005        let transactions = self.blockchain.storage.read().serialized_transactions();
1006        let historical_states = if preserve_historical_states {
1007            Some(self.states.write().serialized_states())
1008        } else {
1009            None
1010        };
1011
1012        let state = self.db.read().await.dump_state(
1013            at,
1014            best_number,
1015            blocks,
1016            transactions,
1017            historical_states,
1018        )?;
1019        state.ok_or_else(|| {
1020            RpcError::invalid_params("Dumping state not supported with the current configuration")
1021                .into()
1022        })
1023    }
1024
1025    /// Write all chain data to serialized bytes buffer
1026    pub async fn dump_state(
1027        &self,
1028        preserve_historical_states: bool,
1029    ) -> Result<Bytes, BlockchainError> {
1030        let state = self.serialized_state(preserve_historical_states).await?;
1031        let mut encoder = GzEncoder::new(Vec::new(), Compression::default());
1032        encoder
1033            .write_all(&serde_json::to_vec(&state).unwrap_or_default())
1034            .map_err(|_| BlockchainError::DataUnavailable)?;
1035        Ok(encoder.finish().unwrap_or_default().into())
1036    }
1037
1038    /// Apply [SerializableState] data to the backend storage.
1039    pub async fn load_state(&self, state: SerializableState) -> Result<bool, BlockchainError> {
1040        // load the blocks and transactions into the storage
1041        self.blockchain.storage.write().load_blocks(state.blocks.clone());
1042        self.blockchain.storage.write().load_transactions(state.transactions.clone());
1043        // reset the block env
1044        if let Some(block) = state.block.clone() {
1045            self.env.write().evm_env.block_env = block.clone();
1046
1047            // Set the current best block number.
1048            // Defaults to block number for compatibility with existing state files.
1049            let fork_num_and_hash = self.get_fork().map(|f| (f.block_number(), f.block_hash()));
1050
1051            let best_number = state.best_block_number.unwrap_or(block.number.saturating_to());
1052            if let Some((number, hash)) = fork_num_and_hash {
1053                trace!(target: "backend", state_block_number=?best_number, fork_block_number=?number);
1054                // If the state.block_number is greater than the fork block number, set best number
1055                // to the state block number.
1056                // Ref: https://github.com/foundry-rs/foundry/issues/9539
1057                if best_number > number {
1058                    self.blockchain.storage.write().best_number = best_number;
1059                    let best_hash =
1060                        self.blockchain.storage.read().hash(best_number.into()).ok_or_else(
1061                            || {
1062                                BlockchainError::RpcError(RpcError::internal_error_with(format!(
1063                                    "Best hash not found for best number {best_number}",
1064                                )))
1065                            },
1066                        )?;
1067                    self.blockchain.storage.write().best_hash = best_hash;
1068                } else {
1069                    // If loading state file on a fork, set best number to the fork block number.
1070                    // Ref: https://github.com/foundry-rs/foundry/pull/9215#issue-2618681838
1071                    self.blockchain.storage.write().best_number = number;
1072                    self.blockchain.storage.write().best_hash = hash;
1073                }
1074            } else {
1075                self.blockchain.storage.write().best_number = best_number;
1076
1077                // Set the current best block hash;
1078                let best_hash =
1079                    self.blockchain.storage.read().hash(best_number.into()).ok_or_else(|| {
1080                        BlockchainError::RpcError(RpcError::internal_error_with(format!(
1081                            "Best hash not found for best number {best_number}",
1082                        )))
1083                    })?;
1084
1085                self.blockchain.storage.write().best_hash = best_hash;
1086            }
1087        }
1088
1089        if let Some(latest) = state.blocks.iter().max_by_key(|b| b.header.number) {
1090            let header = &latest.header;
1091            let next_block_base_fee = self.fees.get_next_block_base_fee_per_gas(
1092                header.gas_used,
1093                header.gas_limit,
1094                header.base_fee_per_gas.unwrap_or_default(),
1095            );
1096            let next_block_excess_blob_gas = self.fees.get_next_block_blob_excess_gas(
1097                header.excess_blob_gas.unwrap_or_default(),
1098                header.blob_gas_used.unwrap_or_default(),
1099            );
1100
1101            // update next base fee
1102            self.fees.set_base_fee(next_block_base_fee);
1103
1104            self.fees.set_blob_excess_gas_and_price(BlobExcessGasAndPrice::new(
1105                next_block_excess_blob_gas,
1106                get_blob_base_fee_update_fraction(
1107                    self.env.read().evm_env.cfg_env.chain_id,
1108                    header.timestamp,
1109                ),
1110            ));
1111        }
1112
1113        if !self.db.write().await.load_state(state.clone())? {
1114            return Err(RpcError::invalid_params(
1115                "Loading state not supported with the current configuration",
1116            )
1117            .into());
1118        }
1119
1120        if let Some(historical_states) = state.historical_states {
1121            self.states.write().load_states(historical_states);
1122        }
1123
1124        Ok(true)
1125    }
1126
1127    /// Deserialize and add all chain data to the backend storage
1128    pub async fn load_state_bytes(&self, buf: Bytes) -> Result<bool, BlockchainError> {
1129        let orig_buf = &buf.0[..];
1130        let mut decoder = GzDecoder::new(orig_buf);
1131        let mut decoded_data = Vec::new();
1132
1133        let state: SerializableState = serde_json::from_slice(if decoder.header().is_some() {
1134            decoder
1135                .read_to_end(decoded_data.as_mut())
1136                .map_err(|_| BlockchainError::FailedToDecodeStateDump)?;
1137            &decoded_data
1138        } else {
1139            &buf.0
1140        })
1141        .map_err(|_| BlockchainError::FailedToDecodeStateDump)?;
1142
1143        self.load_state(state).await
1144    }
1145
1146    /// Returns the environment for the next block
1147    fn next_env(&self) -> Env {
1148        let mut env = self.env.read().clone();
1149        // increase block number for this block
1150        env.evm_env.block_env.number = env.evm_env.block_env.number.saturating_add(U256::from(1));
1151        env.evm_env.block_env.basefee = self.base_fee();
1152        env.evm_env.block_env.timestamp = U256::from(self.time.current_call_timestamp());
1153        env
1154    }
1155
1156    /// Creates an EVM instance with optionally injected precompiles.
1157    fn new_evm_with_inspector_ref<'db, I, DB>(
1158        &self,
1159        db: &'db DB,
1160        env: &Env,
1161        inspector: &'db mut I,
1162    ) -> EitherEvm<WrapDatabaseRef<&'db DB>, &'db mut I, PrecompilesMap>
1163    where
1164        DB: DatabaseRef + ?Sized,
1165        I: Inspector<EthEvmContext<WrapDatabaseRef<&'db DB>>>
1166            + Inspector<OpContext<WrapDatabaseRef<&'db DB>>>,
1167        WrapDatabaseRef<&'db DB>: Database<Error = DatabaseError>,
1168    {
1169        let mut evm = new_evm_with_inspector_ref(db, env, inspector);
1170        self.env.read().networks.inject_precompiles(evm.precompiles_mut());
1171
1172        if let Some(factory) = &self.precompile_factory {
1173            inject_custom_precompiles(&mut evm, factory.precompiles());
1174        }
1175
1176        let cheats = Arc::new(self.cheats.clone());
1177        if cheats.has_recover_overrides() {
1178            let cheat_ecrecover = CheatEcrecover::new(Arc::clone(&cheats));
1179            evm.precompiles_mut().apply_precompile(&EC_RECOVER, move |_| {
1180                Some(DynPrecompile::new_stateful(
1181                    cheat_ecrecover.precompile_id().clone(),
1182                    move |input| cheat_ecrecover.call(input),
1183                ))
1184            });
1185        }
1186
1187        evm
1188    }
1189
1190    /// executes the transactions without writing to the underlying database
1191    pub async fn inspect_tx(
1192        &self,
1193        tx: Arc<PoolTransaction>,
1194    ) -> Result<
1195        (InstructionResult, Option<Output>, u64, State, Vec<revm::primitives::Log>),
1196        BlockchainError,
1197    > {
1198        let mut env = self.next_env();
1199        env.tx = tx.pending_transaction.to_revm_tx_env();
1200
1201        if env.networks.optimism {
1202            env.tx.enveloped_tx =
1203                Some(alloy_rlp::encode(&tx.pending_transaction.transaction.transaction).into());
1204        }
1205
1206        let db = self.db.read().await;
1207        let mut inspector = self.build_inspector();
1208        let mut evm = self.new_evm_with_inspector_ref(&**db, &env, &mut inspector);
1209        let ResultAndState { result, state } = evm.transact(env.tx)?;
1210        let (exit_reason, gas_used, out, logs) = match result {
1211            ExecutionResult::Success { reason, gas_used, logs, output, .. } => {
1212                (reason.into(), gas_used, Some(output), Some(logs))
1213            }
1214            ExecutionResult::Revert { gas_used, output } => {
1215                (InstructionResult::Revert, gas_used, Some(Output::Call(output)), None)
1216            }
1217            ExecutionResult::Halt { reason, gas_used } => {
1218                let eth_reason = op_haltreason_to_instruction_result(reason);
1219                (eth_reason, gas_used, None, None)
1220            }
1221        };
1222
1223        drop(evm);
1224        inspector.print_logs();
1225
1226        if self.print_traces {
1227            inspector.print_traces(self.call_trace_decoder.clone());
1228        }
1229
1230        Ok((exit_reason, out, gas_used, state, logs.unwrap_or_default()))
1231    }
1232
1233    /// Creates the pending block
1234    ///
1235    /// This will execute all transaction in the order they come but will not mine the block
1236    pub async fn pending_block(&self, pool_transactions: Vec<Arc<PoolTransaction>>) -> BlockInfo {
1237        self.with_pending_block(pool_transactions, |_, block| block).await
1238    }
1239
1240    /// Creates the pending block
1241    ///
1242    /// This will execute all transaction in the order they come but will not mine the block
1243    pub async fn with_pending_block<F, T>(
1244        &self,
1245        pool_transactions: Vec<Arc<PoolTransaction>>,
1246        f: F,
1247    ) -> T
1248    where
1249        F: FnOnce(Box<dyn MaybeFullDatabase + '_>, BlockInfo) -> T,
1250    {
1251        let db = self.db.read().await;
1252        let env = self.next_env();
1253
1254        let mut cache_db = CacheDB::new(&*db);
1255
1256        let storage = self.blockchain.storage.read();
1257
1258        let executor = TransactionExecutor {
1259            db: &mut cache_db,
1260            validator: self,
1261            pending: pool_transactions.into_iter(),
1262            block_env: env.evm_env.block_env.clone(),
1263            cfg_env: env.evm_env.cfg_env,
1264            parent_hash: storage.best_hash,
1265            gas_used: 0,
1266            blob_gas_used: 0,
1267            enable_steps_tracing: self.enable_steps_tracing,
1268            print_logs: self.print_logs,
1269            print_traces: self.print_traces,
1270            call_trace_decoder: self.call_trace_decoder.clone(),
1271            precompile_factory: self.precompile_factory.clone(),
1272            networks: self.env.read().networks,
1273            blob_params: self.blob_params(),
1274            cheats: self.cheats().clone(),
1275        };
1276
1277        // create a new pending block
1278        let executed = executor.execute();
1279        f(Box::new(cache_db), executed.block)
1280    }
1281
1282    /// Mines a new block and stores it.
1283    ///
1284    /// this will execute all transaction in the order they come in and return all the markers they
1285    /// provide.
1286    pub async fn mine_block(
1287        &self,
1288        pool_transactions: Vec<Arc<PoolTransaction>>,
1289    ) -> MinedBlockOutcome {
1290        self.do_mine_block(pool_transactions).await
1291    }
1292
1293    async fn do_mine_block(
1294        &self,
1295        pool_transactions: Vec<Arc<PoolTransaction>>,
1296    ) -> MinedBlockOutcome {
1297        let _mining_guard = self.mining.lock().await;
1298        trace!(target: "backend", "creating new block with {} transactions", pool_transactions.len());
1299
1300        let (outcome, header, block_hash) = {
1301            let current_base_fee = self.base_fee();
1302            let current_excess_blob_gas_and_price = self.excess_blob_gas_and_price();
1303
1304            let mut env = self.env.read().clone();
1305
1306            if env.evm_env.block_env.basefee == 0 {
1307                // this is an edge case because the evm fails if `tx.effective_gas_price < base_fee`
1308                // 0 is only possible if it's manually set
1309                env.evm_env.cfg_env.disable_base_fee = true;
1310            }
1311
1312            let block_number = self.blockchain.storage.read().best_number.saturating_add(1);
1313
1314            // increase block number for this block
1315            if is_arbitrum(env.evm_env.cfg_env.chain_id) {
1316                // Temporary set `env.block.number` to `block_number` for Arbitrum chains.
1317                env.evm_env.block_env.number = U256::from(block_number);
1318            } else {
1319                env.evm_env.block_env.number =
1320                    env.evm_env.block_env.number.saturating_add(U256::from(1));
1321            }
1322
1323            env.evm_env.block_env.basefee = current_base_fee;
1324            env.evm_env.block_env.blob_excess_gas_and_price = current_excess_blob_gas_and_price;
1325
1326            let best_hash = self.blockchain.storage.read().best_hash;
1327
1328            let mut input = Vec::with_capacity(40);
1329            input.extend_from_slice(best_hash.as_slice());
1330            input.extend_from_slice(&block_number.to_le_bytes());
1331            env.evm_env.block_env.prevrandao = Some(keccak256(&input));
1332
1333            if self.prune_state_history_config.is_state_history_supported() {
1334                let db = self.db.read().await.current_state();
1335                // store current state before executing all transactions
1336                self.states.write().insert(best_hash, db);
1337            }
1338
1339            let (executed_tx, block_hash) = {
1340                let mut db = self.db.write().await;
1341
1342                // finally set the next block timestamp, this is done just before execution, because
1343                // there can be concurrent requests that can delay acquiring the db lock and we want
1344                // to ensure the timestamp is as close as possible to the actual execution.
1345                env.evm_env.block_env.timestamp = U256::from(self.time.next_timestamp());
1346
1347                let executor = TransactionExecutor {
1348                    db: &mut **db,
1349                    validator: self,
1350                    pending: pool_transactions.into_iter(),
1351                    block_env: env.evm_env.block_env.clone(),
1352                    cfg_env: env.evm_env.cfg_env.clone(),
1353                    parent_hash: best_hash,
1354                    gas_used: 0,
1355                    blob_gas_used: 0,
1356                    enable_steps_tracing: self.enable_steps_tracing,
1357                    print_logs: self.print_logs,
1358                    print_traces: self.print_traces,
1359                    call_trace_decoder: self.call_trace_decoder.clone(),
1360                    networks: self.env.read().networks,
1361                    precompile_factory: self.precompile_factory.clone(),
1362                    blob_params: self.blob_params(),
1363                    cheats: self.cheats().clone(),
1364                };
1365                let executed_tx = executor.execute();
1366
1367                // we also need to update the new blockhash in the db itself
1368                let block_hash = executed_tx.block.block.header.hash_slow();
1369                db.insert_block_hash(U256::from(executed_tx.block.block.header.number), block_hash);
1370
1371                (executed_tx, block_hash)
1372            };
1373
1374            // create the new block with the current timestamp
1375            let ExecutedTransactions { block, included, invalid } = executed_tx;
1376            let BlockInfo { block, transactions, receipts } = block;
1377
1378            let header = block.header.clone();
1379
1380            trace!(
1381                target: "backend",
1382                "Mined block {} with {} tx {:?}",
1383                block_number,
1384                transactions.len(),
1385                transactions.iter().map(|tx| tx.transaction_hash).collect::<Vec<_>>()
1386            );
1387            let mut storage = self.blockchain.storage.write();
1388            // update block metadata
1389            storage.best_number = block_number;
1390            storage.best_hash = block_hash;
1391            // Difficulty is removed and not used after Paris (aka TheMerge). Value is replaced with
1392            // prevrandao. https://github.com/bluealloy/revm/blob/1839b3fce8eaeebb85025576f2519b80615aca1e/crates/interpreter/src/instructions/host_env.rs#L27
1393            if !self.is_eip3675() {
1394                storage.total_difficulty =
1395                    storage.total_difficulty.saturating_add(header.difficulty);
1396            }
1397
1398            storage.blocks.insert(block_hash, block);
1399            storage.hashes.insert(block_number, block_hash);
1400
1401            node_info!("");
1402            // insert all transactions
1403            for (info, receipt) in transactions.into_iter().zip(receipts) {
1404                // log some tx info
1405                node_info!("    Transaction: {:?}", info.transaction_hash);
1406                if let Some(contract) = &info.contract_address {
1407                    node_info!("    Contract created: {contract}");
1408                }
1409                node_info!("    Gas used: {}", receipt.cumulative_gas_used());
1410                if !info.exit.is_ok() {
1411                    let r = RevertDecoder::new().decode(
1412                        info.out.as_ref().map(|b| &b[..]).unwrap_or_default(),
1413                        Some(info.exit),
1414                    );
1415                    node_info!("    Error: reverted with: {r}");
1416                }
1417                node_info!("");
1418
1419                let mined_tx = MinedTransaction { info, receipt, block_hash, block_number };
1420                storage.transactions.insert(mined_tx.info.transaction_hash, mined_tx);
1421            }
1422
1423            // remove old transactions that exceed the transaction block keeper
1424            if let Some(transaction_block_keeper) = self.transaction_block_keeper
1425                && storage.blocks.len() > transaction_block_keeper
1426            {
1427                let to_clear = block_number
1428                    .saturating_sub(transaction_block_keeper.try_into().unwrap_or(u64::MAX));
1429                storage.remove_block_transactions_by_number(to_clear)
1430            }
1431
1432            // we intentionally set the difficulty to `0` for newer blocks
1433            env.evm_env.block_env.difficulty = U256::from(0);
1434
1435            // update env with new values
1436            *self.env.write() = env;
1437
1438            let timestamp = utc_from_secs(header.timestamp);
1439
1440            node_info!("    Block Number: {}", block_number);
1441            node_info!("    Block Hash: {:?}", block_hash);
1442            if timestamp.year() > 9999 {
1443                // rf2822 panics with more than 4 digits
1444                node_info!("    Block Time: {:?}\n", timestamp.to_rfc3339());
1445            } else {
1446                node_info!("    Block Time: {:?}\n", timestamp.to_rfc2822());
1447            }
1448
1449            let outcome = MinedBlockOutcome { block_number, included, invalid };
1450
1451            (outcome, header, block_hash)
1452        };
1453        let next_block_base_fee = self.fees.get_next_block_base_fee_per_gas(
1454            header.gas_used,
1455            header.gas_limit,
1456            header.base_fee_per_gas.unwrap_or_default(),
1457        );
1458        let next_block_excess_blob_gas = self.fees.get_next_block_blob_excess_gas(
1459            header.excess_blob_gas.unwrap_or_default(),
1460            header.blob_gas_used.unwrap_or_default(),
1461        );
1462
1463        // update next base fee
1464        self.fees.set_base_fee(next_block_base_fee);
1465
1466        self.fees.set_blob_excess_gas_and_price(BlobExcessGasAndPrice::new(
1467            next_block_excess_blob_gas,
1468            get_blob_base_fee_update_fraction_by_spec_id(*self.env.read().evm_env.spec_id()),
1469        ));
1470
1471        // notify all listeners
1472        self.notify_on_new_block(header, block_hash);
1473
1474        outcome
1475    }
1476
1477    /// Executes the [TransactionRequest] without writing to the DB
1478    ///
1479    /// # Errors
1480    ///
1481    /// Returns an error if the `block_number` is greater than the current height
1482    pub async fn call(
1483        &self,
1484        request: WithOtherFields<TransactionRequest>,
1485        fee_details: FeeDetails,
1486        block_request: Option<BlockRequest>,
1487        overrides: EvmOverrides,
1488    ) -> Result<(InstructionResult, Option<Output>, u128, State), BlockchainError> {
1489        self.with_database_at(block_request, |state, mut block| {
1490            let block_number = block.number;
1491            let (exit, out, gas, state) = {
1492                let mut cache_db = CacheDB::new(state);
1493                if let Some(state_overrides) = overrides.state {
1494                    apply_state_overrides(state_overrides.into_iter().collect(), &mut cache_db)?;
1495                }
1496                if let Some(block_overrides) = overrides.block {
1497                    cache_db.apply_block_overrides(*block_overrides, &mut block);
1498                }
1499                self.call_with_state(&cache_db, request, fee_details, block)
1500            }?;
1501            trace!(target: "backend", "call return {:?} out: {:?} gas {} on block {}", exit, out, gas, block_number);
1502            Ok((exit, out, gas, state))
1503        }).await?
1504    }
1505
1506    /// ## EVM settings
1507    ///
1508    /// This modifies certain EVM settings to mirror geth's `SkipAccountChecks` when transacting requests, see also: <https://github.com/ethereum/go-ethereum/blob/380688c636a654becc8f114438c2a5d93d2db032/core/state_transition.go#L145-L148>:
1509    ///
1510    ///  - `disable_eip3607` is set to `true`
1511    ///  - `disable_base_fee` is set to `true`
1512    ///  - `tx_gas_limit_cap` is set to `Some(u64::MAX)` indicating no gas limit cap
1513    ///  - `nonce` check is skipped if `request.nonce` is None
1514    fn build_call_env(
1515        &self,
1516        request: WithOtherFields<TransactionRequest>,
1517        fee_details: FeeDetails,
1518        block_env: BlockEnv,
1519    ) -> Env {
1520        let tx_type = request.minimal_tx_type() as u8;
1521
1522        let WithOtherFields::<TransactionRequest> {
1523            inner:
1524                TransactionRequest {
1525                    from,
1526                    to,
1527                    gas,
1528                    value,
1529                    input,
1530                    access_list,
1531                    blob_versioned_hashes,
1532                    authorization_list,
1533                    nonce,
1534                    sidecar: _,
1535                    chain_id,
1536                    transaction_type,
1537                    .. // Rest of the gas fees related fields are taken from `fee_details`
1538                },
1539            other,
1540        } = request;
1541
1542        let FeeDetails {
1543            gas_price,
1544            max_fee_per_gas,
1545            max_priority_fee_per_gas,
1546            max_fee_per_blob_gas,
1547        } = fee_details;
1548
1549        let gas_limit = gas.unwrap_or(block_env.gas_limit);
1550        let mut env = self.env.read().clone();
1551        env.evm_env.block_env = block_env;
1552        // we want to disable this in eth_call, since this is common practice used by other node
1553        // impls and providers <https://github.com/foundry-rs/foundry/issues/4388>
1554        env.evm_env.cfg_env.disable_block_gas_limit = true;
1555        env.evm_env.cfg_env.tx_gas_limit_cap = Some(u64::MAX);
1556
1557        // The basefee should be ignored for calls against state for
1558        // - eth_call
1559        // - eth_estimateGas
1560        // - eth_createAccessList
1561        // - tracing
1562        env.evm_env.cfg_env.disable_base_fee = true;
1563
1564        let gas_price = gas_price.or(max_fee_per_gas).unwrap_or_else(|| {
1565            self.fees().raw_gas_price().saturating_add(MIN_SUGGESTED_PRIORITY_FEE)
1566        });
1567        let caller = from.unwrap_or_default();
1568        let to = to.as_ref().and_then(TxKind::to);
1569        let blob_hashes = blob_versioned_hashes.unwrap_or_default();
1570        let mut base = TxEnv {
1571            caller,
1572            gas_limit,
1573            gas_price,
1574            gas_priority_fee: max_priority_fee_per_gas,
1575            max_fee_per_blob_gas: max_fee_per_blob_gas
1576                .or_else(|| {
1577                    if !blob_hashes.is_empty() {
1578                        env.evm_env.block_env.blob_gasprice()
1579                    } else {
1580                        Some(0)
1581                    }
1582                })
1583                .unwrap_or_default(),
1584            kind: match to {
1585                Some(addr) => TxKind::Call(*addr),
1586                None => TxKind::Create,
1587            },
1588            tx_type,
1589            value: value.unwrap_or_default(),
1590            data: input.into_input().unwrap_or_default(),
1591            chain_id: Some(chain_id.unwrap_or(self.env.read().evm_env.cfg_env.chain_id)),
1592            access_list: access_list.unwrap_or_default(),
1593            blob_hashes,
1594            ..Default::default()
1595        };
1596        base.set_signed_authorization(authorization_list.unwrap_or_default());
1597        env.tx = OpTransaction { base, ..Default::default() };
1598
1599        if let Some(nonce) = nonce {
1600            env.tx.base.nonce = nonce;
1601        } else {
1602            // Disable nonce check in revm
1603            env.evm_env.cfg_env.disable_nonce_check = true;
1604        }
1605
1606        if env.evm_env.block_env.basefee == 0 {
1607            // this is an edge case because the evm fails if `tx.effective_gas_price < base_fee`
1608            // 0 is only possible if it's manually set
1609            env.evm_env.cfg_env.disable_base_fee = true;
1610        }
1611
1612        // Deposit transaction?
1613        if transaction_type == Some(DEPOSIT_TX_TYPE_ID) && has_optimism_fields(&other) {
1614            let deposit = DepositTransactionParts {
1615                source_hash: other
1616                    .get_deserialized::<B256>("sourceHash")
1617                    .map(|sh| sh.unwrap_or_default())
1618                    .unwrap_or_default(),
1619                mint: other
1620                    .get_deserialized::<u128>("mint")
1621                    .map(|m| m.unwrap_or_default())
1622                    .or(None),
1623                is_system_transaction: other
1624                    .get_deserialized::<bool>("isSystemTx")
1625                    .map(|st| st.unwrap_or_default())
1626                    .unwrap_or_default(),
1627            };
1628            env.tx.deposit = deposit;
1629        }
1630
1631        env
1632    }
1633
1634    /// Builds [`Inspector`] with the configured options.
1635    fn build_inspector(&self) -> AnvilInspector {
1636        let mut inspector = AnvilInspector::default();
1637
1638        if self.print_logs {
1639            inspector = inspector.with_log_collector();
1640        }
1641        if self.print_traces {
1642            inspector = inspector.with_trace_printer();
1643        }
1644
1645        inspector
1646    }
1647
1648    /// Simulates the payload by executing the calls in request.
1649    pub async fn simulate(
1650        &self,
1651        request: SimulatePayload,
1652        block_request: Option<BlockRequest>,
1653    ) -> Result<Vec<SimulatedBlock<AnyRpcBlock>>, BlockchainError> {
1654        self.with_database_at(block_request, |state, mut block_env| {
1655            let SimulatePayload {
1656                block_state_calls,
1657                trace_transfers,
1658                validation,
1659                return_full_transactions,
1660            } = request;
1661            let mut cache_db = CacheDB::new(state);
1662            let mut block_res = Vec::with_capacity(block_state_calls.len());
1663
1664            // execute the blocks
1665            for block in block_state_calls {
1666                let SimBlock { block_overrides, state_overrides, calls } = block;
1667                let mut call_res = Vec::with_capacity(calls.len());
1668                let mut log_index = 0;
1669                let mut gas_used = 0;
1670                let mut transactions = Vec::with_capacity(calls.len());
1671                let mut logs= Vec::new();
1672
1673                // apply state overrides before executing the transactions
1674                if let Some(state_overrides) = state_overrides {
1675                    apply_state_overrides(state_overrides, &mut cache_db)?;
1676                }
1677                if let Some(block_overrides) = block_overrides {
1678                    cache_db.apply_block_overrides(block_overrides, &mut block_env);
1679                }
1680
1681                // execute all calls in that block
1682                for (req_idx, request) in calls.into_iter().enumerate() {
1683                    let fee_details = FeeDetails::new(
1684                        request.gas_price,
1685                        request.max_fee_per_gas,
1686                        request.max_priority_fee_per_gas,
1687                        request.max_fee_per_blob_gas,
1688                    )?
1689                    .or_zero_fees();
1690
1691                    let mut env = self.build_call_env(
1692                        WithOtherFields::new(request.clone()),
1693                        fee_details,
1694                        block_env.clone(),
1695                    );
1696
1697                    // Always disable EIP-3607
1698                    env.evm_env.cfg_env.disable_eip3607 = true;
1699
1700                    if !validation {
1701                        env.evm_env.cfg_env.disable_base_fee = !validation;
1702                        env.evm_env.block_env.basefee = 0;
1703                    }
1704
1705                    let mut inspector = self.build_inspector();
1706
1707                    // transact
1708                    let ResultAndState { result, state } = if trace_transfers {
1709                        // prepare inspector to capture transfer inside the evm so they are
1710                        // recorded and included in logs
1711                        inspector = inspector.with_transfers();
1712                        let mut evm= self.new_evm_with_inspector_ref(
1713                            &cache_db,
1714                            &env,
1715                            &mut inspector,
1716                        );
1717
1718                        trace!(target: "backend", env=?env.evm_env, spec=?env.evm_env.spec_id(),"simulate evm env");
1719                        evm.transact(env.tx)?
1720                    } else {
1721                        let mut evm = self.new_evm_with_inspector_ref(
1722                            &cache_db,
1723                            &env,
1724                            &mut inspector,
1725                        );
1726                        trace!(target: "backend", env=?env.evm_env, spec=?env.evm_env.spec_id(),"simulate evm env");
1727                        evm.transact(env.tx)?
1728                    };
1729                    trace!(target: "backend", ?result, ?request, "simulate call");
1730
1731                    inspector.print_logs();
1732                    if self.print_traces {
1733                        inspector.into_print_traces(self.call_trace_decoder.clone());
1734                    }
1735
1736                    // commit the transaction
1737                    cache_db.commit(state);
1738                    gas_used += result.gas_used();
1739
1740                    // create the transaction from a request
1741                    let from = request.from.unwrap_or_default();
1742                    let request = transaction_request_to_typed(WithOtherFields::new(request))
1743                        .ok_or(BlockchainError::MissingRequiredFields)?;
1744                    let tx = build_typed_transaction(
1745                        request,
1746                        Signature::new(Default::default(), Default::default(), false),
1747                    )?;
1748                    let tx_hash = tx.hash();
1749                    let rpc_tx = transaction_build(
1750                        None,
1751                        MaybeImpersonatedTransaction::impersonated(tx, from),
1752                        None,
1753                        None,
1754                        Some(block_env.basefee),
1755                    );
1756                    transactions.push(rpc_tx);
1757
1758                    let return_data = result.output().cloned().unwrap_or_default();
1759                    let sim_res = SimCallResult {
1760                        return_data,
1761                        gas_used: result.gas_used(),
1762                        status: result.is_success(),
1763                        error: result.is_success().not().then(|| {
1764                            alloy_rpc_types::simulate::SimulateError {
1765                                code: -3200,
1766                                message: "execution failed".to_string(),
1767                            }
1768                        }),
1769                        logs: result.clone()
1770                            .into_logs()
1771                            .into_iter()
1772                            .enumerate()
1773                            .map(|(idx, log)| Log {
1774                                inner: log,
1775                                block_number: Some(block_env.number.saturating_to()),
1776                                block_timestamp: Some(block_env.timestamp.saturating_to()),
1777                                transaction_index: Some(req_idx as u64),
1778                                log_index: Some((idx + log_index) as u64),
1779                                removed: false,
1780
1781                                block_hash: None,
1782                                transaction_hash: Some(tx_hash),
1783                            })
1784                            .collect(),
1785                    };
1786                    logs.extend(sim_res.logs.clone().iter().map(|log| log.inner.clone()));
1787                    log_index += sim_res.logs.len();
1788                    call_res.push(sim_res);
1789                }
1790
1791                let transactions_envelopes: Vec<AnyTxEnvelope> = transactions
1792                .iter()
1793                .map(|tx| AnyTxEnvelope::from(tx.clone()))
1794                .collect();
1795                let header = Header {
1796                    logs_bloom: logs_bloom(logs.iter()),
1797                    transactions_root: calculate_transaction_root(&transactions_envelopes),
1798                    receipts_root: calculate_receipt_root(&transactions_envelopes),
1799                    parent_hash: Default::default(),
1800                    beneficiary: block_env.beneficiary,
1801                    state_root: Default::default(),
1802                    difficulty: Default::default(),
1803                    number: block_env.number.saturating_to(),
1804                    gas_limit: block_env.gas_limit,
1805                    gas_used,
1806                    timestamp: block_env.timestamp.saturating_to(),
1807                    extra_data: Default::default(),
1808                    mix_hash: Default::default(),
1809                    nonce: Default::default(),
1810                    base_fee_per_gas: Some(block_env.basefee),
1811                    withdrawals_root: None,
1812                    blob_gas_used: None,
1813                    excess_blob_gas: None,
1814                    parent_beacon_block_root: None,
1815                    requests_hash: None,
1816                    ..Default::default()
1817                };
1818                let mut block = alloy_rpc_types::Block {
1819                    header: AnyRpcHeader {
1820                        hash: header.hash_slow(),
1821                        inner: header.into(),
1822                        total_difficulty: None,
1823                        size: None,
1824                    },
1825                    uncles: vec![],
1826                    transactions: BlockTransactions::Full(transactions),
1827                    withdrawals: None,
1828                };
1829
1830                if !return_full_transactions {
1831                    block.transactions.convert_to_hashes();
1832                }
1833
1834                for res in &mut call_res {
1835                    res.logs.iter_mut().for_each(|log| {
1836                        log.block_hash = Some(block.header.hash);
1837                    });
1838                }
1839
1840                let simulated_block = SimulatedBlock {
1841                    inner: AnyRpcBlock::new(WithOtherFields::new(block)),
1842                    calls: call_res,
1843                };
1844
1845                // update block env
1846                block_env.number += U256::from(1);
1847                block_env.timestamp += U256::from(12);
1848                block_env.basefee = simulated_block
1849                    .inner
1850                    .header
1851                    .next_block_base_fee(BaseFeeParams::ethereum())
1852                    .unwrap_or_default();
1853
1854                block_res.push(simulated_block);
1855            }
1856
1857            Ok(block_res)
1858        })
1859        .await?
1860    }
1861
1862    pub fn call_with_state(
1863        &self,
1864        state: &dyn DatabaseRef,
1865        request: WithOtherFields<TransactionRequest>,
1866        fee_details: FeeDetails,
1867        block_env: BlockEnv,
1868    ) -> Result<(InstructionResult, Option<Output>, u128, State), BlockchainError> {
1869        let mut inspector = self.build_inspector();
1870
1871        let env = self.build_call_env(request, fee_details, block_env);
1872        let mut evm = self.new_evm_with_inspector_ref(state, &env, &mut inspector);
1873        let ResultAndState { result, state } = evm.transact(env.tx)?;
1874        let (exit_reason, gas_used, out) = match result {
1875            ExecutionResult::Success { reason, gas_used, output, .. } => {
1876                (reason.into(), gas_used, Some(output))
1877            }
1878            ExecutionResult::Revert { gas_used, output } => {
1879                (InstructionResult::Revert, gas_used, Some(Output::Call(output)))
1880            }
1881            ExecutionResult::Halt { reason, gas_used } => {
1882                (op_haltreason_to_instruction_result(reason), gas_used, None)
1883            }
1884        };
1885        drop(evm);
1886        inspector.print_logs();
1887
1888        if self.print_traces {
1889            inspector.into_print_traces(self.call_trace_decoder.clone());
1890        }
1891
1892        Ok((exit_reason, out, gas_used as u128, state))
1893    }
1894
1895    pub async fn call_with_tracing(
1896        &self,
1897        request: WithOtherFields<TransactionRequest>,
1898        fee_details: FeeDetails,
1899        block_request: Option<BlockRequest>,
1900        opts: GethDebugTracingCallOptions,
1901    ) -> Result<GethTrace, BlockchainError> {
1902        let GethDebugTracingCallOptions { tracing_options, block_overrides, state_overrides } =
1903            opts;
1904        let GethDebugTracingOptions { config, tracer, tracer_config, .. } = tracing_options;
1905
1906        self.with_database_at(block_request, |state, mut block| {
1907            let block_number = block.number;
1908
1909            let mut cache_db = CacheDB::new(state);
1910            if let Some(state_overrides) = state_overrides {
1911                apply_state_overrides(state_overrides, &mut cache_db)?;
1912            }
1913            if let Some(block_overrides) = block_overrides {
1914                cache_db.apply_block_overrides(block_overrides, &mut block);
1915            }
1916
1917            if let Some(tracer) = tracer {
1918                return match tracer {
1919                    GethDebugTracerType::BuiltInTracer(tracer) => match tracer {
1920                        GethDebugBuiltInTracerType::CallTracer => {
1921                            let call_config = tracer_config
1922                                .into_call_config()
1923                                .map_err(|e| RpcError::invalid_params(e.to_string()))?;
1924
1925                            let mut inspector = self.build_inspector().with_tracing_config(
1926                                TracingInspectorConfig::from_geth_call_config(&call_config),
1927                            );
1928
1929                            let env = self.build_call_env(request, fee_details, block);
1930                            let mut evm =
1931                                self.new_evm_with_inspector_ref(&cache_db, &env, &mut inspector);
1932                            let ResultAndState { result, state: _ } = evm.transact(env.tx)?;
1933
1934                            drop(evm);
1935
1936                            inspector.print_logs();
1937                            if self.print_traces {
1938                                inspector.print_traces(self.call_trace_decoder.clone());
1939                            }
1940
1941                            let tracing_inspector = inspector.tracer.expect("tracer disappeared");
1942
1943                            Ok(tracing_inspector
1944                                .into_geth_builder()
1945                                .geth_call_traces(call_config, result.gas_used())
1946                                .into())
1947                        }
1948                        GethDebugBuiltInTracerType::NoopTracer => Ok(NoopFrame::default().into()),
1949                        GethDebugBuiltInTracerType::FourByteTracer
1950                        | GethDebugBuiltInTracerType::PreStateTracer
1951                        | GethDebugBuiltInTracerType::MuxTracer
1952                        | GethDebugBuiltInTracerType::FlatCallTracer => {
1953                            Err(RpcError::invalid_params("unsupported tracer type").into())
1954                        }
1955                    },
1956                    #[cfg(not(feature = "js-tracer"))]
1957                    GethDebugTracerType::JsTracer(_) => {
1958                        Err(RpcError::invalid_params("unsupported tracer type").into())
1959                    }
1960                    #[cfg(feature = "js-tracer")]
1961                    GethDebugTracerType::JsTracer(code) => {
1962                        use alloy_evm::IntoTxEnv;
1963                        let config = tracer_config.into_json();
1964                        let mut inspector =
1965                            revm_inspectors::tracing::js::JsInspector::new(code, config)
1966                                .map_err(|err| BlockchainError::Message(err.to_string()))?;
1967
1968                        let env = self.build_call_env(request, fee_details, block.clone());
1969                        let mut evm =
1970                            self.new_evm_with_inspector_ref(&cache_db, &env, &mut inspector);
1971                        let result = evm.transact(env.tx.clone())?;
1972                        let res = evm
1973                            .inspector_mut()
1974                            .json_result(result, &env.tx.into_tx_env(), &block, &cache_db)
1975                            .map_err(|err| BlockchainError::Message(err.to_string()))?;
1976
1977                        Ok(GethTrace::JS(res))
1978                    }
1979                };
1980            }
1981
1982            // defaults to StructLog tracer used since no tracer is specified
1983            let mut inspector = self
1984                .build_inspector()
1985                .with_tracing_config(TracingInspectorConfig::from_geth_config(&config));
1986
1987            let env = self.build_call_env(request, fee_details, block);
1988            let mut evm = self.new_evm_with_inspector_ref(&cache_db, &env, &mut inspector);
1989            let ResultAndState { result, state: _ } = evm.transact(env.tx)?;
1990
1991            let (exit_reason, gas_used, out) = match result {
1992                ExecutionResult::Success { reason, gas_used, output, .. } => {
1993                    (reason.into(), gas_used, Some(output))
1994                }
1995                ExecutionResult::Revert { gas_used, output } => {
1996                    (InstructionResult::Revert, gas_used, Some(Output::Call(output)))
1997                }
1998                ExecutionResult::Halt { reason, gas_used } => {
1999                    (op_haltreason_to_instruction_result(reason), gas_used, None)
2000                }
2001            };
2002
2003            drop(evm);
2004            let tracing_inspector = inspector.tracer.expect("tracer disappeared");
2005            let return_value = out.as_ref().map(|o| o.data().clone()).unwrap_or_default();
2006
2007            trace!(target: "backend", ?exit_reason, ?out, %gas_used, %block_number, "trace call");
2008
2009            let res = tracing_inspector
2010                .into_geth_builder()
2011                .geth_traces(gas_used, return_value, config)
2012                .into();
2013
2014            Ok(res)
2015        })
2016        .await?
2017    }
2018
2019    pub fn build_access_list_with_state(
2020        &self,
2021        state: &dyn DatabaseRef,
2022        request: WithOtherFields<TransactionRequest>,
2023        fee_details: FeeDetails,
2024        block_env: BlockEnv,
2025    ) -> Result<(InstructionResult, Option<Output>, u64, AccessList), BlockchainError> {
2026        let mut inspector =
2027            AccessListInspector::new(request.access_list.clone().unwrap_or_default());
2028
2029        let env = self.build_call_env(request, fee_details, block_env);
2030        let mut evm = self.new_evm_with_inspector_ref(state, &env, &mut inspector);
2031        let ResultAndState { result, state: _ } = evm.transact(env.tx)?;
2032        let (exit_reason, gas_used, out) = match result {
2033            ExecutionResult::Success { reason, gas_used, output, .. } => {
2034                (reason.into(), gas_used, Some(output))
2035            }
2036            ExecutionResult::Revert { gas_used, output } => {
2037                (InstructionResult::Revert, gas_used, Some(Output::Call(output)))
2038            }
2039            ExecutionResult::Halt { reason, gas_used } => {
2040                (op_haltreason_to_instruction_result(reason), gas_used, None)
2041            }
2042        };
2043        drop(evm);
2044        let access_list = inspector.access_list();
2045        Ok((exit_reason, out, gas_used, access_list))
2046    }
2047
2048    /// returns all receipts for the given transactions
2049    fn get_receipts(&self, tx_hashes: impl IntoIterator<Item = TxHash>) -> Vec<TypedReceipt> {
2050        let storage = self.blockchain.storage.read();
2051        let mut receipts = vec![];
2052
2053        for hash in tx_hashes {
2054            if let Some(tx) = storage.transactions.get(&hash) {
2055                receipts.push(tx.receipt.clone());
2056            }
2057        }
2058
2059        receipts
2060    }
2061
2062    /// Returns the logs of the block that match the filter
2063    async fn logs_for_block(
2064        &self,
2065        filter: Filter,
2066        hash: B256,
2067    ) -> Result<Vec<Log>, BlockchainError> {
2068        if let Some(block) = self.blockchain.get_block_by_hash(&hash) {
2069            return Ok(self.mined_logs_for_block(filter, block));
2070        }
2071
2072        if let Some(fork) = self.get_fork() {
2073            return Ok(fork.logs(&filter).await?);
2074        }
2075
2076        Ok(Vec::new())
2077    }
2078
2079    /// Returns all `Log`s mined by the node that were emitted in the `block` and match the `Filter`
2080    fn mined_logs_for_block(&self, filter: Filter, block: Block) -> Vec<Log> {
2081        let mut all_logs = Vec::new();
2082        let block_hash = block.header.hash_slow();
2083        let mut block_log_index = 0u32;
2084
2085        let storage = self.blockchain.storage.read();
2086
2087        for tx in block.transactions {
2088            let Some(tx) = storage.transactions.get(&tx.hash()) else {
2089                continue;
2090            };
2091
2092            let logs = tx.receipt.logs();
2093            let transaction_hash = tx.info.transaction_hash;
2094
2095            for log in logs {
2096                if filter.matches(log) {
2097                    all_logs.push(Log {
2098                        inner: log.clone(),
2099                        block_hash: Some(block_hash),
2100                        block_number: Some(block.header.number),
2101                        block_timestamp: Some(block.header.timestamp),
2102                        transaction_hash: Some(transaction_hash),
2103                        transaction_index: Some(tx.info.transaction_index),
2104                        log_index: Some(block_log_index as u64),
2105                        removed: false,
2106                    });
2107                }
2108                block_log_index += 1;
2109            }
2110        }
2111        all_logs
2112    }
2113
2114    /// Returns the logs that match the filter in the given range of blocks
2115    async fn logs_for_range(
2116        &self,
2117        filter: &Filter,
2118        mut from: u64,
2119        to: u64,
2120    ) -> Result<Vec<Log>, BlockchainError> {
2121        let mut all_logs = Vec::new();
2122
2123        // get the range that predates the fork if any
2124        if let Some(fork) = self.get_fork() {
2125            let mut to_on_fork = to;
2126
2127            if !fork.predates_fork(to) {
2128                // adjust the ranges
2129                to_on_fork = fork.block_number();
2130            }
2131
2132            if fork.predates_fork_inclusive(from) {
2133                // this data is only available on the forked client
2134                let filter = filter.clone().from_block(from).to_block(to_on_fork);
2135                all_logs = fork.logs(&filter).await?;
2136
2137                // update the range
2138                from = fork.block_number() + 1;
2139            }
2140        }
2141
2142        for number in from..=to {
2143            if let Some(block) = self.get_block(number) {
2144                all_logs.extend(self.mined_logs_for_block(filter.clone(), block));
2145            }
2146        }
2147
2148        Ok(all_logs)
2149    }
2150
2151    /// Returns the logs according to the filter
2152    pub async fn logs(&self, filter: Filter) -> Result<Vec<Log>, BlockchainError> {
2153        trace!(target: "backend", "get logs [{:?}]", filter);
2154        if let Some(hash) = filter.get_block_hash() {
2155            self.logs_for_block(filter, hash).await
2156        } else {
2157            let best = self.best_number();
2158            let to_block =
2159                self.convert_block_number(filter.block_option.get_to_block().copied()).min(best);
2160            let from_block =
2161                self.convert_block_number(filter.block_option.get_from_block().copied());
2162            if from_block > best {
2163                // requested log range does not exist yet
2164                return Ok(vec![]);
2165            }
2166
2167            self.logs_for_range(&filter, from_block, to_block).await
2168        }
2169    }
2170
2171    pub async fn block_by_hash(&self, hash: B256) -> Result<Option<AnyRpcBlock>, BlockchainError> {
2172        trace!(target: "backend", "get block by hash {:?}", hash);
2173        if let tx @ Some(_) = self.mined_block_by_hash(hash) {
2174            return Ok(tx);
2175        }
2176
2177        if let Some(fork) = self.get_fork() {
2178            return Ok(fork.block_by_hash(hash).await?);
2179        }
2180
2181        Ok(None)
2182    }
2183
2184    pub async fn block_by_hash_full(
2185        &self,
2186        hash: B256,
2187    ) -> Result<Option<AnyRpcBlock>, BlockchainError> {
2188        trace!(target: "backend", "get block by hash {:?}", hash);
2189        if let tx @ Some(_) = self.get_full_block(hash) {
2190            return Ok(tx);
2191        }
2192
2193        if let Some(fork) = self.get_fork() {
2194            return Ok(fork.block_by_hash_full(hash).await?);
2195        }
2196
2197        Ok(None)
2198    }
2199
2200    fn mined_block_by_hash(&self, hash: B256) -> Option<AnyRpcBlock> {
2201        let block = self.blockchain.get_block_by_hash(&hash)?;
2202        Some(self.convert_block(block))
2203    }
2204
2205    pub(crate) async fn mined_transactions_by_block_number(
2206        &self,
2207        number: BlockNumber,
2208    ) -> Option<Vec<AnyRpcTransaction>> {
2209        if let Some(block) = self.get_block(number) {
2210            return self.mined_transactions_in_block(&block);
2211        }
2212        None
2213    }
2214
2215    /// Returns all transactions given a block
2216    pub(crate) fn mined_transactions_in_block(
2217        &self,
2218        block: &Block,
2219    ) -> Option<Vec<AnyRpcTransaction>> {
2220        let mut transactions = Vec::with_capacity(block.transactions.len());
2221        let base_fee = block.header.base_fee_per_gas;
2222        let storage = self.blockchain.storage.read();
2223        for hash in block.transactions.iter().map(|tx| tx.hash()) {
2224            let info = storage.transactions.get(&hash)?.info.clone();
2225            let tx = block.transactions.get(info.transaction_index as usize)?.clone();
2226
2227            let tx = transaction_build(Some(hash), tx, Some(block), Some(info), base_fee);
2228            transactions.push(tx);
2229        }
2230        Some(transactions)
2231    }
2232
2233    pub async fn block_by_number(
2234        &self,
2235        number: BlockNumber,
2236    ) -> Result<Option<AnyRpcBlock>, BlockchainError> {
2237        trace!(target: "backend", "get block by number {:?}", number);
2238        if let tx @ Some(_) = self.mined_block_by_number(number) {
2239            return Ok(tx);
2240        }
2241
2242        if let Some(fork) = self.get_fork() {
2243            let number = self.convert_block_number(Some(number));
2244            if fork.predates_fork_inclusive(number) {
2245                return Ok(fork.block_by_number(number).await?);
2246            }
2247        }
2248
2249        Ok(None)
2250    }
2251
2252    pub async fn block_by_number_full(
2253        &self,
2254        number: BlockNumber,
2255    ) -> Result<Option<AnyRpcBlock>, BlockchainError> {
2256        trace!(target: "backend", "get block by number {:?}", number);
2257        if let tx @ Some(_) = self.get_full_block(number) {
2258            return Ok(tx);
2259        }
2260
2261        if let Some(fork) = self.get_fork() {
2262            let number = self.convert_block_number(Some(number));
2263            if fork.predates_fork_inclusive(number) {
2264                return Ok(fork.block_by_number_full(number).await?);
2265            }
2266        }
2267
2268        Ok(None)
2269    }
2270
2271    pub fn get_block(&self, id: impl Into<BlockId>) -> Option<Block> {
2272        let hash = match id.into() {
2273            BlockId::Hash(hash) => hash.block_hash,
2274            BlockId::Number(number) => {
2275                let storage = self.blockchain.storage.read();
2276                let slots_in_an_epoch = self.slots_in_an_epoch;
2277                match number {
2278                    BlockNumber::Latest => storage.best_hash,
2279                    BlockNumber::Earliest => storage.genesis_hash,
2280                    BlockNumber::Pending => return None,
2281                    BlockNumber::Number(num) => *storage.hashes.get(&num)?,
2282                    BlockNumber::Safe => {
2283                        if storage.best_number > (slots_in_an_epoch) {
2284                            *storage.hashes.get(&(storage.best_number - (slots_in_an_epoch)))?
2285                        } else {
2286                            storage.genesis_hash // treat the genesis block as safe "by definition"
2287                        }
2288                    }
2289                    BlockNumber::Finalized => {
2290                        if storage.best_number > (slots_in_an_epoch * 2) {
2291                            *storage.hashes.get(&(storage.best_number - (slots_in_an_epoch * 2)))?
2292                        } else {
2293                            storage.genesis_hash
2294                        }
2295                    }
2296                }
2297            }
2298        };
2299        self.get_block_by_hash(hash)
2300    }
2301
2302    pub fn get_block_by_hash(&self, hash: B256) -> Option<Block> {
2303        self.blockchain.get_block_by_hash(&hash)
2304    }
2305
2306    pub fn mined_block_by_number(&self, number: BlockNumber) -> Option<AnyRpcBlock> {
2307        let block = self.get_block(number)?;
2308        let mut block = self.convert_block(block);
2309        block.transactions.convert_to_hashes();
2310        Some(block)
2311    }
2312
2313    pub fn get_full_block(&self, id: impl Into<BlockId>) -> Option<AnyRpcBlock> {
2314        let block = self.get_block(id)?;
2315        let transactions = self.mined_transactions_in_block(&block)?;
2316        let mut block = self.convert_block(block);
2317        block.inner.transactions = BlockTransactions::Full(transactions);
2318
2319        Some(block)
2320    }
2321
2322    /// Takes a block as it's stored internally and returns the eth api conform block format.
2323    pub fn convert_block(&self, block: Block) -> AnyRpcBlock {
2324        let size = U256::from(alloy_rlp::encode(&block).len() as u32);
2325
2326        let Block { header, transactions, .. } = block;
2327
2328        let hash = header.hash_slow();
2329        let Header { number, withdrawals_root, .. } = header;
2330
2331        let block = AlloyBlock {
2332            header: AlloyHeader {
2333                inner: AnyHeader::from(header),
2334                hash,
2335                total_difficulty: Some(self.total_difficulty()),
2336                size: Some(size),
2337            },
2338            transactions: alloy_rpc_types::BlockTransactions::Hashes(
2339                transactions.into_iter().map(|tx| tx.hash()).collect(),
2340            ),
2341            uncles: vec![],
2342            withdrawals: withdrawals_root.map(|_| Default::default()),
2343        };
2344
2345        let mut block = WithOtherFields::new(block);
2346
2347        // If Arbitrum, apply chain specifics to converted block.
2348        if is_arbitrum(self.env.read().evm_env.cfg_env.chain_id) {
2349            // Set `l1BlockNumber` field.
2350            block.other.insert("l1BlockNumber".to_string(), number.into());
2351        }
2352
2353        AnyRpcBlock::from(block)
2354    }
2355
2356    /// Converts the `BlockNumber` into a numeric value
2357    ///
2358    /// # Errors
2359    ///
2360    /// returns an error if the requested number is larger than the current height
2361    pub async fn ensure_block_number<T: Into<BlockId>>(
2362        &self,
2363        block_id: Option<T>,
2364    ) -> Result<u64, BlockchainError> {
2365        let current = self.best_number();
2366        let requested =
2367            match block_id.map(Into::into).unwrap_or(BlockId::Number(BlockNumber::Latest)) {
2368                BlockId::Hash(hash) => {
2369                    self.block_by_hash(hash.block_hash)
2370                        .await?
2371                        .ok_or(BlockchainError::BlockNotFound)?
2372                        .header
2373                        .number
2374                }
2375                BlockId::Number(num) => match num {
2376                    BlockNumber::Latest | BlockNumber::Pending => current,
2377                    BlockNumber::Earliest => U64::ZERO.to::<u64>(),
2378                    BlockNumber::Number(num) => num,
2379                    BlockNumber::Safe => current.saturating_sub(self.slots_in_an_epoch),
2380                    BlockNumber::Finalized => current.saturating_sub(self.slots_in_an_epoch * 2),
2381                },
2382            };
2383
2384        if requested > current {
2385            Err(BlockchainError::BlockOutOfRange(current, requested))
2386        } else {
2387            Ok(requested)
2388        }
2389    }
2390
2391    pub fn convert_block_number(&self, block: Option<BlockNumber>) -> u64 {
2392        let current = self.best_number();
2393        match block.unwrap_or(BlockNumber::Latest) {
2394            BlockNumber::Latest | BlockNumber::Pending => current,
2395            BlockNumber::Earliest => 0,
2396            BlockNumber::Number(num) => num,
2397            BlockNumber::Safe => current.saturating_sub(self.slots_in_an_epoch),
2398            BlockNumber::Finalized => current.saturating_sub(self.slots_in_an_epoch * 2),
2399        }
2400    }
2401
2402    /// Helper function to execute a closure with the database at a specific block
2403    pub async fn with_database_at<F, T>(
2404        &self,
2405        block_request: Option<BlockRequest>,
2406        f: F,
2407    ) -> Result<T, BlockchainError>
2408    where
2409        F: FnOnce(Box<dyn MaybeFullDatabase + '_>, BlockEnv) -> T,
2410    {
2411        let block_number = match block_request {
2412            Some(BlockRequest::Pending(pool_transactions)) => {
2413                let result = self
2414                    .with_pending_block(pool_transactions, |state, block| {
2415                        let block = block.block;
2416                        let block = BlockEnv {
2417                            number: U256::from(block.header.number),
2418                            beneficiary: block.header.beneficiary,
2419                            timestamp: U256::from(block.header.timestamp),
2420                            difficulty: block.header.difficulty,
2421                            prevrandao: Some(block.header.mix_hash),
2422                            basefee: block.header.base_fee_per_gas.unwrap_or_default(),
2423                            gas_limit: block.header.gas_limit,
2424                            ..Default::default()
2425                        };
2426                        f(state, block)
2427                    })
2428                    .await;
2429                return Ok(result);
2430            }
2431            Some(BlockRequest::Number(bn)) => Some(BlockNumber::Number(bn)),
2432            None => None,
2433        };
2434        let block_number = self.convert_block_number(block_number);
2435
2436        if block_number < self.env.read().evm_env.block_env.number.saturating_to() {
2437            if let Some((block_hash, block)) = self
2438                .block_by_number(BlockNumber::Number(block_number))
2439                .await?
2440                .map(|block| (block.header.hash, block))
2441                && let Some(state) = self.states.write().get(&block_hash)
2442            {
2443                let block = BlockEnv {
2444                    number: U256::from(block_number),
2445                    beneficiary: block.header.beneficiary,
2446                    timestamp: U256::from(block.header.timestamp),
2447                    difficulty: block.header.difficulty,
2448                    prevrandao: block.header.mix_hash,
2449                    basefee: block.header.base_fee_per_gas.unwrap_or_default(),
2450                    gas_limit: block.header.gas_limit,
2451                    ..Default::default()
2452                };
2453                return Ok(f(Box::new(state), block));
2454            }
2455
2456            warn!(target: "backend", "Not historic state found for block={}", block_number);
2457            return Err(BlockchainError::BlockOutOfRange(
2458                self.env.read().evm_env.block_env.number.saturating_to(),
2459                block_number,
2460            ));
2461        }
2462
2463        let db = self.db.read().await;
2464        let block = self.env.read().evm_env.block_env.clone();
2465        Ok(f(Box::new(&**db), block))
2466    }
2467
2468    pub async fn storage_at(
2469        &self,
2470        address: Address,
2471        index: U256,
2472        block_request: Option<BlockRequest>,
2473    ) -> Result<B256, BlockchainError> {
2474        self.with_database_at(block_request, |db, _| {
2475            trace!(target: "backend", "get storage for {:?} at {:?}", address, index);
2476            let val = db.storage_ref(address, index)?;
2477            Ok(val.into())
2478        })
2479        .await?
2480    }
2481
2482    /// Returns the code of the address
2483    ///
2484    /// If the code is not present and fork mode is enabled then this will try to fetch it from the
2485    /// forked client
2486    pub async fn get_code(
2487        &self,
2488        address: Address,
2489        block_request: Option<BlockRequest>,
2490    ) -> Result<Bytes, BlockchainError> {
2491        self.with_database_at(block_request, |db, _| self.get_code_with_state(&db, address)).await?
2492    }
2493
2494    pub fn get_code_with_state(
2495        &self,
2496        state: &dyn DatabaseRef,
2497        address: Address,
2498    ) -> Result<Bytes, BlockchainError> {
2499        trace!(target: "backend", "get code for {:?}", address);
2500        let account = state.basic_ref(address)?.unwrap_or_default();
2501        if account.code_hash == KECCAK_EMPTY {
2502            // if the code hash is `KECCAK_EMPTY`, we check no further
2503            return Ok(Default::default());
2504        }
2505        let code = if let Some(code) = account.code {
2506            code
2507        } else {
2508            state.code_by_hash_ref(account.code_hash)?
2509        };
2510        Ok(code.bytes()[..code.len()].to_vec().into())
2511    }
2512
2513    /// Returns the balance of the address
2514    ///
2515    /// If the requested number predates the fork then this will fetch it from the endpoint
2516    pub async fn get_balance(
2517        &self,
2518        address: Address,
2519        block_request: Option<BlockRequest>,
2520    ) -> Result<U256, BlockchainError> {
2521        self.with_database_at(block_request, |db, _| self.get_balance_with_state(db, address))
2522            .await?
2523    }
2524
2525    pub async fn get_account_at_block(
2526        &self,
2527        address: Address,
2528        block_request: Option<BlockRequest>,
2529    ) -> Result<Account, BlockchainError> {
2530        self.with_database_at(block_request, |block_db, _| {
2531            let db = block_db.maybe_as_full_db().ok_or(BlockchainError::DataUnavailable)?;
2532            let account = db.get(&address).cloned().unwrap_or_default();
2533            let storage_root = storage_root(&account.storage);
2534            let code_hash = account.info.code_hash;
2535            let balance = account.info.balance;
2536            let nonce = account.info.nonce;
2537            Ok(Account { balance, nonce, code_hash, storage_root })
2538        })
2539        .await?
2540    }
2541
2542    pub fn get_balance_with_state<D>(
2543        &self,
2544        state: D,
2545        address: Address,
2546    ) -> Result<U256, BlockchainError>
2547    where
2548        D: DatabaseRef,
2549    {
2550        trace!(target: "backend", "get balance for {:?}", address);
2551        Ok(state.basic_ref(address)?.unwrap_or_default().balance)
2552    }
2553
2554    /// Returns the nonce of the address
2555    ///
2556    /// If the requested number predates the fork then this will fetch it from the endpoint
2557    pub async fn get_nonce(
2558        &self,
2559        address: Address,
2560        block_request: BlockRequest,
2561    ) -> Result<u64, BlockchainError> {
2562        if let BlockRequest::Pending(pool_transactions) = &block_request
2563            && let Some(value) = get_pool_transactions_nonce(pool_transactions, address)
2564        {
2565            return Ok(value);
2566        }
2567        let final_block_request = match block_request {
2568            BlockRequest::Pending(_) => BlockRequest::Number(self.best_number()),
2569            BlockRequest::Number(bn) => BlockRequest::Number(bn),
2570        };
2571
2572        self.with_database_at(Some(final_block_request), |db, _| {
2573            trace!(target: "backend", "get nonce for {:?}", address);
2574            Ok(db.basic_ref(address)?.unwrap_or_default().nonce)
2575        })
2576        .await?
2577    }
2578
2579    /// Returns the traces for the given transaction
2580    pub async fn trace_transaction(
2581        &self,
2582        hash: B256,
2583    ) -> Result<Vec<LocalizedTransactionTrace>, BlockchainError> {
2584        if let Some(traces) = self.mined_parity_trace_transaction(hash) {
2585            return Ok(traces);
2586        }
2587
2588        if let Some(fork) = self.get_fork() {
2589            return Ok(fork.trace_transaction(hash).await?);
2590        }
2591
2592        Ok(vec![])
2593    }
2594
2595    /// Returns the traces for the given transaction
2596    pub(crate) fn mined_parity_trace_transaction(
2597        &self,
2598        hash: B256,
2599    ) -> Option<Vec<LocalizedTransactionTrace>> {
2600        self.blockchain.storage.read().transactions.get(&hash).map(|tx| tx.parity_traces())
2601    }
2602
2603    /// Returns the traces for the given transaction
2604    pub(crate) fn mined_transaction(&self, hash: B256) -> Option<MinedTransaction> {
2605        self.blockchain.storage.read().transactions.get(&hash).cloned()
2606    }
2607
2608    /// Returns the traces for the given block
2609    pub(crate) fn mined_parity_trace_block(
2610        &self,
2611        block: u64,
2612    ) -> Option<Vec<LocalizedTransactionTrace>> {
2613        let block = self.get_block(block)?;
2614        let mut traces = vec![];
2615        let storage = self.blockchain.storage.read();
2616        for tx in block.transactions {
2617            traces.extend(storage.transactions.get(&tx.hash())?.parity_traces());
2618        }
2619        Some(traces)
2620    }
2621
2622    /// Returns the traces for the given transaction
2623    pub async fn debug_trace_transaction(
2624        &self,
2625        hash: B256,
2626        opts: GethDebugTracingOptions,
2627    ) -> Result<GethTrace, BlockchainError> {
2628        #[cfg(feature = "js-tracer")]
2629        if let Some(tracer_type) = opts.tracer.as_ref()
2630            && tracer_type.is_js()
2631        {
2632            return self
2633                .trace_tx_with_js_tracer(hash, tracer_type.as_str().to_string(), opts.clone())
2634                .await;
2635        }
2636
2637        if let Some(trace) = self.mined_geth_trace_transaction(hash, opts.clone()) {
2638            return trace;
2639        }
2640
2641        if let Some(fork) = self.get_fork() {
2642            return Ok(fork.debug_trace_transaction(hash, opts).await?);
2643        }
2644
2645        Ok(GethTrace::Default(Default::default()))
2646    }
2647
2648    /// Traces the transaction with the js tracer
2649    #[cfg(feature = "js-tracer")]
2650    pub async fn trace_tx_with_js_tracer(
2651        &self,
2652        hash: B256,
2653        code: String,
2654        opts: GethDebugTracingOptions,
2655    ) -> Result<GethTrace, BlockchainError> {
2656        let GethDebugTracingOptions { tracer_config, .. } = opts;
2657
2658        let block = {
2659            let storage = self.blockchain.storage.read();
2660            let MinedTransaction { block_hash, .. } = storage
2661                .transactions
2662                .get(&hash)
2663                .cloned()
2664                .ok_or(BlockchainError::TransactionNotFound)?;
2665
2666            storage.blocks.get(&block_hash).cloned().ok_or(BlockchainError::BlockNotFound)?
2667        };
2668
2669        let index = block
2670            .transactions
2671            .iter()
2672            .position(|tx| tx.hash() == hash)
2673            .expect("transaction not found in block");
2674
2675        let pool_txs: Vec<Arc<PoolTransaction>> = block.transactions[..index]
2676            .iter()
2677            .map(|tx| {
2678                let pending_tx =
2679                    PendingTransaction::from_maybe_impersonated(tx.clone()).expect("is valid");
2680                Arc::new(PoolTransaction {
2681                    pending_transaction: pending_tx,
2682                    requires: vec![],
2683                    provides: vec![],
2684                    priority: crate::eth::pool::transactions::TransactionPriority(0),
2685                })
2686            })
2687            .collect();
2688
2689        let mut states = self.states.write();
2690        let parent_state =
2691            states.get(&block.header.parent_hash).ok_or(BlockchainError::BlockNotFound)?;
2692        let mut cache_db = CacheDB::new(Box::new(parent_state));
2693
2694        // configure the blockenv for the block of the transaction
2695        let mut env = self.env.read().clone();
2696
2697        env.evm_env.block_env = BlockEnv {
2698            number: U256::from(block.header.number),
2699            beneficiary: block.header.beneficiary,
2700            timestamp: U256::from(block.header.timestamp),
2701            difficulty: block.header.difficulty,
2702            prevrandao: Some(block.header.mix_hash),
2703            basefee: block.header.base_fee_per_gas.unwrap_or_default(),
2704            gas_limit: block.header.gas_limit,
2705            ..Default::default()
2706        };
2707
2708        let executor = TransactionExecutor {
2709            db: &mut cache_db,
2710            validator: self,
2711            pending: pool_txs.into_iter(),
2712            block_env: env.evm_env.block_env.clone(),
2713            cfg_env: env.evm_env.cfg_env.clone(),
2714            parent_hash: block.header.parent_hash,
2715            gas_used: 0,
2716            blob_gas_used: 0,
2717            enable_steps_tracing: self.enable_steps_tracing,
2718            print_logs: self.print_logs,
2719            print_traces: self.print_traces,
2720            call_trace_decoder: self.call_trace_decoder.clone(),
2721            precompile_factory: self.precompile_factory.clone(),
2722            networks: self.env.read().networks,
2723            blob_params: self.blob_params(),
2724            cheats: self.cheats().clone(),
2725        };
2726
2727        let _ = executor.execute();
2728
2729        let target_tx = block.transactions[index].clone();
2730        let target_tx = PendingTransaction::from_maybe_impersonated(target_tx)?;
2731        let tx_env = target_tx.to_revm_tx_env();
2732
2733        let config = tracer_config.into_json();
2734        let mut inspector = revm_inspectors::tracing::js::JsInspector::new(code, config)
2735            .map_err(|err| BlockchainError::Message(err.to_string()))?;
2736        let mut evm = self.new_evm_with_inspector_ref(&cache_db, &env, &mut inspector);
2737
2738        let result = evm
2739            .transact(tx_env.clone())
2740            .map_err(|err| BlockchainError::Message(err.to_string()))?;
2741
2742        let trace = inspector
2743            .json_result(
2744                result,
2745                &alloy_evm::IntoTxEnv::into_tx_env(tx_env),
2746                &env.evm_env.block_env,
2747                &cache_db,
2748            )
2749            .map_err(|e| BlockchainError::Message(e.to_string()))?;
2750        Ok(GethTrace::JS(trace))
2751    }
2752
2753    /// Returns code by its hash
2754    pub async fn debug_code_by_hash(
2755        &self,
2756        code_hash: B256,
2757        block_id: Option<BlockId>,
2758    ) -> Result<Option<Bytes>, BlockchainError> {
2759        if let Ok(code) = self.db.read().await.code_by_hash_ref(code_hash) {
2760            return Ok(Some(code.original_bytes()));
2761        }
2762        if let Some(fork) = self.get_fork() {
2763            return Ok(fork.debug_code_by_hash(code_hash, block_id).await?);
2764        }
2765
2766        Ok(None)
2767    }
2768
2769    fn mined_geth_trace_transaction(
2770        &self,
2771        hash: B256,
2772        opts: GethDebugTracingOptions,
2773    ) -> Option<Result<GethTrace, BlockchainError>> {
2774        self.blockchain.storage.read().transactions.get(&hash).map(|tx| tx.geth_trace(opts))
2775    }
2776
2777    /// Returns the traces for the given block
2778    pub async fn trace_block(
2779        &self,
2780        block: BlockNumber,
2781    ) -> Result<Vec<LocalizedTransactionTrace>, BlockchainError> {
2782        let number = self.convert_block_number(Some(block));
2783        if let Some(traces) = self.mined_parity_trace_block(number) {
2784            return Ok(traces);
2785        }
2786
2787        if let Some(fork) = self.get_fork()
2788            && fork.predates_fork(number)
2789        {
2790            return Ok(fork.trace_block(number).await?);
2791        }
2792
2793        Ok(vec![])
2794    }
2795
2796    pub async fn transaction_receipt(
2797        &self,
2798        hash: B256,
2799    ) -> Result<Option<ReceiptResponse>, BlockchainError> {
2800        if let Some(receipt) = self.mined_transaction_receipt(hash) {
2801            return Ok(Some(receipt.inner));
2802        }
2803
2804        if let Some(fork) = self.get_fork() {
2805            let receipt = fork.transaction_receipt(hash).await?;
2806            let number = self.convert_block_number(
2807                receipt.clone().and_then(|r| r.block_number).map(BlockNumber::from),
2808            );
2809
2810            if fork.predates_fork_inclusive(number) {
2811                return Ok(receipt);
2812            }
2813        }
2814
2815        Ok(None)
2816    }
2817
2818    // Returns the traces matching a given filter
2819    pub async fn trace_filter(
2820        &self,
2821        filter: TraceFilter,
2822    ) -> Result<Vec<LocalizedTransactionTrace>, BlockchainError> {
2823        let matcher = filter.matcher();
2824        let start = filter.from_block.unwrap_or(0);
2825        let end = filter.to_block.unwrap_or_else(|| self.best_number());
2826
2827        if start > end {
2828            return Err(BlockchainError::RpcError(RpcError::invalid_params(
2829                "invalid block range, ensure that to block is greater than from block".to_string(),
2830            )));
2831        }
2832
2833        let dist = end - start;
2834        if dist > 300 {
2835            return Err(BlockchainError::RpcError(RpcError::invalid_params(
2836                "block range too large, currently limited to 300".to_string(),
2837            )));
2838        }
2839
2840        // Accumulate tasks for block range
2841        let mut trace_tasks = vec![];
2842        for num in start..=end {
2843            trace_tasks.push(self.trace_block(num.into()));
2844        }
2845
2846        // Execute tasks and filter traces
2847        let traces = futures::future::try_join_all(trace_tasks).await?;
2848        let filtered_traces =
2849            traces.into_iter().flatten().filter(|trace| matcher.matches(&trace.trace));
2850
2851        // Apply after and count
2852        let filtered_traces: Vec<_> = if let Some(after) = filter.after {
2853            filtered_traces.skip(after as usize).collect()
2854        } else {
2855            filtered_traces.collect()
2856        };
2857
2858        let filtered_traces: Vec<_> = if let Some(count) = filter.count {
2859            filtered_traces.into_iter().take(count as usize).collect()
2860        } else {
2861            filtered_traces
2862        };
2863
2864        Ok(filtered_traces)
2865    }
2866
2867    /// Returns all receipts of the block
2868    pub fn mined_receipts(&self, hash: B256) -> Option<Vec<TypedReceipt>> {
2869        let block = self.mined_block_by_hash(hash)?;
2870        let mut receipts = Vec::new();
2871        let storage = self.blockchain.storage.read();
2872        for tx in block.transactions.hashes() {
2873            let receipt = storage.transactions.get(&tx)?.receipt.clone();
2874            receipts.push(receipt);
2875        }
2876        Some(receipts)
2877    }
2878
2879    /// Returns all transaction receipts of the block
2880    pub fn mined_block_receipts(&self, id: impl Into<BlockId>) -> Option<Vec<ReceiptResponse>> {
2881        let mut receipts = Vec::new();
2882        let block = self.get_block(id)?;
2883
2884        for transaction in block.transactions {
2885            let receipt = self.mined_transaction_receipt(transaction.hash())?;
2886            receipts.push(receipt.inner);
2887        }
2888
2889        Some(receipts)
2890    }
2891
2892    /// Returns the transaction receipt for the given hash
2893    pub(crate) fn mined_transaction_receipt(&self, hash: B256) -> Option<MinedTransactionReceipt> {
2894        let MinedTransaction { info, receipt: tx_receipt, block_hash, .. } =
2895            self.blockchain.get_transaction_by_hash(&hash)?;
2896
2897        let index = info.transaction_index as usize;
2898        let block = self.blockchain.get_block_by_hash(&block_hash)?;
2899        let transaction = block.transactions[index].clone();
2900
2901        // Cancun specific
2902        let excess_blob_gas = block.header.excess_blob_gas;
2903        let blob_gas_price =
2904            alloy_eips::eip4844::calc_blob_gasprice(excess_blob_gas.unwrap_or_default());
2905        let blob_gas_used = transaction.blob_gas();
2906
2907        let effective_gas_price = match transaction.transaction {
2908            TypedTransaction::Legacy(t) => t.tx().gas_price,
2909            TypedTransaction::EIP2930(t) => t.tx().gas_price,
2910            TypedTransaction::EIP1559(t) => block
2911                .header
2912                .base_fee_per_gas
2913                .map_or(self.base_fee() as u128, |g| g as u128)
2914                .saturating_add(t.tx().max_priority_fee_per_gas),
2915            TypedTransaction::EIP4844(t) => block
2916                .header
2917                .base_fee_per_gas
2918                .map_or(self.base_fee() as u128, |g| g as u128)
2919                .saturating_add(t.tx().tx().max_priority_fee_per_gas),
2920            TypedTransaction::EIP7702(t) => block
2921                .header
2922                .base_fee_per_gas
2923                .map_or(self.base_fee() as u128, |g| g as u128)
2924                .saturating_add(t.tx().max_priority_fee_per_gas),
2925            TypedTransaction::Deposit(_) => 0_u128,
2926        };
2927
2928        let receipts = self.get_receipts(block.transactions.iter().map(|tx| tx.hash()));
2929        let next_log_index = receipts[..index].iter().map(|r| r.logs().len()).sum::<usize>();
2930
2931        let receipt = tx_receipt.as_receipt_with_bloom().receipt.clone();
2932        let receipt = Receipt {
2933            status: receipt.status,
2934            cumulative_gas_used: receipt.cumulative_gas_used,
2935            logs: receipt
2936                .logs
2937                .into_iter()
2938                .enumerate()
2939                .map(|(index, log)| alloy_rpc_types::Log {
2940                    inner: log,
2941                    block_hash: Some(block_hash),
2942                    block_number: Some(block.header.number),
2943                    block_timestamp: Some(block.header.timestamp),
2944                    transaction_hash: Some(info.transaction_hash),
2945                    transaction_index: Some(info.transaction_index),
2946                    log_index: Some((next_log_index + index) as u64),
2947                    removed: false,
2948                })
2949                .collect(),
2950        };
2951        let receipt_with_bloom =
2952            ReceiptWithBloom { receipt, logs_bloom: tx_receipt.as_receipt_with_bloom().logs_bloom };
2953
2954        let inner = match tx_receipt {
2955            TypedReceipt::EIP1559(_) => TypedReceipt::EIP1559(receipt_with_bloom),
2956            TypedReceipt::Legacy(_) => TypedReceipt::Legacy(receipt_with_bloom),
2957            TypedReceipt::EIP2930(_) => TypedReceipt::EIP2930(receipt_with_bloom),
2958            TypedReceipt::EIP4844(_) => TypedReceipt::EIP4844(receipt_with_bloom),
2959            TypedReceipt::EIP7702(_) => TypedReceipt::EIP7702(receipt_with_bloom),
2960            TypedReceipt::Deposit(r) => TypedReceipt::Deposit(DepositReceipt {
2961                inner: receipt_with_bloom,
2962                deposit_nonce: r.deposit_nonce,
2963                deposit_receipt_version: r.deposit_receipt_version,
2964            }),
2965        };
2966
2967        let inner = TransactionReceipt {
2968            inner,
2969            transaction_hash: info.transaction_hash,
2970            transaction_index: Some(info.transaction_index),
2971            block_number: Some(block.header.number),
2972            gas_used: info.gas_used,
2973            contract_address: info.contract_address,
2974            effective_gas_price,
2975            block_hash: Some(block_hash),
2976            from: info.from,
2977            to: info.to,
2978            blob_gas_price: Some(blob_gas_price),
2979            blob_gas_used,
2980        };
2981
2982        Some(MinedTransactionReceipt { inner, out: info.out.map(|o| o.0.into()) })
2983    }
2984
2985    /// Returns the blocks receipts for the given number
2986    pub async fn block_receipts(
2987        &self,
2988        number: BlockId,
2989    ) -> Result<Option<Vec<ReceiptResponse>>, BlockchainError> {
2990        if let Some(receipts) = self.mined_block_receipts(number) {
2991            return Ok(Some(receipts));
2992        }
2993
2994        if let Some(fork) = self.get_fork() {
2995            let number = match self.ensure_block_number(Some(number)).await {
2996                Err(_) => return Ok(None),
2997                Ok(n) => n,
2998            };
2999
3000            if fork.predates_fork_inclusive(number) {
3001                let receipts = fork.block_receipts(number).await?;
3002
3003                return Ok(receipts);
3004            }
3005        }
3006
3007        Ok(None)
3008    }
3009
3010    pub async fn transaction_by_block_number_and_index(
3011        &self,
3012        number: BlockNumber,
3013        index: Index,
3014    ) -> Result<Option<AnyRpcTransaction>, BlockchainError> {
3015        if let Some(block) = self.mined_block_by_number(number) {
3016            return Ok(self.mined_transaction_by_block_hash_and_index(block.header.hash, index));
3017        }
3018
3019        if let Some(fork) = self.get_fork() {
3020            let number = self.convert_block_number(Some(number));
3021            if fork.predates_fork(number) {
3022                return Ok(fork
3023                    .transaction_by_block_number_and_index(number, index.into())
3024                    .await?);
3025            }
3026        }
3027
3028        Ok(None)
3029    }
3030
3031    pub async fn transaction_by_block_hash_and_index(
3032        &self,
3033        hash: B256,
3034        index: Index,
3035    ) -> Result<Option<AnyRpcTransaction>, BlockchainError> {
3036        if let tx @ Some(_) = self.mined_transaction_by_block_hash_and_index(hash, index) {
3037            return Ok(tx);
3038        }
3039
3040        if let Some(fork) = self.get_fork() {
3041            return Ok(fork.transaction_by_block_hash_and_index(hash, index.into()).await?);
3042        }
3043
3044        Ok(None)
3045    }
3046
3047    pub fn mined_transaction_by_block_hash_and_index(
3048        &self,
3049        block_hash: B256,
3050        index: Index,
3051    ) -> Option<AnyRpcTransaction> {
3052        let (info, block, tx) = {
3053            let storage = self.blockchain.storage.read();
3054            let block = storage.blocks.get(&block_hash).cloned()?;
3055            let index: usize = index.into();
3056            let tx = block.transactions.get(index)?.clone();
3057            let info = storage.transactions.get(&tx.hash())?.info.clone();
3058            (info, block, tx)
3059        };
3060
3061        Some(transaction_build(
3062            Some(info.transaction_hash),
3063            tx,
3064            Some(&block),
3065            Some(info),
3066            block.header.base_fee_per_gas,
3067        ))
3068    }
3069
3070    pub async fn transaction_by_hash(
3071        &self,
3072        hash: B256,
3073    ) -> Result<Option<AnyRpcTransaction>, BlockchainError> {
3074        trace!(target: "backend", "transaction_by_hash={:?}", hash);
3075        if let tx @ Some(_) = self.mined_transaction_by_hash(hash) {
3076            return Ok(tx);
3077        }
3078
3079        if let Some(fork) = self.get_fork() {
3080            return fork
3081                .transaction_by_hash(hash)
3082                .await
3083                .map_err(BlockchainError::AlloyForkProvider);
3084        }
3085
3086        Ok(None)
3087    }
3088
3089    pub fn mined_transaction_by_hash(&self, hash: B256) -> Option<AnyRpcTransaction> {
3090        let (info, block) = {
3091            let storage = self.blockchain.storage.read();
3092            let MinedTransaction { info, block_hash, .. } =
3093                storage.transactions.get(&hash)?.clone();
3094            let block = storage.blocks.get(&block_hash).cloned()?;
3095            (info, block)
3096        };
3097        let tx = block.transactions.get(info.transaction_index as usize)?.clone();
3098
3099        Some(transaction_build(
3100            Some(info.transaction_hash),
3101            tx,
3102            Some(&block),
3103            Some(info),
3104            block.header.base_fee_per_gas,
3105        ))
3106    }
3107
3108    pub fn get_blob_by_tx_hash(&self, hash: B256) -> Result<Option<Vec<alloy_consensus::Blob>>> {
3109        // Try to get the mined transaction by hash
3110        if let Some(tx) = self.mined_transaction_by_hash(hash)
3111            && let Ok(typed_tx) = TypedTransaction::try_from(tx)
3112            && let Some(sidecar) = typed_tx.sidecar()
3113        {
3114            return Ok(Some(sidecar.sidecar.blobs.clone()));
3115        }
3116
3117        Ok(None)
3118    }
3119
3120    pub fn get_blob_by_versioned_hash(&self, hash: B256) -> Result<Option<Blob>> {
3121        let storage = self.blockchain.storage.read();
3122        for block in storage.blocks.values() {
3123            for tx in &block.transactions {
3124                let typed_tx = tx.as_ref();
3125                if let Some(sidecar) = typed_tx.sidecar() {
3126                    for versioned_hash in sidecar.sidecar.versioned_hashes() {
3127                        if versioned_hash == hash
3128                            && let Some(index) =
3129                                sidecar.sidecar.commitments.iter().position(|commitment| {
3130                                    kzg_to_versioned_hash(commitment.as_slice()) == *hash
3131                                })
3132                            && let Some(blob) = sidecar.sidecar.blobs.get(index)
3133                        {
3134                            return Ok(Some(*blob));
3135                        }
3136                    }
3137                }
3138            }
3139        }
3140        Ok(None)
3141    }
3142
3143    /// Overrides the given signature to impersonate the specified address during ecrecover.
3144    pub async fn impersonate_signature(
3145        &self,
3146        signature: Bytes,
3147        address: Address,
3148    ) -> Result<(), BlockchainError> {
3149        self.cheats.add_recover_override(signature, address);
3150        Ok(())
3151    }
3152
3153    /// Prove an account's existence or nonexistence in the state trie.
3154    ///
3155    /// Returns a merkle proof of the account's trie node, `account_key` == keccak(address)
3156    pub async fn prove_account_at(
3157        &self,
3158        address: Address,
3159        keys: Vec<B256>,
3160        block_request: Option<BlockRequest>,
3161    ) -> Result<AccountProof, BlockchainError> {
3162        let block_number = block_request.as_ref().map(|r| r.block_number());
3163
3164        self.with_database_at(block_request, |block_db, _| {
3165            trace!(target: "backend", "get proof for {:?} at {:?}", address, block_number);
3166            let db = block_db.maybe_as_full_db().ok_or(BlockchainError::DataUnavailable)?;
3167            let account = db.get(&address).cloned().unwrap_or_default();
3168
3169            let mut builder = HashBuilder::default()
3170                .with_proof_retainer(ProofRetainer::new(vec![Nibbles::unpack(keccak256(address))]));
3171
3172            for (key, account) in trie_accounts(db) {
3173                builder.add_leaf(key, &account);
3174            }
3175
3176            let _ = builder.root();
3177
3178            let proof = builder
3179                .take_proof_nodes()
3180                .into_nodes_sorted()
3181                .into_iter()
3182                .map(|(_, v)| v)
3183                .collect();
3184            let storage_proofs = prove_storage(&account.storage, &keys);
3185
3186            let account_proof = AccountProof {
3187                address,
3188                balance: account.info.balance,
3189                nonce: account.info.nonce,
3190                code_hash: account.info.code_hash,
3191                storage_hash: storage_root(&account.storage),
3192                account_proof: proof,
3193                storage_proof: keys
3194                    .into_iter()
3195                    .zip(storage_proofs)
3196                    .map(|(key, proof)| {
3197                        let storage_key: U256 = key.into();
3198                        let value = account.storage.get(&storage_key).copied().unwrap_or_default();
3199                        StorageProof { key: JsonStorageKey::Hash(key), value, proof }
3200                    })
3201                    .collect(),
3202            };
3203
3204            Ok(account_proof)
3205        })
3206        .await?
3207    }
3208
3209    /// Returns a new block event stream
3210    pub fn new_block_notifications(&self) -> NewBlockNotifications {
3211        let (tx, rx) = unbounded();
3212        self.new_block_listeners.lock().push(tx);
3213        trace!(target: "backed", "added new block listener");
3214        rx
3215    }
3216
3217    /// Notifies all `new_block_listeners` about the new block
3218    fn notify_on_new_block(&self, header: Header, hash: B256) {
3219        // cleanup closed notification streams first, if the channel is closed we can remove the
3220        // sender half for the set
3221        self.new_block_listeners.lock().retain(|tx| !tx.is_closed());
3222
3223        let notification = NewBlockNotification { hash, header: Arc::new(header) };
3224
3225        self.new_block_listeners
3226            .lock()
3227            .retain(|tx| tx.unbounded_send(notification.clone()).is_ok());
3228    }
3229
3230    /// Reorg the chain to a common height and execute blocks to build new chain.
3231    ///
3232    /// The state of the chain is rewound using `rewind` to the common block, including the db,
3233    /// storage, and env.
3234    ///
3235    /// Finally, `do_mine_block` is called to create the new chain.
3236    pub async fn reorg(
3237        &self,
3238        depth: u64,
3239        tx_pairs: HashMap<u64, Vec<Arc<PoolTransaction>>>,
3240        common_block: Block,
3241    ) -> Result<(), BlockchainError> {
3242        self.rollback(common_block).await?;
3243        // Create the new reorged chain, filling the blocks with transactions if supplied
3244        for i in 0..depth {
3245            let to_be_mined = tx_pairs.get(&i).cloned().unwrap_or_else(Vec::new);
3246            let outcome = self.do_mine_block(to_be_mined).await;
3247            node_info!(
3248                "    Mined reorg block number {}. With {} valid txs and with invalid {} txs",
3249                outcome.block_number,
3250                outcome.included.len(),
3251                outcome.invalid.len()
3252            );
3253        }
3254
3255        Ok(())
3256    }
3257
3258    /// Rollback the chain to a common height.
3259    ///
3260    /// The state of the chain is rewound using `rewind` to the common block, including the db,
3261    /// storage, and env.
3262    pub async fn rollback(&self, common_block: Block) -> Result<(), BlockchainError> {
3263        // Get the database at the common block
3264        let common_state = {
3265            let mut state = self.states.write();
3266            let state_db = state
3267                .get(&common_block.header.hash_slow())
3268                .ok_or(BlockchainError::DataUnavailable)?;
3269            let db_full = state_db.maybe_as_full_db().ok_or(BlockchainError::DataUnavailable)?;
3270            db_full.clone()
3271        };
3272
3273        {
3274            // Set state to common state
3275            self.db.write().await.clear();
3276            for (address, acc) in common_state {
3277                for (key, value) in acc.storage {
3278                    self.db.write().await.set_storage_at(address, key.into(), value.into())?;
3279                }
3280                self.db.write().await.insert_account(address, acc.info);
3281            }
3282        }
3283
3284        {
3285            // Unwind the storage back to the common ancestor
3286            self.blockchain
3287                .storage
3288                .write()
3289                .unwind_to(common_block.header.number, common_block.header.hash_slow());
3290
3291            // Set environment back to common block
3292            let mut env = self.env.write();
3293            env.evm_env.block_env.number = U256::from(common_block.header.number);
3294            env.evm_env.block_env.timestamp = U256::from(common_block.header.timestamp);
3295            env.evm_env.block_env.gas_limit = common_block.header.gas_limit;
3296            env.evm_env.block_env.difficulty = common_block.header.difficulty;
3297            env.evm_env.block_env.prevrandao = Some(common_block.header.mix_hash);
3298
3299            self.time.reset(env.evm_env.block_env.timestamp.saturating_to());
3300        }
3301        Ok(())
3302    }
3303}
3304
3305/// Get max nonce from transaction pool by address.
3306fn get_pool_transactions_nonce(
3307    pool_transactions: &[Arc<PoolTransaction>],
3308    address: Address,
3309) -> Option<u64> {
3310    if let Some(highest_nonce) = pool_transactions
3311        .iter()
3312        .filter(|tx| *tx.pending_transaction.sender() == address)
3313        .map(|tx| tx.pending_transaction.nonce())
3314        .max()
3315    {
3316        let tx_count = highest_nonce.saturating_add(1);
3317        return Some(tx_count);
3318    }
3319    None
3320}
3321
3322#[async_trait::async_trait]
3323impl TransactionValidator for Backend {
3324    async fn validate_pool_transaction(
3325        &self,
3326        tx: &PendingTransaction,
3327    ) -> Result<(), BlockchainError> {
3328        let address = *tx.sender();
3329        let account = self.get_account(address).await?;
3330        let env = self.next_env();
3331        Ok(self.validate_pool_transaction_for(tx, &account, &env)?)
3332    }
3333
3334    fn validate_pool_transaction_for(
3335        &self,
3336        pending: &PendingTransaction,
3337        account: &AccountInfo,
3338        env: &Env,
3339    ) -> Result<(), InvalidTransactionError> {
3340        let tx = &pending.transaction;
3341
3342        if let Some(tx_chain_id) = tx.chain_id() {
3343            let chain_id = self.chain_id();
3344            if chain_id.to::<u64>() != tx_chain_id {
3345                if let Some(legacy) = tx.as_legacy() {
3346                    // <https://github.com/ethereum/EIPs/blob/master/EIPS/eip-155.md>
3347                    if env.evm_env.cfg_env.spec >= SpecId::SPURIOUS_DRAGON
3348                        && legacy.tx().chain_id.is_none()
3349                    {
3350                        warn!(target: "backend", ?chain_id, ?tx_chain_id, "incompatible EIP155-based V");
3351                        return Err(InvalidTransactionError::IncompatibleEIP155);
3352                    }
3353                } else {
3354                    warn!(target: "backend", ?chain_id, ?tx_chain_id, "invalid chain id");
3355                    return Err(InvalidTransactionError::InvalidChainId);
3356                }
3357            }
3358        }
3359
3360        // Nonce validation
3361        let is_deposit_tx =
3362            matches!(&pending.transaction.transaction, TypedTransaction::Deposit(_));
3363        let nonce = tx.nonce();
3364        if nonce < account.nonce && !is_deposit_tx {
3365            warn!(target: "backend", "[{:?}] nonce too low", tx.hash());
3366            return Err(InvalidTransactionError::NonceTooLow);
3367        }
3368
3369        // EIP-4844 structural validation
3370        if env.evm_env.cfg_env.spec >= SpecId::CANCUN && tx.transaction.is_eip4844() {
3371            // Heavy (blob validation) checks
3372            let blob_tx = match &tx.transaction {
3373                TypedTransaction::EIP4844(tx) => tx.tx(),
3374                _ => unreachable!(),
3375            };
3376
3377            let blob_count = blob_tx.tx().blob_versioned_hashes.len();
3378
3379            // Ensure there are blob hashes.
3380            if blob_count == 0 {
3381                return Err(InvalidTransactionError::NoBlobHashes);
3382            }
3383
3384            // Ensure the tx does not exceed the max blobs per block.
3385            let max_blob_count = self.blob_params().max_blob_count as usize;
3386            if blob_count > max_blob_count {
3387                return Err(InvalidTransactionError::TooManyBlobs(blob_count, max_blob_count));
3388            }
3389
3390            // Check for any blob validation errors if not impersonating.
3391            if !self.skip_blob_validation(Some(*pending.sender()))
3392                && let Err(err) = blob_tx.validate(EnvKzgSettings::default().get())
3393            {
3394                return Err(InvalidTransactionError::BlobTransactionValidationError(err));
3395            }
3396        }
3397
3398        // Balance and fee related checks
3399        if !self.disable_pool_balance_checks {
3400            // Gas limit validation
3401            if tx.gas_limit() < MIN_TRANSACTION_GAS as u64 {
3402                warn!(target: "backend", "[{:?}] gas too low", tx.hash());
3403                return Err(InvalidTransactionError::GasTooLow);
3404            }
3405
3406            // Check tx gas limit against block gas limit, if block gas limit is set.
3407            if !env.evm_env.cfg_env.disable_block_gas_limit
3408                && tx.gas_limit() > env.evm_env.block_env.gas_limit
3409            {
3410                warn!(target: "backend", "[{:?}] gas too high", tx.hash());
3411                return Err(InvalidTransactionError::GasTooHigh(ErrDetail {
3412                    detail: String::from("tx.gas_limit > env.block.gas_limit"),
3413                }));
3414            }
3415
3416            // Check tx gas limit against tx gas limit cap (Osaka hard fork and later).
3417            if env.evm_env.cfg_env.tx_gas_limit_cap.is_none()
3418                && tx.gas_limit() > env.evm_env.cfg_env().tx_gas_limit_cap()
3419            {
3420                warn!(target: "backend", "[{:?}] gas too high", tx.hash());
3421                return Err(InvalidTransactionError::GasTooHigh(ErrDetail {
3422                    detail: String::from("tx.gas_limit > env.cfg.tx_gas_limit_cap"),
3423                }));
3424            }
3425
3426            // EIP-1559 fee validation (London hard fork and later).
3427            if env.evm_env.cfg_env.spec >= SpecId::LONDON {
3428                if tx.gas_price() < env.evm_env.block_env.basefee.into() && !is_deposit_tx {
3429                    warn!(target: "backend", "max fee per gas={}, too low, block basefee={}",tx.gas_price(),  env.evm_env.block_env.basefee);
3430                    return Err(InvalidTransactionError::FeeCapTooLow);
3431                }
3432
3433                if let (Some(max_priority_fee_per_gas), Some(max_fee_per_gas)) =
3434                    (tx.essentials().max_priority_fee_per_gas, tx.essentials().max_fee_per_gas)
3435                    && max_priority_fee_per_gas > max_fee_per_gas
3436                {
3437                    warn!(target: "backend", "max priority fee per gas={}, too high, max fee per gas={}", max_priority_fee_per_gas, max_fee_per_gas);
3438                    return Err(InvalidTransactionError::TipAboveFeeCap);
3439                }
3440            }
3441
3442            // EIP-4844 blob fee validation
3443            if env.evm_env.cfg_env.spec >= SpecId::CANCUN
3444                && tx.transaction.is_eip4844()
3445                && let Some(max_fee_per_blob_gas) = tx.essentials().max_fee_per_blob_gas
3446                && let Some(blob_gas_and_price) = &env.evm_env.block_env.blob_excess_gas_and_price
3447                && max_fee_per_blob_gas < blob_gas_and_price.blob_gasprice
3448            {
3449                warn!(target: "backend", "max fee per blob gas={}, too low, block blob gas price={}", max_fee_per_blob_gas, blob_gas_and_price.blob_gasprice);
3450                return Err(InvalidTransactionError::BlobFeeCapTooLow);
3451            }
3452
3453            let max_cost = tx.max_cost();
3454            let value = tx.value();
3455            match &tx.transaction {
3456                TypedTransaction::Deposit(deposit_tx) => {
3457                    // Deposit transactions
3458                    // https://specs.optimism.io/protocol/deposits.html#execution
3459                    // 1. no gas cost check required since already have prepaid gas from L1
3460                    // 2. increment account balance by deposited amount before checking for
3461                    //    sufficient funds `tx.value <= existing account value + deposited value`
3462                    if value > account.balance + U256::from(deposit_tx.mint) {
3463                        warn!(target: "backend", "[{:?}] insufficient balance={}, required={} account={:?}", tx.hash(), account.balance + U256::from(deposit_tx.mint), value, *pending.sender());
3464                        return Err(InvalidTransactionError::InsufficientFunds);
3465                    }
3466                }
3467                _ => {
3468                    // check sufficient funds: `gas * price + value`
3469                    let req_funds =
3470                        max_cost.checked_add(value.saturating_to()).ok_or_else(|| {
3471                            warn!(target: "backend", "[{:?}] cost too high", tx.hash());
3472                            InvalidTransactionError::InsufficientFunds
3473                        })?;
3474                    if account.balance < U256::from(req_funds) {
3475                        warn!(target: "backend", "[{:?}] insufficient allowance={}, required={} account={:?}", tx.hash(), account.balance, req_funds, *pending.sender());
3476                        return Err(InvalidTransactionError::InsufficientFunds);
3477                    }
3478                }
3479            }
3480        }
3481        Ok(())
3482    }
3483
3484    fn validate_for(
3485        &self,
3486        tx: &PendingTransaction,
3487        account: &AccountInfo,
3488        env: &Env,
3489    ) -> Result<(), InvalidTransactionError> {
3490        self.validate_pool_transaction_for(tx, account, env)?;
3491        if tx.nonce() > account.nonce {
3492            return Err(InvalidTransactionError::NonceTooHigh);
3493        }
3494        Ok(())
3495    }
3496}
3497
3498/// Creates a `AnyRpcTransaction` as it's expected for the `eth` RPC api from storage data
3499pub fn transaction_build(
3500    tx_hash: Option<B256>,
3501    eth_transaction: MaybeImpersonatedTransaction,
3502    block: Option<&Block>,
3503    info: Option<TransactionInfo>,
3504    base_fee: Option<u64>,
3505) -> AnyRpcTransaction {
3506    if let TypedTransaction::Deposit(ref deposit_tx) = eth_transaction.transaction {
3507        let dep_tx = deposit_tx;
3508
3509        let ser = serde_json::to_value(dep_tx).expect("could not serialize TxDeposit");
3510        let maybe_deposit_fields = OtherFields::try_from(ser);
3511
3512        match maybe_deposit_fields {
3513            Ok(mut fields) => {
3514                // Add zeroed signature fields for backwards compatibility
3515                // https://specs.optimism.io/protocol/deposits.html#the-deposited-transaction-type
3516                fields.insert("v".to_string(), serde_json::to_value("0x0").unwrap());
3517                fields.insert("r".to_string(), serde_json::to_value(B256::ZERO).unwrap());
3518                fields.insert(String::from("s"), serde_json::to_value(B256::ZERO).unwrap());
3519                fields.insert(String::from("nonce"), serde_json::to_value("0x0").unwrap());
3520
3521                let inner = UnknownTypedTransaction {
3522                    ty: AnyTxType(DEPOSIT_TX_TYPE_ID),
3523                    fields,
3524                    memo: Default::default(),
3525                };
3526
3527                let envelope = AnyTxEnvelope::Unknown(UnknownTxEnvelope {
3528                    hash: eth_transaction.hash(),
3529                    inner,
3530                });
3531
3532                let tx = Transaction {
3533                    inner: Recovered::new_unchecked(envelope, deposit_tx.from),
3534                    block_hash: block
3535                        .as_ref()
3536                        .map(|block| B256::from(keccak256(alloy_rlp::encode(&block.header)))),
3537                    block_number: block.as_ref().map(|block| block.header.number),
3538                    transaction_index: info.as_ref().map(|info| info.transaction_index),
3539                    effective_gas_price: None,
3540                };
3541
3542                return AnyRpcTransaction::from(WithOtherFields::new(tx));
3543            }
3544            Err(_) => {
3545                error!(target: "backend", "failed to serialize deposit transaction");
3546            }
3547        }
3548    }
3549
3550    let mut transaction: Transaction = eth_transaction.clone().into();
3551
3552    let effective_gas_price = if !eth_transaction.is_dynamic_fee() {
3553        transaction.effective_gas_price(base_fee)
3554    } else if block.is_none() && info.is_none() {
3555        // transaction is not mined yet, gas price is considered just `max_fee_per_gas`
3556        transaction.max_fee_per_gas()
3557    } else {
3558        // if transaction is already mined, gas price is considered base fee + priority
3559        // fee: the effective gas price.
3560        let base_fee = base_fee.map_or(0u128, |g| g as u128);
3561        let max_priority_fee_per_gas = transaction.max_priority_fee_per_gas().unwrap_or(0);
3562
3563        base_fee.saturating_add(max_priority_fee_per_gas)
3564    };
3565
3566    transaction.effective_gas_price = Some(effective_gas_price);
3567
3568    let envelope = transaction.inner;
3569
3570    // if a specific hash was provided we update the transaction's hash
3571    // This is important for impersonated transactions since they all use the
3572    // `BYPASS_SIGNATURE` which would result in different hashes
3573    // Note: for impersonated transactions this only concerns pending transactions because
3574    // there's // no `info` yet.
3575    let hash = tx_hash.unwrap_or(*envelope.tx_hash());
3576
3577    let envelope = match envelope.into_inner() {
3578        TxEnvelope::Legacy(signed_tx) => {
3579            let (t, sig, _) = signed_tx.into_parts();
3580            let new_signed = Signed::new_unchecked(t, sig, hash);
3581            AnyTxEnvelope::Ethereum(TxEnvelope::Legacy(new_signed))
3582        }
3583        TxEnvelope::Eip1559(signed_tx) => {
3584            let (t, sig, _) = signed_tx.into_parts();
3585            let new_signed = Signed::new_unchecked(t, sig, hash);
3586            AnyTxEnvelope::Ethereum(TxEnvelope::Eip1559(new_signed))
3587        }
3588        TxEnvelope::Eip2930(signed_tx) => {
3589            let (t, sig, _) = signed_tx.into_parts();
3590            let new_signed = Signed::new_unchecked(t, sig, hash);
3591            AnyTxEnvelope::Ethereum(TxEnvelope::Eip2930(new_signed))
3592        }
3593        TxEnvelope::Eip4844(signed_tx) => {
3594            let (t, sig, _) = signed_tx.into_parts();
3595            let new_signed = Signed::new_unchecked(t, sig, hash);
3596            AnyTxEnvelope::Ethereum(TxEnvelope::Eip4844(new_signed))
3597        }
3598        TxEnvelope::Eip7702(signed_tx) => {
3599            let (t, sig, _) = signed_tx.into_parts();
3600            let new_signed = Signed::new_unchecked(t, sig, hash);
3601            AnyTxEnvelope::Ethereum(TxEnvelope::Eip7702(new_signed))
3602        }
3603    };
3604
3605    let tx = Transaction {
3606        inner: Recovered::new_unchecked(
3607            envelope,
3608            eth_transaction.recover().expect("can recover signed tx"),
3609        ),
3610        block_hash: block
3611            .as_ref()
3612            .map(|block| B256::from(keccak256(alloy_rlp::encode(&block.header)))),
3613        block_number: block.as_ref().map(|block| block.header.number),
3614        transaction_index: info.as_ref().map(|info| info.transaction_index),
3615        // deprecated
3616        effective_gas_price: Some(effective_gas_price),
3617    };
3618    AnyRpcTransaction::from(WithOtherFields::new(tx))
3619}
3620
3621/// Prove a storage key's existence or nonexistence in the account's storage trie.
3622///
3623/// `storage_key` is the hash of the desired storage key, meaning
3624/// this will only work correctly under a secure trie.
3625/// `storage_key` == keccak(key)
3626pub fn prove_storage(storage: &HashMap<U256, U256>, keys: &[B256]) -> Vec<Vec<Bytes>> {
3627    let keys: Vec<_> = keys.iter().map(|key| Nibbles::unpack(keccak256(key))).collect();
3628
3629    let mut builder = HashBuilder::default().with_proof_retainer(ProofRetainer::new(keys.clone()));
3630
3631    for (key, value) in trie_storage(storage) {
3632        builder.add_leaf(key, &value);
3633    }
3634
3635    let _ = builder.root();
3636
3637    let mut proofs = Vec::new();
3638    let all_proof_nodes = builder.take_proof_nodes();
3639
3640    for proof_key in keys {
3641        // Iterate over all proof nodes and find the matching ones.
3642        // The filtered results are guaranteed to be in order.
3643        let matching_proof_nodes =
3644            all_proof_nodes.matching_nodes_sorted(&proof_key).into_iter().map(|(_, node)| node);
3645        proofs.push(matching_proof_nodes.collect());
3646    }
3647
3648    proofs
3649}
3650
3651pub fn is_arbitrum(chain_id: u64) -> bool {
3652    if let Ok(chain) = NamedChain::try_from(chain_id) {
3653        return chain.is_arbitrum();
3654    }
3655    false
3656}
3657
3658pub fn op_haltreason_to_instruction_result(op_reason: OpHaltReason) -> InstructionResult {
3659    match op_reason {
3660        OpHaltReason::Base(eth_h) => eth_h.into(),
3661        OpHaltReason::FailedDeposit => InstructionResult::Stop,
3662    }
3663}
3664
3665#[cfg(test)]
3666mod tests {
3667    use crate::{NodeConfig, spawn};
3668
3669    #[tokio::test]
3670    async fn test_deterministic_block_mining() {
3671        // Test that mine_block produces deterministic block hashes with same initial conditions
3672        let genesis_timestamp = 1743944919u64;
3673
3674        // Create two identical backends
3675        let config_a = NodeConfig::test().with_genesis_timestamp(genesis_timestamp.into());
3676        let config_b = NodeConfig::test().with_genesis_timestamp(genesis_timestamp.into());
3677
3678        let (api_a, _handle_a) = spawn(config_a).await;
3679        let (api_b, _handle_b) = spawn(config_b).await;
3680
3681        // Mine empty blocks (no transactions) on both backends
3682        let outcome_a_1 = api_a.backend.mine_block(vec![]).await;
3683        let outcome_b_1 = api_b.backend.mine_block(vec![]).await;
3684
3685        // Both should mine the same block number
3686        assert_eq!(outcome_a_1.block_number, outcome_b_1.block_number);
3687
3688        // Get the actual blocks to compare hashes
3689        let block_a_1 =
3690            api_a.block_by_number(outcome_a_1.block_number.into()).await.unwrap().unwrap();
3691        let block_b_1 =
3692            api_b.block_by_number(outcome_b_1.block_number.into()).await.unwrap().unwrap();
3693
3694        // The block hashes should be identical
3695        assert_eq!(
3696            block_a_1.header.hash, block_b_1.header.hash,
3697            "Block hashes should be deterministic. Got {} vs {}",
3698            block_a_1.header.hash, block_b_1.header.hash
3699        );
3700
3701        // Mine another block to ensure it remains deterministic
3702        let outcome_a_2 = api_a.backend.mine_block(vec![]).await;
3703        let outcome_b_2 = api_b.backend.mine_block(vec![]).await;
3704
3705        let block_a_2 =
3706            api_a.block_by_number(outcome_a_2.block_number.into()).await.unwrap().unwrap();
3707        let block_b_2 =
3708            api_b.block_by_number(outcome_b_2.block_number.into()).await.unwrap().unwrap();
3709
3710        assert_eq!(
3711            block_a_2.header.hash, block_b_2.header.hash,
3712            "Second block hashes should also be deterministic. Got {} vs {}",
3713            block_a_2.header.hash, block_b_2.header.hash
3714        );
3715
3716        // Ensure the blocks are different (sanity check)
3717        assert_ne!(
3718            block_a_1.header.hash, block_a_2.header.hash,
3719            "Different blocks should have different hashes"
3720        );
3721    }
3722}