anvil/eth/backend/mem/
mod.rs

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