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