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