reth_rpc_eth_api/helpers/
call.rs

1//! Loads a pending block from database. Helper trait for `eth_` transaction, call and trace RPC
2//! methods.
3
4use super::{LoadBlock, LoadPendingBlock, LoadState, LoadTransaction, SpawnBlocking, Trace};
5use crate::{
6    helpers::estimate::EstimateCall, FromEvmError, FullEthApiTypes, RpcBlock, RpcNodeCore,
7};
8use alloy_consensus::BlockHeader;
9use alloy_eips::eip2930::AccessListResult;
10use alloy_primitives::{Bytes, B256, U256};
11use alloy_rpc_types_eth::{
12    simulate::{SimBlock, SimulatePayload, SimulatedBlock},
13    state::{EvmOverrides, StateOverride},
14    transaction::TransactionRequest,
15    BlockId, Bundle, EthCallResponse, StateContext, TransactionInfo,
16};
17use futures::Future;
18use reth_errors::{ProviderError, RethError};
19use reth_evm::{
20    ConfigureEvm, Evm, EvmEnv, EvmEnvFor, HaltReasonFor, InspectorFor, SpecFor, TransactionEnv,
21    TxEnvFor,
22};
23use reth_node_api::{BlockBody, NodePrimitives};
24use reth_primitives_traits::{Recovered, SealedHeader, SignedTransaction};
25use reth_revm::{
26    database::StateProviderDatabase,
27    db::{CacheDB, State},
28    DatabaseRef,
29};
30use reth_rpc_eth_types::{
31    cache::db::{StateCacheDbRefMutWrapper, StateProviderTraitObjWrapper},
32    error::{api::FromEvmHalt, ensure_success, FromEthApiError},
33    revm_utils::{apply_block_overrides, apply_state_overrides, caller_gas_allowance},
34    simulate::{self, EthSimulateError},
35    EthApiError, RevertError, RpcInvalidTransactionError, StateCacheDb,
36};
37use reth_storage_api::{BlockIdReader, ProviderHeader, ProviderTx};
38use revm::{
39    context_interface::{
40        result::{ExecutionResult, ResultAndState},
41        Transaction,
42    },
43    Database, DatabaseCommit,
44};
45use revm_inspectors::{access_list::AccessListInspector, transfer::TransferInspector};
46use tracing::trace;
47
48/// Result type for `eth_simulateV1` RPC method.
49pub type SimulatedBlocksResult<N, E> = Result<Vec<SimulatedBlock<RpcBlock<N>>>, E>;
50
51/// Execution related functions for the [`EthApiServer`](crate::EthApiServer) trait in
52/// the `eth_` namespace.
53pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthApiTypes {
54    /// Estimate gas needed for execution of the `request` at the [`BlockId`].
55    fn estimate_gas_at(
56        &self,
57        request: TransactionRequest,
58        at: BlockId,
59        state_override: Option<StateOverride>,
60    ) -> impl Future<Output = Result<U256, Self::Error>> + Send {
61        EstimateCall::estimate_gas_at(self, request, at, state_override)
62    }
63
64    /// `eth_simulateV1` executes an arbitrary number of transactions on top of the requested state.
65    /// The transactions are packed into individual blocks. Overrides can be provided.
66    ///
67    /// See also: <https://github.com/ethereum/go-ethereum/pull/27720>
68    fn simulate_v1(
69        &self,
70        payload: SimulatePayload,
71        block: Option<BlockId>,
72    ) -> impl Future<Output = SimulatedBlocksResult<Self::NetworkTypes, Self::Error>> + Send {
73        async move {
74            if payload.block_state_calls.len() > self.max_simulate_blocks() as usize {
75                return Err(EthApiError::InvalidParams("too many blocks.".to_string()).into())
76            }
77
78            let block = block.unwrap_or_default();
79
80            let SimulatePayload {
81                block_state_calls,
82                trace_transfers,
83                validation,
84                return_full_transactions,
85            } = payload;
86
87            if block_state_calls.is_empty() {
88                return Err(EthApiError::InvalidParams(String::from("calls are empty.")).into())
89            }
90
91            let base_block =
92                self.recovered_block(block).await?.ok_or(EthApiError::HeaderNotFound(block))?;
93            let mut parent = base_block.sealed_header().clone();
94
95            let this = self.clone();
96            self.spawn_with_state_at_block(block, move |state| {
97                let mut db =
98                    State::builder().with_database(StateProviderDatabase::new(state)).build();
99                let mut blocks: Vec<SimulatedBlock<RpcBlock<Self::NetworkTypes>>> =
100                    Vec::with_capacity(block_state_calls.len());
101                for block in block_state_calls {
102                    let mut evm_env = this
103                        .evm_config()
104                        .next_evm_env(&parent, &this.next_env_attributes(&parent)?)
105                        .map_err(RethError::other)
106                        .map_err(Self::Error::from_eth_err)?;
107
108                    // Always disable EIP-3607
109                    evm_env.cfg_env.disable_eip3607 = true;
110
111                    if !validation {
112                        // If not explicitly required, we disable nonce check <https://github.com/paradigmxyz/reth/issues/16108>
113                        evm_env.cfg_env.disable_nonce_check = true;
114                        evm_env.cfg_env.disable_base_fee = true;
115                        evm_env.block_env.basefee = 0;
116                    }
117
118                    let SimBlock { block_overrides, state_overrides, calls } = block;
119
120                    if let Some(block_overrides) = block_overrides {
121                        // ensure we don't allow uncapped gas limit per block
122                        if let Some(gas_limit_override) = block_overrides.gas_limit {
123                            if gas_limit_override > evm_env.block_env.gas_limit &&
124                                gas_limit_override > this.call_gas_limit()
125                            {
126                                return Err(
127                                    EthApiError::other(EthSimulateError::GasLimitReached).into()
128                                )
129                            }
130                        }
131                        apply_block_overrides(block_overrides, &mut db, &mut evm_env.block_env);
132                    }
133                    if let Some(state_overrides) = state_overrides {
134                        apply_state_overrides(state_overrides, &mut db)?;
135                    }
136
137                    let block_env = evm_env.block_env.clone();
138                    let chain_id = evm_env.cfg_env.chain_id;
139
140                    let default_gas_limit = {
141                        let total_specified_gas = calls.iter().filter_map(|tx| tx.gas).sum::<u64>();
142                        let txs_without_gas_limit =
143                            calls.iter().filter(|tx| tx.gas.is_none()).count();
144
145                        if total_specified_gas > block_env.gas_limit {
146                            return Err(EthApiError::Other(Box::new(
147                                EthSimulateError::BlockGasLimitExceeded,
148                            ))
149                            .into())
150                        }
151
152                        if txs_without_gas_limit > 0 {
153                            (block_env.gas_limit - total_specified_gas) /
154                                txs_without_gas_limit as u64
155                        } else {
156                            0
157                        }
158                    };
159
160                    let ctx = this
161                        .evm_config()
162                        .context_for_next_block(&parent, this.next_env_attributes(&parent)?);
163                    let (result, results) = if trace_transfers {
164                        // prepare inspector to capture transfer inside the evm so they are recorded
165                        // and included in logs
166                        let inspector = TransferInspector::new(false).with_logs(true);
167                        let evm = this
168                            .evm_config()
169                            .evm_with_env_and_inspector(&mut db, evm_env, inspector);
170                        let builder = this.evm_config().create_block_builder(evm, &parent, ctx);
171                        simulate::execute_transactions(
172                            builder,
173                            calls,
174                            default_gas_limit,
175                            chain_id,
176                            this.tx_resp_builder(),
177                        )?
178                    } else {
179                        let evm = this.evm_config().evm_with_env(&mut db, evm_env);
180                        let builder = this.evm_config().create_block_builder(evm, &parent, ctx);
181                        simulate::execute_transactions(
182                            builder,
183                            calls,
184                            default_gas_limit,
185                            chain_id,
186                            this.tx_resp_builder(),
187                        )?
188                    };
189
190                    let block = simulate::build_simulated_block(
191                        result.block,
192                        results,
193                        return_full_transactions,
194                        this.tx_resp_builder(),
195                    )?;
196
197                    parent = SealedHeader::new(
198                        block.inner.header.inner.clone(),
199                        block.inner.header.hash,
200                    );
201
202                    blocks.push(block);
203                }
204
205                Ok(blocks)
206            })
207            .await
208        }
209    }
210
211    /// Executes the call request (`eth_call`) and returns the output
212    fn call(
213        &self,
214        request: TransactionRequest,
215        block_number: Option<BlockId>,
216        overrides: EvmOverrides,
217    ) -> impl Future<Output = Result<Bytes, Self::Error>> + Send {
218        async move {
219            let (res, _env) =
220                self.transact_call_at(request, block_number.unwrap_or_default(), overrides).await?;
221
222            ensure_success(res.result)
223        }
224    }
225
226    /// Simulate arbitrary number of transactions at an arbitrary blockchain index, with the
227    /// optionality of state overrides
228    fn call_many(
229        &self,
230        bundles: Vec<Bundle>,
231        state_context: Option<StateContext>,
232        mut state_override: Option<StateOverride>,
233    ) -> impl Future<Output = Result<Vec<Vec<EthCallResponse>>, Self::Error>> + Send {
234        async move {
235            // Check if the vector of bundles is empty
236            if bundles.is_empty() {
237                return Err(EthApiError::InvalidParams(String::from("bundles are empty.")).into());
238            }
239
240            let StateContext { transaction_index, block_number } =
241                state_context.unwrap_or_default();
242            let transaction_index = transaction_index.unwrap_or_default();
243
244            let mut target_block = block_number.unwrap_or_default();
245            let is_block_target_pending = target_block.is_pending();
246
247            // if it's not pending, we should always use block_hash over block_number to ensure that
248            // different provider calls query data related to the same block.
249            if !is_block_target_pending {
250                target_block = self
251                    .provider()
252                    .block_hash_for_id(target_block)
253                    .map_err(|_| EthApiError::HeaderNotFound(target_block))?
254                    .ok_or_else(|| EthApiError::HeaderNotFound(target_block))?
255                    .into();
256            }
257
258            let ((evm_env, _), block) = futures::try_join!(
259                self.evm_env_at(target_block),
260                self.recovered_block(target_block)
261            )?;
262
263            let block = block.ok_or(EthApiError::HeaderNotFound(target_block))?;
264
265            // we're essentially replaying the transactions in the block here, hence we need the
266            // state that points to the beginning of the block, which is the state at
267            // the parent block
268            let mut at = block.parent_hash();
269            let mut replay_block_txs = true;
270
271            let num_txs =
272                transaction_index.index().unwrap_or_else(|| block.body().transactions().len());
273            // but if all transactions are to be replayed, we can use the state at the block itself,
274            // however only if we're not targeting the pending block, because for pending we can't
275            // rely on the block's state being available
276            if !is_block_target_pending && num_txs == block.body().transactions().len() {
277                at = block.hash();
278                replay_block_txs = false;
279            }
280
281            let this = self.clone();
282            self.spawn_with_state_at_block(at.into(), move |state| {
283                let mut all_results = Vec::with_capacity(bundles.len());
284                let mut db = CacheDB::new(StateProviderDatabase::new(state));
285
286                if replay_block_txs {
287                    // only need to replay the transactions in the block if not all transactions are
288                    // to be replayed
289                    let block_transactions = block.transactions_recovered().take(num_txs);
290                    for tx in block_transactions {
291                        let tx_env = RpcNodeCore::evm_config(&this).tx_env(tx);
292                        let (res, _) = this.transact(&mut db, evm_env.clone(), tx_env)?;
293                        db.commit(res.state);
294                    }
295                }
296
297                // transact all bundles
298                for bundle in bundles {
299                    let Bundle { transactions, block_override } = bundle;
300                    if transactions.is_empty() {
301                        // Skip empty bundles
302                        continue;
303                    }
304
305                    let mut bundle_results = Vec::with_capacity(transactions.len());
306                    let block_overrides = block_override.map(Box::new);
307
308                    // transact all transactions in the bundle
309                    for tx in transactions {
310                        // Apply overrides, state overrides are only applied for the first tx in the
311                        // request
312                        let overrides =
313                            EvmOverrides::new(state_override.take(), block_overrides.clone());
314
315                        let (current_evm_env, prepared_tx) =
316                            this.prepare_call_env(evm_env.clone(), tx, &mut db, overrides)?;
317                        let (res, _) = this.transact(&mut db, current_evm_env, prepared_tx)?;
318
319                        match ensure_success::<_, Self::Error>(res.result) {
320                            Ok(output) => {
321                                bundle_results
322                                    .push(EthCallResponse { value: Some(output), error: None });
323                            }
324                            Err(err) => {
325                                bundle_results.push(EthCallResponse {
326                                    value: None,
327                                    error: Some(err.to_string()),
328                                });
329                            }
330                        }
331
332                        // Commit state changes after each transaction to allow subsequent calls to
333                        // see the updates
334                        db.commit(res.state);
335                    }
336
337                    all_results.push(bundle_results);
338                }
339
340                Ok(all_results)
341            })
342            .await
343        }
344    }
345
346    /// Creates [`AccessListResult`] for the [`TransactionRequest`] at the given
347    /// [`BlockId`], or latest block.
348    fn create_access_list_at(
349        &self,
350        request: TransactionRequest,
351        block_number: Option<BlockId>,
352        state_override: Option<StateOverride>,
353    ) -> impl Future<Output = Result<AccessListResult, Self::Error>> + Send
354    where
355        Self: Trace,
356    {
357        async move {
358            let block_id = block_number.unwrap_or_default();
359            let (evm_env, at) = self.evm_env_at(block_id).await?;
360
361            self.spawn_blocking_io(move |this| {
362                this.create_access_list_with(evm_env, at, request, state_override)
363            })
364            .await
365        }
366    }
367
368    /// Creates [`AccessListResult`] for the [`TransactionRequest`] at the given
369    /// [`BlockId`].
370    fn create_access_list_with(
371        &self,
372        mut evm_env: EvmEnvFor<Self::Evm>,
373        at: BlockId,
374        mut request: TransactionRequest,
375        state_override: Option<StateOverride>,
376    ) -> Result<AccessListResult, Self::Error>
377    where
378        Self: Trace,
379    {
380        let state = self.state_at_block_id(at)?;
381        let mut db = CacheDB::new(StateProviderDatabase::new(state));
382
383        if let Some(state_overrides) = state_override {
384            apply_state_overrides(state_overrides, &mut db)?;
385        }
386
387        let mut tx_env = self.create_txn_env(&evm_env, request.clone(), &mut db)?;
388
389        // we want to disable this in eth_createAccessList, since this is common practice used by
390        // other node impls and providers <https://github.com/foundry-rs/foundry/issues/4388>
391        evm_env.cfg_env.disable_block_gas_limit = true;
392
393        // The basefee should be ignored for eth_createAccessList
394        // See:
395        // <https://github.com/ethereum/go-ethereum/blob/8990c92aea01ca07801597b00c0d83d4e2d9b811/internal/ethapi/api.go#L1476-L1476>
396        evm_env.cfg_env.disable_base_fee = true;
397
398        // Disabled because eth_createAccessList is sometimes used with non-eoa senders
399        evm_env.cfg_env.disable_eip3607 = true;
400
401        if request.gas.is_none() && tx_env.gas_price() > 0 {
402            let cap = caller_gas_allowance(&mut db, &tx_env)?;
403            // no gas limit was provided in the request, so we need to cap the request's gas limit
404            tx_env.set_gas_limit(cap.min(evm_env.block_env.gas_limit));
405        }
406
407        // can consume the list since we're not using the request anymore
408        let initial = request.access_list.take().unwrap_or_default();
409
410        let mut inspector = AccessListInspector::new(initial);
411
412        let (result, (evm_env, mut tx_env)) =
413            self.inspect(&mut db, evm_env, tx_env, &mut inspector)?;
414        let access_list = inspector.into_access_list();
415        tx_env.set_access_list(access_list.clone());
416        match result.result {
417            ExecutionResult::Halt { reason, gas_used } => {
418                let error =
419                    Some(Self::Error::from_evm_halt(reason, tx_env.gas_limit()).to_string());
420                return Ok(AccessListResult { access_list, gas_used: U256::from(gas_used), error })
421            }
422            ExecutionResult::Revert { output, gas_used } => {
423                let error = Some(RevertError::new(output).to_string());
424                return Ok(AccessListResult { access_list, gas_used: U256::from(gas_used), error })
425            }
426            ExecutionResult::Success { .. } => {}
427        };
428
429        // transact again to get the exact gas used
430        let (result, (_, tx_env)) = self.transact(&mut db, evm_env, tx_env)?;
431        let res = match result.result {
432            ExecutionResult::Halt { reason, gas_used } => {
433                let error =
434                    Some(Self::Error::from_evm_halt(reason, tx_env.gas_limit()).to_string());
435                AccessListResult { access_list, gas_used: U256::from(gas_used), error }
436            }
437            ExecutionResult::Revert { output, gas_used } => {
438                let error = Some(RevertError::new(output).to_string());
439                AccessListResult { access_list, gas_used: U256::from(gas_used), error }
440            }
441            ExecutionResult::Success { gas_used, .. } => {
442                AccessListResult { access_list, gas_used: U256::from(gas_used), error: None }
443            }
444        };
445
446        Ok(res)
447    }
448}
449
450/// Executes code on state.
451pub trait Call:
452    LoadState<
453        Evm: ConfigureEvm<
454            Primitives: NodePrimitives<
455                BlockHeader = ProviderHeader<Self::Provider>,
456                SignedTx = ProviderTx<Self::Provider>,
457            >,
458        >,
459        Error: FromEvmError<Self::Evm>,
460    > + SpawnBlocking
461{
462    /// Returns default gas limit to use for `eth_call` and tracing RPC methods.
463    ///
464    /// Data access in default trait method implementations.
465    fn call_gas_limit(&self) -> u64;
466
467    /// Returns the maximum number of blocks accepted for `eth_simulateV1`.
468    fn max_simulate_blocks(&self) -> u64;
469
470    /// Executes the closure with the state that corresponds to the given [`BlockId`].
471    fn with_state_at_block<F, R>(&self, at: BlockId, f: F) -> Result<R, Self::Error>
472    where
473        F: FnOnce(StateProviderTraitObjWrapper<'_>) -> Result<R, Self::Error>,
474    {
475        let state = self.state_at_block_id(at)?;
476        f(StateProviderTraitObjWrapper(&state))
477    }
478
479    /// Executes the `TxEnv` against the given [Database] without committing state
480    /// changes.
481    #[expect(clippy::type_complexity)]
482    fn transact<DB>(
483        &self,
484        db: DB,
485        evm_env: EvmEnvFor<Self::Evm>,
486        tx_env: TxEnvFor<Self::Evm>,
487    ) -> Result<
488        (ResultAndState<HaltReasonFor<Self::Evm>>, (EvmEnvFor<Self::Evm>, TxEnvFor<Self::Evm>)),
489        Self::Error,
490    >
491    where
492        DB: Database<Error = ProviderError>,
493    {
494        let mut evm = self.evm_config().evm_with_env(db, evm_env.clone());
495        let res = evm.transact(tx_env.clone()).map_err(Self::Error::from_evm_err)?;
496
497        Ok((res, (evm_env, tx_env)))
498    }
499
500    /// Executes the [`EvmEnv`] against the given [Database] without committing state
501    /// changes.
502    #[expect(clippy::type_complexity)]
503    fn transact_with_inspector<DB, I>(
504        &self,
505        db: DB,
506        evm_env: EvmEnvFor<Self::Evm>,
507        tx_env: TxEnvFor<Self::Evm>,
508        inspector: I,
509    ) -> Result<
510        (ResultAndState<HaltReasonFor<Self::Evm>>, (EvmEnvFor<Self::Evm>, TxEnvFor<Self::Evm>)),
511        Self::Error,
512    >
513    where
514        DB: Database<Error = ProviderError>,
515        I: InspectorFor<Self::Evm, DB>,
516    {
517        let mut evm = self.evm_config().evm_with_env_and_inspector(db, evm_env.clone(), inspector);
518        let res = evm.transact(tx_env.clone()).map_err(Self::Error::from_evm_err)?;
519
520        Ok((res, (evm_env, tx_env)))
521    }
522
523    /// Executes the call request at the given [`BlockId`].
524    #[expect(clippy::type_complexity)]
525    fn transact_call_at(
526        &self,
527        request: TransactionRequest,
528        at: BlockId,
529        overrides: EvmOverrides,
530    ) -> impl Future<
531        Output = Result<
532            (ResultAndState<HaltReasonFor<Self::Evm>>, (EvmEnvFor<Self::Evm>, TxEnvFor<Self::Evm>)),
533            Self::Error,
534        >,
535    > + Send
536    where
537        Self: LoadPendingBlock,
538    {
539        let this = self.clone();
540        self.spawn_with_call_at(request, at, overrides, move |db, evm_env, tx_env| {
541            this.transact(db, evm_env, tx_env)
542        })
543    }
544
545    /// Executes the closure with the state that corresponds to the given [`BlockId`] on a new task
546    fn spawn_with_state_at_block<F, R>(
547        &self,
548        at: BlockId,
549        f: F,
550    ) -> impl Future<Output = Result<R, Self::Error>> + Send
551    where
552        F: FnOnce(StateProviderTraitObjWrapper<'_>) -> Result<R, Self::Error> + Send + 'static,
553        R: Send + 'static,
554    {
555        self.spawn_tracing(move |this| {
556            let state = this.state_at_block_id(at)?;
557            f(StateProviderTraitObjWrapper(&state))
558        })
559    }
560
561    /// Prepares the state and env for the given [`TransactionRequest`] at the given [`BlockId`] and
562    /// executes the closure on a new task returning the result of the closure.
563    ///
564    /// This returns the configured [`EvmEnv`] for the given [`TransactionRequest`] at
565    /// the given [`BlockId`] and with configured call settings: `prepare_call_env`.
566    ///
567    /// This is primarily used by `eth_call`.
568    ///
569    /// # Blocking behaviour
570    ///
571    /// This assumes executing the call is relatively more expensive on IO than CPU because it
572    /// transacts a single transaction on an empty in memory database. Because `eth_call`s are
573    /// usually allowed to consume a lot of gas, this also allows a lot of memory operations so
574    /// we assume this is not primarily CPU bound and instead spawn the call on a regular tokio task
575    /// instead, where blocking IO is less problematic.
576    fn spawn_with_call_at<F, R>(
577        &self,
578        request: TransactionRequest,
579        at: BlockId,
580        overrides: EvmOverrides,
581        f: F,
582    ) -> impl Future<Output = Result<R, Self::Error>> + Send
583    where
584        Self: LoadPendingBlock,
585        F: FnOnce(
586                StateCacheDbRefMutWrapper<'_, '_>,
587                EvmEnvFor<Self::Evm>,
588                TxEnvFor<Self::Evm>,
589            ) -> Result<R, Self::Error>
590            + Send
591            + 'static,
592        R: Send + 'static,
593    {
594        async move {
595            let (evm_env, at) = self.evm_env_at(at).await?;
596            let this = self.clone();
597            self.spawn_blocking_io(move |_| {
598                let state = this.state_at_block_id(at)?;
599                let mut db =
600                    CacheDB::new(StateProviderDatabase::new(StateProviderTraitObjWrapper(&state)));
601
602                let (evm_env, tx_env) =
603                    this.prepare_call_env(evm_env, request, &mut db, overrides)?;
604
605                f(StateCacheDbRefMutWrapper(&mut db), evm_env, tx_env)
606            })
607            .await
608        }
609    }
610
611    /// Retrieves the transaction if it exists and executes it.
612    ///
613    /// Before the transaction is executed, all previous transaction in the block are applied to the
614    /// state by executing them first.
615    /// The callback `f` is invoked with the [`ResultAndState`] after the transaction was executed
616    /// and the database that points to the beginning of the transaction.
617    ///
618    /// Note: Implementers should use a threadpool where blocking is allowed, such as
619    /// [`BlockingTaskPool`](reth_tasks::pool::BlockingTaskPool).
620    fn spawn_replay_transaction<F, R>(
621        &self,
622        hash: B256,
623        f: F,
624    ) -> impl Future<Output = Result<Option<R>, Self::Error>> + Send
625    where
626        Self: LoadBlock + LoadTransaction,
627        F: FnOnce(
628                TransactionInfo,
629                ResultAndState<HaltReasonFor<Self::Evm>>,
630                StateCacheDb<'_>,
631            ) -> Result<R, Self::Error>
632            + Send
633            + 'static,
634        R: Send + 'static,
635    {
636        async move {
637            let (transaction, block) = match self.transaction_and_block(hash).await? {
638                None => return Ok(None),
639                Some(res) => res,
640            };
641            let (tx, tx_info) = transaction.split();
642
643            let (evm_env, _) = self.evm_env_at(block.hash().into()).await?;
644
645            // we need to get the state of the parent block because we're essentially replaying the
646            // block the transaction is included in
647            let parent_block = block.parent_hash();
648
649            let this = self.clone();
650            self.spawn_with_state_at_block(parent_block.into(), move |state| {
651                let mut db = CacheDB::new(StateProviderDatabase::new(state));
652                let block_txs = block.transactions_recovered();
653
654                // replay all transactions prior to the targeted transaction
655                this.replay_transactions_until(&mut db, evm_env.clone(), block_txs, *tx.tx_hash())?;
656
657                let tx_env = RpcNodeCore::evm_config(&this).tx_env(tx);
658
659                let (res, _) = this.transact(&mut db, evm_env, tx_env)?;
660                f(tx_info, res, db)
661            })
662            .await
663            .map(Some)
664        }
665    }
666
667    /// Replays all the transactions until the target transaction is found.
668    ///
669    /// All transactions before the target transaction are executed and their changes are written to
670    /// the _runtime_ db ([`CacheDB`]).
671    ///
672    /// Note: This assumes the target transaction is in the given iterator.
673    /// Returns the index of the target transaction in the given iterator.
674    fn replay_transactions_until<'a, DB, I>(
675        &self,
676        db: &mut DB,
677        evm_env: EvmEnvFor<Self::Evm>,
678        transactions: I,
679        target_tx_hash: B256,
680    ) -> Result<usize, Self::Error>
681    where
682        DB: Database<Error = ProviderError> + DatabaseCommit,
683        I: IntoIterator<Item = Recovered<&'a ProviderTx<Self::Provider>>>,
684    {
685        let mut evm = self.evm_config().evm_with_env(db, evm_env);
686        let mut index = 0;
687        for tx in transactions {
688            if *tx.tx_hash() == target_tx_hash {
689                // reached the target transaction
690                break
691            }
692
693            let tx_env = self.evm_config().tx_env(tx);
694            evm.transact_commit(tx_env).map_err(Self::Error::from_evm_err)?;
695            index += 1;
696        }
697        Ok(index)
698    }
699
700    /// Configures a new `TxEnv`  for the [`TransactionRequest`]
701    ///
702    /// All `TxEnv` fields are derived from the given [`TransactionRequest`], if fields are
703    /// `None`, they fall back to the [`EvmEnv`]'s settings.
704    fn create_txn_env(
705        &self,
706        evm_env: &EvmEnv<SpecFor<Self::Evm>>,
707        request: TransactionRequest,
708        db: impl Database<Error: Into<EthApiError>>,
709    ) -> Result<TxEnvFor<Self::Evm>, Self::Error>;
710
711    /// Prepares the [`EvmEnv`] for execution of calls.
712    ///
713    /// Does not commit any changes to the underlying database.
714    ///
715    /// ## EVM settings
716    ///
717    /// This modifies certain EVM settings to mirror geth's `SkipAccountChecks` when transacting requests, see also: <https://github.com/ethereum/go-ethereum/blob/380688c636a654becc8f114438c2a5d93d2db032/core/state_transition.go#L145-L148>:
718    ///
719    ///  - `disable_eip3607` is set to `true`
720    ///  - `disable_base_fee` is set to `true`
721    ///  - `nonce` is set to `None`
722    ///
723    /// In addition, this changes the block's gas limit to the configured [`Self::call_gas_limit`].
724    #[expect(clippy::type_complexity)]
725    fn prepare_call_env<DB>(
726        &self,
727        mut evm_env: EvmEnvFor<Self::Evm>,
728        mut request: TransactionRequest,
729        db: &mut CacheDB<DB>,
730        overrides: EvmOverrides,
731    ) -> Result<(EvmEnvFor<Self::Evm>, TxEnvFor<Self::Evm>), Self::Error>
732    where
733        DB: DatabaseRef,
734        EthApiError: From<<DB as DatabaseRef>::Error>,
735    {
736        if request.gas > Some(self.call_gas_limit()) {
737            // configured gas exceeds limit
738            return Err(
739                EthApiError::InvalidTransaction(RpcInvalidTransactionError::GasTooHigh).into()
740            )
741        }
742
743        // apply configured gas cap
744        evm_env.block_env.gas_limit = self.call_gas_limit();
745
746        // Disabled because eth_call is sometimes used with eoa senders
747        // See <https://github.com/paradigmxyz/reth/issues/1959>
748        evm_env.cfg_env.disable_eip3607 = true;
749
750        // The basefee should be ignored for eth_call
751        // See:
752        // <https://github.com/ethereum/go-ethereum/blob/ee8e83fa5f6cb261dad2ed0a7bbcde4930c41e6c/internal/ethapi/api.go#L985>
753        evm_env.cfg_env.disable_base_fee = true;
754
755        // set nonce to None so that the correct nonce is chosen by the EVM
756        request.nonce = None;
757
758        if let Some(block_overrides) = overrides.block {
759            apply_block_overrides(*block_overrides, db, &mut evm_env.block_env);
760        }
761        if let Some(state_overrides) = overrides.state {
762            apply_state_overrides(state_overrides, db)?;
763        }
764
765        let request_gas = request.gas;
766        let request_from = request.from;
767        let mut tx_env = self.create_txn_env(&evm_env, request, &mut *db)?;
768
769        if request_from.is_none() {
770            // No from address was provided in the request, so we use the default gas limit
771        } else if request_gas.is_none() {
772            // No gas limit was provided in the request, so we need to cap the transaction gas limit
773            if tx_env.gas_price() > 0 {
774                // If gas price is specified, cap transaction gas limit with caller allowance
775                trace!(target: "rpc::eth::call", ?tx_env, "Applying gas limit cap with caller allowance");
776                let cap = caller_gas_allowance(db, &tx_env)?;
777                // ensure we cap gas_limit to the block's
778                tx_env.set_gas_limit(cap.min(evm_env.block_env.gas_limit));
779            }
780        }
781
782        Ok((evm_env, tx_env))
783    }
784}