anvil/eth/backend/mem/
mod.rs

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