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 core::fmt;
5
6use super::{LoadBlock, LoadPendingBlock, LoadState, LoadTransaction, SpawnBlocking, Trace};
7use crate::{
8    helpers::estimate::EstimateCall, FromEvmError, FullEthApiTypes, RpcBlock, RpcNodeCore,
9};
10use alloy_consensus::{transaction::TxHashRef, BlockHeader};
11use alloy_eips::eip2930::AccessListResult;
12use alloy_evm::overrides::{apply_block_overrides, apply_state_overrides, OverrideBlockHashes};
13use alloy_network::TransactionBuilder;
14use alloy_primitives::{Bytes, B256, U256};
15use alloy_rpc_types_eth::{
16    simulate::{SimBlock, SimulatePayload, SimulatedBlock},
17    state::{EvmOverrides, StateOverride},
18    BlockId, Bundle, EthCallResponse, StateContext, TransactionInfo,
19};
20use futures::Future;
21use reth_errors::{ProviderError, RethError};
22use reth_evm::{
23    ConfigureEvm, Evm, EvmEnv, EvmEnvFor, HaltReasonFor, InspectorFor, SpecFor, TransactionEnv,
24    TxEnvFor,
25};
26use reth_node_api::BlockBody;
27use reth_primitives_traits::Recovered;
28use reth_revm::{
29    database::StateProviderDatabase,
30    db::{CacheDB, State},
31};
32use reth_rpc_convert::{RpcConvert, RpcTxReq};
33use reth_rpc_eth_types::{
34    cache::db::{StateCacheDbRefMutWrapper, StateProviderTraitObjWrapper},
35    error::{api::FromEvmHalt, ensure_success, FromEthApiError},
36    simulate::{self, EthSimulateError},
37    EthApiError, RevertError, StateCacheDb,
38};
39use reth_storage_api::{BlockIdReader, ProviderTx};
40use revm::{
41    context_interface::{
42        result::{ExecutionResult, ResultAndState},
43        Transaction,
44    },
45    Database, DatabaseCommit,
46};
47use revm_inspectors::{access_list::AccessListInspector, transfer::TransferInspector};
48use tracing::{trace, warn};
49
50/// Result type for `eth_simulateV1` RPC method.
51pub type SimulatedBlocksResult<N, E> = Result<Vec<SimulatedBlock<RpcBlock<N>>>, E>;
52
53/// Execution related functions for the [`EthApiServer`](crate::EthApiServer) trait in
54/// the `eth_` namespace.
55pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthApiTypes {
56    /// Estimate gas needed for execution of the `request` at the [`BlockId`].
57    fn estimate_gas_at(
58        &self,
59        request: RpcTxReq<<Self::RpcConvert as RpcConvert>::Network>,
60        at: BlockId,
61        state_override: Option<StateOverride>,
62    ) -> impl Future<Output = Result<U256, Self::Error>> + Send {
63        EstimateCall::estimate_gas_at(self, request, at, state_override)
64    }
65
66    /// `eth_simulateV1` executes an arbitrary number of transactions on top of the requested state.
67    /// The transactions are packed into individual blocks. Overrides can be provided.
68    ///
69    /// See also: <https://github.com/ethereum/go-ethereum/pull/27720>
70    fn simulate_v1(
71        &self,
72        payload: SimulatePayload<RpcTxReq<<Self::RpcConvert as RpcConvert>::Network>>,
73        block: Option<BlockId>,
74    ) -> impl Future<Output = SimulatedBlocksResult<Self::NetworkTypes, Self::Error>> + Send {
75        async move {
76            if payload.block_state_calls.len() > self.max_simulate_blocks() as usize {
77                return Err(EthApiError::InvalidParams("too many blocks.".to_string()).into())
78            }
79
80            let block = block.unwrap_or_default();
81
82            let SimulatePayload {
83                block_state_calls,
84                trace_transfers,
85                validation,
86                return_full_transactions,
87            } = payload;
88
89            if block_state_calls.is_empty() {
90                return Err(EthApiError::InvalidParams(String::from("calls are empty.")).into())
91            }
92
93            let base_block =
94                self.recovered_block(block).await?.ok_or(EthApiError::HeaderNotFound(block))?;
95            let mut parent = base_block.sealed_header().clone();
96
97            let this = self.clone();
98            self.spawn_with_state_at_block(block, move |state| {
99                let mut db =
100                    State::builder().with_database(StateProviderDatabase::new(state)).build();
101                let mut blocks: Vec<SimulatedBlock<RpcBlock<Self::NetworkTypes>>> =
102                    Vec::with_capacity(block_state_calls.len());
103                for block in block_state_calls {
104                    let mut evm_env = this
105                        .evm_config()
106                        .next_evm_env(&parent, &this.next_env_attributes(&parent)?)
107                        .map_err(RethError::other)
108                        .map_err(Self::Error::from_eth_err)?;
109
110                    // Always disable EIP-3607
111                    evm_env.cfg_env.disable_eip3607 = true;
112
113                    if !validation {
114                        // If not explicitly required, we disable nonce check <https://github.com/paradigmxyz/reth/issues/16108>
115                        evm_env.cfg_env.disable_nonce_check = true;
116                        evm_env.cfg_env.disable_base_fee = true;
117                        evm_env.cfg_env.tx_gas_limit_cap = Some(u64::MAX);
118                        evm_env.block_env.basefee = 0;
119                    }
120
121                    let SimBlock { block_overrides, state_overrides, calls } = block;
122
123                    if let Some(block_overrides) = block_overrides {
124                        // ensure we don't allow uncapped gas limit per block
125                        if let Some(gas_limit_override) = block_overrides.gas_limit &&
126                            gas_limit_override > evm_env.block_env.gas_limit &&
127                            gas_limit_override > this.call_gas_limit()
128                        {
129                            return Err(EthApiError::other(EthSimulateError::GasLimitReached).into())
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                            .map_err(Self::Error::from_eth_err)?;
136                    }
137
138                    let block_gas_limit = evm_env.block_env.gas_limit;
139                    let chain_id = evm_env.cfg_env.chain_id;
140
141                    let default_gas_limit = {
142                        let total_specified_gas =
143                            calls.iter().filter_map(|tx| tx.as_ref().gas_limit()).sum::<u64>();
144                        let txs_without_gas_limit =
145                            calls.iter().filter(|tx| tx.as_ref().gas_limit().is_none()).count();
146
147                        if total_specified_gas > block_gas_limit {
148                            return Err(EthApiError::Other(Box::new(
149                                EthSimulateError::BlockGasLimitExceeded,
150                            ))
151                            .into())
152                        }
153
154                        if txs_without_gas_limit > 0 {
155                            (block_gas_limit - total_specified_gas) / txs_without_gas_limit as u64
156                        } else {
157                            0
158                        }
159                    };
160
161                    let ctx = this
162                        .evm_config()
163                        .context_for_next_block(&parent, this.next_env_attributes(&parent)?)
164                        .map_err(RethError::other)
165                        .map_err(Self::Error::from_eth_err)?;
166                    let (result, results) = if trace_transfers {
167                        // prepare inspector to capture transfer inside the evm so they are recorded
168                        // and included in logs
169                        let inspector = TransferInspector::new(false).with_logs(true);
170                        let evm = this
171                            .evm_config()
172                            .evm_with_env_and_inspector(&mut db, evm_env, inspector);
173                        let builder = this.evm_config().create_block_builder(evm, &parent, ctx);
174                        simulate::execute_transactions(
175                            builder,
176                            calls,
177                            default_gas_limit,
178                            chain_id,
179                            this.tx_resp_builder(),
180                        )?
181                    } else {
182                        let evm = this.evm_config().evm_with_env(&mut db, evm_env);
183                        let builder = this.evm_config().create_block_builder(evm, &parent, ctx);
184                        simulate::execute_transactions(
185                            builder,
186                            calls,
187                            default_gas_limit,
188                            chain_id,
189                            this.tx_resp_builder(),
190                        )?
191                    };
192
193                    parent = result.block.clone_sealed_header();
194
195                    let block = simulate::build_simulated_block(
196                        result.block,
197                        results,
198                        return_full_transactions.into(),
199                        this.tx_resp_builder(),
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: RpcTxReq<<Self::RpcConvert as RpcConvert>::Network>,
215        block_number: Option<BlockId>,
216        overrides: EvmOverrides,
217    ) -> impl Future<Output = Result<Bytes, Self::Error>> + Send {
218        async move {
219            let res =
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<RpcTxReq<<Self::RpcConvert as RpcConvert>::Network>>>,
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 [`RpcTxReq`] at the given
347    /// [`BlockId`], or latest block.
348    fn create_access_list_at(
349        &self,
350        request: RpcTxReq<<Self::RpcConvert as RpcConvert>::Network>,
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_fut(move |this| async move {
362                this.create_access_list_with(evm_env, at, request, state_override).await
363            })
364            .await
365        }
366    }
367
368    /// Creates [`AccessListResult`] for the [`RpcTxReq`] at the given
369    /// [`BlockId`].
370    fn create_access_list_with(
371        &self,
372        mut evm_env: EvmEnvFor<Self::Evm>,
373        at: BlockId,
374        request: RpcTxReq<<Self::RpcConvert as RpcConvert>::Network>,
375        state_override: Option<StateOverride>,
376    ) -> impl Future<Output = Result<AccessListResult, Self::Error>> + Send
377    where
378        Self: Trace,
379    {
380        self.spawn_blocking_io_fut(move |this| async move {
381            let state = this.state_at_block_id(at).await?;
382            let mut db = CacheDB::new(StateProviderDatabase::new(state));
383
384            if let Some(state_overrides) = state_override {
385                apply_state_overrides(state_overrides, &mut db)
386                    .map_err(Self::Error::from_eth_err)?;
387            }
388
389            let mut tx_env = this.create_txn_env(&evm_env, request.clone(), &mut db)?;
390
391            // we want to disable this in eth_createAccessList, since this is common practice used
392            // by other node impls and providers <https://github.com/foundry-rs/foundry/issues/4388>
393            evm_env.cfg_env.disable_block_gas_limit = true;
394
395            // The basefee should be ignored for eth_createAccessList
396            // See:
397            // <https://github.com/ethereum/go-ethereum/blob/8990c92aea01ca07801597b00c0d83d4e2d9b811/internal/ethapi/api.go#L1476-L1476>
398            evm_env.cfg_env.disable_base_fee = true;
399
400            // Disabled because eth_createAccessList is sometimes used with non-eoa senders
401            evm_env.cfg_env.disable_eip3607 = true;
402
403            if request.as_ref().gas_limit().is_none() && tx_env.gas_price() > 0 {
404                let cap = this.caller_gas_allowance(&mut db, &evm_env, &tx_env)?;
405                // no gas limit was provided in the request, so we need to cap the request's gas
406                // limit
407                tx_env.set_gas_limit(cap.min(evm_env.block_env.gas_limit));
408            }
409
410            // can consume the list since we're not using the request anymore
411            let initial = request.as_ref().access_list().cloned().unwrap_or_default();
412
413            let mut inspector = AccessListInspector::new(initial);
414
415            let result = this.inspect(&mut db, evm_env.clone(), tx_env.clone(), &mut inspector)?;
416            let access_list = inspector.into_access_list();
417            tx_env.set_access_list(access_list.clone());
418            match result.result {
419                ExecutionResult::Halt { reason, gas_used } => {
420                    let error =
421                        Some(Self::Error::from_evm_halt(reason, tx_env.gas_limit()).to_string());
422                    return Ok(AccessListResult {
423                        access_list,
424                        gas_used: U256::from(gas_used),
425                        error,
426                    })
427                }
428                ExecutionResult::Revert { output, gas_used } => {
429                    let error = Some(RevertError::new(output).to_string());
430                    return Ok(AccessListResult {
431                        access_list,
432                        gas_used: U256::from(gas_used),
433                        error,
434                    })
435                }
436                ExecutionResult::Success { .. } => {}
437            };
438
439            // transact again to get the exact gas used
440            let gas_limit = tx_env.gas_limit();
441            let result = this.transact(&mut db, evm_env, tx_env)?;
442            let res = match result.result {
443                ExecutionResult::Halt { reason, gas_used } => {
444                    let error = Some(Self::Error::from_evm_halt(reason, gas_limit).to_string());
445                    AccessListResult { access_list, gas_used: U256::from(gas_used), error }
446                }
447                ExecutionResult::Revert { output, gas_used } => {
448                    let error = Some(RevertError::new(output).to_string());
449                    AccessListResult { access_list, gas_used: U256::from(gas_used), error }
450                }
451                ExecutionResult::Success { gas_used, .. } => {
452                    AccessListResult { access_list, gas_used: U256::from(gas_used), error: None }
453                }
454            };
455
456            Ok(res)
457        })
458    }
459}
460
461/// Executes code on state.
462pub trait Call:
463    LoadState<
464        RpcConvert: RpcConvert<TxEnv = TxEnvFor<Self::Evm>, Spec = SpecFor<Self::Evm>>,
465        Error: FromEvmError<Self::Evm>
466                   + From<<Self::RpcConvert as RpcConvert>::Error>
467                   + From<ProviderError>,
468    > + SpawnBlocking
469{
470    /// Returns default gas limit to use for `eth_call` and tracing RPC methods.
471    ///
472    /// Data access in default trait method implementations.
473    fn call_gas_limit(&self) -> u64;
474
475    /// Returns the maximum number of blocks accepted for `eth_simulateV1`.
476    fn max_simulate_blocks(&self) -> u64;
477
478    /// Returns the max gas limit that the caller can afford given a transaction environment.
479    fn caller_gas_allowance(
480        &self,
481        mut db: impl Database<Error: Into<EthApiError>>,
482        _evm_env: &EvmEnvFor<Self::Evm>,
483        tx_env: &TxEnvFor<Self::Evm>,
484    ) -> Result<u64, Self::Error> {
485        alloy_evm::call::caller_gas_allowance(&mut db, tx_env).map_err(Self::Error::from_eth_err)
486    }
487
488    /// Executes the closure with the state that corresponds to the given [`BlockId`].
489    fn with_state_at_block<F, R>(
490        &self,
491        at: BlockId,
492        f: F,
493    ) -> impl Future<Output = Result<R, Self::Error>> + Send
494    where
495        R: Send + 'static,
496        F: FnOnce(Self, StateProviderTraitObjWrapper<'_>) -> Result<R, Self::Error>
497            + Send
498            + 'static,
499    {
500        self.spawn_blocking_io_fut(move |this| async move {
501            let state = this.state_at_block_id(at).await?;
502            f(this, StateProviderTraitObjWrapper(&state))
503        })
504    }
505
506    /// Executes the `TxEnv` against the given [Database] without committing state
507    /// changes.
508    fn transact<DB>(
509        &self,
510        db: DB,
511        evm_env: EvmEnvFor<Self::Evm>,
512        tx_env: TxEnvFor<Self::Evm>,
513    ) -> Result<ResultAndState<HaltReasonFor<Self::Evm>>, Self::Error>
514    where
515        DB: Database<Error = ProviderError> + fmt::Debug,
516    {
517        let mut evm = self.evm_config().evm_with_env(db, evm_env);
518        let res = evm.transact(tx_env).map_err(Self::Error::from_evm_err)?;
519
520        Ok(res)
521    }
522
523    /// Executes the [`EvmEnv`] against the given [Database] without committing state
524    /// changes.
525    fn transact_with_inspector<DB, I>(
526        &self,
527        db: DB,
528        evm_env: EvmEnvFor<Self::Evm>,
529        tx_env: TxEnvFor<Self::Evm>,
530        inspector: I,
531    ) -> Result<ResultAndState<HaltReasonFor<Self::Evm>>, Self::Error>
532    where
533        DB: Database<Error = ProviderError> + fmt::Debug,
534        I: InspectorFor<Self::Evm, DB>,
535    {
536        let mut evm = self.evm_config().evm_with_env_and_inspector(db, evm_env, inspector);
537        let res = evm.transact(tx_env).map_err(Self::Error::from_evm_err)?;
538
539        Ok(res)
540    }
541
542    /// Executes the call request at the given [`BlockId`].
543    fn transact_call_at(
544        &self,
545        request: RpcTxReq<<Self::RpcConvert as RpcConvert>::Network>,
546        at: BlockId,
547        overrides: EvmOverrides,
548    ) -> impl Future<Output = Result<ResultAndState<HaltReasonFor<Self::Evm>>, Self::Error>> + Send
549    where
550        Self: LoadPendingBlock,
551    {
552        let this = self.clone();
553        self.spawn_with_call_at(request, at, overrides, move |db, evm_env, tx_env| {
554            this.transact(db, evm_env, tx_env)
555        })
556    }
557
558    /// Executes the closure with the state that corresponds to the given [`BlockId`] on a new task
559    fn spawn_with_state_at_block<F, R>(
560        &self,
561        at: BlockId,
562        f: F,
563    ) -> impl Future<Output = Result<R, Self::Error>> + Send
564    where
565        F: FnOnce(StateProviderTraitObjWrapper<'_>) -> Result<R, Self::Error> + Send + 'static,
566        R: Send + 'static,
567    {
568        self.spawn_blocking_io_fut(move |this| async move {
569            let state = this.state_at_block_id(at).await?;
570            f(StateProviderTraitObjWrapper(&state))
571        })
572    }
573
574    /// Prepares the state and env for the given [`RpcTxReq`] at the given [`BlockId`] and
575    /// executes the closure on a new task returning the result of the closure.
576    ///
577    /// This returns the configured [`EvmEnv`] for the given [`RpcTxReq`] at
578    /// the given [`BlockId`] and with configured call settings: `prepare_call_env`.
579    ///
580    /// This is primarily used by `eth_call`.
581    ///
582    /// # Blocking behaviour
583    ///
584    /// This assumes executing the call is relatively more expensive on IO than CPU because it
585    /// transacts a single transaction on an empty in memory database. Because `eth_call`s are
586    /// usually allowed to consume a lot of gas, this also allows a lot of memory operations so
587    /// we assume this is not primarily CPU bound and instead spawn the call on a regular tokio task
588    /// instead, where blocking IO is less problematic.
589    fn spawn_with_call_at<F, R>(
590        &self,
591        request: RpcTxReq<<Self::RpcConvert as RpcConvert>::Network>,
592        at: BlockId,
593        overrides: EvmOverrides,
594        f: F,
595    ) -> impl Future<Output = Result<R, Self::Error>> + Send
596    where
597        Self: LoadPendingBlock,
598        F: FnOnce(
599                StateCacheDbRefMutWrapper<'_, '_>,
600                EvmEnvFor<Self::Evm>,
601                TxEnvFor<Self::Evm>,
602            ) -> Result<R, Self::Error>
603            + Send
604            + 'static,
605        R: Send + 'static,
606    {
607        async move {
608            let (evm_env, at) = self.evm_env_at(at).await?;
609            let this = self.clone();
610            self.spawn_blocking_io_fut(move |_| async move {
611                let state = this.state_at_block_id(at).await?;
612                let mut db =
613                    CacheDB::new(StateProviderDatabase::new(StateProviderTraitObjWrapper(&state)));
614
615                let (evm_env, tx_env) =
616                    this.prepare_call_env(evm_env, request, &mut db, overrides)?;
617
618                f(StateCacheDbRefMutWrapper(&mut db), evm_env, tx_env)
619            })
620            .await
621        }
622    }
623
624    /// Retrieves the transaction if it exists and executes it.
625    ///
626    /// Before the transaction is executed, all previous transaction in the block are applied to the
627    /// state by executing them first.
628    /// The callback `f` is invoked with the [`ResultAndState`] after the transaction was executed
629    /// and the database that points to the beginning of the transaction.
630    ///
631    /// Note: Implementers should use a threadpool where blocking is allowed, such as
632    /// [`BlockingTaskPool`](reth_tasks::pool::BlockingTaskPool).
633    fn spawn_replay_transaction<F, R>(
634        &self,
635        hash: B256,
636        f: F,
637    ) -> impl Future<Output = Result<Option<R>, Self::Error>> + Send
638    where
639        Self: LoadBlock + LoadTransaction,
640        F: FnOnce(
641                TransactionInfo,
642                ResultAndState<HaltReasonFor<Self::Evm>>,
643                StateCacheDb<'_>,
644            ) -> Result<R, Self::Error>
645            + Send
646            + 'static,
647        R: Send + 'static,
648    {
649        async move {
650            let (transaction, block) = match self.transaction_and_block(hash).await? {
651                None => return Ok(None),
652                Some(res) => res,
653            };
654            let (tx, tx_info) = transaction.split();
655
656            let (evm_env, _) = self.evm_env_at(block.hash().into()).await?;
657
658            // we need to get the state of the parent block because we're essentially replaying the
659            // block the transaction is included in
660            let parent_block = block.parent_hash();
661
662            let this = self.clone();
663            self.spawn_with_state_at_block(parent_block.into(), move |state| {
664                let mut db = CacheDB::new(StateProviderDatabase::new(state));
665                let block_txs = block.transactions_recovered();
666
667                // replay all transactions prior to the targeted transaction
668                this.replay_transactions_until(&mut db, evm_env.clone(), block_txs, *tx.tx_hash())?;
669
670                let tx_env = RpcNodeCore::evm_config(&this).tx_env(tx);
671
672                let res = this.transact(&mut db, evm_env, tx_env)?;
673                f(tx_info, res, db)
674            })
675            .await
676            .map(Some)
677        }
678    }
679
680    /// Replays all the transactions until the target transaction is found.
681    ///
682    /// All transactions before the target transaction are executed and their changes are written to
683    /// the _runtime_ db ([`CacheDB`]).
684    ///
685    /// Note: This assumes the target transaction is in the given iterator.
686    /// Returns the index of the target transaction in the given iterator.
687    fn replay_transactions_until<'a, DB, I>(
688        &self,
689        db: &mut DB,
690        evm_env: EvmEnvFor<Self::Evm>,
691        transactions: I,
692        target_tx_hash: B256,
693    ) -> Result<usize, Self::Error>
694    where
695        DB: Database<Error = ProviderError> + DatabaseCommit + core::fmt::Debug,
696        I: IntoIterator<Item = Recovered<&'a ProviderTx<Self::Provider>>>,
697    {
698        let mut evm = self.evm_config().evm_with_env(db, evm_env);
699        let mut index = 0;
700        for tx in transactions {
701            if *tx.tx_hash() == target_tx_hash {
702                // reached the target transaction
703                break
704            }
705
706            let tx_env = self.evm_config().tx_env(tx);
707            evm.transact_commit(tx_env).map_err(Self::Error::from_evm_err)?;
708            index += 1;
709        }
710        Ok(index)
711    }
712
713    ///
714    /// All `TxEnv` fields are derived from the given [`RpcTxReq`], if fields are
715    /// `None`, they fall back to the [`EvmEnv`]'s settings.
716    fn create_txn_env(
717        &self,
718        evm_env: &EvmEnv<SpecFor<Self::Evm>>,
719        mut request: RpcTxReq<<Self::RpcConvert as RpcConvert>::Network>,
720        mut db: impl Database<Error: Into<EthApiError>>,
721    ) -> Result<TxEnvFor<Self::Evm>, Self::Error> {
722        if request.as_ref().nonce().is_none() {
723            let nonce = db
724                .basic(request.as_ref().from().unwrap_or_default())
725                .map_err(Into::into)?
726                .map(|acc| acc.nonce)
727                .unwrap_or_default();
728            request.as_mut().set_nonce(nonce);
729        }
730
731        Ok(self.tx_resp_builder().tx_env(request, &evm_env.cfg_env, &evm_env.block_env)?)
732    }
733
734    /// Prepares the [`EvmEnv`] for execution of calls.
735    ///
736    /// Does not commit any changes to the underlying database.
737    ///
738    /// ## EVM settings
739    ///
740    /// 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>:
741    ///
742    ///  - `disable_eip3607` is set to `true`
743    ///  - `disable_base_fee` is set to `true`
744    ///  - `nonce` is set to `None`
745    ///
746    /// In addition, this changes the block's gas limit to the configured [`Self::call_gas_limit`].
747    #[expect(clippy::type_complexity)]
748    fn prepare_call_env<DB>(
749        &self,
750        mut evm_env: EvmEnvFor<Self::Evm>,
751        mut request: RpcTxReq<<Self::RpcConvert as RpcConvert>::Network>,
752        db: &mut DB,
753        overrides: EvmOverrides,
754    ) -> Result<(EvmEnvFor<Self::Evm>, TxEnvFor<Self::Evm>), Self::Error>
755    where
756        DB: Database + DatabaseCommit + OverrideBlockHashes,
757        EthApiError: From<<DB as Database>::Error>,
758    {
759        // track whether the request has a gas limit set
760        let request_has_gas_limit = request.as_ref().gas_limit().is_some();
761
762        if let Some(requested_gas) = request.as_ref().gas_limit() {
763            let global_gas_cap = self.call_gas_limit();
764            if global_gas_cap != 0 && global_gas_cap < requested_gas {
765                warn!(target: "rpc::eth::call", ?request, ?global_gas_cap, "Capping gas limit to global gas cap");
766                request.as_mut().set_gas_limit(global_gas_cap);
767            }
768        } else {
769            // cap request's gas limit to call gas limit
770            request.as_mut().set_gas_limit(self.call_gas_limit());
771        }
772
773        // Disable block gas limit check to allow executing transactions with higher gas limit (call
774        // gas limit): https://github.com/paradigmxyz/reth/issues/18577
775        evm_env.cfg_env.disable_block_gas_limit = true;
776
777        // Disabled because eth_call is sometimes used with eoa senders
778        // See <https://github.com/paradigmxyz/reth/issues/1959>
779        evm_env.cfg_env.disable_eip3607 = true;
780
781        // The basefee should be ignored for eth_call
782        // See:
783        // <https://github.com/ethereum/go-ethereum/blob/ee8e83fa5f6cb261dad2ed0a7bbcde4930c41e6c/internal/ethapi/api.go#L985>
784        evm_env.cfg_env.disable_base_fee = true;
785
786        // Disable EIP-7825 transaction gas limit to support larger transactions
787        evm_env.cfg_env.tx_gas_limit_cap = Some(u64::MAX);
788
789        // set nonce to None so that the correct nonce is chosen by the EVM
790        request.as_mut().take_nonce();
791
792        if let Some(block_overrides) = overrides.block {
793            apply_block_overrides(*block_overrides, db, &mut evm_env.block_env);
794        }
795        if let Some(state_overrides) = overrides.state {
796            apply_state_overrides(state_overrides, db)
797                .map_err(EthApiError::from_state_overrides_err)?;
798        }
799
800        let mut tx_env = self.create_txn_env(&evm_env, request, &mut *db)?;
801
802        // lower the basefee to 0 to avoid breaking EVM invariants (basefee < gasprice): <https://github.com/ethereum/go-ethereum/blob/355228b011ef9a85ebc0f21e7196f892038d49f0/internal/ethapi/api.go#L700-L704>
803        if tx_env.gas_price() == 0 {
804            evm_env.block_env.basefee = 0;
805        }
806
807        if !request_has_gas_limit {
808            // No gas limit was provided in the request, so we need to cap the transaction gas limit
809            if tx_env.gas_price() > 0 {
810                // If gas price is specified, cap transaction gas limit with caller allowance
811                trace!(target: "rpc::eth::call", ?tx_env, "Applying gas limit cap with caller allowance");
812                let cap = self.caller_gas_allowance(db, &evm_env, &tx_env)?;
813                // ensure we cap gas_limit to the block's
814                tx_env.set_gas_limit(cap.min(evm_env.block_env.gas_limit));
815            }
816        }
817
818        Ok((evm_env, tx_env))
819    }
820}