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, FromEthApiError, FromEvmError, FullEthApiTypes,
7    IntoEthApiError, RpcBlock, RpcNodeCore,
8};
9use alloy_consensus::BlockHeader;
10use alloy_eips::{eip1559::calc_next_block_base_fee, eip2930::AccessListResult};
11use alloy_primitives::{Address, Bytes, TxKind, B256, U256};
12use alloy_rpc_types_eth::{
13    simulate::{SimBlock, SimulatePayload, SimulatedBlock},
14    state::{EvmOverrides, StateOverride},
15    transaction::TransactionRequest,
16    BlockId, Bundle, EthCallResponse, StateContext, TransactionInfo,
17};
18use futures::Future;
19use reth_chainspec::EthChainSpec;
20use reth_evm::{ConfigureEvm, ConfigureEvmEnv};
21use reth_node_api::BlockBody;
22use reth_primitives_traits::SignedTransaction;
23use reth_provider::{BlockIdReader, ChainSpecProvider, ProviderHeader};
24use reth_revm::{
25    database::StateProviderDatabase,
26    db::CacheDB,
27    primitives::{
28        BlockEnv, CfgEnvWithHandlerCfg, EnvWithHandlerCfg, ExecutionResult, ResultAndState, TxEnv,
29    },
30    DatabaseRef,
31};
32use reth_rpc_eth_types::{
33    cache::db::{StateCacheDbRefMutWrapper, StateProviderTraitObjWrapper},
34    error::ensure_success,
35    revm_utils::{
36        apply_block_overrides, apply_state_overrides, caller_gas_allowance, get_precompiles,
37        CallFees,
38    },
39    simulate::{self, EthSimulateError},
40    EthApiError, RevertError, RpcInvalidTransactionError, StateCacheDb,
41};
42use revm::{Database, DatabaseCommit, GetInspector};
43use revm_inspectors::{access_list::AccessListInspector, transfer::TransferInspector};
44use revm_primitives::RngMode;
45use tracing::trace;
46
47/// Result type for `eth_simulateV1` RPC method.
48pub type SimulatedBlocksResult<N, E> = Result<Vec<SimulatedBlock<RpcBlock<N>>>, E>;
49
50/// Execution related functions for the [`EthApiServer`](crate::EthApiServer) trait in
51/// the `eth_` namespace.
52pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthApiTypes {
53    /// Estimate gas needed for execution of the `request` at the [`BlockId`].
54    fn estimate_gas_at(
55        &self,
56        request: TransactionRequest,
57        at: BlockId,
58        state_override: Option<StateOverride>,
59    ) -> impl Future<Output = Result<U256, Self::Error>> + Send {
60        EstimateCall::estimate_gas_at(self, request, at, state_override)
61    }
62
63    /// `eth_simulateV1` executes an arbitrary number of transactions on top of the requested state.
64    /// The transactions are packed into individual blocks. Overrides can be provided.
65    ///
66    /// See also: <https://github.com/ethereum/go-ethereum/pull/27720>
67    #[allow(clippy::type_complexity)]
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 SimulatePayload {
79                block_state_calls,
80                trace_transfers,
81                validation,
82                return_full_transactions,
83            } = payload;
84
85            if block_state_calls.is_empty() {
86                return Err(EthApiError::InvalidParams(String::from("calls are empty.")).into())
87            }
88
89            // Build cfg and block env, we'll reuse those.
90            let (mut cfg, mut block_env, block) =
91                self.evm_env_at(block.unwrap_or_default()).await?;
92
93            // Gas cap for entire operation
94            let total_gas_limit = self.call_gas_limit();
95
96            let base_block =
97                self.block_with_senders(block).await?.ok_or(EthApiError::HeaderNotFound(block))?;
98            let mut parent_hash = base_block.header.hash();
99
100            // Only enforce base fee if validation is enabled
101            cfg.disable_base_fee = !validation;
102            // Always disable EIP-3607
103            cfg.disable_eip3607 = true;
104
105            let this = self.clone();
106            self.spawn_with_state_at_block(block, move |state| {
107                let mut db = CacheDB::new(StateProviderDatabase::new(state));
108                let mut blocks: Vec<SimulatedBlock<RpcBlock<Self::NetworkTypes>>> =
109                    Vec::with_capacity(block_state_calls.len());
110                let mut gas_used = 0;
111                for block in block_state_calls {
112                    // Increase number and timestamp for every new block
113                    block_env.number += U256::from(1);
114                    block_env.timestamp += U256::from(1);
115
116                    if validation {
117                        let chain_spec = RpcNodeCore::provider(&this).chain_spec();
118                        let base_fee_params =
119                            chain_spec.base_fee_params_at_timestamp(block_env.timestamp.to());
120                        let base_fee = if let Some(latest) = blocks.last() {
121                            let header = &latest.inner.header;
122                            calc_next_block_base_fee(
123                                header.gas_used(),
124                                header.gas_limit(),
125                                header.base_fee_per_gas().unwrap_or_default(),
126                                base_fee_params,
127                            )
128                        } else {
129                            base_block
130                                .header
131                                .next_block_base_fee(base_fee_params)
132                                .unwrap_or_default()
133                        };
134                        block_env.basefee = U256::from(base_fee);
135                    } else {
136                        block_env.basefee = U256::ZERO;
137                    }
138
139                    let SimBlock { block_overrides, state_overrides, mut calls } = block;
140
141                    if let Some(block_overrides) = block_overrides {
142                        apply_block_overrides(block_overrides, &mut db, &mut block_env);
143                    }
144                    if let Some(state_overrides) = state_overrides {
145                        apply_state_overrides(state_overrides, &mut db)?;
146                    }
147
148                    if (total_gas_limit - gas_used) < block_env.gas_limit.to() {
149                        return Err(
150                            EthApiError::Other(Box::new(EthSimulateError::GasLimitReached)).into()
151                        )
152                    }
153
154                    // Resolve transactions, populate missing fields and enforce calls correctness.
155                    let transactions = simulate::resolve_transactions(
156                        &mut calls,
157                        validation,
158                        block_env.gas_limit.to(),
159                        cfg.chain_id,
160                        &mut db,
161                        this.tx_resp_builder(),
162                    )?;
163
164                    let mut calls = calls.into_iter().peekable();
165                    let mut senders = Vec::with_capacity(transactions.len());
166                    let mut results = Vec::with_capacity(calls.len());
167
168                    while let Some(tx) = calls.next() {
169                        let env = this.build_call_evm_env(cfg.clone(), block_env.clone(), tx)?;
170
171                        let (res, env) = {
172                            if trace_transfers {
173                                this.transact_with_inspector(
174                                    &mut db,
175                                    env,
176                                    TransferInspector::new(false).with_logs(true),
177                                )?
178                            } else {
179                                this.transact(&mut db, env)?
180                            }
181                        };
182
183                        if calls.peek().is_some() {
184                            // need to apply the state changes of this call before executing the
185                            // next call
186                            db.commit(res.state);
187                        }
188
189                        senders.push(env.tx.caller);
190                        results.push(res.result);
191                    }
192
193                    let (block, _) = this.assemble_block_and_receipts(
194                        &block_env,
195                        parent_hash,
196                        // state root calculation is skipped for performance reasons
197                        B256::ZERO,
198                        transactions,
199                        results.clone(),
200                    );
201
202                    let block: SimulatedBlock<RpcBlock<Self::NetworkTypes>> =
203                        simulate::build_simulated_block(
204                            senders,
205                            results,
206                            return_full_transactions,
207                            this.tx_resp_builder(),
208                            block,
209                        )?;
210
211                    parent_hash = block.inner.header.hash;
212                    gas_used += block.inner.header.gas_used();
213
214                    blocks.push(block);
215                }
216
217                Ok(blocks)
218            })
219            .await
220        }
221    }
222
223    /// Executes the call request (`eth_call`) and returns the output
224    fn call(
225        &self,
226        request: TransactionRequest,
227        block_number: Option<BlockId>,
228        overrides: EvmOverrides,
229    ) -> impl Future<Output = Result<Bytes, Self::Error>> + Send {
230        async move {
231            let (res, _env) =
232                self.transact_call_at(request, block_number.unwrap_or_default(), overrides).await?;
233
234            ensure_success(res.result).map_err(Self::Error::from_eth_err)
235        }
236    }
237
238    /// Simulate arbitrary number of transactions at an arbitrary blockchain index, with the
239    /// optionality of state overrides
240    fn call_many(
241        &self,
242        bundle: Bundle,
243        state_context: Option<StateContext>,
244        mut state_override: Option<StateOverride>,
245    ) -> impl Future<Output = Result<Vec<EthCallResponse>, Self::Error>> + Send {
246        async move {
247            let Bundle { transactions, block_override } = bundle;
248            if transactions.is_empty() {
249                return Err(
250                    EthApiError::InvalidParams(String::from("transactions are empty.")).into()
251                )
252            }
253
254            let StateContext { transaction_index, block_number } =
255                state_context.unwrap_or_default();
256            let transaction_index = transaction_index.unwrap_or_default();
257
258            let mut target_block = block_number.unwrap_or_default();
259            let is_block_target_pending = target_block.is_pending();
260
261            // if it's not pending, we should always use block_hash over block_number to ensure that
262            // different provider calls query data related to the same block.
263            if !is_block_target_pending {
264                target_block = self
265                    .provider()
266                    .block_hash_for_id(target_block)
267                    .map_err(|_| EthApiError::HeaderNotFound(target_block))?
268                    .ok_or_else(|| EthApiError::HeaderNotFound(target_block))?
269                    .into();
270            }
271
272            let ((cfg, block_env, _), block) = futures::try_join!(
273                self.evm_env_at(target_block),
274                self.block_with_senders(target_block)
275            )?;
276
277            let block = block.ok_or(EthApiError::HeaderNotFound(target_block))?;
278
279            // we're essentially replaying the transactions in the block here, hence we need the
280            // state that points to the beginning of the block, which is the state at
281            // the parent block
282            let mut at = block.parent_hash();
283            let mut replay_block_txs = true;
284
285            let num_txs =
286                transaction_index.index().unwrap_or_else(|| block.body.transactions().len());
287            // but if all transactions are to be replayed, we can use the state at the block itself,
288            // however only if we're not targeting the pending block, because for pending we can't
289            // rely on the block's state being available
290            if !is_block_target_pending && num_txs == block.body.transactions().len() {
291                at = block.hash();
292                replay_block_txs = false;
293            }
294
295            let this = self.clone();
296            self.spawn_with_state_at_block(at.into(), move |state| {
297                let mut results = Vec::with_capacity(transactions.len());
298                let mut db = CacheDB::new(StateProviderDatabase::new(state));
299
300                if replay_block_txs {
301                    // only need to replay the transactions in the block if not all transactions are
302                    // to be replayed
303                    let transactions = block.transactions_with_sender().take(num_txs);
304                    for (signer, tx) in transactions {
305                        let env = EnvWithHandlerCfg::new_with_cfg_env(
306                            cfg.clone(),
307                            block_env.clone(),
308                            RpcNodeCore::evm_config(&this)
309                                .tx_env(tx, *signer)
310                                .map_err(|_| EthApiError::FailedToDecodeSignedTransaction)?,
311                        );
312                        let (res, _) = this.transact(&mut db, env)?;
313                        db.commit(res.state);
314                    }
315                }
316
317                let block_overrides = block_override.map(Box::new);
318
319                let mut transactions = transactions.into_iter().peekable();
320                while let Some(tx) = transactions.next() {
321                    // apply state overrides only once, before the first transaction
322                    let state_overrides = state_override.take();
323                    let overrides = EvmOverrides::new(state_overrides, block_overrides.clone());
324
325                    let env = this
326                        .prepare_call_env(cfg.clone(), block_env.clone(), tx, &mut db, overrides)
327                        .map(Into::into)?;
328                    let (res, _) = this.transact(&mut db, env)?;
329
330                    match ensure_success(res.result) {
331                        Ok(output) => {
332                            results.push(EthCallResponse { value: Some(output), error: None });
333                        }
334                        Err(err) => {
335                            results.push(EthCallResponse {
336                                value: None,
337                                error: Some(err.to_string()),
338                            });
339                        }
340                    }
341
342                    if transactions.peek().is_some() {
343                        // need to apply the state changes of this call before executing the next
344                        // call
345                        db.commit(res.state);
346                    }
347                }
348
349                Ok(results)
350            })
351            .await
352        }
353    }
354
355    /// Creates [`AccessListResult`] for the [`TransactionRequest`] at the given
356    /// [`BlockId`], or latest block.
357    fn create_access_list_at(
358        &self,
359        request: TransactionRequest,
360        block_number: Option<BlockId>,
361    ) -> impl Future<Output = Result<AccessListResult, Self::Error>> + Send
362    where
363        Self: Trace,
364    {
365        async move {
366            let block_id = block_number.unwrap_or_default();
367            let (cfg, block, at) = self.evm_env_at(block_id).await?;
368
369            self.spawn_blocking_io(move |this| {
370                this.create_access_list_with(cfg, block, at, request)
371            })
372            .await
373        }
374    }
375
376    /// Creates [`AccessListResult`] for the [`TransactionRequest`] at the given
377    /// [`BlockId`].
378    fn create_access_list_with(
379        &self,
380        cfg: CfgEnvWithHandlerCfg,
381        block: BlockEnv,
382        at: BlockId,
383        mut request: TransactionRequest,
384    ) -> Result<AccessListResult, Self::Error>
385    where
386        Self: Trace,
387    {
388        let state = self.state_at_block_id(at)?;
389
390        let mut env = self.build_call_evm_env(cfg, block, request.clone())?;
391
392        // we want to disable this in eth_createAccessList, since this is common practice used by
393        // other node impls and providers <https://github.com/foundry-rs/foundry/issues/4388>
394        env.cfg.disable_block_gas_limit = true;
395
396        // The basefee should be ignored for eth_createAccessList
397        // See:
398        // <https://github.com/ethereum/go-ethereum/blob/8990c92aea01ca07801597b00c0d83d4e2d9b811/internal/ethapi/api.go#L1476-L1476>
399        env.cfg.disable_base_fee = true;
400
401        let mut db = CacheDB::new(StateProviderDatabase::new(state));
402
403        if request.gas.is_none() && env.tx.gas_price > U256::ZERO {
404            let cap = caller_gas_allowance(&mut db, &env.tx)?;
405            // no gas limit was provided in the request, so we need to cap the request's gas limit
406            env.tx.gas_limit = cap.min(env.block.gas_limit).saturating_to();
407        }
408
409        let from = request.from.unwrap_or_default();
410        let to = if let Some(TxKind::Call(to)) = request.to {
411            to
412        } else {
413            let nonce =
414                db.basic_ref(from).map_err(Self::Error::from_eth_err)?.unwrap_or_default().nonce;
415            from.create(nonce)
416        };
417
418        // can consume the list since we're not using the request anymore
419        let initial = request.access_list.take().unwrap_or_default();
420
421        let precompiles = get_precompiles(env.handler_cfg.spec_id);
422        let mut inspector = AccessListInspector::new(initial, from, to, precompiles);
423
424        let (result, mut env) = self.inspect(&mut db, env, &mut inspector)?;
425        let access_list = inspector.into_access_list();
426        env.tx.access_list = access_list.to_vec();
427        match result.result {
428            ExecutionResult::Halt { reason, gas_used } => {
429                let error =
430                    Some(RpcInvalidTransactionError::halt(reason, env.tx.gas_limit).to_string());
431                return Ok(AccessListResult { access_list, gas_used: U256::from(gas_used), error })
432            }
433            ExecutionResult::Revert { output, gas_used } => {
434                let error = Some(RevertError::new(output).to_string());
435                return Ok(AccessListResult { access_list, gas_used: U256::from(gas_used), error })
436            }
437            ExecutionResult::Success { .. } => {}
438        };
439
440        // transact again to get the exact gas used
441        let (result, env) = self.transact(&mut db, env)?;
442        let res = match result.result {
443            ExecutionResult::Halt { reason, gas_used } => {
444                let error =
445                    Some(RpcInvalidTransactionError::halt(reason, env.tx.gas_limit).to_string());
446                AccessListResult { access_list, gas_used: U256::from(gas_used), error }
447            }
448            ExecutionResult::Revert { output, gas_used } => {
449                let error = Some(RevertError::new(output).to_string());
450                AccessListResult { access_list, gas_used: U256::from(gas_used), error }
451            }
452            ExecutionResult::Success { gas_used, .. } => {
453                AccessListResult { access_list, gas_used: U256::from(gas_used), error: None }
454            }
455        };
456
457        Ok(res)
458    }
459}
460
461/// Executes code on state.
462pub trait Call:
463    LoadState<Evm: ConfigureEvm<Header = ProviderHeader<Self::Provider>>> + SpawnBlocking
464{
465    /// Returns default gas limit to use for `eth_call` and tracing RPC methods.
466    ///
467    /// Data access in default trait method implementations.
468    fn call_gas_limit(&self) -> u64;
469
470    /// Returns the maximum number of blocks accepted for `eth_simulateV1`.
471    fn max_simulate_blocks(&self) -> u64;
472
473    /// Executes the closure with the state that corresponds to the given [`BlockId`].
474    fn with_state_at_block<F, R>(&self, at: BlockId, f: F) -> Result<R, Self::Error>
475    where
476        F: FnOnce(StateProviderTraitObjWrapper<'_>) -> Result<R, Self::Error>,
477    {
478        let state = self.state_at_block_id(at)?;
479        f(StateProviderTraitObjWrapper(&state))
480    }
481
482    /// Executes the [`EnvWithHandlerCfg`] against the given [Database] without committing state
483    /// changes.
484    fn transact<DB>(
485        &self,
486        db: DB,
487        env: EnvWithHandlerCfg,
488    ) -> Result<(ResultAndState, EnvWithHandlerCfg), Self::Error>
489    where
490        DB: Database,
491        EthApiError: From<DB::Error>,
492    {
493        let mut evm = self.evm_config().evm_with_env(db, env);
494        let res = evm.transact().map_err(Self::Error::from_evm_err)?;
495        let (_, env) = evm.into_db_and_env_with_handler_cfg();
496        Ok((res, env))
497    }
498
499    /// Executes the [`EnvWithHandlerCfg`] against the given [Database] without committing state
500    /// changes.
501    fn transact_with_inspector<DB>(
502        &self,
503        db: DB,
504        env: EnvWithHandlerCfg,
505        inspector: impl GetInspector<DB>,
506    ) -> Result<(ResultAndState, EnvWithHandlerCfg), Self::Error>
507    where
508        DB: Database,
509        EthApiError: From<DB::Error>,
510    {
511        let mut evm = self.evm_config().evm_with_env_and_inspector(db, env, inspector);
512        let res = evm.transact().map_err(Self::Error::from_evm_err)?;
513        let (_, env) = evm.into_db_and_env_with_handler_cfg();
514        Ok((res, env))
515    }
516
517    /// Executes the call request at the given [`BlockId`].
518    fn transact_call_at(
519        &self,
520        request: TransactionRequest,
521        at: BlockId,
522        overrides: EvmOverrides,
523    ) -> impl Future<Output = Result<(ResultAndState, EnvWithHandlerCfg), Self::Error>> + Send
524    where
525        Self: LoadPendingBlock,
526    {
527        let this = self.clone();
528        self.spawn_with_call_at(request, at, overrides, move |db, env| this.transact(db, env))
529    }
530
531    /// Executes the closure with the state that corresponds to the given [`BlockId`] on a new task
532    fn spawn_with_state_at_block<F, R>(
533        &self,
534        at: BlockId,
535        f: F,
536    ) -> impl Future<Output = Result<R, Self::Error>> + Send
537    where
538        F: FnOnce(StateProviderTraitObjWrapper<'_>) -> Result<R, Self::Error> + Send + 'static,
539        R: Send + 'static,
540    {
541        self.spawn_tracing(move |this| {
542            let state = this.state_at_block_id(at)?;
543            f(StateProviderTraitObjWrapper(&state))
544        })
545    }
546
547    /// Prepares the state and env for the given [`TransactionRequest`] at the given [`BlockId`] and
548    /// executes the closure on a new task returning the result of the closure.
549    ///
550    /// This returns the configured [`EnvWithHandlerCfg`] for the given [`TransactionRequest`] at
551    /// the given [`BlockId`] and with configured call settings: `prepare_call_env`.
552    ///
553    /// This is primarily used by `eth_call`.
554    ///
555    /// # Blocking behaviour
556    ///
557    /// This assumes executing the call is relatively more expensive on IO than CPU because it
558    /// transacts a single transaction on an empty in memory database. Because `eth_call`s are
559    /// usually allowed to consume a lot of gas, this also allows a lot of memory operations so
560    /// we assume this is not primarily CPU bound and instead spawn the call on a regular tokio task
561    /// instead, where blocking IO is less problematic.
562    fn spawn_with_call_at<F, R>(
563        &self,
564        request: TransactionRequest,
565        at: BlockId,
566        overrides: EvmOverrides,
567        f: F,
568    ) -> impl Future<Output = Result<R, Self::Error>> + Send
569    where
570        Self: LoadPendingBlock,
571        F: FnOnce(StateCacheDbRefMutWrapper<'_, '_>, EnvWithHandlerCfg) -> Result<R, Self::Error>
572            + Send
573            + 'static,
574        R: Send + 'static,
575    {
576        async move {
577            let (cfg, block_env, at) = self.evm_env_at(at).await?;
578            let this = self.clone();
579            self.spawn_blocking_io(move |_| {
580                let state = this.state_at_block_id(at)?;
581                let mut db =
582                    CacheDB::new(StateProviderDatabase::new(StateProviderTraitObjWrapper(&state)));
583
584                let env = this.prepare_call_env(cfg, block_env, request, &mut db, overrides)?;
585
586                f(StateCacheDbRefMutWrapper(&mut db), env)
587            })
588            .await
589        }
590    }
591
592    /// Retrieves the transaction if it exists and executes it.
593    ///
594    /// Before the transaction is executed, all previous transaction in the block are applied to the
595    /// state by executing them first.
596    /// The callback `f` is invoked with the [`ResultAndState`] after the transaction was executed
597    /// and the database that points to the beginning of the transaction.
598    ///
599    /// Note: Implementers should use a threadpool where blocking is allowed, such as
600    /// [`BlockingTaskPool`](reth_tasks::pool::BlockingTaskPool).
601    fn spawn_replay_transaction<F, R>(
602        &self,
603        hash: B256,
604        f: F,
605    ) -> impl Future<Output = Result<Option<R>, Self::Error>> + Send
606    where
607        Self: LoadBlock + LoadTransaction,
608        F: FnOnce(TransactionInfo, ResultAndState, StateCacheDb<'_>) -> Result<R, Self::Error>
609            + Send
610            + 'static,
611        R: Send + 'static,
612    {
613        async move {
614            let (transaction, block) = match self.transaction_and_block(hash).await? {
615                None => return Ok(None),
616                Some(res) => res,
617            };
618            let (tx, tx_info) = transaction.split();
619
620            let (cfg, block_env, _) = self.evm_env_at(block.hash().into()).await?;
621
622            // we need to get the state of the parent block because we're essentially replaying the
623            // block the transaction is included in
624            let parent_block = block.parent_hash();
625
626            let this = self.clone();
627            self.spawn_with_state_at_block(parent_block.into(), move |state| {
628                let mut db = CacheDB::new(StateProviderDatabase::new(state));
629                let block_txs = block.transactions_with_sender();
630
631                // replay all transactions prior to the targeted transaction
632                this.replay_transactions_until(
633                    &mut db,
634                    cfg.clone(),
635                    block_env.clone(),
636                    block_txs,
637                    *tx.tx_hash(),
638                )?;
639
640                let env = EnvWithHandlerCfg::new_with_cfg_env(
641                    cfg,
642                    block_env,
643                    RpcNodeCore::evm_config(&this)
644                        .tx_env(tx.as_signed(), tx.signer())
645                        .map_err(|_| EthApiError::FailedToDecodeSignedTransaction)?,
646                );
647
648                let (res, _) = this.transact(&mut db, env)?;
649                f(tx_info, res, db)
650            })
651            .await
652            .map(Some)
653        }
654    }
655
656    /// Replays all the transactions until the target transaction is found.
657    ///
658    /// All transactions before the target transaction are executed and their changes are written to
659    /// the _runtime_ db ([`CacheDB`]).
660    ///
661    /// Note: This assumes the target transaction is in the given iterator.
662    /// Returns the index of the target transaction in the given iterator.
663    fn replay_transactions_until<'a, DB, I>(
664        &self,
665        db: &mut DB,
666        cfg: CfgEnvWithHandlerCfg,
667        block_env: BlockEnv,
668        transactions: I,
669        target_tx_hash: B256,
670    ) -> Result<usize, Self::Error>
671    where
672        DB: Database + DatabaseCommit,
673        EthApiError: From<DB::Error>,
674        I: IntoIterator<Item = (&'a Address, &'a <Self::Evm as ConfigureEvmEnv>::Transaction)>,
675        <Self::Evm as ConfigureEvmEnv>::Transaction: SignedTransaction,
676    {
677        let env = EnvWithHandlerCfg::new_with_cfg_env(cfg, block_env, Default::default());
678
679        let mut evm = self.evm_config().evm_with_env(db, env);
680        let mut index = 0;
681        for (sender, tx) in transactions {
682            if *tx.tx_hash() == target_tx_hash {
683                // reached the target transaction
684                break
685            }
686
687            self.evm_config()
688                .fill_tx_env(evm.tx_mut(), tx, *sender)
689                .map_err(|_| EthApiError::FailedToDecodeSignedTransaction)?;
690            evm.transact_commit().map_err(Self::Error::from_evm_err)?;
691            index += 1;
692        }
693        Ok(index)
694    }
695
696    /// Configures a new [`TxEnv`]  for the [`TransactionRequest`]
697    ///
698    /// All [`TxEnv`] fields are derived from the given [`TransactionRequest`], if fields are
699    /// `None`, they fall back to the [`BlockEnv`]'s settings.
700    fn create_txn_env(
701        &self,
702        block_env: &BlockEnv,
703        request: TransactionRequest,
704    ) -> Result<TxEnv, Self::Error> {
705        // Ensure that if versioned hashes are set, they're not empty
706        if request.blob_versioned_hashes.as_ref().is_some_and(|hashes| hashes.is_empty()) {
707            return Err(RpcInvalidTransactionError::BlobTransactionMissingBlobHashes.into_eth_err())
708        }
709
710        let TransactionRequest {
711            from,
712            to,
713            gas_price,
714            max_fee_per_gas,
715            max_priority_fee_per_gas,
716            gas,
717            value,
718            input,
719            nonce,
720            access_list,
721            chain_id,
722            blob_versioned_hashes,
723            max_fee_per_blob_gas,
724            authorization_list,
725            seismic_elements,
726            ..
727        } = request;
728
729        let CallFees { max_priority_fee_per_gas, gas_price, max_fee_per_blob_gas } =
730            CallFees::ensure_fees(
731                gas_price.map(U256::from),
732                max_fee_per_gas.map(U256::from),
733                max_priority_fee_per_gas.map(U256::from),
734                block_env.basefee,
735                blob_versioned_hashes.as_deref(),
736                max_fee_per_blob_gas.map(U256::from),
737                block_env.get_blob_gasprice().map(U256::from),
738            )?;
739
740        let gas_limit = gas.unwrap_or_else(|| {
741            // Use maximum allowed gas limit. The reason for this
742            // is that both Erigon and Geth use pre-configured gas cap even if
743            // it's possible to derive the gas limit from the block:
744            // <https://github.com/ledgerwatch/erigon/blob/eae2d9a79cb70dbe30b3a6b79c436872e4605458/cmd/rpcdaemon/commands/trace_adhoc.go#L956
745            // https://github.com/ledgerwatch/erigon/blob/eae2d9a79cb70dbe30b3a6b79c436872e4605458/eth/ethconfig/config.go#L94>
746            block_env.gas_limit.saturating_to()
747        });
748
749        let input =
750            input.try_into_unique_input().map_err(Self::Error::from_eth_err)?.unwrap_or_default();
751        let input = if let Some(elements) = seismic_elements {
752            self.evm_config().decrypt(&input, &elements).map_err(|_| {
753                EthApiError::InvalidParams("failed to decrypt seismic transaction".to_string())
754                    .into()
755            })?
756        } else {
757            input
758        };
759
760        #[allow(clippy::needless_update)]
761        let env = TxEnv {
762            gas_limit,
763            nonce,
764            caller: from.unwrap_or_default(),
765            gas_price,
766            gas_priority_fee: max_priority_fee_per_gas,
767            transact_to: to.unwrap_or(TxKind::Create),
768            value: value.unwrap_or_default(),
769            data: input,
770            chain_id,
771            access_list: access_list.unwrap_or_default().into(),
772            // EIP-4844 fields
773            blob_hashes: blob_versioned_hashes.unwrap_or_default(),
774            max_fee_per_blob_gas,
775            // EIP-7702 fields
776            authorization_list: authorization_list.map(Into::into),
777            rng_mode: RngMode::Simulation,
778            ..Default::default()
779        };
780
781        Ok(env)
782    }
783
784    /// Creates a new [`EnvWithHandlerCfg`] to be used for executing the [`TransactionRequest`] in
785    /// `eth_call`.
786    ///
787    /// Note: this does _not_ access the Database to check the sender.
788    fn build_call_evm_env(
789        &self,
790        cfg: CfgEnvWithHandlerCfg,
791        block: BlockEnv,
792        request: TransactionRequest,
793    ) -> Result<EnvWithHandlerCfg, Self::Error> {
794        let tx = self.create_txn_env(&block, request)?;
795        Ok(EnvWithHandlerCfg::new_with_cfg_env(cfg, block, tx))
796    }
797
798    /// Prepares the [`EnvWithHandlerCfg`] for execution of calls.
799    ///
800    /// Does not commit any changes to the underlying database.
801    ///
802    /// ## EVM settings
803    ///
804    /// 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>:
805    ///
806    ///  - `disable_eip3607` is set to `true`
807    ///  - `disable_base_fee` is set to `true`
808    ///  - `nonce` is set to `None`
809    ///
810    /// In addition, this changes the block's gas limit to the configured [`Self::call_gas_limit`].
811    fn prepare_call_env<DB>(
812        &self,
813        mut cfg: CfgEnvWithHandlerCfg,
814        mut block: BlockEnv,
815        mut request: TransactionRequest,
816        db: &mut CacheDB<DB>,
817        overrides: EvmOverrides,
818    ) -> Result<EnvWithHandlerCfg, Self::Error>
819    where
820        DB: DatabaseRef,
821        EthApiError: From<<DB as DatabaseRef>::Error>,
822    {
823        if request.gas > Some(self.call_gas_limit()) {
824            // configured gas exceeds limit
825            return Err(
826                EthApiError::InvalidTransaction(RpcInvalidTransactionError::GasTooHigh).into()
827            )
828        }
829
830        // apply configured gas cap
831        block.gas_limit = U256::from(self.call_gas_limit());
832
833        // Disabled because eth_call is sometimes used with eoa senders
834        // See <https://github.com/paradigmxyz/reth/issues/1959>
835        cfg.disable_eip3607 = true;
836
837        // The basefee should be ignored for eth_call
838        // See:
839        // <https://github.com/ethereum/go-ethereum/blob/ee8e83fa5f6cb261dad2ed0a7bbcde4930c41e6c/internal/ethapi/api.go#L985>
840        cfg.disable_base_fee = true;
841
842        // set nonce to None so that the correct nonce is chosen by the EVM
843        request.nonce = None;
844
845        if let Some(block_overrides) = overrides.block {
846            apply_block_overrides(*block_overrides, db, &mut block);
847        }
848        if let Some(state_overrides) = overrides.state {
849            apply_state_overrides(state_overrides, db)?;
850        }
851
852        let request_gas = request.gas;
853        let mut env = self.build_call_evm_env(cfg, block, request)?;
854
855        if request_gas.is_none() {
856            // No gas limit was provided in the request, so we need to cap the transaction gas limit
857            if env.tx.gas_price > U256::ZERO {
858                // If gas price is specified, cap transaction gas limit with caller allowance
859                trace!(target: "rpc::eth::call", ?env, "Applying gas limit cap with caller allowance");
860                let cap = caller_gas_allowance(db, &env.tx)?;
861                // ensure we cap gas_limit to the block's
862                env.tx.gas_limit = cap.min(env.block.gas_limit).saturating_to();
863            }
864        }
865
866        Ok(env)
867    }
868}