reth_rpc_eth_api/helpers/
trace.rs

1//! Loads a pending block from database. Helper trait for `eth_` call and trace RPC methods.
2
3use super::{Call, LoadBlock, LoadPendingBlock, LoadState, LoadTransaction};
4use crate::FromEvmError;
5use alloy_consensus::BlockHeader;
6use alloy_primitives::B256;
7use alloy_rpc_types_eth::{BlockId, TransactionInfo};
8use futures::Future;
9use reth_chainspec::ChainSpecProvider;
10use reth_errors::ProviderError;
11use reth_evm::{
12    system_calls::SystemCaller, ConfigureEvm, Database, Evm, EvmEnvFor, HaltReasonFor,
13    InspectorFor, TxEnvFor,
14};
15use reth_node_api::NodePrimitives;
16use reth_primitives_traits::{BlockBody, RecoveredBlock, SignedTransaction};
17use reth_provider::{BlockReader, ProviderBlock, ProviderHeader, ProviderTx};
18use reth_revm::{database::StateProviderDatabase, db::CacheDB};
19use reth_rpc_eth_types::{
20    cache::db::{StateCacheDb, StateCacheDbRefMutWrapper, StateProviderTraitObjWrapper},
21    EthApiError,
22};
23use revm::{
24    context_interface::result::{ExecutionResult, ResultAndState},
25    state::EvmState,
26    DatabaseCommit,
27};
28use revm_inspectors::tracing::{TracingInspector, TracingInspectorConfig};
29use std::sync::Arc;
30
31/// Executes CPU heavy tasks.
32pub trait Trace:
33    LoadState<
34    Provider: BlockReader,
35    Evm: ConfigureEvm<
36        Primitives: NodePrimitives<
37            BlockHeader = ProviderHeader<Self::Provider>,
38            SignedTx = ProviderTx<Self::Provider>,
39        >,
40    >,
41    Error: FromEvmError<Self::Evm>,
42>
43{
44    /// Executes the [`reth_evm::EvmEnv`] against the given [Database] without committing state
45    /// changes.
46    #[expect(clippy::type_complexity)]
47    fn inspect<DB, I>(
48        &self,
49        db: DB,
50        evm_env: EvmEnvFor<Self::Evm>,
51        tx_env: TxEnvFor<Self::Evm>,
52        inspector: I,
53    ) -> Result<
54        (ResultAndState<HaltReasonFor<Self::Evm>>, (EvmEnvFor<Self::Evm>, TxEnvFor<Self::Evm>)),
55        Self::Error,
56    >
57    where
58        DB: Database<Error = ProviderError>,
59        I: InspectorFor<Self::Evm, DB>,
60    {
61        let mut evm = self.evm_config().evm_with_env_and_inspector(db, evm_env.clone(), inspector);
62        let res = evm.transact(tx_env.clone()).map_err(Self::Error::from_evm_err)?;
63        Ok((res, (evm_env, tx_env)))
64    }
65
66    /// Executes the transaction on top of the given [`BlockId`] with a tracer configured by the
67    /// config.
68    ///
69    /// The callback is then called with the [`TracingInspector`] and the [`ResultAndState`] after
70    /// the configured [`reth_evm::EvmEnv`] was inspected.
71    ///
72    /// Caution: this is blocking
73    fn trace_at<F, R>(
74        &self,
75        evm_env: EvmEnvFor<Self::Evm>,
76        tx_env: TxEnvFor<Self::Evm>,
77        config: TracingInspectorConfig,
78        at: BlockId,
79        f: F,
80    ) -> Result<R, Self::Error>
81    where
82        Self: Call,
83        F: FnOnce(
84            TracingInspector,
85            ResultAndState<HaltReasonFor<Self::Evm>>,
86        ) -> Result<R, Self::Error>,
87    {
88        self.with_state_at_block(at, |state| {
89            let mut db = CacheDB::new(StateProviderDatabase::new(state));
90            let mut inspector = TracingInspector::new(config);
91            let (res, _) = self.inspect(&mut db, evm_env, tx_env, &mut inspector)?;
92            f(inspector, res)
93        })
94    }
95
96    /// Same as [`trace_at`](Self::trace_at) but also provides the used database to the callback.
97    ///
98    /// Executes the transaction on top of the given [`BlockId`] with a tracer configured by the
99    /// config.
100    ///
101    /// The callback is then called with the [`TracingInspector`] and the [`ResultAndState`] after
102    /// the configured [`reth_evm::EvmEnv`] was inspected.
103    fn spawn_trace_at_with_state<F, R>(
104        &self,
105        evm_env: EvmEnvFor<Self::Evm>,
106        tx_env: TxEnvFor<Self::Evm>,
107        config: TracingInspectorConfig,
108        at: BlockId,
109        f: F,
110    ) -> impl Future<Output = Result<R, Self::Error>> + Send
111    where
112        Self: LoadPendingBlock + Call,
113        F: FnOnce(
114                TracingInspector,
115                ResultAndState<HaltReasonFor<Self::Evm>>,
116                StateCacheDb<'_>,
117            ) -> Result<R, Self::Error>
118            + Send
119            + 'static,
120        R: Send + 'static,
121    {
122        let this = self.clone();
123        self.spawn_with_state_at_block(at, move |state| {
124            let mut db = CacheDB::new(StateProviderDatabase::new(state));
125            let mut inspector = TracingInspector::new(config);
126            let (res, _) = this.inspect(&mut db, evm_env, tx_env, &mut inspector)?;
127            f(inspector, res, db)
128        })
129    }
130
131    /// Retrieves the transaction if it exists and returns its trace.
132    ///
133    /// Before the transaction is traced, all previous transaction in the block are applied to the
134    /// state by executing them first.
135    /// The callback `f` is invoked with the [`ResultAndState`] after the transaction was executed
136    /// and the database that points to the beginning of the transaction.
137    ///
138    /// Note: Implementers should use a threadpool where blocking is allowed, such as
139    /// [`BlockingTaskPool`](reth_tasks::pool::BlockingTaskPool).
140    fn spawn_trace_transaction_in_block<F, R>(
141        &self,
142        hash: B256,
143        config: TracingInspectorConfig,
144        f: F,
145    ) -> impl Future<Output = Result<Option<R>, Self::Error>> + Send
146    where
147        Self: LoadPendingBlock + LoadTransaction + Call,
148        F: FnOnce(
149                TransactionInfo,
150                TracingInspector,
151                ResultAndState<HaltReasonFor<Self::Evm>>,
152                StateCacheDb<'_>,
153            ) -> Result<R, Self::Error>
154            + Send
155            + 'static,
156        R: Send + 'static,
157    {
158        self.spawn_trace_transaction_in_block_with_inspector(hash, TracingInspector::new(config), f)
159    }
160
161    /// Retrieves the transaction if it exists and returns its trace.
162    ///
163    /// Before the transaction is traced, all previous transaction in the block are applied to the
164    /// state by executing them first.
165    /// The callback `f` is invoked with the [`ResultAndState`] after the transaction was executed
166    /// and the database that points to the beginning of the transaction.
167    ///
168    /// Note: Implementers should use a threadpool where blocking is allowed, such as
169    /// [`BlockingTaskPool`](reth_tasks::pool::BlockingTaskPool).
170    fn spawn_trace_transaction_in_block_with_inspector<Insp, F, R>(
171        &self,
172        hash: B256,
173        mut inspector: Insp,
174        f: F,
175    ) -> impl Future<Output = Result<Option<R>, Self::Error>> + Send
176    where
177        Self: LoadPendingBlock + LoadTransaction + Call,
178        F: FnOnce(
179                TransactionInfo,
180                Insp,
181                ResultAndState<HaltReasonFor<Self::Evm>>,
182                StateCacheDb<'_>,
183            ) -> Result<R, Self::Error>
184            + Send
185            + 'static,
186        Insp:
187            for<'a, 'b> InspectorFor<Self::Evm, StateCacheDbRefMutWrapper<'a, 'b>> + Send + 'static,
188        R: Send + 'static,
189    {
190        async move {
191            let (transaction, block) = match self.transaction_and_block(hash).await? {
192                None => return Ok(None),
193                Some(res) => res,
194            };
195            let (tx, tx_info) = transaction.split();
196
197            let (evm_env, _) = self.evm_env_at(block.hash().into()).await?;
198
199            // we need to get the state of the parent block because we're essentially replaying the
200            // block the transaction is included in
201            let parent_block = block.parent_hash();
202
203            let this = self.clone();
204            self.spawn_with_state_at_block(parent_block.into(), move |state| {
205                let mut db = CacheDB::new(StateProviderDatabase::new(state));
206                let block_txs = block.transactions_recovered();
207
208                this.apply_pre_execution_changes(&block, &mut db, &evm_env)?;
209
210                // replay all transactions prior to the targeted transaction
211                this.replay_transactions_until(&mut db, evm_env.clone(), block_txs, *tx.tx_hash())?;
212
213                let tx_env = this.evm_config().tx_env(tx);
214                let (res, _) = this.inspect(
215                    StateCacheDbRefMutWrapper(&mut db),
216                    evm_env,
217                    tx_env,
218                    &mut inspector,
219                )?;
220                f(tx_info, inspector, res, db)
221            })
222            .await
223            .map(Some)
224        }
225    }
226
227    /// Executes all transactions of a block up to a given index.
228    ///
229    /// If a `highest_index` is given, this will only execute the first `highest_index`
230    /// transactions, in other words, it will stop executing transactions after the
231    /// `highest_index`th transaction. If `highest_index` is `None`, all transactions
232    /// are executed.
233    fn trace_block_until<F, R>(
234        &self,
235        block_id: BlockId,
236        block: Option<Arc<RecoveredBlock<ProviderBlock<Self::Provider>>>>,
237        highest_index: Option<u64>,
238        config: TracingInspectorConfig,
239        f: F,
240    ) -> impl Future<Output = Result<Option<Vec<R>>, Self::Error>> + Send
241    where
242        Self: LoadBlock,
243        F: Fn(
244                TransactionInfo,
245                TracingInspector,
246                ExecutionResult<HaltReasonFor<Self::Evm>>,
247                &EvmState,
248                &StateCacheDb<'_>,
249            ) -> Result<R, Self::Error>
250            + Send
251            + 'static,
252        R: Send + 'static,
253    {
254        self.trace_block_until_with_inspector(
255            block_id,
256            block,
257            highest_index,
258            move || TracingInspector::new(config),
259            f,
260        )
261    }
262
263    /// Executes all transactions of a block.
264    ///
265    /// If a `highest_index` is given, this will only execute the first `highest_index`
266    /// transactions, in other words, it will stop executing transactions after the
267    /// `highest_index`th transaction.
268    ///
269    /// Note: This expect tx index to be 0-indexed, so the first transaction is at index 0.
270    ///
271    /// This accepts a `inspector_setup` closure that returns the inspector to be used for tracing
272    /// the transactions.
273    fn trace_block_until_with_inspector<Setup, Insp, F, R>(
274        &self,
275        block_id: BlockId,
276        block: Option<Arc<RecoveredBlock<ProviderBlock<Self::Provider>>>>,
277        highest_index: Option<u64>,
278        mut inspector_setup: Setup,
279        f: F,
280    ) -> impl Future<Output = Result<Option<Vec<R>>, Self::Error>> + Send
281    where
282        Self: LoadBlock,
283        F: Fn(
284                TransactionInfo,
285                Insp,
286                ExecutionResult<HaltReasonFor<Self::Evm>>,
287                &EvmState,
288                &StateCacheDb<'_>,
289            ) -> Result<R, Self::Error>
290            + Send
291            + 'static,
292        Setup: FnMut() -> Insp + Send + 'static,
293        Insp:
294            for<'a, 'b> InspectorFor<Self::Evm, StateCacheDbRefMutWrapper<'a, 'b>> + Send + 'static,
295        R: Send + 'static,
296    {
297        async move {
298            let block = async {
299                if block.is_some() {
300                    return Ok(block)
301                }
302                self.recovered_block(block_id).await
303            };
304
305            let ((evm_env, _), block) = futures::try_join!(self.evm_env_at(block_id), block)?;
306
307            let Some(block) = block else { return Ok(None) };
308
309            if block.body().transactions().is_empty() {
310                // nothing to trace
311                return Ok(Some(Vec::new()))
312            }
313
314            // replay all transactions of the block
315            self.spawn_tracing(move |this| {
316                // we need to get the state of the parent block because we're replaying this block
317                // on top of its parent block's state
318                let state_at = block.parent_hash();
319                let block_hash = block.hash();
320
321                let block_number = evm_env.block_env.number;
322                let base_fee = evm_env.block_env.basefee;
323
324                // now get the state
325                let state = this.state_at_block_id(state_at.into())?;
326                let mut db =
327                    CacheDB::new(StateProviderDatabase::new(StateProviderTraitObjWrapper(&state)));
328
329                this.apply_pre_execution_changes(&block, &mut db, &evm_env)?;
330
331                // prepare transactions, we do everything upfront to reduce time spent with open
332                // state
333                let max_transactions =
334                    highest_index.map_or(block.body().transaction_count(), |highest| {
335                        // we need + 1 because the index is 0-based
336                        highest as usize + 1
337                    });
338                let mut results = Vec::with_capacity(max_transactions);
339
340                let mut transactions = block
341                    .transactions_recovered()
342                    .take(max_transactions)
343                    .enumerate()
344                    .map(|(idx, tx)| {
345                        let tx_info = TransactionInfo {
346                            hash: Some(*tx.tx_hash()),
347                            index: Some(idx as u64),
348                            block_hash: Some(block_hash),
349                            block_number: Some(block_number),
350                            base_fee: Some(base_fee),
351                        };
352                        let tx_env = this.evm_config().tx_env(tx);
353                        (tx_info, tx_env)
354                    })
355                    .peekable();
356
357                while let Some((tx_info, tx)) = transactions.next() {
358                    let mut inspector = inspector_setup();
359                    let (res, _) = this.inspect(
360                        StateCacheDbRefMutWrapper(&mut db),
361                        evm_env.clone(),
362                        tx,
363                        &mut inspector,
364                    )?;
365                    let ResultAndState { result, state } = res;
366                    results.push(f(tx_info, inspector, result, &state, &db)?);
367
368                    // need to apply the state changes of this transaction before executing the
369                    // next transaction, but only if there's a next transaction
370                    if transactions.peek().is_some() {
371                        // commit the state changes to the DB
372                        db.commit(state)
373                    }
374                }
375
376                Ok(Some(results))
377            })
378            .await
379        }
380    }
381
382    /// Executes all transactions of a block and returns a list of callback results invoked for each
383    /// transaction in the block.
384    ///
385    /// This
386    /// 1. fetches all transactions of the block
387    /// 2. configures the EVM evn
388    /// 3. loops over all transactions and executes them
389    /// 4. calls the callback with the transaction info, the execution result, the changed state
390    ///    _after_ the transaction [`StateProviderDatabase`] and the database that points to the
391    ///    state right _before_ the transaction.
392    fn trace_block_with<F, R>(
393        &self,
394        block_id: BlockId,
395        block: Option<Arc<RecoveredBlock<ProviderBlock<Self::Provider>>>>,
396        config: TracingInspectorConfig,
397        f: F,
398    ) -> impl Future<Output = Result<Option<Vec<R>>, Self::Error>> + Send
399    where
400        Self: LoadBlock,
401        // This is the callback that's invoked for each transaction with the inspector, the result,
402        // state and db
403        F: Fn(
404                TransactionInfo,
405                TracingInspector,
406                ExecutionResult<HaltReasonFor<Self::Evm>>,
407                &EvmState,
408                &StateCacheDb<'_>,
409            ) -> Result<R, Self::Error>
410            + Send
411            + 'static,
412        R: Send + 'static,
413    {
414        self.trace_block_until(block_id, block, None, config, f)
415    }
416
417    /// Executes all transactions of a block and returns a list of callback results invoked for each
418    /// transaction in the block.
419    ///
420    /// This
421    /// 1. fetches all transactions of the block
422    /// 2. configures the EVM evn
423    /// 3. loops over all transactions and executes them
424    /// 4. calls the callback with the transaction info, the execution result, the changed state
425    ///    _after_ the transaction [`EvmState`] and the database that points to the state right
426    ///    _before_ the transaction, in other words the state the transaction was executed on:
427    ///    `changed_state = tx(cached_state)`
428    ///
429    /// This accepts a `inspector_setup` closure that returns the inspector to be used for tracing
430    /// a transaction. This is invoked for each transaction.
431    fn trace_block_inspector<Setup, Insp, F, R>(
432        &self,
433        block_id: BlockId,
434        block: Option<Arc<RecoveredBlock<ProviderBlock<Self::Provider>>>>,
435        insp_setup: Setup,
436        f: F,
437    ) -> impl Future<Output = Result<Option<Vec<R>>, Self::Error>> + Send
438    where
439        Self: LoadBlock,
440        // This is the callback that's invoked for each transaction with the inspector, the result,
441        // state and db
442        F: Fn(
443                TransactionInfo,
444                Insp,
445                ExecutionResult<HaltReasonFor<Self::Evm>>,
446                &EvmState,
447                &StateCacheDb<'_>,
448            ) -> Result<R, Self::Error>
449            + Send
450            + 'static,
451        Setup: FnMut() -> Insp + Send + 'static,
452        Insp:
453            for<'a, 'b> InspectorFor<Self::Evm, StateCacheDbRefMutWrapper<'a, 'b>> + Send + 'static,
454        R: Send + 'static,
455    {
456        self.trace_block_until_with_inspector(block_id, block, None, insp_setup, f)
457    }
458
459    /// Applies chain-specific state transitions required before executing a block.
460    ///
461    /// Note: This should only be called when tracing an entire block vs individual transactions.
462    /// When tracing transaction on top of an already committed block state, those transitions are
463    /// already applied.
464    fn apply_pre_execution_changes<DB: Send + Database + DatabaseCommit>(
465        &self,
466        block: &RecoveredBlock<ProviderBlock<Self::Provider>>,
467        db: &mut DB,
468        evm_env: &EvmEnvFor<Self::Evm>,
469    ) -> Result<(), Self::Error> {
470        let mut system_caller = SystemCaller::new(self.provider().chain_spec());
471
472        // apply relevant system calls
473        let mut evm = self.evm_config().evm_with_env(db, evm_env.clone());
474        system_caller
475            .apply_pre_execution_changes(block.header(), &mut evm)
476            .map_err(|_| EthApiError::EvmCustom("failed to apply 4788 system call".to_string()))?;
477
478        Ok(())
479    }
480}