anvil/eth/
api.rs

1use super::{
2    backend::{
3        db::MaybeFullDatabase,
4        mem::{state, BlockRequest, State},
5    },
6    sign::build_typed_transaction,
7};
8use crate::{
9    eth::{
10        backend::{
11            self,
12            db::SerializableState,
13            mem::{MIN_CREATE_GAS, MIN_TRANSACTION_GAS},
14            notifications::NewBlockNotifications,
15            validate::TransactionValidator,
16        },
17        error::{
18            BlockchainError, FeeHistoryError, InvalidTransactionError, Result, ToRpcResponseResult,
19        },
20        fees::{FeeDetails, FeeHistoryCache, MIN_SUGGESTED_PRIORITY_FEE},
21        macros::node_info,
22        miner::FixedBlockTimeMiner,
23        pool::{
24            transactions::{
25                to_marker, PoolTransaction, TransactionOrder, TransactionPriority, TxMarker,
26            },
27            Pool,
28        },
29        sign::{self, Signer},
30    },
31    filter::{EthFilter, Filters, LogsFilter},
32    mem::transaction_build,
33    ClientFork, LoggingManager, Miner, MiningMode, StorageInfo,
34};
35use alloy_consensus::{
36    transaction::{eip4844::TxEip4844Variant, Recovered},
37    Account,
38};
39use alloy_dyn_abi::TypedData;
40use alloy_eips::eip2718::Encodable2718;
41use alloy_network::{
42    eip2718::Decodable2718, AnyRpcBlock, AnyRpcTransaction, BlockResponse, Ethereum, NetworkWallet,
43    TransactionBuilder, TransactionResponse,
44};
45use alloy_primitives::{
46    map::{HashMap, HashSet},
47    Address, Bytes, Signature, TxHash, TxKind, B256, B64, U256, U64,
48};
49use alloy_provider::utils::{
50    eip1559_default_estimator, EIP1559_FEE_ESTIMATION_PAST_BLOCKS,
51    EIP1559_FEE_ESTIMATION_REWARD_PERCENTILE,
52};
53use alloy_rpc_types::{
54    anvil::{
55        ForkedNetwork, Forking, Metadata, MineOptions, NodeEnvironment, NodeForkConfig, NodeInfo,
56    },
57    request::TransactionRequest,
58    simulate::{SimulatePayload, SimulatedBlock},
59    state::{AccountOverride, EvmOverrides, StateOverridesBuilder},
60    trace::{
61        filter::TraceFilter,
62        geth::{GethDebugTracingCallOptions, GethDebugTracingOptions, GethTrace},
63        parity::LocalizedTransactionTrace,
64    },
65    txpool::{TxpoolContent, TxpoolInspect, TxpoolInspectSummary, TxpoolStatus},
66    AccessList, AccessListResult, BlockId, BlockNumberOrTag as BlockNumber, BlockTransactions,
67    EIP1186AccountProofResponse, FeeHistory, Filter, FilteredParams, Index, Log, Work,
68};
69use alloy_serde::WithOtherFields;
70use alloy_sol_types::{sol, SolCall, SolValue};
71use alloy_transport::TransportErrorKind;
72use anvil_core::{
73    eth::{
74        block::BlockInfo,
75        transaction::{
76            transaction_request_to_typed, PendingTransaction, ReceiptResponse, TypedTransaction,
77            TypedTransactionRequest,
78        },
79        wallet::{WalletCapabilities, WalletError},
80        EthRequest,
81    },
82    types::{ReorgOptions, TransactionData},
83};
84use anvil_rpc::{error::RpcError, response::ResponseResult};
85use foundry_common::provider::ProviderBuilder;
86use foundry_evm::{backend::DatabaseError, decode::RevertDecoder};
87use futures::{
88    channel::{mpsc::Receiver, oneshot},
89    StreamExt,
90};
91use parking_lot::RwLock;
92use revm::{
93    bytecode::Bytecode,
94    context::BlockEnv,
95    context_interface::{block::BlobExcessGasAndPrice, result::Output},
96    database::{CacheDB, DatabaseRef},
97    interpreter::{return_ok, return_revert, InstructionResult},
98    primitives::eip7702::PER_EMPTY_ACCOUNT_COST,
99};
100use std::{future::Future, sync::Arc, time::Duration};
101use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver};
102
103/// The client version: `anvil/v{major}.{minor}.{patch}`
104pub const CLIENT_VERSION: &str = concat!("anvil/v", env!("CARGO_PKG_VERSION"));
105
106/// The entry point for executing eth api RPC call - The Eth RPC interface.
107///
108/// This type is cheap to clone and can be used concurrently
109#[derive(Clone)]
110pub struct EthApi {
111    /// The transaction pool
112    pool: Arc<Pool>,
113    /// Holds all blockchain related data
114    /// In-Memory only for now
115    pub backend: Arc<backend::mem::Backend>,
116    /// Whether this node is mining
117    is_mining: bool,
118    /// available signers
119    signers: Arc<Vec<Box<dyn Signer>>>,
120    /// data required for `eth_feeHistory`
121    fee_history_cache: FeeHistoryCache,
122    /// max number of items kept in fee cache
123    fee_history_limit: u64,
124    /// access to the actual miner
125    ///
126    /// This access is required in order to adjust miner settings based on requests received from
127    /// custom RPC endpoints
128    miner: Miner,
129    /// allows to enabled/disable logging
130    logger: LoggingManager,
131    /// Tracks all active filters
132    filters: Filters,
133    /// How transactions are ordered in the pool
134    transaction_order: Arc<RwLock<TransactionOrder>>,
135    /// Whether we're listening for RPC calls
136    net_listening: bool,
137    /// The instance ID. Changes on every reset.
138    instance_id: Arc<RwLock<B256>>,
139}
140
141impl EthApi {
142    /// Creates a new instance
143    #[expect(clippy::too_many_arguments)]
144    pub fn new(
145        pool: Arc<Pool>,
146        backend: Arc<backend::mem::Backend>,
147        signers: Arc<Vec<Box<dyn Signer>>>,
148        fee_history_cache: FeeHistoryCache,
149        fee_history_limit: u64,
150        miner: Miner,
151        logger: LoggingManager,
152        filters: Filters,
153        transactions_order: TransactionOrder,
154    ) -> Self {
155        Self {
156            pool,
157            backend,
158            is_mining: true,
159            signers,
160            fee_history_cache,
161            fee_history_limit,
162            miner,
163            logger,
164            filters,
165            net_listening: true,
166            transaction_order: Arc::new(RwLock::new(transactions_order)),
167            instance_id: Arc::new(RwLock::new(B256::random())),
168        }
169    }
170
171    /// Executes the [EthRequest] and returns an RPC [ResponseResult].
172    pub async fn execute(&self, request: EthRequest) -> ResponseResult {
173        trace!(target: "rpc::api", "executing eth request");
174        let response = match request.clone() {
175            EthRequest::Web3ClientVersion(()) => self.client_version().to_rpc_result(),
176            EthRequest::Web3Sha3(content) => self.sha3(content).to_rpc_result(),
177            EthRequest::EthGetAccount(addr, block) => {
178                self.get_account(addr, block).await.to_rpc_result()
179            }
180            EthRequest::EthGetAccountInfo(addr, block) => {
181                self.get_account_info(addr, block).await.to_rpc_result()
182            }
183            EthRequest::EthGetBalance(addr, block) => {
184                self.balance(addr, block).await.to_rpc_result()
185            }
186            EthRequest::EthGetTransactionByHash(hash) => {
187                self.transaction_by_hash(hash).await.to_rpc_result()
188            }
189            EthRequest::EthSendTransaction(request) => {
190                self.send_transaction(*request).await.to_rpc_result()
191            }
192            EthRequest::EthChainId(_) => self.eth_chain_id().to_rpc_result(),
193            EthRequest::EthNetworkId(_) => self.network_id().to_rpc_result(),
194            EthRequest::NetListening(_) => self.net_listening().to_rpc_result(),
195            EthRequest::EthGasPrice(_) => self.eth_gas_price().to_rpc_result(),
196            EthRequest::EthMaxPriorityFeePerGas(_) => {
197                self.gas_max_priority_fee_per_gas().to_rpc_result()
198            }
199            EthRequest::EthBlobBaseFee(_) => self.blob_base_fee().to_rpc_result(),
200            EthRequest::EthAccounts(_) => self.accounts().to_rpc_result(),
201            EthRequest::EthBlockNumber(_) => self.block_number().to_rpc_result(),
202            EthRequest::EthGetStorageAt(addr, slot, block) => {
203                self.storage_at(addr, slot, block).await.to_rpc_result()
204            }
205            EthRequest::EthGetBlockByHash(hash, full) => {
206                if full {
207                    self.block_by_hash_full(hash).await.to_rpc_result()
208                } else {
209                    self.block_by_hash(hash).await.to_rpc_result()
210                }
211            }
212            EthRequest::EthGetBlockByNumber(num, full) => {
213                if full {
214                    self.block_by_number_full(num).await.to_rpc_result()
215                } else {
216                    self.block_by_number(num).await.to_rpc_result()
217                }
218            }
219            EthRequest::EthGetTransactionCount(addr, block) => {
220                self.transaction_count(addr, block).await.to_rpc_result()
221            }
222            EthRequest::EthGetTransactionCountByHash(hash) => {
223                self.block_transaction_count_by_hash(hash).await.to_rpc_result()
224            }
225            EthRequest::EthGetTransactionCountByNumber(num) => {
226                self.block_transaction_count_by_number(num).await.to_rpc_result()
227            }
228            EthRequest::EthGetUnclesCountByHash(hash) => {
229                self.block_uncles_count_by_hash(hash).await.to_rpc_result()
230            }
231            EthRequest::EthGetUnclesCountByNumber(num) => {
232                self.block_uncles_count_by_number(num).await.to_rpc_result()
233            }
234            EthRequest::EthGetCodeAt(addr, block) => {
235                self.get_code(addr, block).await.to_rpc_result()
236            }
237            EthRequest::EthGetProof(addr, keys, block) => {
238                self.get_proof(addr, keys, block).await.to_rpc_result()
239            }
240            EthRequest::EthSign(addr, content) => self.sign(addr, content).await.to_rpc_result(),
241            EthRequest::PersonalSign(content, addr) => {
242                self.sign(addr, content).await.to_rpc_result()
243            }
244            EthRequest::EthSignTransaction(request) => {
245                self.sign_transaction(*request).await.to_rpc_result()
246            }
247            EthRequest::EthSignTypedData(addr, data) => {
248                self.sign_typed_data(addr, data).await.to_rpc_result()
249            }
250            EthRequest::EthSignTypedDataV3(addr, data) => {
251                self.sign_typed_data_v3(addr, data).await.to_rpc_result()
252            }
253            EthRequest::EthSignTypedDataV4(addr, data) => {
254                self.sign_typed_data_v4(addr, &data).await.to_rpc_result()
255            }
256            EthRequest::EthSendRawTransaction(tx) => {
257                self.send_raw_transaction(tx).await.to_rpc_result()
258            }
259            EthRequest::EthCall(call, block, state_override, block_overrides) => self
260                .call(call, block, EvmOverrides::new(state_override, block_overrides))
261                .await
262                .to_rpc_result(),
263            EthRequest::EthSimulateV1(simulation, block) => {
264                self.simulate_v1(simulation, block).await.to_rpc_result()
265            }
266            EthRequest::EthCreateAccessList(call, block) => {
267                self.create_access_list(call, block).await.to_rpc_result()
268            }
269            EthRequest::EthEstimateGas(call, block, state_override, block_overrides) => self
270                .estimate_gas(call, block, EvmOverrides::new(state_override, block_overrides))
271                .await
272                .to_rpc_result(),
273            EthRequest::EthGetRawTransactionByHash(hash) => {
274                self.raw_transaction(hash).await.to_rpc_result()
275            }
276            EthRequest::EthGetRawTransactionByBlockHashAndIndex(hash, index) => {
277                self.raw_transaction_by_block_hash_and_index(hash, index).await.to_rpc_result()
278            }
279            EthRequest::EthGetRawTransactionByBlockNumberAndIndex(num, index) => {
280                self.raw_transaction_by_block_number_and_index(num, index).await.to_rpc_result()
281            }
282            EthRequest::EthGetTransactionByBlockHashAndIndex(hash, index) => {
283                self.transaction_by_block_hash_and_index(hash, index).await.to_rpc_result()
284            }
285            EthRequest::EthGetTransactionByBlockNumberAndIndex(num, index) => {
286                self.transaction_by_block_number_and_index(num, index).await.to_rpc_result()
287            }
288            EthRequest::EthGetTransactionReceipt(tx) => {
289                self.transaction_receipt(tx).await.to_rpc_result()
290            }
291            EthRequest::EthGetBlockReceipts(number) => {
292                self.block_receipts(number).await.to_rpc_result()
293            }
294            EthRequest::EthGetUncleByBlockHashAndIndex(hash, index) => {
295                self.uncle_by_block_hash_and_index(hash, index).await.to_rpc_result()
296            }
297            EthRequest::EthGetUncleByBlockNumberAndIndex(num, index) => {
298                self.uncle_by_block_number_and_index(num, index).await.to_rpc_result()
299            }
300            EthRequest::EthGetLogs(filter) => self.logs(filter).await.to_rpc_result(),
301            EthRequest::EthGetWork(_) => self.work().to_rpc_result(),
302            EthRequest::EthSyncing(_) => self.syncing().to_rpc_result(),
303            EthRequest::EthSubmitWork(nonce, pow, digest) => {
304                self.submit_work(nonce, pow, digest).to_rpc_result()
305            }
306            EthRequest::EthSubmitHashRate(rate, id) => {
307                self.submit_hashrate(rate, id).to_rpc_result()
308            }
309            EthRequest::EthFeeHistory(count, newest, reward_percentiles) => {
310                self.fee_history(count, newest, reward_percentiles).await.to_rpc_result()
311            }
312            // non eth-standard rpc calls
313            EthRequest::DebugGetRawTransaction(hash) => {
314                self.raw_transaction(hash).await.to_rpc_result()
315            }
316            // non eth-standard rpc calls
317            EthRequest::DebugTraceTransaction(tx, opts) => {
318                self.debug_trace_transaction(tx, opts).await.to_rpc_result()
319            }
320            // non eth-standard rpc calls
321            EthRequest::DebugTraceCall(tx, block, opts) => {
322                self.debug_trace_call(tx, block, opts).await.to_rpc_result()
323            }
324            EthRequest::TraceTransaction(tx) => self.trace_transaction(tx).await.to_rpc_result(),
325            EthRequest::TraceBlock(block) => self.trace_block(block).await.to_rpc_result(),
326            EthRequest::TraceFilter(filter) => self.trace_filter(filter).await.to_rpc_result(),
327            EthRequest::ImpersonateAccount(addr) => {
328                self.anvil_impersonate_account(addr).await.to_rpc_result()
329            }
330            EthRequest::StopImpersonatingAccount(addr) => {
331                self.anvil_stop_impersonating_account(addr).await.to_rpc_result()
332            }
333            EthRequest::AutoImpersonateAccount(enable) => {
334                self.anvil_auto_impersonate_account(enable).await.to_rpc_result()
335            }
336            EthRequest::GetAutoMine(()) => self.anvil_get_auto_mine().to_rpc_result(),
337            EthRequest::Mine(blocks, interval) => {
338                self.anvil_mine(blocks, interval).await.to_rpc_result()
339            }
340            EthRequest::SetAutomine(enabled) => {
341                self.anvil_set_auto_mine(enabled).await.to_rpc_result()
342            }
343            EthRequest::SetIntervalMining(interval) => {
344                self.anvil_set_interval_mining(interval).to_rpc_result()
345            }
346            EthRequest::GetIntervalMining(()) => self.anvil_get_interval_mining().to_rpc_result(),
347            EthRequest::DropTransaction(tx) => {
348                self.anvil_drop_transaction(tx).await.to_rpc_result()
349            }
350            EthRequest::DropAllTransactions() => {
351                self.anvil_drop_all_transactions().await.to_rpc_result()
352            }
353            EthRequest::Reset(fork) => {
354                self.anvil_reset(fork.and_then(|p| p.params)).await.to_rpc_result()
355            }
356            EthRequest::SetBalance(addr, val) => {
357                self.anvil_set_balance(addr, val).await.to_rpc_result()
358            }
359            EthRequest::AddBalance(addr, val) => {
360                self.anvil_add_balance(addr, val).await.to_rpc_result()
361            }
362            EthRequest::DealERC20(addr, token_addr, val) => {
363                self.anvil_deal_erc20(addr, token_addr, val).await.to_rpc_result()
364            }
365            EthRequest::SetCode(addr, code) => {
366                self.anvil_set_code(addr, code).await.to_rpc_result()
367            }
368            EthRequest::SetNonce(addr, nonce) => {
369                self.anvil_set_nonce(addr, nonce).await.to_rpc_result()
370            }
371            EthRequest::SetStorageAt(addr, slot, val) => {
372                self.anvil_set_storage_at(addr, slot, val).await.to_rpc_result()
373            }
374            EthRequest::SetCoinbase(addr) => self.anvil_set_coinbase(addr).await.to_rpc_result(),
375            EthRequest::SetChainId(id) => self.anvil_set_chain_id(id).await.to_rpc_result(),
376            EthRequest::SetLogging(log) => self.anvil_set_logging(log).await.to_rpc_result(),
377            EthRequest::SetMinGasPrice(gas) => {
378                self.anvil_set_min_gas_price(gas).await.to_rpc_result()
379            }
380            EthRequest::SetNextBlockBaseFeePerGas(gas) => {
381                self.anvil_set_next_block_base_fee_per_gas(gas).await.to_rpc_result()
382            }
383            EthRequest::DumpState(preserve_historical_states) => self
384                .anvil_dump_state(preserve_historical_states.and_then(|s| s.params))
385                .await
386                .to_rpc_result(),
387            EthRequest::LoadState(buf) => self.anvil_load_state(buf).await.to_rpc_result(),
388            EthRequest::NodeInfo(_) => self.anvil_node_info().await.to_rpc_result(),
389            EthRequest::AnvilMetadata(_) => self.anvil_metadata().await.to_rpc_result(),
390            EthRequest::EvmSnapshot(_) => self.evm_snapshot().await.to_rpc_result(),
391            EthRequest::EvmRevert(id) => self.evm_revert(id).await.to_rpc_result(),
392            EthRequest::EvmIncreaseTime(time) => self.evm_increase_time(time).await.to_rpc_result(),
393            EthRequest::EvmSetNextBlockTimeStamp(time) => {
394                if time >= U256::from(u64::MAX) {
395                    return ResponseResult::Error(RpcError::invalid_params(
396                        "The timestamp is too big",
397                    ))
398                }
399                let time = time.to::<u64>();
400                self.evm_set_next_block_timestamp(time).to_rpc_result()
401            }
402            EthRequest::EvmSetTime(timestamp) => {
403                if timestamp >= U256::from(u64::MAX) {
404                    return ResponseResult::Error(RpcError::invalid_params(
405                        "The timestamp is too big",
406                    ))
407                }
408                let time = timestamp.to::<u64>();
409                self.evm_set_time(time).to_rpc_result()
410            }
411            EthRequest::EvmSetBlockGasLimit(gas_limit) => {
412                self.evm_set_block_gas_limit(gas_limit).to_rpc_result()
413            }
414            EthRequest::EvmSetBlockTimeStampInterval(time) => {
415                self.evm_set_block_timestamp_interval(time).to_rpc_result()
416            }
417            EthRequest::EvmRemoveBlockTimeStampInterval(()) => {
418                self.evm_remove_block_timestamp_interval().to_rpc_result()
419            }
420            EthRequest::EvmMine(mine) => {
421                self.evm_mine(mine.and_then(|p| p.params)).await.to_rpc_result()
422            }
423            EthRequest::EvmMineDetailed(mine) => {
424                self.evm_mine_detailed(mine.and_then(|p| p.params)).await.to_rpc_result()
425            }
426            EthRequest::SetRpcUrl(url) => self.anvil_set_rpc_url(url).to_rpc_result(),
427            EthRequest::EthSendUnsignedTransaction(tx) => {
428                self.eth_send_unsigned_transaction(*tx).await.to_rpc_result()
429            }
430            EthRequest::EnableTraces(_) => self.anvil_enable_traces().await.to_rpc_result(),
431            EthRequest::EthNewFilter(filter) => self.new_filter(filter).await.to_rpc_result(),
432            EthRequest::EthGetFilterChanges(id) => self.get_filter_changes(&id).await,
433            EthRequest::EthNewBlockFilter(_) => self.new_block_filter().await.to_rpc_result(),
434            EthRequest::EthNewPendingTransactionFilter(_) => {
435                self.new_pending_transaction_filter().await.to_rpc_result()
436            }
437            EthRequest::EthGetFilterLogs(id) => self.get_filter_logs(&id).await.to_rpc_result(),
438            EthRequest::EthUninstallFilter(id) => self.uninstall_filter(&id).await.to_rpc_result(),
439            EthRequest::TxPoolStatus(_) => self.txpool_status().await.to_rpc_result(),
440            EthRequest::TxPoolInspect(_) => self.txpool_inspect().await.to_rpc_result(),
441            EthRequest::TxPoolContent(_) => self.txpool_content().await.to_rpc_result(),
442            EthRequest::ErigonGetHeaderByNumber(num) => {
443                self.erigon_get_header_by_number(num).await.to_rpc_result()
444            }
445            EthRequest::OtsGetApiLevel(_) => self.ots_get_api_level().await.to_rpc_result(),
446            EthRequest::OtsGetInternalOperations(hash) => {
447                self.ots_get_internal_operations(hash).await.to_rpc_result()
448            }
449            EthRequest::OtsHasCode(addr, num) => self.ots_has_code(addr, num).await.to_rpc_result(),
450            EthRequest::OtsTraceTransaction(hash) => {
451                self.ots_trace_transaction(hash).await.to_rpc_result()
452            }
453            EthRequest::OtsGetTransactionError(hash) => {
454                self.ots_get_transaction_error(hash).await.to_rpc_result()
455            }
456            EthRequest::OtsGetBlockDetails(num) => {
457                self.ots_get_block_details(num).await.to_rpc_result()
458            }
459            EthRequest::OtsGetBlockDetailsByHash(hash) => {
460                self.ots_get_block_details_by_hash(hash).await.to_rpc_result()
461            }
462            EthRequest::OtsGetBlockTransactions(num, page, page_size) => {
463                self.ots_get_block_transactions(num, page, page_size).await.to_rpc_result()
464            }
465            EthRequest::OtsSearchTransactionsBefore(address, num, page_size) => {
466                self.ots_search_transactions_before(address, num, page_size).await.to_rpc_result()
467            }
468            EthRequest::OtsSearchTransactionsAfter(address, num, page_size) => {
469                self.ots_search_transactions_after(address, num, page_size).await.to_rpc_result()
470            }
471            EthRequest::OtsGetTransactionBySenderAndNonce(address, nonce) => {
472                self.ots_get_transaction_by_sender_and_nonce(address, nonce).await.to_rpc_result()
473            }
474            EthRequest::OtsGetContractCreator(address) => {
475                self.ots_get_contract_creator(address).await.to_rpc_result()
476            }
477            EthRequest::RemovePoolTransactions(address) => {
478                self.anvil_remove_pool_transactions(address).await.to_rpc_result()
479            }
480            EthRequest::Reorg(reorg_options) => {
481                self.anvil_reorg(reorg_options).await.to_rpc_result()
482            }
483            EthRequest::Rollback(depth) => self.anvil_rollback(depth).await.to_rpc_result(),
484            EthRequest::WalletGetCapabilities(()) => self.get_capabilities().to_rpc_result(),
485            EthRequest::WalletSendTransaction(tx) => {
486                self.wallet_send_transaction(*tx).await.to_rpc_result()
487            }
488            EthRequest::AnvilAddCapability(addr) => self.anvil_add_capability(addr).to_rpc_result(),
489            EthRequest::AnvilSetExecutor(executor_pk) => {
490                self.anvil_set_executor(executor_pk).to_rpc_result()
491            }
492        };
493
494        if let ResponseResult::Error(err) = &response {
495            node_info!("\nRPC request failed:");
496            node_info!("    Request: {:?}", request);
497            node_info!("    Error: {}\n", err);
498        }
499
500        response
501    }
502
503    fn sign_request(
504        &self,
505        from: &Address,
506        request: TypedTransactionRequest,
507    ) -> Result<TypedTransaction> {
508        match request {
509            TypedTransactionRequest::Deposit(_) => {
510                let nil_signature = Signature::from_scalars_and_parity(
511                    B256::with_last_byte(1),
512                    B256::with_last_byte(1),
513                    false,
514                );
515                return build_typed_transaction(request, nil_signature)
516            }
517            _ => {
518                for signer in self.signers.iter() {
519                    if signer.accounts().contains(from) {
520                        let signature = signer.sign_transaction(request.clone(), from)?;
521                        return build_typed_transaction(request, signature)
522                    }
523                }
524            }
525        }
526        Err(BlockchainError::NoSignerAvailable)
527    }
528
529    async fn block_request(&self, block_number: Option<BlockId>) -> Result<BlockRequest> {
530        let block_request = match block_number {
531            Some(BlockId::Number(BlockNumber::Pending)) => {
532                let pending_txs = self.pool.ready_transactions().collect();
533                BlockRequest::Pending(pending_txs)
534            }
535            _ => {
536                let number = self.backend.ensure_block_number(block_number).await?;
537                BlockRequest::Number(number)
538            }
539        };
540        Ok(block_request)
541    }
542
543    async fn inner_raw_transaction(&self, hash: B256) -> Result<Option<Bytes>> {
544        match self.pool.get_transaction(hash) {
545            Some(tx) => Ok(Some(tx.transaction.encoded_2718().into())),
546            None => match self.backend.transaction_by_hash(hash).await? {
547                Some(tx) => Ok(Some(tx.inner.inner.encoded_2718().into())),
548                None => Ok(None),
549            },
550        }
551    }
552
553    /// Returns the current client version.
554    ///
555    /// Handler for ETH RPC call: `web3_clientVersion`
556    pub fn client_version(&self) -> Result<String> {
557        node_info!("web3_clientVersion");
558        Ok(CLIENT_VERSION.to_string())
559    }
560
561    /// Returns Keccak-256 (not the standardized SHA3-256) of the given data.
562    ///
563    /// Handler for ETH RPC call: `web3_sha3`
564    pub fn sha3(&self, bytes: Bytes) -> Result<String> {
565        node_info!("web3_sha3");
566        let hash = alloy_primitives::keccak256(bytes.as_ref());
567        Ok(alloy_primitives::hex::encode_prefixed(&hash[..]))
568    }
569
570    /// Returns protocol version encoded as a string (quotes are necessary).
571    ///
572    /// Handler for ETH RPC call: `eth_protocolVersion`
573    pub fn protocol_version(&self) -> Result<u64> {
574        node_info!("eth_protocolVersion");
575        Ok(1)
576    }
577
578    /// Returns the number of hashes per second that the node is mining with.
579    ///
580    /// Handler for ETH RPC call: `eth_hashrate`
581    pub fn hashrate(&self) -> Result<U256> {
582        node_info!("eth_hashrate");
583        Ok(U256::ZERO)
584    }
585
586    /// Returns the client coinbase address.
587    ///
588    /// Handler for ETH RPC call: `eth_coinbase`
589    pub fn author(&self) -> Result<Address> {
590        node_info!("eth_coinbase");
591        Ok(self.backend.coinbase())
592    }
593
594    /// Returns true if client is actively mining new blocks.
595    ///
596    /// Handler for ETH RPC call: `eth_mining`
597    pub fn is_mining(&self) -> Result<bool> {
598        node_info!("eth_mining");
599        Ok(self.is_mining)
600    }
601
602    /// Returns the chain ID used for transaction signing at the
603    /// current best block. None is returned if not
604    /// available.
605    ///
606    /// Handler for ETH RPC call: `eth_chainId`
607    pub fn eth_chain_id(&self) -> Result<Option<U64>> {
608        node_info!("eth_chainId");
609        Ok(Some(self.backend.chain_id().to::<U64>()))
610    }
611
612    /// Returns the same as `chain_id`
613    ///
614    /// Handler for ETH RPC call: `eth_networkId`
615    pub fn network_id(&self) -> Result<Option<String>> {
616        node_info!("eth_networkId");
617        let chain_id = self.backend.chain_id().to::<u64>();
618        Ok(Some(format!("{chain_id}")))
619    }
620
621    /// Returns true if client is actively listening for network connections.
622    ///
623    /// Handler for ETH RPC call: `net_listening`
624    pub fn net_listening(&self) -> Result<bool> {
625        node_info!("net_listening");
626        Ok(self.net_listening)
627    }
628
629    /// Returns the current gas price
630    fn eth_gas_price(&self) -> Result<U256> {
631        node_info!("eth_gasPrice");
632        Ok(U256::from(self.gas_price()))
633    }
634
635    /// Returns the current gas price
636    pub fn gas_price(&self) -> u128 {
637        if self.backend.is_eip1559() {
638            if self.backend.is_min_priority_fee_enforced() {
639                (self.backend.base_fee() as u128).saturating_add(self.lowest_suggestion_tip())
640            } else {
641                self.backend.base_fee() as u128
642            }
643        } else {
644            self.backend.fees().raw_gas_price()
645        }
646    }
647
648    /// Returns the excess blob gas and current blob gas price
649    pub fn excess_blob_gas_and_price(&self) -> Result<Option<BlobExcessGasAndPrice>> {
650        Ok(self.backend.excess_blob_gas_and_price())
651    }
652
653    /// Returns a fee per gas that is an estimate of how much you can pay as a priority fee, or
654    /// 'tip', to get a transaction included in the current block.
655    ///
656    /// Handler for ETH RPC call: `eth_maxPriorityFeePerGas`
657    pub fn gas_max_priority_fee_per_gas(&self) -> Result<U256> {
658        self.max_priority_fee_per_gas()
659    }
660
661    /// Returns the base fee per blob required to send a EIP-4844 tx.
662    ///
663    /// Handler for ETH RPC call: `eth_blobBaseFee`
664    pub fn blob_base_fee(&self) -> Result<U256> {
665        Ok(U256::from(self.backend.fees().base_fee_per_blob_gas()))
666    }
667
668    /// Returns the block gas limit
669    pub fn gas_limit(&self) -> U256 {
670        U256::from(self.backend.gas_limit())
671    }
672
673    /// Returns the accounts list
674    ///
675    /// Handler for ETH RPC call: `eth_accounts`
676    pub fn accounts(&self) -> Result<Vec<Address>> {
677        node_info!("eth_accounts");
678        let mut unique = HashSet::new();
679        let mut accounts: Vec<Address> = Vec::new();
680        for signer in self.signers.iter() {
681            accounts.extend(signer.accounts().into_iter().filter(|acc| unique.insert(*acc)));
682        }
683        accounts.extend(
684            self.backend
685                .cheats()
686                .impersonated_accounts()
687                .into_iter()
688                .filter(|acc| unique.insert(*acc)),
689        );
690        Ok(accounts.into_iter().collect())
691    }
692
693    /// Returns the number of most recent block.
694    ///
695    /// Handler for ETH RPC call: `eth_blockNumber`
696    pub fn block_number(&self) -> Result<U256> {
697        node_info!("eth_blockNumber");
698        Ok(U256::from(self.backend.best_number()))
699    }
700
701    /// Returns balance of the given account.
702    ///
703    /// Handler for ETH RPC call: `eth_getBalance`
704    pub async fn balance(&self, address: Address, block_number: Option<BlockId>) -> Result<U256> {
705        node_info!("eth_getBalance");
706        let block_request = self.block_request(block_number).await?;
707
708        // check if the number predates the fork, if in fork mode
709        if let BlockRequest::Number(number) = block_request {
710            if let Some(fork) = self.get_fork() {
711                if fork.predates_fork(number) {
712                    return Ok(fork.get_balance(address, number).await?)
713                }
714            }
715        }
716
717        self.backend.get_balance(address, Some(block_request)).await
718    }
719
720    /// Returns the ethereum account.
721    ///
722    /// Handler for ETH RPC call: `eth_getAccount`
723    pub async fn get_account(
724        &self,
725        address: Address,
726        block_number: Option<BlockId>,
727    ) -> Result<Account> {
728        node_info!("eth_getAccount");
729        let block_request = self.block_request(block_number).await?;
730
731        // check if the number predates the fork, if in fork mode
732        if let BlockRequest::Number(number) = block_request {
733            if let Some(fork) = self.get_fork() {
734                if fork.predates_fork(number) {
735                    return Ok(fork.get_account(address, number).await?)
736                }
737            }
738        }
739
740        self.backend.get_account_at_block(address, Some(block_request)).await
741    }
742
743    /// Returns the account information including balance, nonce, code and storage
744    pub async fn get_account_info(
745        &self,
746        address: Address,
747        block_number: Option<BlockId>,
748    ) -> Result<alloy_rpc_types::eth::AccountInfo> {
749        node_info!("eth_getAccountInfo");
750        let account = self
751            .backend
752            .get_account_at_block(address, Some(self.block_request(block_number).await?))
753            .await?;
754        let code =
755            self.backend.get_code(address, Some(self.block_request(block_number).await?)).await?;
756        Ok(alloy_rpc_types::eth::AccountInfo {
757            balance: account.balance,
758            nonce: account.nonce,
759            code,
760        })
761    }
762    /// Returns content of the storage at given address.
763    ///
764    /// Handler for ETH RPC call: `eth_getStorageAt`
765    pub async fn storage_at(
766        &self,
767        address: Address,
768        index: U256,
769        block_number: Option<BlockId>,
770    ) -> Result<B256> {
771        node_info!("eth_getStorageAt");
772        let block_request = self.block_request(block_number).await?;
773
774        // check if the number predates the fork, if in fork mode
775        if let BlockRequest::Number(number) = block_request {
776            if let Some(fork) = self.get_fork() {
777                if fork.predates_fork(number) {
778                    return Ok(B256::from(
779                        fork.storage_at(address, index, Some(BlockNumber::Number(number))).await?,
780                    ));
781                }
782            }
783        }
784
785        self.backend.storage_at(address, index, Some(block_request)).await
786    }
787
788    /// Returns block with given hash.
789    ///
790    /// Handler for ETH RPC call: `eth_getBlockByHash`
791    pub async fn block_by_hash(&self, hash: B256) -> Result<Option<AnyRpcBlock>> {
792        node_info!("eth_getBlockByHash");
793        self.backend.block_by_hash(hash).await
794    }
795
796    /// Returns a _full_ block with given hash.
797    ///
798    /// Handler for ETH RPC call: `eth_getBlockByHash`
799    pub async fn block_by_hash_full(&self, hash: B256) -> Result<Option<AnyRpcBlock>> {
800        node_info!("eth_getBlockByHash");
801        self.backend.block_by_hash_full(hash).await
802    }
803
804    /// Returns block with given number.
805    ///
806    /// Handler for ETH RPC call: `eth_getBlockByNumber`
807    pub async fn block_by_number(&self, number: BlockNumber) -> Result<Option<AnyRpcBlock>> {
808        node_info!("eth_getBlockByNumber");
809        if number == BlockNumber::Pending {
810            return Ok(Some(self.pending_block().await));
811        }
812
813        self.backend.block_by_number(number).await
814    }
815
816    /// Returns a _full_ block with given number
817    ///
818    /// Handler for ETH RPC call: `eth_getBlockByNumber`
819    pub async fn block_by_number_full(&self, number: BlockNumber) -> Result<Option<AnyRpcBlock>> {
820        node_info!("eth_getBlockByNumber");
821        if number == BlockNumber::Pending {
822            return Ok(self.pending_block_full().await);
823        }
824        self.backend.block_by_number_full(number).await
825    }
826
827    /// Returns the number of transactions sent from given address at given time (block number).
828    ///
829    /// Also checks the pending transactions if `block_number` is
830    /// `BlockId::Number(BlockNumber::Pending)`
831    ///
832    /// Handler for ETH RPC call: `eth_getTransactionCount`
833    pub async fn transaction_count(
834        &self,
835        address: Address,
836        block_number: Option<BlockId>,
837    ) -> Result<U256> {
838        node_info!("eth_getTransactionCount");
839        self.get_transaction_count(address, block_number).await.map(U256::from)
840    }
841
842    /// Returns the number of transactions in a block with given hash.
843    ///
844    /// Handler for ETH RPC call: `eth_getBlockTransactionCountByHash`
845    pub async fn block_transaction_count_by_hash(&self, hash: B256) -> Result<Option<U256>> {
846        node_info!("eth_getBlockTransactionCountByHash");
847        let block = self.backend.block_by_hash(hash).await?;
848        let txs = block.map(|b| match b.transactions() {
849            BlockTransactions::Full(txs) => U256::from(txs.len()),
850            BlockTransactions::Hashes(txs) => U256::from(txs.len()),
851            BlockTransactions::Uncle => U256::from(0),
852        });
853        Ok(txs)
854    }
855
856    /// Returns the number of transactions in a block with given block number.
857    ///
858    /// Handler for ETH RPC call: `eth_getBlockTransactionCountByNumber`
859    pub async fn block_transaction_count_by_number(
860        &self,
861        block_number: BlockNumber,
862    ) -> Result<Option<U256>> {
863        node_info!("eth_getBlockTransactionCountByNumber");
864        let block_request = self.block_request(Some(block_number.into())).await?;
865        if let BlockRequest::Pending(txs) = block_request {
866            let block = self.backend.pending_block(txs).await;
867            return Ok(Some(U256::from(block.transactions.len())));
868        }
869        let block = self.backend.block_by_number(block_number).await?;
870        let txs = block.map(|b| match b.transactions() {
871            BlockTransactions::Full(txs) => U256::from(txs.len()),
872            BlockTransactions::Hashes(txs) => U256::from(txs.len()),
873            BlockTransactions::Uncle => U256::from(0),
874        });
875        Ok(txs)
876    }
877
878    /// Returns the number of uncles in a block with given hash.
879    ///
880    /// Handler for ETH RPC call: `eth_getUncleCountByBlockHash`
881    pub async fn block_uncles_count_by_hash(&self, hash: B256) -> Result<U256> {
882        node_info!("eth_getUncleCountByBlockHash");
883        let block =
884            self.backend.block_by_hash(hash).await?.ok_or(BlockchainError::BlockNotFound)?;
885        Ok(U256::from(block.uncles.len()))
886    }
887
888    /// Returns the number of uncles in a block with given block number.
889    ///
890    /// Handler for ETH RPC call: `eth_getUncleCountByBlockNumber`
891    pub async fn block_uncles_count_by_number(&self, block_number: BlockNumber) -> Result<U256> {
892        node_info!("eth_getUncleCountByBlockNumber");
893        let block = self
894            .backend
895            .block_by_number(block_number)
896            .await?
897            .ok_or(BlockchainError::BlockNotFound)?;
898        Ok(U256::from(block.uncles.len()))
899    }
900
901    /// Returns the code at given address at given time (block number).
902    ///
903    /// Handler for ETH RPC call: `eth_getCode`
904    pub async fn get_code(&self, address: Address, block_number: Option<BlockId>) -> Result<Bytes> {
905        node_info!("eth_getCode");
906        let block_request = self.block_request(block_number).await?;
907        // check if the number predates the fork, if in fork mode
908        if let BlockRequest::Number(number) = block_request {
909            if let Some(fork) = self.get_fork() {
910                if fork.predates_fork(number) {
911                    return Ok(fork.get_code(address, number).await?)
912                }
913            }
914        }
915        self.backend.get_code(address, Some(block_request)).await
916    }
917
918    /// Returns the account and storage values of the specified account including the Merkle-proof.
919    /// This call can be used to verify that the data you are pulling from is not tampered with.
920    ///
921    /// Handler for ETH RPC call: `eth_getProof`
922    pub async fn get_proof(
923        &self,
924        address: Address,
925        keys: Vec<B256>,
926        block_number: Option<BlockId>,
927    ) -> Result<EIP1186AccountProofResponse> {
928        node_info!("eth_getProof");
929        let block_request = self.block_request(block_number).await?;
930
931        // If we're in forking mode, or still on the forked block (no blocks mined yet) then we can
932        // delegate the call.
933        if let BlockRequest::Number(number) = block_request {
934            if let Some(fork) = self.get_fork() {
935                if fork.predates_fork_inclusive(number) {
936                    return Ok(fork.get_proof(address, keys, Some(number.into())).await?)
937                }
938            }
939        }
940
941        let proof = self.backend.prove_account_at(address, keys, Some(block_request)).await?;
942        Ok(proof)
943    }
944
945    /// Signs data via [EIP-712](https://github.com/ethereum/EIPs/blob/master/EIPS/eip-712.md).
946    ///
947    /// Handler for ETH RPC call: `eth_signTypedData`
948    pub async fn sign_typed_data(
949        &self,
950        _address: Address,
951        _data: serde_json::Value,
952    ) -> Result<String> {
953        node_info!("eth_signTypedData");
954        Err(BlockchainError::RpcUnimplemented)
955    }
956
957    /// Signs data via [EIP-712](https://github.com/ethereum/EIPs/blob/master/EIPS/eip-712.md).
958    ///
959    /// Handler for ETH RPC call: `eth_signTypedData_v3`
960    pub async fn sign_typed_data_v3(
961        &self,
962        _address: Address,
963        _data: serde_json::Value,
964    ) -> Result<String> {
965        node_info!("eth_signTypedData_v3");
966        Err(BlockchainError::RpcUnimplemented)
967    }
968
969    /// Signs data via [EIP-712](https://github.com/ethereum/EIPs/blob/master/EIPS/eip-712.md), and includes full support of arrays and recursive data structures.
970    ///
971    /// Handler for ETH RPC call: `eth_signTypedData_v4`
972    pub async fn sign_typed_data_v4(&self, address: Address, data: &TypedData) -> Result<String> {
973        node_info!("eth_signTypedData_v4");
974        let signer = self.get_signer(address).ok_or(BlockchainError::NoSignerAvailable)?;
975        let signature = signer.sign_typed_data(address, data).await?;
976        let signature = alloy_primitives::hex::encode(signature.as_bytes());
977        Ok(format!("0x{signature}"))
978    }
979
980    /// The sign method calculates an Ethereum specific signature
981    ///
982    /// Handler for ETH RPC call: `eth_sign`
983    pub async fn sign(&self, address: Address, content: impl AsRef<[u8]>) -> Result<String> {
984        node_info!("eth_sign");
985        let signer = self.get_signer(address).ok_or(BlockchainError::NoSignerAvailable)?;
986        let signature =
987            alloy_primitives::hex::encode(signer.sign(address, content.as_ref()).await?.as_bytes());
988        Ok(format!("0x{signature}"))
989    }
990
991    /// Signs a transaction
992    ///
993    /// Handler for ETH RPC call: `eth_signTransaction`
994    pub async fn sign_transaction(
995        &self,
996        mut request: WithOtherFields<TransactionRequest>,
997    ) -> Result<String> {
998        node_info!("eth_signTransaction");
999
1000        let from = request.from.map(Ok).unwrap_or_else(|| {
1001            self.accounts()?.first().cloned().ok_or(BlockchainError::NoSignerAvailable)
1002        })?;
1003
1004        let (nonce, _) = self.request_nonce(&request, from).await?;
1005
1006        if request.gas.is_none() {
1007            // estimate if not provided
1008            if let Ok(gas) = self.estimate_gas(request.clone(), None, EvmOverrides::default()).await
1009            {
1010                request.gas = Some(gas.to());
1011            }
1012        }
1013
1014        let request = self.build_typed_tx_request(request, nonce)?;
1015
1016        let signed_transaction = self.sign_request(&from, request)?.encoded_2718();
1017        Ok(alloy_primitives::hex::encode_prefixed(signed_transaction))
1018    }
1019
1020    /// Sends a transaction
1021    ///
1022    /// Handler for ETH RPC call: `eth_sendTransaction`
1023    pub async fn send_transaction(
1024        &self,
1025        mut request: WithOtherFields<TransactionRequest>,
1026    ) -> Result<TxHash> {
1027        node_info!("eth_sendTransaction");
1028
1029        let from = request.from.map(Ok).unwrap_or_else(|| {
1030            self.accounts()?.first().cloned().ok_or(BlockchainError::NoSignerAvailable)
1031        })?;
1032        let (nonce, on_chain_nonce) = self.request_nonce(&request, from).await?;
1033
1034        if request.gas.is_none() {
1035            // estimate if not provided
1036            if let Ok(gas) = self.estimate_gas(request.clone(), None, EvmOverrides::default()).await
1037            {
1038                request.gas = Some(gas.to());
1039            }
1040        }
1041
1042        let request = self.build_typed_tx_request(request, nonce)?;
1043
1044        // if the sender is currently impersonated we need to "bypass" signing
1045        let pending_transaction = if self.is_impersonated(from) {
1046            let bypass_signature = self.impersonated_signature(&request);
1047            let transaction = sign::build_typed_transaction(request, bypass_signature)?;
1048            self.ensure_typed_transaction_supported(&transaction)?;
1049            trace!(target : "node", ?from, "eth_sendTransaction: impersonating");
1050            PendingTransaction::with_impersonated(transaction, from)
1051        } else {
1052            let transaction = self.sign_request(&from, request)?;
1053            self.ensure_typed_transaction_supported(&transaction)?;
1054            PendingTransaction::new(transaction)?
1055        };
1056        // pre-validate
1057        self.backend.validate_pool_transaction(&pending_transaction).await?;
1058
1059        let requires = required_marker(nonce, on_chain_nonce, from);
1060        let provides = vec![to_marker(nonce, from)];
1061        debug_assert!(requires != provides);
1062
1063        self.add_pending_transaction(pending_transaction, requires, provides)
1064    }
1065
1066    /// Sends signed transaction, returning its hash.
1067    ///
1068    /// Handler for ETH RPC call: `eth_sendRawTransaction`
1069    pub async fn send_raw_transaction(&self, tx: Bytes) -> Result<TxHash> {
1070        node_info!("eth_sendRawTransaction");
1071        let mut data = tx.as_ref();
1072        if data.is_empty() {
1073            return Err(BlockchainError::EmptyRawTransactionData);
1074        }
1075
1076        let transaction = TypedTransaction::decode_2718(&mut data)
1077            .map_err(|_| BlockchainError::FailedToDecodeSignedTransaction)?;
1078
1079        self.ensure_typed_transaction_supported(&transaction)?;
1080
1081        let pending_transaction = PendingTransaction::new(transaction)?;
1082
1083        // pre-validate
1084        self.backend.validate_pool_transaction(&pending_transaction).await?;
1085
1086        let on_chain_nonce = self.backend.current_nonce(*pending_transaction.sender()).await?;
1087        let from = *pending_transaction.sender();
1088        let nonce = pending_transaction.transaction.nonce();
1089        let requires = required_marker(nonce, on_chain_nonce, from);
1090
1091        let priority = self.transaction_priority(&pending_transaction.transaction);
1092        let pool_transaction = PoolTransaction {
1093            requires,
1094            provides: vec![to_marker(nonce, *pending_transaction.sender())],
1095            pending_transaction,
1096            priority,
1097        };
1098
1099        let tx = self.pool.add_transaction(pool_transaction)?;
1100        trace!(target: "node", "Added transaction: [{:?}] sender={:?}", tx.hash(), from);
1101        Ok(*tx.hash())
1102    }
1103
1104    /// Call contract, returning the output data.
1105    ///
1106    /// Handler for ETH RPC call: `eth_call`
1107    pub async fn call(
1108        &self,
1109        request: WithOtherFields<TransactionRequest>,
1110        block_number: Option<BlockId>,
1111        overrides: EvmOverrides,
1112    ) -> Result<Bytes> {
1113        node_info!("eth_call");
1114        let block_request = self.block_request(block_number).await?;
1115        // check if the number predates the fork, if in fork mode
1116        if let BlockRequest::Number(number) = block_request {
1117            if let Some(fork) = self.get_fork() {
1118                if fork.predates_fork(number) {
1119                    if overrides.has_state() || overrides.has_block() {
1120                        return Err(BlockchainError::EvmOverrideError(
1121                            "not available on past forked blocks".to_string(),
1122                        ));
1123                    }
1124                    return Ok(fork.call(&request, Some(number.into())).await?)
1125                }
1126            }
1127        }
1128
1129        let fees = FeeDetails::new(
1130            request.gas_price,
1131            request.max_fee_per_gas,
1132            request.max_priority_fee_per_gas,
1133            request.max_fee_per_blob_gas,
1134        )?
1135        .or_zero_fees();
1136        // this can be blocking for a bit, especially in forking mode
1137        // <https://github.com/foundry-rs/foundry/issues/6036>
1138        self.on_blocking_task(|this| async move {
1139            let (exit, out, gas, _) =
1140                this.backend.call(request, fees, Some(block_request), overrides).await?;
1141            trace!(target : "node", "Call status {:?}, gas {}", exit, gas);
1142
1143            ensure_return_ok(exit, &out)
1144        })
1145        .await
1146    }
1147
1148    pub async fn simulate_v1(
1149        &self,
1150        request: SimulatePayload,
1151        block_number: Option<BlockId>,
1152    ) -> Result<Vec<SimulatedBlock<AnyRpcBlock>>> {
1153        node_info!("eth_simulateV1");
1154        let block_request = self.block_request(block_number).await?;
1155        // check if the number predates the fork, if in fork mode
1156        if let BlockRequest::Number(number) = block_request {
1157            if let Some(fork) = self.get_fork() {
1158                if fork.predates_fork(number) {
1159                    return Ok(fork.simulate_v1(&request, Some(number.into())).await?)
1160                }
1161            }
1162        }
1163
1164        // this can be blocking for a bit, especially in forking mode
1165        // <https://github.com/foundry-rs/foundry/issues/6036>
1166        self.on_blocking_task(|this| async move {
1167            let simulated_blocks = this.backend.simulate(request, Some(block_request)).await?;
1168            trace!(target : "node", "Simulate status {:?}", simulated_blocks);
1169
1170            Ok(simulated_blocks)
1171        })
1172        .await
1173    }
1174
1175    /// This method creates an EIP2930 type accessList based on a given Transaction. The accessList
1176    /// contains all storage slots and addresses read and written by the transaction, except for the
1177    /// sender account and the precompiles.
1178    ///
1179    /// It returns list of addresses and storage keys used by the transaction, plus the gas
1180    /// consumed when the access list is added. That is, it gives you the list of addresses and
1181    /// storage keys that will be used by that transaction, plus the gas consumed if the access
1182    /// list is included. Like eth_estimateGas, this is an estimation; the list could change
1183    /// when the transaction is actually mined. Adding an accessList to your transaction does
1184    /// not necessary result in lower gas usage compared to a transaction without an access
1185    /// list.
1186    ///
1187    /// Handler for ETH RPC call: `eth_createAccessList`
1188    pub async fn create_access_list(
1189        &self,
1190        mut request: WithOtherFields<TransactionRequest>,
1191        block_number: Option<BlockId>,
1192    ) -> Result<AccessListResult> {
1193        node_info!("eth_createAccessList");
1194        let block_request = self.block_request(block_number).await?;
1195        // check if the number predates the fork, if in fork mode
1196        if let BlockRequest::Number(number) = block_request {
1197            if let Some(fork) = self.get_fork() {
1198                if fork.predates_fork(number) {
1199                    return Ok(fork.create_access_list(&request, Some(number.into())).await?)
1200                }
1201            }
1202        }
1203
1204        self.backend
1205            .with_database_at(Some(block_request), |state, block_env| {
1206                let (exit, out, _, access_list) = self.backend.build_access_list_with_state(
1207                    &state,
1208                    request.clone(),
1209                    FeeDetails::zero(),
1210                    block_env.clone(),
1211                )?;
1212                ensure_return_ok(exit, &out)?;
1213
1214                // execute again but with access list set
1215                request.access_list = Some(access_list.clone());
1216
1217                let (exit, out, gas_used, _) = self.backend.call_with_state(
1218                    &state,
1219                    request.clone(),
1220                    FeeDetails::zero(),
1221                    block_env,
1222                )?;
1223                ensure_return_ok(exit, &out)?;
1224
1225                Ok(AccessListResult {
1226                    access_list: AccessList(access_list.0),
1227                    gas_used: U256::from(gas_used),
1228                    error: None,
1229                })
1230            })
1231            .await?
1232    }
1233
1234    /// Estimate gas needed for execution of given contract.
1235    /// If no block parameter is given, it will use the pending block by default
1236    ///
1237    /// Handler for ETH RPC call: `eth_estimateGas`
1238    pub async fn estimate_gas(
1239        &self,
1240        request: WithOtherFields<TransactionRequest>,
1241        block_number: Option<BlockId>,
1242        overrides: EvmOverrides,
1243    ) -> Result<U256> {
1244        node_info!("eth_estimateGas");
1245        self.do_estimate_gas(
1246            request,
1247            block_number.or_else(|| Some(BlockNumber::Pending.into())),
1248            overrides,
1249        )
1250        .await
1251        .map(U256::from)
1252    }
1253
1254    /// Get transaction by its hash.
1255    ///
1256    /// This will check the storage for a matching transaction, if no transaction exists in storage
1257    /// this will also scan the mempool for a matching pending transaction
1258    ///
1259    /// Handler for ETH RPC call: `eth_getTransactionByHash`
1260    pub async fn transaction_by_hash(&self, hash: B256) -> Result<Option<AnyRpcTransaction>> {
1261        node_info!("eth_getTransactionByHash");
1262        let mut tx = self.pool.get_transaction(hash).map(|pending| {
1263            let from = *pending.sender();
1264            let tx = transaction_build(
1265                Some(*pending.hash()),
1266                pending.transaction,
1267                None,
1268                None,
1269                Some(self.backend.base_fee()),
1270            );
1271
1272            let WithOtherFields { inner: mut tx, other } = tx.0;
1273            // we set the from field here explicitly to the set sender of the pending transaction,
1274            // in case the transaction is impersonated.
1275            tx.inner = Recovered::new_unchecked(tx.inner.into_inner(), from);
1276
1277            AnyRpcTransaction(WithOtherFields { inner: tx, other })
1278        });
1279        if tx.is_none() {
1280            tx = self.backend.transaction_by_hash(hash).await?
1281        }
1282
1283        Ok(tx)
1284    }
1285
1286    /// Returns transaction at given block hash and index.
1287    ///
1288    /// Handler for ETH RPC call: `eth_getTransactionByBlockHashAndIndex`
1289    pub async fn transaction_by_block_hash_and_index(
1290        &self,
1291        hash: B256,
1292        index: Index,
1293    ) -> Result<Option<AnyRpcTransaction>> {
1294        node_info!("eth_getTransactionByBlockHashAndIndex");
1295        self.backend.transaction_by_block_hash_and_index(hash, index).await
1296    }
1297
1298    /// Returns transaction by given block number and index.
1299    ///
1300    /// Handler for ETH RPC call: `eth_getTransactionByBlockNumberAndIndex`
1301    pub async fn transaction_by_block_number_and_index(
1302        &self,
1303        block: BlockNumber,
1304        idx: Index,
1305    ) -> Result<Option<AnyRpcTransaction>> {
1306        node_info!("eth_getTransactionByBlockNumberAndIndex");
1307        self.backend.transaction_by_block_number_and_index(block, idx).await
1308    }
1309
1310    /// Returns transaction receipt by transaction hash.
1311    ///
1312    /// Handler for ETH RPC call: `eth_getTransactionReceipt`
1313    pub async fn transaction_receipt(&self, hash: B256) -> Result<Option<ReceiptResponse>> {
1314        node_info!("eth_getTransactionReceipt");
1315        let tx = self.pool.get_transaction(hash);
1316        if tx.is_some() {
1317            return Ok(None);
1318        }
1319        self.backend.transaction_receipt(hash).await
1320    }
1321
1322    /// Returns block receipts by block number.
1323    ///
1324    /// Handler for ETH RPC call: `eth_getBlockReceipts`
1325    pub async fn block_receipts(&self, number: BlockId) -> Result<Option<Vec<ReceiptResponse>>> {
1326        node_info!("eth_getBlockReceipts");
1327        self.backend.block_receipts(number).await
1328    }
1329
1330    /// Returns an uncles at given block and index.
1331    ///
1332    /// Handler for ETH RPC call: `eth_getUncleByBlockHashAndIndex`
1333    pub async fn uncle_by_block_hash_and_index(
1334        &self,
1335        block_hash: B256,
1336        idx: Index,
1337    ) -> Result<Option<AnyRpcBlock>> {
1338        node_info!("eth_getUncleByBlockHashAndIndex");
1339        let number =
1340            self.backend.ensure_block_number(Some(BlockId::Hash(block_hash.into()))).await?;
1341        if let Some(fork) = self.get_fork() {
1342            if fork.predates_fork_inclusive(number) {
1343                return Ok(fork.uncle_by_block_hash_and_index(block_hash, idx.into()).await?)
1344            }
1345        }
1346        // It's impossible to have uncles outside of fork mode
1347        Ok(None)
1348    }
1349
1350    /// Returns an uncles at given block and index.
1351    ///
1352    /// Handler for ETH RPC call: `eth_getUncleByBlockNumberAndIndex`
1353    pub async fn uncle_by_block_number_and_index(
1354        &self,
1355        block_number: BlockNumber,
1356        idx: Index,
1357    ) -> Result<Option<AnyRpcBlock>> {
1358        node_info!("eth_getUncleByBlockNumberAndIndex");
1359        let number = self.backend.ensure_block_number(Some(BlockId::Number(block_number))).await?;
1360        if let Some(fork) = self.get_fork() {
1361            if fork.predates_fork_inclusive(number) {
1362                return Ok(fork.uncle_by_block_number_and_index(number, idx.into()).await?)
1363            }
1364        }
1365        // It's impossible to have uncles outside of fork mode
1366        Ok(None)
1367    }
1368
1369    /// Returns logs matching given filter object.
1370    ///
1371    /// Handler for ETH RPC call: `eth_getLogs`
1372    pub async fn logs(&self, filter: Filter) -> Result<Vec<Log>> {
1373        node_info!("eth_getLogs");
1374        self.backend.logs(filter).await
1375    }
1376
1377    /// Returns the hash of the current block, the seedHash, and the boundary condition to be met.
1378    ///
1379    /// Handler for ETH RPC call: `eth_getWork`
1380    pub fn work(&self) -> Result<Work> {
1381        node_info!("eth_getWork");
1382        Err(BlockchainError::RpcUnimplemented)
1383    }
1384
1385    /// Returns the sync status, always be fails.
1386    ///
1387    /// Handler for ETH RPC call: `eth_syncing`
1388    pub fn syncing(&self) -> Result<bool> {
1389        node_info!("eth_syncing");
1390        Ok(false)
1391    }
1392
1393    /// Used for submitting a proof-of-work solution.
1394    ///
1395    /// Handler for ETH RPC call: `eth_submitWork`
1396    pub fn submit_work(&self, _: B64, _: B256, _: B256) -> Result<bool> {
1397        node_info!("eth_submitWork");
1398        Err(BlockchainError::RpcUnimplemented)
1399    }
1400
1401    /// Used for submitting mining hashrate.
1402    ///
1403    /// Handler for ETH RPC call: `eth_submitHashrate`
1404    pub fn submit_hashrate(&self, _: U256, _: B256) -> Result<bool> {
1405        node_info!("eth_submitHashrate");
1406        Err(BlockchainError::RpcUnimplemented)
1407    }
1408
1409    /// Introduced in EIP-1559 for getting information on the appropriate priority fee to use.
1410    ///
1411    /// Handler for ETH RPC call: `eth_feeHistory`
1412    pub async fn fee_history(
1413        &self,
1414        block_count: U256,
1415        newest_block: BlockNumber,
1416        reward_percentiles: Vec<f64>,
1417    ) -> Result<FeeHistory> {
1418        node_info!("eth_feeHistory");
1419        // max number of blocks in the requested range
1420
1421        let current = self.backend.best_number();
1422        let slots_in_an_epoch = 32u64;
1423
1424        let number = match newest_block {
1425            BlockNumber::Latest | BlockNumber::Pending => current,
1426            BlockNumber::Earliest => 0,
1427            BlockNumber::Number(n) => n,
1428            BlockNumber::Safe => current.saturating_sub(slots_in_an_epoch),
1429            BlockNumber::Finalized => current.saturating_sub(slots_in_an_epoch * 2),
1430        };
1431
1432        // check if the number predates the fork, if in fork mode
1433        if let Some(fork) = self.get_fork() {
1434            // if we're still at the forked block we don't have any history and can't compute it
1435            // efficiently, instead we fetch it from the fork
1436            if fork.predates_fork_inclusive(number) {
1437                return fork
1438                    .fee_history(block_count.to(), BlockNumber::Number(number), &reward_percentiles)
1439                    .await
1440                    .map_err(BlockchainError::AlloyForkProvider);
1441            }
1442        }
1443
1444        const MAX_BLOCK_COUNT: u64 = 1024u64;
1445        let block_count = block_count.to::<u64>().min(MAX_BLOCK_COUNT);
1446
1447        // highest and lowest block num in the requested range
1448        let highest = number;
1449        let lowest = highest.saturating_sub(block_count.saturating_sub(1));
1450
1451        // only support ranges that are in cache range
1452        if lowest < self.backend.best_number().saturating_sub(self.fee_history_limit) {
1453            return Err(FeeHistoryError::InvalidBlockRange.into());
1454        }
1455
1456        let mut response = FeeHistory {
1457            oldest_block: lowest,
1458            base_fee_per_gas: Vec::new(),
1459            gas_used_ratio: Vec::new(),
1460            reward: Some(Default::default()),
1461            base_fee_per_blob_gas: Default::default(),
1462            blob_gas_used_ratio: Default::default(),
1463        };
1464        let mut rewards = Vec::new();
1465
1466        {
1467            let fee_history = self.fee_history_cache.lock();
1468
1469            // iter over the requested block range
1470            for n in lowest..=highest {
1471                // <https://eips.ethereum.org/EIPS/eip-1559>
1472                if let Some(block) = fee_history.get(&n) {
1473                    response.base_fee_per_gas.push(block.base_fee);
1474                    response.base_fee_per_blob_gas.push(block.base_fee_per_blob_gas.unwrap_or(0));
1475                    response.blob_gas_used_ratio.push(block.blob_gas_used_ratio);
1476                    response.gas_used_ratio.push(block.gas_used_ratio);
1477
1478                    // requested percentiles
1479                    if !reward_percentiles.is_empty() {
1480                        let mut block_rewards = Vec::new();
1481                        let resolution_per_percentile: f64 = 2.0;
1482                        for p in &reward_percentiles {
1483                            let p = p.clamp(0.0, 100.0);
1484                            let index = ((p.round() / 2f64) * 2f64) * resolution_per_percentile;
1485                            let reward = block.rewards.get(index as usize).map_or(0, |r| *r);
1486                            block_rewards.push(reward);
1487                        }
1488                        rewards.push(block_rewards);
1489                    }
1490                }
1491            }
1492        }
1493
1494        response.reward = Some(rewards);
1495
1496        // add the next block's base fee to the response
1497        // The spec states that `base_fee_per_gas` "[..] includes the next block after the
1498        // newest of the returned range, because this value can be derived from the
1499        // newest block"
1500        response.base_fee_per_gas.push(self.backend.fees().base_fee() as u128);
1501
1502        // Same goes for the `base_fee_per_blob_gas`:
1503        // > [..] includes the next block after the newest of the returned range, because this
1504        // > value can be derived from the newest block.
1505        response.base_fee_per_blob_gas.push(self.backend.fees().base_fee_per_blob_gas());
1506
1507        Ok(response)
1508    }
1509
1510    /// Introduced in EIP-1159, a Geth-specific and simplified priority fee oracle.
1511    /// Leverages the already existing fee history cache.
1512    ///
1513    /// Returns a suggestion for a gas tip cap for dynamic fee transactions.
1514    ///
1515    /// Handler for ETH RPC call: `eth_maxPriorityFeePerGas`
1516    pub fn max_priority_fee_per_gas(&self) -> Result<U256> {
1517        node_info!("eth_maxPriorityFeePerGas");
1518        Ok(U256::from(self.lowest_suggestion_tip()))
1519    }
1520
1521    /// Returns the suggested fee cap.
1522    ///
1523    /// Returns at least [MIN_SUGGESTED_PRIORITY_FEE]
1524    fn lowest_suggestion_tip(&self) -> u128 {
1525        let block_number = self.backend.best_number();
1526        let latest_cached_block = self.fee_history_cache.lock().get(&block_number).cloned();
1527
1528        match latest_cached_block {
1529            Some(block) => block.rewards.iter().copied().min(),
1530            None => self.fee_history_cache.lock().values().flat_map(|b| b.rewards.clone()).min(),
1531        }
1532        .map(|fee| fee.max(MIN_SUGGESTED_PRIORITY_FEE))
1533        .unwrap_or(MIN_SUGGESTED_PRIORITY_FEE)
1534    }
1535
1536    /// Creates a filter object, based on filter options, to notify when the state changes (logs).
1537    ///
1538    /// Handler for ETH RPC call: `eth_newFilter`
1539    pub async fn new_filter(&self, filter: Filter) -> Result<String> {
1540        node_info!("eth_newFilter");
1541        // all logs that are already available that match the filter if the filter's block range is
1542        // in the past
1543        let historic = if filter.block_option.get_from_block().is_some() {
1544            self.backend.logs(filter.clone()).await?
1545        } else {
1546            vec![]
1547        };
1548        let filter = EthFilter::Logs(Box::new(LogsFilter {
1549            blocks: self.new_block_notifications(),
1550            storage: self.storage_info(),
1551            filter: FilteredParams::new(Some(filter)),
1552            historic: Some(historic),
1553        }));
1554        Ok(self.filters.add_filter(filter).await)
1555    }
1556
1557    /// Creates a filter in the node, to notify when a new block arrives.
1558    ///
1559    /// Handler for ETH RPC call: `eth_newBlockFilter`
1560    pub async fn new_block_filter(&self) -> Result<String> {
1561        node_info!("eth_newBlockFilter");
1562        let filter = EthFilter::Blocks(self.new_block_notifications());
1563        Ok(self.filters.add_filter(filter).await)
1564    }
1565
1566    /// Creates a filter in the node, to notify when new pending transactions arrive.
1567    ///
1568    /// Handler for ETH RPC call: `eth_newPendingTransactionFilter`
1569    pub async fn new_pending_transaction_filter(&self) -> Result<String> {
1570        node_info!("eth_newPendingTransactionFilter");
1571        let filter = EthFilter::PendingTransactions(self.new_ready_transactions());
1572        Ok(self.filters.add_filter(filter).await)
1573    }
1574
1575    /// Polling method for a filter, which returns an array of logs which occurred since last poll.
1576    ///
1577    /// Handler for ETH RPC call: `eth_getFilterChanges`
1578    pub async fn get_filter_changes(&self, id: &str) -> ResponseResult {
1579        node_info!("eth_getFilterChanges");
1580        self.filters.get_filter_changes(id).await
1581    }
1582
1583    /// Returns an array of all logs matching filter with given id.
1584    ///
1585    /// Handler for ETH RPC call: `eth_getFilterLogs`
1586    pub async fn get_filter_logs(&self, id: &str) -> Result<Vec<Log>> {
1587        node_info!("eth_getFilterLogs");
1588        if let Some(filter) = self.filters.get_log_filter(id).await {
1589            self.backend.logs(filter).await
1590        } else {
1591            Ok(Vec::new())
1592        }
1593    }
1594
1595    /// Handler for ETH RPC call: `eth_uninstallFilter`
1596    pub async fn uninstall_filter(&self, id: &str) -> Result<bool> {
1597        node_info!("eth_uninstallFilter");
1598        Ok(self.filters.uninstall_filter(id).await.is_some())
1599    }
1600
1601    /// Returns EIP-2718 encoded raw transaction
1602    ///
1603    /// Handler for RPC call: `debug_getRawTransaction`
1604    pub async fn raw_transaction(&self, hash: B256) -> Result<Option<Bytes>> {
1605        node_info!("debug_getRawTransaction");
1606        self.inner_raw_transaction(hash).await
1607    }
1608
1609    /// Returns EIP-2718 encoded raw transaction by block hash and index
1610    ///
1611    /// Handler for RPC call: `eth_getRawTransactionByBlockHashAndIndex`
1612    pub async fn raw_transaction_by_block_hash_and_index(
1613        &self,
1614        block_hash: B256,
1615        index: Index,
1616    ) -> Result<Option<Bytes>> {
1617        node_info!("eth_getRawTransactionByBlockHashAndIndex");
1618        match self.backend.transaction_by_block_hash_and_index(block_hash, index).await? {
1619            Some(tx) => self.inner_raw_transaction(tx.tx_hash()).await,
1620            None => Ok(None),
1621        }
1622    }
1623
1624    /// Returns EIP-2718 encoded raw transaction by block number and index
1625    ///
1626    /// Handler for RPC call: `eth_getRawTransactionByBlockNumberAndIndex`
1627    pub async fn raw_transaction_by_block_number_and_index(
1628        &self,
1629        block_number: BlockNumber,
1630        index: Index,
1631    ) -> Result<Option<Bytes>> {
1632        node_info!("eth_getRawTransactionByBlockNumberAndIndex");
1633        match self.backend.transaction_by_block_number_and_index(block_number, index).await? {
1634            Some(tx) => self.inner_raw_transaction(tx.tx_hash()).await,
1635            None => Ok(None),
1636        }
1637    }
1638
1639    /// Returns traces for the transaction hash for geth's tracing endpoint
1640    ///
1641    /// Handler for RPC call: `debug_traceTransaction`
1642    pub async fn debug_trace_transaction(
1643        &self,
1644        tx_hash: B256,
1645        opts: GethDebugTracingOptions,
1646    ) -> Result<GethTrace> {
1647        node_info!("debug_traceTransaction");
1648        self.backend.debug_trace_transaction(tx_hash, opts).await
1649    }
1650
1651    /// Returns traces for the transaction for geth's tracing endpoint
1652    ///
1653    /// Handler for RPC call: `debug_traceCall`
1654    pub async fn debug_trace_call(
1655        &self,
1656        request: WithOtherFields<TransactionRequest>,
1657        block_number: Option<BlockId>,
1658        opts: GethDebugTracingCallOptions,
1659    ) -> Result<GethTrace> {
1660        node_info!("debug_traceCall");
1661        let block_request = self.block_request(block_number).await?;
1662        let fees = FeeDetails::new(
1663            request.gas_price,
1664            request.max_fee_per_gas,
1665            request.max_priority_fee_per_gas,
1666            request.max_fee_per_blob_gas,
1667        )?
1668        .or_zero_fees();
1669
1670        let result: std::result::Result<GethTrace, BlockchainError> =
1671            self.backend.call_with_tracing(request, fees, Some(block_request), opts).await;
1672        result
1673    }
1674
1675    /// Returns traces for the transaction hash via parity's tracing endpoint
1676    ///
1677    /// Handler for RPC call: `trace_transaction`
1678    pub async fn trace_transaction(&self, tx_hash: B256) -> Result<Vec<LocalizedTransactionTrace>> {
1679        node_info!("trace_transaction");
1680        self.backend.trace_transaction(tx_hash).await
1681    }
1682
1683    /// Returns traces for the transaction hash via parity's tracing endpoint
1684    ///
1685    /// Handler for RPC call: `trace_block`
1686    pub async fn trace_block(&self, block: BlockNumber) -> Result<Vec<LocalizedTransactionTrace>> {
1687        node_info!("trace_block");
1688        self.backend.trace_block(block).await
1689    }
1690
1691    /// Returns filtered traces over blocks
1692    ///
1693    /// Handler for RPC call: `trace_filter`
1694    pub async fn trace_filter(
1695        &self,
1696        filter: TraceFilter,
1697    ) -> Result<Vec<LocalizedTransactionTrace>> {
1698        node_info!("trace_filter");
1699        self.backend.trace_filter(filter).await
1700    }
1701}
1702
1703// == impl EthApi anvil endpoints ==
1704
1705impl EthApi {
1706    /// Send transactions impersonating specific account and contract addresses.
1707    ///
1708    /// Handler for ETH RPC call: `anvil_impersonateAccount`
1709    pub async fn anvil_impersonate_account(&self, address: Address) -> Result<()> {
1710        node_info!("anvil_impersonateAccount");
1711        self.backend.impersonate(address);
1712        Ok(())
1713    }
1714
1715    /// Stops impersonating an account if previously set with `anvil_impersonateAccount`.
1716    ///
1717    /// Handler for ETH RPC call: `anvil_stopImpersonatingAccount`
1718    pub async fn anvil_stop_impersonating_account(&self, address: Address) -> Result<()> {
1719        node_info!("anvil_stopImpersonatingAccount");
1720        self.backend.stop_impersonating(address);
1721        Ok(())
1722    }
1723
1724    /// If set to true will make every account impersonated
1725    ///
1726    /// Handler for ETH RPC call: `anvil_autoImpersonateAccount`
1727    pub async fn anvil_auto_impersonate_account(&self, enabled: bool) -> Result<()> {
1728        node_info!("anvil_autoImpersonateAccount");
1729        self.backend.auto_impersonate_account(enabled);
1730        Ok(())
1731    }
1732
1733    /// Returns true if auto mining is enabled, and false.
1734    ///
1735    /// Handler for ETH RPC call: `anvil_getAutomine`
1736    pub fn anvil_get_auto_mine(&self) -> Result<bool> {
1737        node_info!("anvil_getAutomine");
1738        Ok(self.miner.is_auto_mine())
1739    }
1740
1741    /// Returns the value of mining interval, if set.
1742    ///
1743    /// Handler for ETH RPC call: `anvil_getIntervalMining`.
1744    pub fn anvil_get_interval_mining(&self) -> Result<Option<u64>> {
1745        node_info!("anvil_getIntervalMining");
1746        Ok(self.miner.get_interval())
1747    }
1748
1749    /// Enables or disables, based on the single boolean argument, the automatic mining of new
1750    /// blocks with each new transaction submitted to the network.
1751    ///
1752    /// Handler for ETH RPC call: `evm_setAutomine`
1753    pub async fn anvil_set_auto_mine(&self, enable_automine: bool) -> Result<()> {
1754        node_info!("evm_setAutomine");
1755        if self.miner.is_auto_mine() {
1756            if enable_automine {
1757                return Ok(());
1758            }
1759            self.miner.set_mining_mode(MiningMode::None);
1760        } else if enable_automine {
1761            let listener = self.pool.add_ready_listener();
1762            let mode = MiningMode::instant(1_000, listener);
1763            self.miner.set_mining_mode(mode);
1764        }
1765        Ok(())
1766    }
1767
1768    /// Mines a series of blocks.
1769    ///
1770    /// Handler for ETH RPC call: `anvil_mine`
1771    pub async fn anvil_mine(&self, num_blocks: Option<U256>, interval: Option<U256>) -> Result<()> {
1772        node_info!("anvil_mine");
1773        let interval = interval.map(|i| i.to::<u64>());
1774        let blocks = num_blocks.unwrap_or(U256::from(1));
1775        if blocks.is_zero() {
1776            return Ok(());
1777        }
1778
1779        self.on_blocking_task(|this| async move {
1780            // mine all the blocks
1781            for _ in 0..blocks.to::<u64>() {
1782                // If we have an interval, jump forwards in time to the "next" timestamp
1783                if let Some(interval) = interval {
1784                    this.backend.time().increase_time(interval);
1785                }
1786                this.mine_one().await;
1787            }
1788            Ok(())
1789        })
1790        .await?;
1791
1792        Ok(())
1793    }
1794
1795    /// Sets the mining behavior to interval with the given interval (seconds)
1796    ///
1797    /// Handler for ETH RPC call: `evm_setIntervalMining`
1798    pub fn anvil_set_interval_mining(&self, secs: u64) -> Result<()> {
1799        node_info!("evm_setIntervalMining");
1800        let mining_mode = if secs == 0 {
1801            MiningMode::None
1802        } else {
1803            let block_time = Duration::from_secs(secs);
1804
1805            // This ensures that memory limits are stricter in interval-mine mode
1806            self.backend.update_interval_mine_block_time(block_time);
1807
1808            MiningMode::FixedBlockTime(FixedBlockTimeMiner::new(block_time))
1809        };
1810        self.miner.set_mining_mode(mining_mode);
1811        Ok(())
1812    }
1813
1814    /// Removes transactions from the pool
1815    ///
1816    /// Handler for RPC call: `anvil_dropTransaction`
1817    pub async fn anvil_drop_transaction(&self, tx_hash: B256) -> Result<Option<B256>> {
1818        node_info!("anvil_dropTransaction");
1819        Ok(self.pool.drop_transaction(tx_hash).map(|tx| tx.hash()))
1820    }
1821
1822    /// Removes all transactions from the pool
1823    ///
1824    /// Handler for RPC call: `anvil_dropAllTransactions`
1825    pub async fn anvil_drop_all_transactions(&self) -> Result<()> {
1826        node_info!("anvil_dropAllTransactions");
1827        self.pool.clear();
1828        Ok(())
1829    }
1830
1831    /// Reset the fork to a fresh forked state, and optionally update the fork config.
1832    ///
1833    /// If `forking` is `None` then this will disable forking entirely.
1834    ///
1835    /// Handler for RPC call: `anvil_reset`
1836    pub async fn anvil_reset(&self, forking: Option<Forking>) -> Result<()> {
1837        node_info!("anvil_reset");
1838        if let Some(forking) = forking {
1839            // if we're resetting the fork we need to reset the instance id
1840            self.reset_instance_id();
1841            self.backend.reset_fork(forking).await
1842        } else {
1843            Err(BlockchainError::RpcUnimplemented)
1844        }
1845    }
1846
1847    pub async fn anvil_set_chain_id(&self, chain_id: u64) -> Result<()> {
1848        node_info!("anvil_setChainId");
1849        self.backend.set_chain_id(chain_id);
1850        Ok(())
1851    }
1852
1853    /// Modifies the balance of an account.
1854    ///
1855    /// Handler for RPC call: `anvil_setBalance`
1856    pub async fn anvil_set_balance(&self, address: Address, balance: U256) -> Result<()> {
1857        node_info!("anvil_setBalance");
1858        self.backend.set_balance(address, balance).await?;
1859        Ok(())
1860    }
1861
1862    /// Increases the balance of an account.
1863    ///
1864    /// Handler for RPC call: `anvil_addBalance`
1865    pub async fn anvil_add_balance(&self, address: Address, balance: U256) -> Result<()> {
1866        node_info!("anvil_addBalance");
1867        let current_balance = self.backend.get_balance(address, None).await?;
1868        self.backend.set_balance(address, current_balance + balance).await?;
1869        Ok(())
1870    }
1871
1872    /// Deals ERC20 tokens to a address
1873    ///
1874    /// Handler for RPC call: `anvil_dealERC20`
1875    pub async fn anvil_deal_erc20(
1876        &self,
1877        address: Address,
1878        token_address: Address,
1879        balance: U256,
1880    ) -> Result<()> {
1881        node_info!("anvil_dealERC20");
1882
1883        sol! {
1884            #[sol(rpc)]
1885            contract IERC20 {
1886                function balanceOf(address target) external view returns (uint256);
1887            }
1888        }
1889
1890        let calldata = IERC20::balanceOfCall { target: address }.abi_encode();
1891        let tx = TransactionRequest::default().with_to(token_address).with_input(calldata.clone());
1892
1893        // first collect all the slots that are used by the balanceOf call
1894        let access_list_result =
1895            self.create_access_list(WithOtherFields::new(tx.clone()), None).await?;
1896        let access_list = access_list_result.access_list;
1897
1898        // now we can iterate over all the accessed slots and try to find the one that contains the
1899        // balance by overriding the slot and checking the `balanceOfCall` of
1900        for item in access_list.0 {
1901            if item.address != token_address {
1902                continue;
1903            };
1904            for slot in &item.storage_keys {
1905                let account_override = AccountOverride::default()
1906                    .with_state_diff(std::iter::once((*slot, B256::from(balance.to_be_bytes()))));
1907
1908                let state_override = StateOverridesBuilder::default()
1909                    .append(token_address, account_override)
1910                    .build();
1911
1912                let evm_override = EvmOverrides::state(Some(state_override));
1913
1914                let Ok(result) =
1915                    self.call(WithOtherFields::new(tx.clone()), None, evm_override).await
1916                else {
1917                    // overriding this slot failed
1918                    continue;
1919                };
1920
1921                let Ok(result_balance) = U256::abi_decode(&result) else {
1922                    // response returned something other than a U256
1923                    continue;
1924                };
1925
1926                if result_balance == balance {
1927                    self.anvil_set_storage_at(
1928                        token_address,
1929                        U256::from_be_bytes(slot.0),
1930                        B256::from(balance.to_be_bytes()),
1931                    )
1932                    .await?;
1933                    return Ok(());
1934                }
1935            }
1936        }
1937
1938        // unable to set the balance
1939        Err(BlockchainError::Message("Unable to set ERC20 balance, no slot found".to_string()))
1940    }
1941
1942    /// Sets the code of a contract.
1943    ///
1944    /// Handler for RPC call: `anvil_setCode`
1945    pub async fn anvil_set_code(&self, address: Address, code: Bytes) -> Result<()> {
1946        node_info!("anvil_setCode");
1947        self.backend.set_code(address, code).await?;
1948        Ok(())
1949    }
1950
1951    /// Sets the nonce of an address.
1952    ///
1953    /// Handler for RPC call: `anvil_setNonce`
1954    pub async fn anvil_set_nonce(&self, address: Address, nonce: U256) -> Result<()> {
1955        node_info!("anvil_setNonce");
1956        self.backend.set_nonce(address, nonce).await?;
1957        Ok(())
1958    }
1959
1960    /// Writes a single slot of the account's storage.
1961    ///
1962    /// Handler for RPC call: `anvil_setStorageAt`
1963    pub async fn anvil_set_storage_at(
1964        &self,
1965        address: Address,
1966        slot: U256,
1967        val: B256,
1968    ) -> Result<bool> {
1969        node_info!("anvil_setStorageAt");
1970        self.backend.set_storage_at(address, slot, val).await?;
1971        Ok(true)
1972    }
1973
1974    /// Enable or disable logging.
1975    ///
1976    /// Handler for RPC call: `anvil_setLoggingEnabled`
1977    pub async fn anvil_set_logging(&self, enable: bool) -> Result<()> {
1978        node_info!("anvil_setLoggingEnabled");
1979        self.logger.set_enabled(enable);
1980        Ok(())
1981    }
1982
1983    /// Set the minimum gas price for the node.
1984    ///
1985    /// Handler for RPC call: `anvil_setMinGasPrice`
1986    pub async fn anvil_set_min_gas_price(&self, gas: U256) -> Result<()> {
1987        node_info!("anvil_setMinGasPrice");
1988        if self.backend.is_eip1559() {
1989            return Err(RpcError::invalid_params(
1990                "anvil_setMinGasPrice is not supported when EIP-1559 is active",
1991            )
1992            .into());
1993        }
1994        self.backend.set_gas_price(gas.to());
1995        Ok(())
1996    }
1997
1998    /// Sets the base fee of the next block.
1999    ///
2000    /// Handler for RPC call: `anvil_setNextBlockBaseFeePerGas`
2001    pub async fn anvil_set_next_block_base_fee_per_gas(&self, basefee: U256) -> Result<()> {
2002        node_info!("anvil_setNextBlockBaseFeePerGas");
2003        if !self.backend.is_eip1559() {
2004            return Err(RpcError::invalid_params(
2005                "anvil_setNextBlockBaseFeePerGas is only supported when EIP-1559 is active",
2006            )
2007            .into());
2008        }
2009        self.backend.set_base_fee(basefee.to());
2010        Ok(())
2011    }
2012
2013    /// Sets the coinbase address.
2014    ///
2015    /// Handler for RPC call: `anvil_setCoinbase`
2016    pub async fn anvil_set_coinbase(&self, address: Address) -> Result<()> {
2017        node_info!("anvil_setCoinbase");
2018        self.backend.set_coinbase(address);
2019        Ok(())
2020    }
2021
2022    /// Create a buffer that represents all state on the chain, which can be loaded to separate
2023    /// process by calling `anvil_loadState`
2024    ///
2025    /// Handler for RPC call: `anvil_dumpState`
2026    pub async fn anvil_dump_state(
2027        &self,
2028        preserve_historical_states: Option<bool>,
2029    ) -> Result<Bytes> {
2030        node_info!("anvil_dumpState");
2031        self.backend.dump_state(preserve_historical_states.unwrap_or(false)).await
2032    }
2033
2034    /// Returns the current state
2035    pub async fn serialized_state(
2036        &self,
2037        preserve_historical_states: bool,
2038    ) -> Result<SerializableState> {
2039        self.backend.serialized_state(preserve_historical_states).await
2040    }
2041
2042    /// Append chain state buffer to current chain. Will overwrite any conflicting addresses or
2043    /// storage.
2044    ///
2045    /// Handler for RPC call: `anvil_loadState`
2046    pub async fn anvil_load_state(&self, buf: Bytes) -> Result<bool> {
2047        node_info!("anvil_loadState");
2048        self.backend.load_state_bytes(buf).await
2049    }
2050
2051    /// Retrieves the Anvil node configuration params.
2052    ///
2053    /// Handler for RPC call: `anvil_nodeInfo`
2054    pub async fn anvil_node_info(&self) -> Result<NodeInfo> {
2055        node_info!("anvil_nodeInfo");
2056
2057        let env = self.backend.env().read();
2058        let fork_config = self.backend.get_fork();
2059        let tx_order = self.transaction_order.read();
2060        let hard_fork: &str = env.evm_env.cfg_env.spec.into();
2061
2062        Ok(NodeInfo {
2063            current_block_number: self.backend.best_number(),
2064            current_block_timestamp: env.evm_env.block_env.timestamp,
2065            current_block_hash: self.backend.best_hash(),
2066            hard_fork: hard_fork.to_string(),
2067            transaction_order: match *tx_order {
2068                TransactionOrder::Fifo => "fifo".to_string(),
2069                TransactionOrder::Fees => "fees".to_string(),
2070            },
2071            environment: NodeEnvironment {
2072                base_fee: self.backend.base_fee() as u128,
2073                chain_id: self.backend.chain_id().to::<u64>(),
2074                gas_limit: self.backend.gas_limit(),
2075                gas_price: self.gas_price(),
2076            },
2077            fork_config: fork_config
2078                .map(|fork| {
2079                    let config = fork.config.read();
2080
2081                    NodeForkConfig {
2082                        fork_url: Some(config.eth_rpc_url.clone()),
2083                        fork_block_number: Some(config.block_number),
2084                        fork_retry_backoff: Some(config.backoff.as_millis()),
2085                    }
2086                })
2087                .unwrap_or_default(),
2088        })
2089    }
2090
2091    /// Retrieves metadata about the Anvil instance.
2092    ///
2093    /// Handler for RPC call: `anvil_metadata`
2094    pub async fn anvil_metadata(&self) -> Result<Metadata> {
2095        node_info!("anvil_metadata");
2096        let fork_config = self.backend.get_fork();
2097
2098        Ok(Metadata {
2099            client_version: CLIENT_VERSION.to_string(),
2100            chain_id: self.backend.chain_id().to::<u64>(),
2101            latest_block_hash: self.backend.best_hash(),
2102            latest_block_number: self.backend.best_number(),
2103            instance_id: *self.instance_id.read(),
2104            forked_network: fork_config.map(|cfg| ForkedNetwork {
2105                chain_id: cfg.chain_id(),
2106                fork_block_number: cfg.block_number(),
2107                fork_block_hash: cfg.block_hash(),
2108            }),
2109            snapshots: self.backend.list_state_snapshots(),
2110        })
2111    }
2112
2113    pub async fn anvil_remove_pool_transactions(&self, address: Address) -> Result<()> {
2114        node_info!("anvil_removePoolTransactions");
2115        self.pool.remove_transactions_by_address(address);
2116        Ok(())
2117    }
2118
2119    /// Reorg the chain to a specific depth and mine new blocks back to the canonical height.
2120    ///
2121    /// e.g depth = 3
2122    ///     A  -> B  -> C  -> D  -> E
2123    ///     A  -> B  -> C' -> D' -> E'
2124    ///
2125    /// Depth specifies the height to reorg the chain back to. Depth must not exceed the current
2126    /// chain height, i.e. can't reorg past the genesis block.
2127    ///
2128    /// Optionally supply a list of transaction and block pairs that will populate the reorged
2129    /// blocks. The maximum block number of the pairs must not exceed the specified depth.
2130    ///
2131    /// Handler for RPC call: `anvil_reorg`
2132    pub async fn anvil_reorg(&self, options: ReorgOptions) -> Result<()> {
2133        node_info!("anvil_reorg");
2134        let depth = options.depth;
2135        let tx_block_pairs = options.tx_block_pairs;
2136
2137        // Check reorg depth doesn't exceed current chain height
2138        let current_height = self.backend.best_number();
2139        let common_height = current_height.checked_sub(depth).ok_or(BlockchainError::RpcError(
2140            RpcError::invalid_params(format!(
2141                "Reorg depth must not exceed current chain height: current height {current_height}, depth {depth}"
2142            )),
2143        ))?;
2144
2145        // Get the common ancestor block
2146        let common_block =
2147            self.backend.get_block(common_height).ok_or(BlockchainError::BlockNotFound)?;
2148
2149        // Convert the transaction requests to pool transactions if they exist, otherwise use empty
2150        // hashmap
2151        let block_pool_txs = if tx_block_pairs.is_empty() {
2152            HashMap::default()
2153        } else {
2154            let mut pairs = tx_block_pairs;
2155
2156            // Check the maximum block supplied number will not exceed the reorged chain height
2157            if let Some((_, num)) = pairs.iter().find(|(_, num)| *num >= depth) {
2158                return Err(BlockchainError::RpcError(RpcError::invalid_params(format!(
2159                    "Block number for reorg tx will exceed the reorged chain height. Block number {num} must not exceed (depth-1) {}",
2160                    depth-1
2161                ))));
2162            }
2163
2164            // Sort by block number to make it easier to manage new nonces
2165            pairs.sort_by_key(|a| a.1);
2166
2167            // Manage nonces for each signer
2168            // address -> cumulative nonce
2169            let mut nonces: HashMap<Address, u64> = HashMap::default();
2170
2171            let mut txs: HashMap<u64, Vec<Arc<PoolTransaction>>> = HashMap::default();
2172            for pair in pairs {
2173                let (tx_data, block_index) = pair;
2174
2175                let pending = match tx_data {
2176                    TransactionData::Raw(bytes) => {
2177                        let mut data = bytes.as_ref();
2178                        let decoded = TypedTransaction::decode_2718(&mut data)
2179                            .map_err(|_| BlockchainError::FailedToDecodeSignedTransaction)?;
2180                        PendingTransaction::new(decoded)?
2181                    }
2182
2183                    TransactionData::JSON(req) => {
2184                        let mut tx_req = WithOtherFields::new(req);
2185                        let from = tx_req.from.map(Ok).unwrap_or_else(|| {
2186                            self.accounts()?
2187                                .first()
2188                                .cloned()
2189                                .ok_or(BlockchainError::NoSignerAvailable)
2190                        })?;
2191
2192                        // Get the nonce at the common block
2193                        let curr_nonce = nonces.entry(from).or_insert(
2194                            self.get_transaction_count(
2195                                from,
2196                                Some(common_block.header.number.into()),
2197                            )
2198                            .await?,
2199                        );
2200
2201                        // Estimate gas
2202                        if tx_req.gas.is_none() {
2203                            if let Ok(gas) = self
2204                                .estimate_gas(tx_req.clone(), None, EvmOverrides::default())
2205                                .await
2206                            {
2207                                tx_req.gas = Some(gas.to());
2208                            }
2209                        }
2210
2211                        // Build typed transaction request
2212                        let typed = self.build_typed_tx_request(tx_req, *curr_nonce)?;
2213
2214                        // Increment nonce
2215                        *curr_nonce += 1;
2216
2217                        // Handle signer and convert to pending transaction
2218                        if self.is_impersonated(from) {
2219                            let bypass_signature = self.impersonated_signature(&typed);
2220                            let transaction =
2221                                sign::build_typed_transaction(typed, bypass_signature)?;
2222                            self.ensure_typed_transaction_supported(&transaction)?;
2223                            PendingTransaction::with_impersonated(transaction, from)
2224                        } else {
2225                            let transaction = self.sign_request(&from, typed)?;
2226                            self.ensure_typed_transaction_supported(&transaction)?;
2227                            PendingTransaction::new(transaction)?
2228                        }
2229                    }
2230                };
2231
2232                let pooled = PoolTransaction::new(pending);
2233                txs.entry(block_index).or_default().push(Arc::new(pooled));
2234            }
2235
2236            txs
2237        };
2238
2239        self.backend.reorg(depth, block_pool_txs, common_block).await?;
2240        Ok(())
2241    }
2242
2243    /// Rollback the chain to a specific depth.
2244    ///
2245    /// e.g depth = 3
2246    ///     A  -> B  -> C  -> D  -> E
2247    ///     A  -> B
2248    ///
2249    /// Depth specifies the height to rollback the chain back to. Depth must not exceed the current
2250    /// chain height, i.e. can't rollback past the genesis block.
2251    ///
2252    /// Handler for RPC call: `anvil_rollback`
2253    pub async fn anvil_rollback(&self, depth: Option<u64>) -> Result<()> {
2254        node_info!("anvil_rollback");
2255        let depth = depth.unwrap_or(1);
2256
2257        // Check reorg depth doesn't exceed current chain height
2258        let current_height = self.backend.best_number();
2259        let common_height = current_height.checked_sub(depth).ok_or(BlockchainError::RpcError(
2260            RpcError::invalid_params(format!(
2261                "Rollback depth must not exceed current chain height: current height {current_height}, depth {depth}"
2262            )),
2263        ))?;
2264
2265        // Get the common ancestor block
2266        let common_block =
2267            self.backend.get_block(common_height).ok_or(BlockchainError::BlockNotFound)?;
2268
2269        self.backend.rollback(common_block).await?;
2270        Ok(())
2271    }
2272
2273    /// Snapshot the state of the blockchain at the current block.
2274    ///
2275    /// Handler for RPC call: `evm_snapshot`
2276    pub async fn evm_snapshot(&self) -> Result<U256> {
2277        node_info!("evm_snapshot");
2278        Ok(self.backend.create_state_snapshot().await)
2279    }
2280
2281    /// Revert the state of the blockchain to a previous snapshot.
2282    /// Takes a single parameter, which is the snapshot id to revert to.
2283    ///
2284    /// Handler for RPC call: `evm_revert`
2285    pub async fn evm_revert(&self, id: U256) -> Result<bool> {
2286        node_info!("evm_revert");
2287        self.backend.revert_state_snapshot(id).await
2288    }
2289
2290    /// Jump forward in time by the given amount of time, in seconds.
2291    ///
2292    /// Handler for RPC call: `evm_increaseTime`
2293    pub async fn evm_increase_time(&self, seconds: U256) -> Result<i64> {
2294        node_info!("evm_increaseTime");
2295        Ok(self.backend.time().increase_time(seconds.try_into().unwrap_or(u64::MAX)) as i64)
2296    }
2297
2298    /// Similar to `evm_increaseTime` but takes the exact timestamp that you want in the next block
2299    ///
2300    /// Handler for RPC call: `evm_setNextBlockTimestamp`
2301    pub fn evm_set_next_block_timestamp(&self, seconds: u64) -> Result<()> {
2302        node_info!("evm_setNextBlockTimestamp");
2303        self.backend.time().set_next_block_timestamp(seconds)
2304    }
2305
2306    /// Sets the specific timestamp and returns the number of seconds between the given timestamp
2307    /// and the current time.
2308    ///
2309    /// Handler for RPC call: `evm_setTime`
2310    pub fn evm_set_time(&self, timestamp: u64) -> Result<u64> {
2311        node_info!("evm_setTime");
2312        let now = self.backend.time().current_call_timestamp();
2313        self.backend.time().reset(timestamp);
2314
2315        // number of seconds between the given timestamp and the current time.
2316        let offset = timestamp.saturating_sub(now);
2317        Ok(Duration::from_millis(offset).as_secs())
2318    }
2319
2320    /// Set the next block gas limit
2321    ///
2322    /// Handler for RPC call: `evm_setBlockGasLimit`
2323    pub fn evm_set_block_gas_limit(&self, gas_limit: U256) -> Result<bool> {
2324        node_info!("evm_setBlockGasLimit");
2325        self.backend.set_gas_limit(gas_limit.to());
2326        Ok(true)
2327    }
2328
2329    /// Sets an interval for the block timestamp
2330    ///
2331    /// Handler for RPC call: `anvil_setBlockTimestampInterval`
2332    pub fn evm_set_block_timestamp_interval(&self, seconds: u64) -> Result<()> {
2333        node_info!("anvil_setBlockTimestampInterval");
2334        self.backend.time().set_block_timestamp_interval(seconds);
2335        Ok(())
2336    }
2337
2338    /// Sets an interval for the block timestamp
2339    ///
2340    /// Handler for RPC call: `anvil_removeBlockTimestampInterval`
2341    pub fn evm_remove_block_timestamp_interval(&self) -> Result<bool> {
2342        node_info!("anvil_removeBlockTimestampInterval");
2343        Ok(self.backend.time().remove_block_timestamp_interval())
2344    }
2345
2346    /// Mine blocks, instantly.
2347    ///
2348    /// Handler for RPC call: `evm_mine`
2349    ///
2350    /// This will mine the blocks regardless of the configured mining mode.
2351    /// **Note**: ganache returns `0x0` here as placeholder for additional meta-data in the future.
2352    pub async fn evm_mine(&self, opts: Option<MineOptions>) -> Result<String> {
2353        node_info!("evm_mine");
2354
2355        self.do_evm_mine(opts).await?;
2356
2357        Ok("0x0".to_string())
2358    }
2359
2360    /// Mine blocks, instantly and return the mined blocks.
2361    ///
2362    /// Handler for RPC call: `evm_mine_detailed`
2363    ///
2364    /// This will mine the blocks regardless of the configured mining mode.
2365    ///
2366    /// **Note**: This behaves exactly as [Self::evm_mine] but returns different output, for
2367    /// compatibility reasons, this is a separate call since `evm_mine` is not an anvil original.
2368    /// and `ganache` may change the `0x0` placeholder.
2369    pub async fn evm_mine_detailed(&self, opts: Option<MineOptions>) -> Result<Vec<AnyRpcBlock>> {
2370        node_info!("evm_mine_detailed");
2371
2372        let mined_blocks = self.do_evm_mine(opts).await?;
2373
2374        let mut blocks = Vec::with_capacity(mined_blocks as usize);
2375
2376        let latest = self.backend.best_number();
2377        for offset in (0..mined_blocks).rev() {
2378            let block_num = latest - offset;
2379            if let Some(mut block) =
2380                self.backend.block_by_number_full(BlockNumber::Number(block_num)).await?
2381            {
2382                let block_txs = match block.transactions_mut() {
2383                    BlockTransactions::Full(txs) => txs,
2384                    BlockTransactions::Hashes(_) | BlockTransactions::Uncle => unreachable!(),
2385                };
2386                for tx in block_txs.iter_mut() {
2387                    if let Some(receipt) = self.backend.mined_transaction_receipt(tx.tx_hash()) {
2388                        if let Some(output) = receipt.out {
2389                            // insert revert reason if failure
2390                            if !receipt
2391                                .inner
2392                                .inner
2393                                .as_receipt_with_bloom()
2394                                .receipt
2395                                .status
2396                                .coerce_status()
2397                            {
2398                                if let Some(reason) =
2399                                    RevertDecoder::new().maybe_decode(&output, None)
2400                                {
2401                                    tx.other.insert(
2402                                        "revertReason".to_string(),
2403                                        serde_json::to_value(reason).expect("Infallible"),
2404                                    );
2405                                }
2406                            }
2407                            tx.other.insert(
2408                                "output".to_string(),
2409                                serde_json::to_value(output).expect("Infallible"),
2410                            );
2411                        }
2412                    }
2413                }
2414                block.transactions = BlockTransactions::Full(block_txs.to_vec());
2415                blocks.push(block);
2416            }
2417        }
2418
2419        Ok(blocks)
2420    }
2421
2422    /// Sets the reported block number
2423    ///
2424    /// Handler for ETH RPC call: `anvil_setBlock`
2425    pub fn anvil_set_block(&self, block_number: u64) -> Result<()> {
2426        node_info!("anvil_setBlock");
2427        self.backend.set_block_number(block_number);
2428        Ok(())
2429    }
2430
2431    /// Sets the backend rpc url
2432    ///
2433    /// Handler for ETH RPC call: `anvil_setRpcUrl`
2434    pub fn anvil_set_rpc_url(&self, url: String) -> Result<()> {
2435        node_info!("anvil_setRpcUrl");
2436        if let Some(fork) = self.backend.get_fork() {
2437            let mut config = fork.config.write();
2438            // let interval = config.provider.get_interval();
2439            let new_provider = Arc::new(
2440                ProviderBuilder::new(&url).max_retry(10).initial_backoff(1000).build().map_err(
2441                    |_| {
2442                        TransportErrorKind::custom_str(
2443                            format!("Failed to parse invalid url {url}").as_str(),
2444                        )
2445                    },
2446                    // TODO: Add interval
2447                )?, // .interval(interval),
2448            );
2449            config.provider = new_provider;
2450            trace!(target: "backend", "Updated fork rpc from \"{}\" to \"{}\"", config.eth_rpc_url, url);
2451            config.eth_rpc_url = url;
2452        }
2453        Ok(())
2454    }
2455
2456    /// Turn on call traces for transactions that are returned to the user when they execute a
2457    /// transaction (instead of just txhash/receipt)
2458    ///
2459    /// Handler for ETH RPC call: `anvil_enableTraces`
2460    pub async fn anvil_enable_traces(&self) -> Result<()> {
2461        node_info!("anvil_enableTraces");
2462        Err(BlockchainError::RpcUnimplemented)
2463    }
2464
2465    /// Execute a transaction regardless of signature status
2466    ///
2467    /// Handler for ETH RPC call: `eth_sendUnsignedTransaction`
2468    pub async fn eth_send_unsigned_transaction(
2469        &self,
2470        request: WithOtherFields<TransactionRequest>,
2471    ) -> Result<TxHash> {
2472        node_info!("eth_sendUnsignedTransaction");
2473        // either use the impersonated account of the request's `from` field
2474        let from = request.from.ok_or(BlockchainError::NoSignerAvailable)?;
2475
2476        let (nonce, on_chain_nonce) = self.request_nonce(&request, from).await?;
2477
2478        let request = self.build_typed_tx_request(request, nonce)?;
2479
2480        let bypass_signature = self.impersonated_signature(&request);
2481        let transaction = sign::build_typed_transaction(request, bypass_signature)?;
2482
2483        self.ensure_typed_transaction_supported(&transaction)?;
2484
2485        let pending_transaction = PendingTransaction::with_impersonated(transaction, from);
2486
2487        // pre-validate
2488        self.backend.validate_pool_transaction(&pending_transaction).await?;
2489
2490        let requires = required_marker(nonce, on_chain_nonce, from);
2491        let provides = vec![to_marker(nonce, from)];
2492
2493        self.add_pending_transaction(pending_transaction, requires, provides)
2494    }
2495
2496    /// Returns the number of transactions currently pending for inclusion in the next block(s), as
2497    /// well as the ones that are being scheduled for future execution only.
2498    /// Ref: [Here](https://geth.ethereum.org/docs/rpc/ns-txpool#txpool_status)
2499    ///
2500    /// Handler for ETH RPC call: `txpool_status`
2501    pub async fn txpool_status(&self) -> Result<TxpoolStatus> {
2502        node_info!("txpool_status");
2503        Ok(self.pool.txpool_status())
2504    }
2505
2506    /// Returns a summary of all the transactions currently pending for inclusion in the next
2507    /// block(s), as well as the ones that are being scheduled for future execution only.
2508    ///
2509    /// See [here](https://geth.ethereum.org/docs/rpc/ns-txpool#txpool_inspect) for more details
2510    ///
2511    /// Handler for ETH RPC call: `txpool_inspect`
2512    pub async fn txpool_inspect(&self) -> Result<TxpoolInspect> {
2513        node_info!("txpool_inspect");
2514        let mut inspect = TxpoolInspect::default();
2515
2516        fn convert(tx: Arc<PoolTransaction>) -> TxpoolInspectSummary {
2517            let tx = &tx.pending_transaction.transaction;
2518            let to = tx.to();
2519            let gas_price = tx.gas_price();
2520            let value = tx.value();
2521            let gas = tx.gas_limit();
2522            TxpoolInspectSummary { to, value, gas, gas_price }
2523        }
2524
2525        // Note: naming differs geth vs anvil:
2526        //
2527        // _Pending transactions_ are transactions that are ready to be processed and included in
2528        // the block. _Queued transactions_ are transactions where the transaction nonce is
2529        // not in sequence. The transaction nonce is an incrementing number for each transaction
2530        // with the same From address.
2531        for pending in self.pool.ready_transactions() {
2532            let entry = inspect.pending.entry(*pending.pending_transaction.sender()).or_default();
2533            let key = pending.pending_transaction.nonce().to_string();
2534            entry.insert(key, convert(pending));
2535        }
2536        for queued in self.pool.pending_transactions() {
2537            let entry = inspect.pending.entry(*queued.pending_transaction.sender()).or_default();
2538            let key = queued.pending_transaction.nonce().to_string();
2539            entry.insert(key, convert(queued));
2540        }
2541        Ok(inspect)
2542    }
2543
2544    /// Returns the details of all transactions currently pending for inclusion in the next
2545    /// block(s), as well as the ones that are being scheduled for future execution only.
2546    ///
2547    /// See [here](https://geth.ethereum.org/docs/rpc/ns-txpool#txpool_content) for more details
2548    ///
2549    /// Handler for ETH RPC call: `txpool_inspect`
2550    pub async fn txpool_content(&self) -> Result<TxpoolContent<AnyRpcTransaction>> {
2551        node_info!("txpool_content");
2552        let mut content = TxpoolContent::<AnyRpcTransaction>::default();
2553        fn convert(tx: Arc<PoolTransaction>) -> Result<AnyRpcTransaction> {
2554            let from = *tx.pending_transaction.sender();
2555            let tx = transaction_build(
2556                Some(tx.hash()),
2557                tx.pending_transaction.transaction.clone(),
2558                None,
2559                None,
2560                None,
2561            );
2562
2563            let WithOtherFields { inner: mut tx, other } = tx.0;
2564
2565            // we set the from field here explicitly to the set sender of the pending transaction,
2566            // in case the transaction is impersonated.
2567            tx.inner = Recovered::new_unchecked(tx.inner.into_inner(), from);
2568
2569            let tx = AnyRpcTransaction(WithOtherFields { inner: tx, other });
2570
2571            Ok(tx)
2572        }
2573
2574        for pending in self.pool.ready_transactions() {
2575            let entry = content.pending.entry(*pending.pending_transaction.sender()).or_default();
2576            let key = pending.pending_transaction.nonce().to_string();
2577            entry.insert(key, convert(pending)?);
2578        }
2579        for queued in self.pool.pending_transactions() {
2580            let entry = content.pending.entry(*queued.pending_transaction.sender()).or_default();
2581            let key = queued.pending_transaction.nonce().to_string();
2582            entry.insert(key, convert(queued)?);
2583        }
2584
2585        Ok(content)
2586    }
2587}
2588
2589// ===== impl Wallet endppoints =====
2590impl EthApi {
2591    /// Get the capabilities of the wallet.
2592    ///
2593    /// See also [EIP-5792][eip-5792].
2594    ///
2595    /// [eip-5792]: https://eips.ethereum.org/EIPS/eip-5792
2596    pub fn get_capabilities(&self) -> Result<WalletCapabilities> {
2597        node_info!("wallet_getCapabilities");
2598        Ok(self.backend.get_capabilities())
2599    }
2600
2601    pub async fn wallet_send_transaction(
2602        &self,
2603        mut request: WithOtherFields<TransactionRequest>,
2604    ) -> Result<TxHash> {
2605        node_info!("wallet_sendTransaction");
2606
2607        // Validate the request
2608        // reject transactions that have a non-zero value to prevent draining the executor.
2609        if request.value.is_some_and(|val| val > U256::ZERO) {
2610            return Err(WalletError::ValueNotZero.into())
2611        }
2612
2613        // reject transactions that have from set, as this will be the executor.
2614        if request.from.is_some() {
2615            return Err(WalletError::FromSet.into());
2616        }
2617
2618        // reject transaction requests that have nonce set, as this is managed by the executor.
2619        if request.nonce.is_some() {
2620            return Err(WalletError::NonceSet.into());
2621        }
2622
2623        let capabilities = self.backend.get_capabilities();
2624        let valid_delegations: &[Address] = capabilities
2625            .get(self.chain_id())
2626            .map(|caps| caps.delegation.addresses.as_ref())
2627            .unwrap_or_default();
2628
2629        if let Some(authorizations) = &request.authorization_list {
2630            if authorizations.iter().any(|auth| !valid_delegations.contains(&auth.address)) {
2631                return Err(WalletError::InvalidAuthorization.into());
2632            }
2633        }
2634
2635        // validate the destination address
2636        match (request.authorization_list.is_some(), request.to) {
2637            // if this is an eip-1559 tx, ensure that it is an account that delegates to a
2638            // whitelisted address
2639            (false, Some(TxKind::Call(addr))) => {
2640                let acc = self.backend.get_account(addr).await?;
2641
2642                let delegated_address = acc
2643                    .code
2644                    .map(|code| match code {
2645                        Bytecode::Eip7702(c) => c.address(),
2646                        _ => Address::ZERO,
2647                    })
2648                    .unwrap_or_default();
2649
2650                // not a whitelisted address, or not an eip-7702 bytecode
2651                if delegated_address == Address::ZERO ||
2652                    !valid_delegations.contains(&delegated_address)
2653                {
2654                    return Err(WalletError::IllegalDestination.into());
2655                }
2656            }
2657            // if it's an eip-7702 tx, let it through
2658            (true, _) => (),
2659            // create tx's disallowed
2660            _ => return Err(WalletError::IllegalDestination.into()),
2661        }
2662
2663        let wallet = self.backend.executor_wallet().ok_or(WalletError::InternalError)?;
2664
2665        let from = NetworkWallet::<Ethereum>::default_signer_address(&wallet);
2666
2667        let nonce = self.get_transaction_count(from, Some(BlockId::latest())).await?;
2668
2669        request.nonce = Some(nonce);
2670
2671        let chain_id = self.chain_id();
2672
2673        request.chain_id = Some(chain_id);
2674
2675        request.from = Some(from);
2676
2677        let gas_limit_fut =
2678            self.estimate_gas(request.clone(), Some(BlockId::latest()), EvmOverrides::default());
2679
2680        let fees_fut = self.fee_history(
2681            U256::from(EIP1559_FEE_ESTIMATION_PAST_BLOCKS),
2682            BlockNumber::Latest,
2683            vec![EIP1559_FEE_ESTIMATION_REWARD_PERCENTILE],
2684        );
2685
2686        let (gas_limit, fees) = tokio::join!(gas_limit_fut, fees_fut);
2687
2688        let gas_limit = gas_limit?;
2689        let fees = fees?;
2690
2691        request.gas = Some(gas_limit.to());
2692
2693        let base_fee = fees.latest_block_base_fee().unwrap_or_default();
2694
2695        let estimation = eip1559_default_estimator(base_fee, &fees.reward.unwrap_or_default());
2696
2697        request.max_fee_per_gas = Some(estimation.max_fee_per_gas);
2698        request.max_priority_fee_per_gas = Some(estimation.max_priority_fee_per_gas);
2699        request.gas_price = None;
2700
2701        let envelope = request.build(&wallet).await.map_err(|_| WalletError::InternalError)?;
2702
2703        self.send_raw_transaction(envelope.encoded_2718().into()).await
2704    }
2705
2706    /// Add an address to the delegation capability of wallet.
2707    ///
2708    /// This entails that the executor will now be able to sponsor transactions to this address.
2709    pub fn anvil_add_capability(&self, address: Address) -> Result<()> {
2710        node_info!("anvil_addCapability");
2711        self.backend.add_capability(address);
2712        Ok(())
2713    }
2714
2715    pub fn anvil_set_executor(&self, executor_pk: String) -> Result<Address> {
2716        node_info!("anvil_setExecutor");
2717        self.backend.set_executor(executor_pk)
2718    }
2719}
2720
2721impl EthApi {
2722    /// Executes the future on a new blocking task.
2723    async fn on_blocking_task<C, F, R>(&self, c: C) -> Result<R>
2724    where
2725        C: FnOnce(Self) -> F,
2726        F: Future<Output = Result<R>> + Send + 'static,
2727        R: Send + 'static,
2728    {
2729        let (tx, rx) = oneshot::channel();
2730        let this = self.clone();
2731        let f = c(this);
2732        tokio::task::spawn_blocking(move || {
2733            tokio::runtime::Handle::current().block_on(async move {
2734                let res = f.await;
2735                let _ = tx.send(res);
2736            })
2737        });
2738        rx.await.map_err(|_| BlockchainError::Internal("blocking task panicked".to_string()))?
2739    }
2740
2741    /// Executes the `evm_mine` and returns the number of blocks mined
2742    async fn do_evm_mine(&self, opts: Option<MineOptions>) -> Result<u64> {
2743        let mut blocks_to_mine = 1u64;
2744
2745        if let Some(opts) = opts {
2746            let timestamp = match opts {
2747                MineOptions::Timestamp(timestamp) => timestamp,
2748                MineOptions::Options { timestamp, blocks } => {
2749                    if let Some(blocks) = blocks {
2750                        blocks_to_mine = blocks;
2751                    }
2752                    timestamp
2753                }
2754            };
2755            if let Some(timestamp) = timestamp {
2756                // timestamp was explicitly provided to be the next timestamp
2757                self.evm_set_next_block_timestamp(timestamp)?;
2758            }
2759        }
2760
2761        // this can be blocking for a bit, especially in forking mode
2762        // <https://github.com/foundry-rs/foundry/issues/6036>
2763        self.on_blocking_task(|this| async move {
2764            // mine all the blocks
2765            for _ in 0..blocks_to_mine {
2766                this.mine_one().await;
2767            }
2768            Ok(())
2769        })
2770        .await?;
2771
2772        Ok(blocks_to_mine)
2773    }
2774
2775    async fn do_estimate_gas(
2776        &self,
2777        request: WithOtherFields<TransactionRequest>,
2778        block_number: Option<BlockId>,
2779        overrides: EvmOverrides,
2780    ) -> Result<u128> {
2781        let block_request = self.block_request(block_number).await?;
2782        // check if the number predates the fork, if in fork mode
2783        if let BlockRequest::Number(number) = block_request {
2784            if let Some(fork) = self.get_fork() {
2785                if fork.predates_fork(number) {
2786                    if overrides.has_state() || overrides.has_block() {
2787                        return Err(BlockchainError::EvmOverrideError(
2788                            "not available on past forked blocks".to_string(),
2789                        ));
2790                    }
2791                    return Ok(fork.estimate_gas(&request, Some(number.into())).await?)
2792                }
2793            }
2794        }
2795
2796        // this can be blocking for a bit, especially in forking mode
2797        // <https://github.com/foundry-rs/foundry/issues/6036>
2798        self.on_blocking_task(|this| async move {
2799            this.backend
2800                .with_database_at(Some(block_request), |state, mut block| {
2801                    let mut cache_db = CacheDB::new(state);
2802                    if let Some(state_overrides) = overrides.state {
2803                        state::apply_state_overrides(
2804                            state_overrides.into_iter().collect(),
2805                            &mut cache_db,
2806                        )?;
2807                    }
2808                    if let Some(block_overrides) = overrides.block {
2809                        state::apply_block_overrides(*block_overrides, &mut cache_db, &mut block);
2810                    }
2811                    this.do_estimate_gas_with_state(request, cache_db.as_dyn(), block)
2812                })
2813                .await?
2814        })
2815        .await
2816    }
2817
2818    /// Estimates the gas usage of the `request` with the state.
2819    ///
2820    /// This will execute the transaction request and find the best gas limit via binary search.
2821    fn do_estimate_gas_with_state(
2822        &self,
2823        mut request: WithOtherFields<TransactionRequest>,
2824        state: &dyn DatabaseRef<Error = DatabaseError>,
2825        block_env: BlockEnv,
2826    ) -> Result<u128> {
2827        // If the request is a simple native token transfer we can optimize
2828        // We assume it's a transfer if we have no input data.
2829        let to = request.to.as_ref().and_then(TxKind::to);
2830
2831        // check certain fields to see if the request could be a simple transfer
2832        let maybe_transfer = request.input.input().is_none() &&
2833            request.access_list.is_none() &&
2834            request.blob_versioned_hashes.is_none();
2835
2836        if maybe_transfer {
2837            if let Some(to) = to {
2838                if let Ok(target_code) = self.backend.get_code_with_state(&state, *to) {
2839                    if target_code.as_ref().is_empty() {
2840                        return Ok(MIN_TRANSACTION_GAS);
2841                    }
2842                }
2843            }
2844        }
2845
2846        let fees = FeeDetails::new(
2847            request.gas_price,
2848            request.max_fee_per_gas,
2849            request.max_priority_fee_per_gas,
2850            request.max_fee_per_blob_gas,
2851        )?
2852        .or_zero_fees();
2853
2854        // get the highest possible gas limit, either the request's set value or the currently
2855        // configured gas limit
2856        let mut highest_gas_limit = request.gas.map_or(block_env.gas_limit.into(), |g| g as u128);
2857
2858        let gas_price = fees.gas_price.unwrap_or_default();
2859        // If we have non-zero gas price, cap gas limit by sender balance
2860        if gas_price > 0 {
2861            if let Some(from) = request.from {
2862                let mut available_funds = self.backend.get_balance_with_state(state, from)?;
2863                if let Some(value) = request.value {
2864                    if value > available_funds {
2865                        return Err(InvalidTransactionError::InsufficientFunds.into());
2866                    }
2867                    // safe: value < available_funds
2868                    available_funds -= value;
2869                }
2870                // amount of gas the sender can afford with the `gas_price`
2871                let allowance =
2872                    available_funds.checked_div(U256::from(gas_price)).unwrap_or_default();
2873                highest_gas_limit = std::cmp::min(highest_gas_limit, allowance.saturating_to());
2874            }
2875        }
2876
2877        let mut call_to_estimate = request.clone();
2878        call_to_estimate.gas = Some(highest_gas_limit as u64);
2879
2880        // execute the call without writing to db
2881        let ethres =
2882            self.backend.call_with_state(&state, call_to_estimate, fees.clone(), block_env.clone());
2883
2884        let gas_used = match ethres.try_into()? {
2885            GasEstimationCallResult::Success(gas) => Ok(gas),
2886            GasEstimationCallResult::OutOfGas => {
2887                Err(InvalidTransactionError::BasicOutOfGas(highest_gas_limit).into())
2888            }
2889            GasEstimationCallResult::Revert(output) => {
2890                Err(InvalidTransactionError::Revert(output).into())
2891            }
2892            GasEstimationCallResult::EvmError(err) => {
2893                warn!(target: "node", "estimation failed due to {:?}", err);
2894                Err(BlockchainError::EvmError(err))
2895            }
2896        }?;
2897
2898        // at this point we know the call succeeded but want to find the _best_ (lowest) gas the
2899        // transaction succeeds with. we find this by doing a binary search over the
2900        // possible range NOTE: this is the gas the transaction used, which is less than the
2901        // transaction requires to succeed
2902
2903        // Get the starting lowest gas needed depending on the transaction kind.
2904        let mut lowest_gas_limit = determine_base_gas_by_kind(&request);
2905
2906        // pick a point that's close to the estimated gas
2907        let mut mid_gas_limit =
2908            std::cmp::min(gas_used * 3, (highest_gas_limit + lowest_gas_limit) / 2);
2909
2910        // Binary search for the ideal gas limit
2911        while (highest_gas_limit - lowest_gas_limit) > 1 {
2912            request.gas = Some(mid_gas_limit as u64);
2913            let ethres = self.backend.call_with_state(
2914                &state,
2915                request.clone(),
2916                fees.clone(),
2917                block_env.clone(),
2918            );
2919
2920            match ethres.try_into()? {
2921                GasEstimationCallResult::Success(_) => {
2922                    // If the transaction succeeded, we can set a ceiling for the highest gas limit
2923                    // at the current midpoint, as spending any more gas would
2924                    // make no sense (as the TX would still succeed).
2925                    highest_gas_limit = mid_gas_limit;
2926                }
2927                GasEstimationCallResult::OutOfGas |
2928                GasEstimationCallResult::Revert(_) |
2929                GasEstimationCallResult::EvmError(_) => {
2930                    // If the transaction failed, we can set a floor for the lowest gas limit at the
2931                    // current midpoint, as spending any less gas would make no
2932                    // sense (as the TX would still revert due to lack of gas).
2933                    //
2934                    // We don't care about the reason here, as we known that transaction is correct
2935                    // as it succeeded earlier
2936                    lowest_gas_limit = mid_gas_limit;
2937                }
2938            };
2939            // new midpoint
2940            mid_gas_limit = (highest_gas_limit + lowest_gas_limit) / 2;
2941        }
2942
2943        trace!(target : "node", "Estimated Gas for call {:?}", highest_gas_limit);
2944
2945        Ok(highest_gas_limit)
2946    }
2947
2948    /// Updates the `TransactionOrder`
2949    pub fn set_transaction_order(&self, order: TransactionOrder) {
2950        *self.transaction_order.write() = order;
2951    }
2952
2953    /// Returns the priority of the transaction based on the current `TransactionOrder`
2954    fn transaction_priority(&self, tx: &TypedTransaction) -> TransactionPriority {
2955        self.transaction_order.read().priority(tx)
2956    }
2957
2958    /// Returns the chain ID used for transaction
2959    pub fn chain_id(&self) -> u64 {
2960        self.backend.chain_id().to::<u64>()
2961    }
2962
2963    /// Returns the configured fork, if any.
2964    pub fn get_fork(&self) -> Option<ClientFork> {
2965        self.backend.get_fork()
2966    }
2967
2968    /// Returns the current instance's ID.
2969    pub fn instance_id(&self) -> B256 {
2970        *self.instance_id.read()
2971    }
2972
2973    /// Resets the instance ID.
2974    pub fn reset_instance_id(&self) {
2975        *self.instance_id.write() = B256::random();
2976    }
2977
2978    /// Returns the first signer that can sign for the given address
2979    #[expect(clippy::borrowed_box)]
2980    pub fn get_signer(&self, address: Address) -> Option<&Box<dyn Signer>> {
2981        self.signers.iter().find(|signer| signer.is_signer_for(address))
2982    }
2983
2984    /// Returns a new block event stream that yields Notifications when a new block was added
2985    pub fn new_block_notifications(&self) -> NewBlockNotifications {
2986        self.backend.new_block_notifications()
2987    }
2988
2989    /// Returns a new listeners for ready transactions
2990    pub fn new_ready_transactions(&self) -> Receiver<TxHash> {
2991        self.pool.add_ready_listener()
2992    }
2993
2994    /// Returns a listener for pending transactions, yielding full transactions
2995    pub fn full_pending_transactions(&self) -> UnboundedReceiver<AnyRpcTransaction> {
2996        let (tx, rx) = unbounded_channel();
2997        let mut hashes = self.new_ready_transactions();
2998
2999        let this = self.clone();
3000
3001        tokio::spawn(async move {
3002            while let Some(hash) = hashes.next().await {
3003                if let Ok(Some(txn)) = this.transaction_by_hash(hash).await {
3004                    if tx.send(txn).is_err() {
3005                        break;
3006                    }
3007                }
3008            }
3009        });
3010
3011        rx
3012    }
3013
3014    /// Returns a new accessor for certain storage elements
3015    pub fn storage_info(&self) -> StorageInfo {
3016        StorageInfo::new(Arc::clone(&self.backend))
3017    }
3018
3019    /// Returns true if forked
3020    pub fn is_fork(&self) -> bool {
3021        self.backend.is_fork()
3022    }
3023
3024    /// Mines exactly one block
3025    pub async fn mine_one(&self) {
3026        let transactions = self.pool.ready_transactions().collect::<Vec<_>>();
3027        let outcome = self.backend.mine_block(transactions).await;
3028
3029        trace!(target: "node", blocknumber = ?outcome.block_number, "mined block");
3030        self.pool.on_mined_block(outcome);
3031    }
3032
3033    /// Returns the pending block with tx hashes
3034    async fn pending_block(&self) -> AnyRpcBlock {
3035        let transactions = self.pool.ready_transactions().collect::<Vec<_>>();
3036        let info = self.backend.pending_block(transactions).await;
3037        self.backend.convert_block(info.block)
3038    }
3039
3040    /// Returns the full pending block with `Transaction` objects
3041    async fn pending_block_full(&self) -> Option<AnyRpcBlock> {
3042        let transactions = self.pool.ready_transactions().collect::<Vec<_>>();
3043        let BlockInfo { block, transactions, receipts: _ } =
3044            self.backend.pending_block(transactions).await;
3045
3046        let mut partial_block = self.backend.convert_block(block.clone());
3047
3048        let mut block_transactions = Vec::with_capacity(block.transactions.len());
3049        let base_fee = self.backend.base_fee();
3050
3051        for info in transactions {
3052            let tx = block.transactions.get(info.transaction_index as usize)?.clone();
3053
3054            let tx = transaction_build(
3055                Some(info.transaction_hash),
3056                tx,
3057                Some(&block),
3058                Some(info),
3059                Some(base_fee),
3060            );
3061            block_transactions.push(tx);
3062        }
3063
3064        partial_block.transactions = BlockTransactions::from(block_transactions);
3065
3066        Some(partial_block)
3067    }
3068
3069    fn build_typed_tx_request(
3070        &self,
3071        request: WithOtherFields<TransactionRequest>,
3072        nonce: u64,
3073    ) -> Result<TypedTransactionRequest> {
3074        let chain_id = request.chain_id.unwrap_or_else(|| self.chain_id());
3075        let max_fee_per_gas = request.max_fee_per_gas;
3076        let max_fee_per_blob_gas = request.max_fee_per_blob_gas;
3077        let gas_price = request.gas_price;
3078
3079        let gas_limit = request.gas.unwrap_or_else(|| self.backend.gas_limit());
3080        let from = request.from;
3081
3082        let request = match transaction_request_to_typed(request) {
3083            Some(TypedTransactionRequest::Legacy(mut m)) => {
3084                m.nonce = nonce;
3085                m.chain_id = Some(chain_id);
3086                m.gas_limit = gas_limit;
3087                if gas_price.is_none() {
3088                    m.gas_price = self.gas_price();
3089                }
3090                TypedTransactionRequest::Legacy(m)
3091            }
3092            Some(TypedTransactionRequest::EIP2930(mut m)) => {
3093                m.nonce = nonce;
3094                m.chain_id = chain_id;
3095                m.gas_limit = gas_limit;
3096                if gas_price.is_none() {
3097                    m.gas_price = self.gas_price();
3098                }
3099                TypedTransactionRequest::EIP2930(m)
3100            }
3101            Some(TypedTransactionRequest::EIP1559(mut m)) => {
3102                m.nonce = nonce;
3103                m.chain_id = chain_id;
3104                m.gas_limit = gas_limit;
3105                if max_fee_per_gas.is_none() {
3106                    m.max_fee_per_gas = self.gas_price();
3107                }
3108                TypedTransactionRequest::EIP1559(m)
3109            }
3110            Some(TypedTransactionRequest::EIP7702(mut m)) => {
3111                m.nonce = nonce;
3112                m.chain_id = chain_id;
3113                m.gas_limit = gas_limit;
3114                if max_fee_per_gas.is_none() {
3115                    m.max_fee_per_gas = self.gas_price();
3116                }
3117                TypedTransactionRequest::EIP7702(m)
3118            }
3119            Some(TypedTransactionRequest::EIP4844(m)) => {
3120                TypedTransactionRequest::EIP4844(match m {
3121                    // We only accept the TxEip4844 variant which has the sidecar.
3122                    TxEip4844Variant::TxEip4844WithSidecar(mut m) => {
3123                        m.tx.nonce = nonce;
3124                        m.tx.chain_id = chain_id;
3125                        m.tx.gas_limit = gas_limit;
3126                        if max_fee_per_gas.is_none() {
3127                            m.tx.max_fee_per_gas = self.gas_price();
3128                        }
3129                        if max_fee_per_blob_gas.is_none() {
3130                            m.tx.max_fee_per_blob_gas = self
3131                                .excess_blob_gas_and_price()
3132                                .unwrap_or_default()
3133                                .map_or(0, |g| g.blob_gasprice)
3134                        }
3135                        TxEip4844Variant::TxEip4844WithSidecar(m)
3136                    }
3137                    TxEip4844Variant::TxEip4844(mut tx) => {
3138                        if !self.backend.skip_blob_validation(from) {
3139                            return Err(BlockchainError::FailedToDecodeTransaction)
3140                        }
3141
3142                        // Allows 4844 with no sidecar when impersonation is active.
3143                        tx.nonce = nonce;
3144                        tx.chain_id = chain_id;
3145                        tx.gas_limit = gas_limit;
3146                        if max_fee_per_gas.is_none() {
3147                            tx.max_fee_per_gas = self.gas_price();
3148                        }
3149                        if max_fee_per_blob_gas.is_none() {
3150                            tx.max_fee_per_blob_gas = self
3151                                .excess_blob_gas_and_price()
3152                                .unwrap_or_default()
3153                                .map_or(0, |g| g.blob_gasprice)
3154                        }
3155
3156                        TxEip4844Variant::TxEip4844(tx)
3157                    }
3158                })
3159            }
3160            Some(TypedTransactionRequest::Deposit(mut m)) => {
3161                m.gas_limit = gas_limit;
3162                TypedTransactionRequest::Deposit(m)
3163            }
3164            None => return Err(BlockchainError::FailedToDecodeTransaction),
3165        };
3166        Ok(request)
3167    }
3168
3169    /// Returns true if the `addr` is currently impersonated
3170    pub fn is_impersonated(&self, addr: Address) -> bool {
3171        self.backend.cheats().is_impersonated(addr)
3172    }
3173
3174    /// The signature used to bypass signing via the `eth_sendUnsignedTransaction` cheat RPC
3175    fn impersonated_signature(&self, request: &TypedTransactionRequest) -> Signature {
3176        match request {
3177            // Only the legacy transaction type requires v to be in {27, 28}, thus
3178            // requiring the use of Parity::NonEip155
3179            TypedTransactionRequest::Legacy(_) => Signature::from_scalars_and_parity(
3180                B256::with_last_byte(1),
3181                B256::with_last_byte(1),
3182                false,
3183            ),
3184            TypedTransactionRequest::EIP2930(_) |
3185            TypedTransactionRequest::EIP1559(_) |
3186            TypedTransactionRequest::EIP7702(_) |
3187            TypedTransactionRequest::EIP4844(_) |
3188            TypedTransactionRequest::Deposit(_) => Signature::from_scalars_and_parity(
3189                B256::with_last_byte(1),
3190                B256::with_last_byte(1),
3191                false,
3192            ),
3193        }
3194    }
3195
3196    /// Returns the nonce of the `address` depending on the `block_number`
3197    async fn get_transaction_count(
3198        &self,
3199        address: Address,
3200        block_number: Option<BlockId>,
3201    ) -> Result<u64> {
3202        let block_request = self.block_request(block_number).await?;
3203
3204        if let BlockRequest::Number(number) = block_request {
3205            if let Some(fork) = self.get_fork() {
3206                if fork.predates_fork(number) {
3207                    return Ok(fork.get_nonce(address, number).await?)
3208                }
3209            }
3210        }
3211
3212        self.backend.get_nonce(address, block_request).await
3213    }
3214
3215    /// Returns the nonce for this request
3216    ///
3217    /// This returns a tuple of `(request nonce, highest nonce)`
3218    /// If the nonce field of the `request` is `None` then the tuple will be `(highest nonce,
3219    /// highest nonce)`.
3220    ///
3221    /// This will also check the tx pool for pending transactions from the sender.
3222    async fn request_nonce(
3223        &self,
3224        request: &TransactionRequest,
3225        from: Address,
3226    ) -> Result<(u64, u64)> {
3227        let highest_nonce =
3228            self.get_transaction_count(from, Some(BlockId::Number(BlockNumber::Pending))).await?;
3229        let nonce = request.nonce.unwrap_or(highest_nonce);
3230
3231        Ok((nonce, highest_nonce))
3232    }
3233
3234    /// Adds the given transaction to the pool
3235    fn add_pending_transaction(
3236        &self,
3237        pending_transaction: PendingTransaction,
3238        requires: Vec<TxMarker>,
3239        provides: Vec<TxMarker>,
3240    ) -> Result<TxHash> {
3241        let from = *pending_transaction.sender();
3242        let priority = self.transaction_priority(&pending_transaction.transaction);
3243        let pool_transaction =
3244            PoolTransaction { requires, provides, pending_transaction, priority };
3245        let tx = self.pool.add_transaction(pool_transaction)?;
3246        trace!(target: "node", "Added transaction: [{:?}] sender={:?}", tx.hash(), from);
3247        Ok(*tx.hash())
3248    }
3249
3250    /// Returns the current state root
3251    pub async fn state_root(&self) -> Option<B256> {
3252        self.backend.get_db().read().await.maybe_state_root()
3253    }
3254
3255    /// additional validation against hardfork
3256    fn ensure_typed_transaction_supported(&self, tx: &TypedTransaction) -> Result<()> {
3257        match &tx {
3258            TypedTransaction::EIP2930(_) => self.backend.ensure_eip2930_active(),
3259            TypedTransaction::EIP1559(_) => self.backend.ensure_eip1559_active(),
3260            TypedTransaction::EIP4844(_) => self.backend.ensure_eip4844_active(),
3261            TypedTransaction::EIP7702(_) => self.backend.ensure_eip7702_active(),
3262            TypedTransaction::Deposit(_) => self.backend.ensure_op_deposits_active(),
3263            TypedTransaction::Legacy(_) => Ok(()),
3264        }
3265    }
3266}
3267
3268fn required_marker(provided_nonce: u64, on_chain_nonce: u64, from: Address) -> Vec<TxMarker> {
3269    if provided_nonce == on_chain_nonce {
3270        return Vec::new();
3271    }
3272    let prev_nonce = provided_nonce.saturating_sub(1);
3273    if on_chain_nonce <= prev_nonce {
3274        vec![to_marker(prev_nonce, from)]
3275    } else {
3276        Vec::new()
3277    }
3278}
3279
3280fn convert_transact_out(out: &Option<Output>) -> Bytes {
3281    match out {
3282        None => Default::default(),
3283        Some(Output::Call(out)) => out.to_vec().into(),
3284        Some(Output::Create(out, _)) => out.to_vec().into(),
3285    }
3286}
3287
3288/// Returns an error if the `exit` code is _not_ ok
3289fn ensure_return_ok(exit: InstructionResult, out: &Option<Output>) -> Result<Bytes> {
3290    let out = convert_transact_out(out);
3291    match exit {
3292        return_ok!() => Ok(out),
3293        return_revert!() => Err(InvalidTransactionError::Revert(Some(out.0.into())).into()),
3294        reason => Err(BlockchainError::EvmError(reason)),
3295    }
3296}
3297
3298/// Determines the minimum gas needed for a transaction depending on the transaction kind.
3299fn determine_base_gas_by_kind(request: &WithOtherFields<TransactionRequest>) -> u128 {
3300    match transaction_request_to_typed(request.clone()) {
3301        Some(request) => match request {
3302            TypedTransactionRequest::Legacy(req) => match req.to {
3303                TxKind::Call(_) => MIN_TRANSACTION_GAS,
3304                TxKind::Create => MIN_CREATE_GAS,
3305            },
3306            TypedTransactionRequest::EIP1559(req) => match req.to {
3307                TxKind::Call(_) => MIN_TRANSACTION_GAS,
3308                TxKind::Create => MIN_CREATE_GAS,
3309            },
3310            TypedTransactionRequest::EIP7702(req) => {
3311                MIN_TRANSACTION_GAS +
3312                    req.authorization_list.len() as u128 * PER_EMPTY_ACCOUNT_COST as u128
3313            }
3314            TypedTransactionRequest::EIP2930(req) => match req.to {
3315                TxKind::Call(_) => MIN_TRANSACTION_GAS,
3316                TxKind::Create => MIN_CREATE_GAS,
3317            },
3318            TypedTransactionRequest::EIP4844(_) => MIN_TRANSACTION_GAS,
3319            TypedTransactionRequest::Deposit(req) => match req.to {
3320                TxKind::Call(_) => MIN_TRANSACTION_GAS,
3321                TxKind::Create => MIN_CREATE_GAS,
3322            },
3323        },
3324        // Tighten the gas limit upwards if we don't know the transaction type to avoid deployments
3325        // failing.
3326        _ => MIN_CREATE_GAS,
3327    }
3328}
3329
3330/// Keeps result of a call to revm EVM used for gas estimation
3331enum GasEstimationCallResult {
3332    Success(u128),
3333    OutOfGas,
3334    Revert(Option<Bytes>),
3335    EvmError(InstructionResult),
3336}
3337
3338/// Converts the result of a call to revm EVM into a [`GasEstimationCallResult`].
3339impl TryFrom<Result<(InstructionResult, Option<Output>, u128, State)>> for GasEstimationCallResult {
3340    type Error = BlockchainError;
3341
3342    fn try_from(res: Result<(InstructionResult, Option<Output>, u128, State)>) -> Result<Self> {
3343        match res {
3344            // Exceptional case: init used too much gas, treated as out of gas error
3345            Err(BlockchainError::InvalidTransaction(InvalidTransactionError::GasTooHigh(_))) => {
3346                Ok(Self::OutOfGas)
3347            }
3348            Err(err) => Err(err),
3349            Ok((exit, output, gas, _)) => match exit {
3350                return_ok!() | InstructionResult::CallOrCreate => Ok(Self::Success(gas)),
3351
3352                InstructionResult::Revert => Ok(Self::Revert(output.map(|o| o.into_data()))),
3353
3354                InstructionResult::OutOfGas |
3355                InstructionResult::MemoryOOG |
3356                InstructionResult::MemoryLimitOOG |
3357                InstructionResult::PrecompileOOG |
3358                InstructionResult::InvalidOperandOOG |
3359                InstructionResult::ReentrancySentryOOG => Ok(Self::OutOfGas),
3360
3361                InstructionResult::OpcodeNotFound |
3362                InstructionResult::CallNotAllowedInsideStatic |
3363                InstructionResult::StateChangeDuringStaticCall |
3364                InstructionResult::InvalidExtDelegateCallTarget |
3365                InstructionResult::InvalidEXTCALLTarget |
3366                InstructionResult::InvalidFEOpcode |
3367                InstructionResult::InvalidJump |
3368                InstructionResult::NotActivated |
3369                InstructionResult::StackUnderflow |
3370                InstructionResult::StackOverflow |
3371                InstructionResult::OutOfOffset |
3372                InstructionResult::CreateCollision |
3373                InstructionResult::OverflowPayment |
3374                InstructionResult::PrecompileError |
3375                InstructionResult::NonceOverflow |
3376                InstructionResult::CreateContractSizeLimit |
3377                InstructionResult::CreateContractStartingWithEF |
3378                InstructionResult::CreateInitCodeSizeLimit |
3379                InstructionResult::FatalExternalError |
3380                InstructionResult::OutOfFunds |
3381                InstructionResult::CallTooDeep => Ok(Self::EvmError(exit)),
3382
3383                // Handle Revm EOF InstructionResults: not supported
3384                InstructionResult::ReturnContractInNotInitEOF |
3385                InstructionResult::EOFOpcodeDisabledInLegacy |
3386                InstructionResult::SubRoutineStackOverflow |
3387                InstructionResult::CreateInitCodeStartingEF00 |
3388                InstructionResult::InvalidEOFInitCode |
3389                InstructionResult::EofAuxDataOverflow |
3390                InstructionResult::EofAuxDataTooSmall => Ok(Self::EvmError(exit)),
3391            },
3392        }
3393    }
3394}