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