reth_rpc/
debug.rs

1use alloy_consensus::{transaction::SignerRecoverable, BlockHeader};
2use alloy_eips::{eip2718::Encodable2718, BlockId, BlockNumberOrTag};
3use alloy_genesis::ChainConfig;
4use alloy_primitives::{Address, Bytes, B256};
5use alloy_rlp::{Decodable, Encodable};
6use alloy_rpc_types_debug::ExecutionWitness;
7use alloy_rpc_types_eth::{
8    state::EvmOverrides, transaction::TransactionRequest, Block as RpcBlock, BlockError, Bundle,
9    StateContext, TransactionInfo,
10};
11use alloy_rpc_types_trace::geth::{
12    call::FlatCallFrame, BlockTraceResult, FourByteFrame, GethDebugBuiltInTracerType,
13    GethDebugTracerType, GethDebugTracingCallOptions, GethDebugTracingOptions, GethTrace,
14    NoopFrame, TraceResult,
15};
16use async_trait::async_trait;
17use jsonrpsee::core::RpcResult;
18use reth_chainspec::{ChainSpecProvider, EthChainSpec, EthereumHardforks};
19use reth_evm::{execute::Executor, ConfigureEvm, EvmEnvFor, TxEnvFor};
20use reth_primitives_traits::{
21    Block as _, BlockBody, NodePrimitives, ReceiptWithBloom, RecoveredBlock, SignedTransaction,
22};
23use reth_revm::{
24    database::StateProviderDatabase,
25    db::{CacheDB, State},
26    witness::ExecutionWitnessRecord,
27};
28use reth_rpc_api::DebugApiServer;
29use reth_rpc_eth_api::{
30    helpers::{EthTransactions, TraceExt},
31    EthApiTypes, FromEthApiError, RpcNodeCore,
32};
33use reth_rpc_eth_types::{EthApiError, StateCacheDb};
34use reth_rpc_server_types::{result::internal_rpc_err, ToRpcResult};
35use reth_storage_api::{
36    BlockIdReader, BlockReaderIdExt, HeaderProvider, ProviderBlock, ReceiptProviderIdExt,
37    StateProofProvider, StateProvider, StateProviderFactory, TransactionVariant,
38};
39use reth_tasks::pool::BlockingTaskGuard;
40use revm::{context_interface::Transaction, state::EvmState, DatabaseCommit};
41use revm_inspectors::tracing::{
42    FourByteInspector, MuxInspector, TracingInspector, TracingInspectorConfig, TransactionContext,
43};
44use std::sync::Arc;
45use tokio::sync::{AcquireError, OwnedSemaphorePermit};
46
47/// `debug` API implementation.
48///
49/// This type provides the functionality for handling `debug` related requests.
50pub struct DebugApi<Eth, BlockExecutor> {
51    inner: Arc<DebugApiInner<Eth, BlockExecutor>>,
52}
53
54// === impl DebugApi ===
55
56impl<Eth, Evm> DebugApi<Eth, Evm> {
57    /// Create a new instance of the [`DebugApi`]
58    pub fn new(eth: Eth, blocking_task_guard: BlockingTaskGuard, evm_config: Evm) -> Self {
59        let inner = Arc::new(DebugApiInner { eth_api: eth, blocking_task_guard, evm_config });
60        Self { inner }
61    }
62
63    /// Access the underlying `Eth` API.
64    pub fn eth_api(&self) -> &Eth {
65        &self.inner.eth_api
66    }
67}
68
69impl<Eth: RpcNodeCore, BlockExecutor> DebugApi<Eth, BlockExecutor> {
70    /// Access the underlying provider.
71    pub fn provider(&self) -> &Eth::Provider {
72        self.inner.eth_api.provider()
73    }
74}
75
76// === impl DebugApi ===
77
78impl<Eth, Evm> DebugApi<Eth, Evm>
79where
80    Eth: EthApiTypes + TraceExt + 'static,
81    Evm: ConfigureEvm<Primitives: NodePrimitives<Block = ProviderBlock<Eth::Provider>>> + 'static,
82{
83    /// Acquires a permit to execute a tracing call.
84    async fn acquire_trace_permit(&self) -> Result<OwnedSemaphorePermit, AcquireError> {
85        self.inner.blocking_task_guard.clone().acquire_owned().await
86    }
87
88    /// Trace the entire block asynchronously
89    async fn trace_block(
90        &self,
91        block: Arc<RecoveredBlock<ProviderBlock<Eth::Provider>>>,
92        evm_env: EvmEnvFor<Eth::Evm>,
93        opts: GethDebugTracingOptions,
94    ) -> Result<Vec<TraceResult>, Eth::Error> {
95        // replay all transactions of the block
96        let this = self.clone();
97        self.eth_api()
98            .spawn_with_state_at_block(block.parent_hash().into(), move |state| {
99                let mut results = Vec::with_capacity(block.body().transactions().len());
100                let mut db = CacheDB::new(StateProviderDatabase::new(state));
101
102                this.eth_api().apply_pre_execution_changes(&block, &mut db, &evm_env)?;
103
104                let mut transactions = block.transactions_recovered().enumerate().peekable();
105                let mut inspector = None;
106                while let Some((index, tx)) = transactions.next() {
107                    let tx_hash = *tx.tx_hash();
108
109                    let tx_env = this.eth_api().evm_config().tx_env(tx);
110
111                    let (result, state_changes) = this.trace_transaction(
112                        &opts,
113                        evm_env.clone(),
114                        tx_env,
115                        &mut db,
116                        Some(TransactionContext {
117                            block_hash: Some(block.hash()),
118                            tx_hash: Some(tx_hash),
119                            tx_index: Some(index),
120                        }),
121                        &mut inspector,
122                    )?;
123
124                    inspector = inspector.map(|insp| insp.fused());
125
126                    results.push(TraceResult::Success { result, tx_hash: Some(tx_hash) });
127                    if transactions.peek().is_some() {
128                        // need to apply the state changes of this transaction before executing the
129                        // next transaction
130                        db.commit(state_changes)
131                    }
132                }
133
134                Ok(results)
135            })
136            .await
137    }
138
139    /// Replays the given block and returns the trace of each transaction.
140    ///
141    /// This expects a rlp encoded block
142    ///
143    /// Note, the parent of this block must be present, or it will fail.
144    pub async fn debug_trace_raw_block(
145        &self,
146        rlp_block: Bytes,
147        opts: GethDebugTracingOptions,
148    ) -> Result<Vec<TraceResult>, Eth::Error> {
149        let block: ProviderBlock<Eth::Provider> = Decodable::decode(&mut rlp_block.as_ref())
150            .map_err(BlockError::RlpDecodeRawBlock)
151            .map_err(Eth::Error::from_eth_err)?;
152
153        let evm_env = self.eth_api().evm_config().evm_env(block.header());
154
155        // Depending on EIP-2 we need to recover the transactions differently
156        let senders =
157            if self.provider().chain_spec().is_homestead_active_at_block(block.header().number()) {
158                block
159                    .body()
160                    .transactions()
161                    .iter()
162                    .map(|tx| tx.recover_signer().map_err(Eth::Error::from_eth_err))
163                    .collect::<Result<Vec<_>, _>>()?
164                    .into_iter()
165                    .collect()
166            } else {
167                block
168                    .body()
169                    .transactions()
170                    .iter()
171                    .map(|tx| tx.recover_signer_unchecked().map_err(Eth::Error::from_eth_err))
172                    .collect::<Result<Vec<_>, _>>()?
173                    .into_iter()
174                    .collect()
175            };
176
177        self.trace_block(Arc::new(block.into_recovered_with_signers(senders)), evm_env, opts).await
178    }
179
180    /// Replays a block and returns the trace of each transaction.
181    pub async fn debug_trace_block(
182        &self,
183        block_id: BlockId,
184        opts: GethDebugTracingOptions,
185    ) -> Result<Vec<TraceResult>, Eth::Error> {
186        let block_hash = self
187            .provider()
188            .block_hash_for_id(block_id)
189            .map_err(Eth::Error::from_eth_err)?
190            .ok_or(EthApiError::HeaderNotFound(block_id))?;
191
192        let ((evm_env, _), block) = futures::try_join!(
193            self.eth_api().evm_env_at(block_hash.into()),
194            self.eth_api().recovered_block(block_hash.into()),
195        )?;
196
197        let block = block.ok_or(EthApiError::HeaderNotFound(block_id))?;
198
199        self.trace_block(block, evm_env, opts).await
200    }
201
202    /// Trace the transaction according to the provided options.
203    ///
204    /// Ref: <https://geth.ethereum.org/docs/developers/evm-tracing/built-in-tracers>
205    pub async fn debug_trace_transaction(
206        &self,
207        tx_hash: B256,
208        opts: GethDebugTracingOptions,
209    ) -> Result<GethTrace, Eth::Error> {
210        let (transaction, block) = match self.eth_api().transaction_and_block(tx_hash).await? {
211            None => return Err(EthApiError::TransactionNotFound.into()),
212            Some(res) => res,
213        };
214        let (evm_env, _) = self.eth_api().evm_env_at(block.hash().into()).await?;
215
216        // we need to get the state of the parent block because we're essentially replaying the
217        // block the transaction is included in
218        let state_at: BlockId = block.parent_hash().into();
219        let block_hash = block.hash();
220
221        let this = self.clone();
222        self.eth_api()
223            .spawn_with_state_at_block(state_at, move |state| {
224                let block_txs = block.transactions_recovered();
225
226                // configure env for the target transaction
227                let tx = transaction.into_recovered();
228
229                let mut db = CacheDB::new(StateProviderDatabase::new(state));
230
231                this.eth_api().apply_pre_execution_changes(&block, &mut db, &evm_env)?;
232
233                // replay all transactions prior to the targeted transaction
234                let index = this.eth_api().replay_transactions_until(
235                    &mut db,
236                    evm_env.clone(),
237                    block_txs,
238                    *tx.tx_hash(),
239                )?;
240
241                let tx_env = this.eth_api().evm_config().tx_env(&tx);
242
243                this.trace_transaction(
244                    &opts,
245                    evm_env,
246                    tx_env,
247                    &mut db,
248                    Some(TransactionContext {
249                        block_hash: Some(block_hash),
250                        tx_index: Some(index),
251                        tx_hash: Some(*tx.tx_hash()),
252                    }),
253                    &mut None,
254                )
255                .map(|(trace, _)| trace)
256            })
257            .await
258    }
259
260    /// The `debug_traceCall` method lets you run an `eth_call` within the context of the given
261    /// block execution using the final state of parent block as the base.
262    ///
263    /// Differences compare to `eth_call`:
264    ///  - `debug_traceCall` executes with __enabled__ basefee check, `eth_call` does not: <https://github.com/paradigmxyz/reth/issues/6240>
265    pub async fn debug_trace_call(
266        &self,
267        call: TransactionRequest,
268        block_id: Option<BlockId>,
269        opts: GethDebugTracingCallOptions,
270    ) -> Result<GethTrace, Eth::Error> {
271        let at = block_id.unwrap_or_default();
272        let GethDebugTracingCallOptions { tracing_options, state_overrides, block_overrides } =
273            opts;
274        let overrides = EvmOverrides::new(state_overrides, block_overrides.map(Box::new));
275        let GethDebugTracingOptions { config, tracer, tracer_config, .. } = tracing_options;
276
277        let this = self.clone();
278        if let Some(tracer) = tracer {
279            return match tracer {
280                GethDebugTracerType::BuiltInTracer(tracer) => match tracer {
281                    GethDebugBuiltInTracerType::FourByteTracer => {
282                        let mut inspector = FourByteInspector::default();
283                        let inspector = self
284                            .eth_api()
285                            .spawn_with_call_at(call, at, overrides, move |db, evm_env, tx_env| {
286                                this.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?;
287                                Ok(inspector)
288                            })
289                            .await?;
290                        Ok(FourByteFrame::from(&inspector).into())
291                    }
292                    GethDebugBuiltInTracerType::CallTracer => {
293                        let call_config = tracer_config
294                            .into_call_config()
295                            .map_err(|_| EthApiError::InvalidTracerConfig)?;
296
297                        let mut inspector = TracingInspector::new(
298                            TracingInspectorConfig::from_geth_call_config(&call_config),
299                        );
300
301                        let frame = self
302                            .eth_api()
303                            .spawn_with_call_at(call, at, overrides, move |db, evm_env, tx_env| {
304                                let (res, (_, tx_env)) =
305                                    this.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?;
306                                let frame = inspector
307                                    .with_transaction_gas_limit(tx_env.gas_limit())
308                                    .into_geth_builder()
309                                    .geth_call_traces(call_config, res.result.gas_used());
310                                Ok(frame.into())
311                            })
312                            .await?;
313                        Ok(frame)
314                    }
315                    GethDebugBuiltInTracerType::PreStateTracer => {
316                        let prestate_config = tracer_config
317                            .into_pre_state_config()
318                            .map_err(|_| EthApiError::InvalidTracerConfig)?;
319                        let mut inspector = TracingInspector::new(
320                            TracingInspectorConfig::from_geth_prestate_config(&prestate_config),
321                        );
322
323                        let frame = self
324                            .eth_api()
325                            .spawn_with_call_at(call, at, overrides, move |db, evm_env, tx_env| {
326                                // wrapper is hack to get around 'higher-ranked lifetime error',
327                                // see <https://github.com/rust-lang/rust/issues/100013>
328                                let db = db.0;
329
330                                let (res, (_, tx_env)) = this.eth_api().inspect(
331                                    &mut *db,
332                                    evm_env,
333                                    tx_env,
334                                    &mut inspector,
335                                )?;
336                                let frame = inspector
337                                    .with_transaction_gas_limit(tx_env.gas_limit())
338                                    .into_geth_builder()
339                                    .geth_prestate_traces(&res, &prestate_config, db)
340                                    .map_err(Eth::Error::from_eth_err)?;
341                                Ok(frame)
342                            })
343                            .await?;
344                        Ok(frame.into())
345                    }
346                    GethDebugBuiltInTracerType::NoopTracer => Ok(NoopFrame::default().into()),
347                    GethDebugBuiltInTracerType::MuxTracer => {
348                        let mux_config = tracer_config
349                            .into_mux_config()
350                            .map_err(|_| EthApiError::InvalidTracerConfig)?;
351
352                        let mut inspector = MuxInspector::try_from_config(mux_config)
353                            .map_err(Eth::Error::from_eth_err)?;
354
355                        let frame = self
356                            .inner
357                            .eth_api
358                            .spawn_with_call_at(call, at, overrides, move |db, evm_env, tx_env| {
359                                // wrapper is hack to get around 'higher-ranked lifetime error', see
360                                // <https://github.com/rust-lang/rust/issues/100013>
361                                let db = db.0;
362
363                                let tx_info = TransactionInfo {
364                                    block_number: Some(evm_env.block_env.number),
365                                    base_fee: Some(evm_env.block_env.basefee),
366                                    hash: None,
367                                    block_hash: None,
368                                    index: None,
369                                };
370
371                                let (res, _) = this.eth_api().inspect(
372                                    &mut *db,
373                                    evm_env,
374                                    tx_env,
375                                    &mut inspector,
376                                )?;
377                                let frame = inspector
378                                    .try_into_mux_frame(&res, db, tx_info)
379                                    .map_err(Eth::Error::from_eth_err)?;
380                                Ok(frame.into())
381                            })
382                            .await?;
383                        Ok(frame)
384                    }
385                    GethDebugBuiltInTracerType::FlatCallTracer => {
386                        let flat_call_config = tracer_config
387                            .into_flat_call_config()
388                            .map_err(|_| EthApiError::InvalidTracerConfig)?;
389
390                        let mut inspector = TracingInspector::new(
391                            TracingInspectorConfig::from_flat_call_config(&flat_call_config),
392                        );
393
394                        let frame: FlatCallFrame = self
395                            .inner
396                            .eth_api
397                            .spawn_with_call_at(call, at, overrides, move |db, evm_env, tx_env| {
398                                let (_res, (_, tx_env)) =
399                                    this.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?;
400                                let tx_info = TransactionInfo::default();
401                                let frame: FlatCallFrame = inspector
402                                    .with_transaction_gas_limit(tx_env.gas_limit())
403                                    .into_parity_builder()
404                                    .into_localized_transaction_traces(tx_info);
405                                Ok(frame)
406                            })
407                            .await?;
408
409                        Ok(frame.into())
410                    }
411                },
412                #[cfg(not(feature = "js-tracer"))]
413                GethDebugTracerType::JsTracer(_) => {
414                    Err(EthApiError::Unsupported("JS Tracer is not enabled").into())
415                }
416                #[cfg(feature = "js-tracer")]
417                GethDebugTracerType::JsTracer(code) => {
418                    let config = tracer_config.into_json();
419
420                    let (_, at) = self.eth_api().evm_env_at(at).await?;
421
422                    let res = self
423                        .eth_api()
424                        .spawn_with_call_at(call, at, overrides, move |db, evm_env, tx_env| {
425                            // wrapper is hack to get around 'higher-ranked lifetime error', see
426                            // <https://github.com/rust-lang/rust/issues/100013>
427                            let db = db.0;
428
429                            let mut inspector =
430                                revm_inspectors::tracing::js::JsInspector::new(code, config)
431                                    .map_err(Eth::Error::from_eth_err)?;
432                            let (res, _) = this.eth_api().inspect(
433                                &mut *db,
434                                evm_env.clone(),
435                                tx_env.clone(),
436                                &mut inspector,
437                            )?;
438                            inspector
439                                .json_result(res, &tx_env, &evm_env.block_env, db)
440                                .map_err(Eth::Error::from_eth_err)
441                        })
442                        .await?;
443
444                    Ok(GethTrace::JS(res))
445                }
446            }
447        }
448
449        // default structlog tracer
450        let inspector_config = TracingInspectorConfig::from_geth_config(&config);
451
452        let mut inspector = TracingInspector::new(inspector_config);
453
454        let (res, tx_gas_limit, inspector) = self
455            .eth_api()
456            .spawn_with_call_at(call, at, overrides, move |db, evm_env, tx_env| {
457                let (res, (_, tx_env)) =
458                    this.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?;
459                Ok((res, tx_env.gas_limit(), inspector))
460            })
461            .await?;
462        let gas_used = res.result.gas_used();
463        let return_value = res.result.into_output().unwrap_or_default();
464        let frame = inspector
465            .with_transaction_gas_limit(tx_gas_limit)
466            .into_geth_builder()
467            .geth_traces(gas_used, return_value, config);
468
469        Ok(frame.into())
470    }
471
472    /// The `debug_traceCallMany` method lets you run an `eth_callMany` within the context of the
473    /// given block execution using the first n transactions in the given block as base.
474    /// Each following bundle increments block number by 1 and block timestamp by 12 seconds
475    pub async fn debug_trace_call_many(
476        &self,
477        bundles: Vec<Bundle>,
478        state_context: Option<StateContext>,
479        opts: Option<GethDebugTracingCallOptions>,
480    ) -> Result<Vec<Vec<GethTrace>>, Eth::Error> {
481        if bundles.is_empty() {
482            return Err(EthApiError::InvalidParams(String::from("bundles are empty.")).into())
483        }
484
485        let StateContext { transaction_index, block_number } = state_context.unwrap_or_default();
486        let transaction_index = transaction_index.unwrap_or_default();
487
488        let target_block = block_number.unwrap_or_default();
489        let ((mut evm_env, _), block) = futures::try_join!(
490            self.eth_api().evm_env_at(target_block),
491            self.eth_api().recovered_block(target_block),
492        )?;
493
494        let opts = opts.unwrap_or_default();
495        let block = block.ok_or(EthApiError::HeaderNotFound(target_block))?;
496        let GethDebugTracingCallOptions { tracing_options, mut state_overrides, .. } = opts;
497
498        // we're essentially replaying the transactions in the block here, hence we need the state
499        // that points to the beginning of the block, which is the state at the parent block
500        let mut at = block.parent_hash();
501        let mut replay_block_txs = true;
502
503        // if a transaction index is provided, we need to replay the transactions until the index
504        let num_txs =
505            transaction_index.index().unwrap_or_else(|| block.body().transactions().len());
506        // but if all transactions are to be replayed, we can use the state at the block itself
507        // this works with the exception of the PENDING block, because its state might not exist if
508        // built locally
509        if !target_block.is_pending() && num_txs == block.body().transactions().len() {
510            at = block.hash();
511            replay_block_txs = false;
512        }
513
514        let this = self.clone();
515
516        self.eth_api()
517            .spawn_with_state_at_block(at.into(), move |state| {
518                // the outer vec for the bundles
519                let mut all_bundles = Vec::with_capacity(bundles.len());
520                let mut db = CacheDB::new(StateProviderDatabase::new(state));
521
522                if replay_block_txs {
523                    // only need to replay the transactions in the block if not all transactions are
524                    // to be replayed
525                    let transactions = block.transactions_recovered().take(num_txs);
526
527                    // Execute all transactions until index
528                    for tx in transactions {
529                        let tx_env = this.eth_api().evm_config().tx_env(tx);
530                        let (res, _) = this.eth_api().transact(&mut db, evm_env.clone(), tx_env)?;
531                        db.commit(res.state);
532                    }
533                }
534
535                // Trace all bundles
536                let mut bundles = bundles.into_iter().peekable();
537                while let Some(bundle) = bundles.next() {
538                    let mut results = Vec::with_capacity(bundle.transactions.len());
539                    let Bundle { transactions, block_override } = bundle;
540
541                    let block_overrides = block_override.map(Box::new);
542                    let mut inspector = None;
543
544                    let mut transactions = transactions.into_iter().peekable();
545                    while let Some(tx) = transactions.next() {
546                        // apply state overrides only once, before the first transaction
547                        let state_overrides = state_overrides.take();
548                        let overrides = EvmOverrides::new(state_overrides, block_overrides.clone());
549
550                        let (evm_env, tx_env) = this.eth_api().prepare_call_env(
551                            evm_env.clone(),
552                            tx,
553                            &mut db,
554                            overrides,
555                        )?;
556
557                        let (trace, state) = this.trace_transaction(
558                            &tracing_options,
559                            evm_env,
560                            tx_env,
561                            &mut db,
562                            None,
563                            &mut inspector,
564                        )?;
565
566                        inspector = inspector.map(|insp| insp.fused());
567
568                        // If there is more transactions, commit the database
569                        // If there is no transactions, but more bundles, commit to the database too
570                        if transactions.peek().is_some() || bundles.peek().is_some() {
571                            db.commit(state);
572                        }
573                        results.push(trace);
574                    }
575                    // Increment block_env number and timestamp for the next bundle
576                    evm_env.block_env.number += 1;
577                    evm_env.block_env.timestamp += 12;
578
579                    all_bundles.push(results);
580                }
581                Ok(all_bundles)
582            })
583            .await
584    }
585
586    /// Generates an execution witness for the given block hash. see
587    /// [`Self::debug_execution_witness`] for more info.
588    pub async fn debug_execution_witness_by_block_hash(
589        &self,
590        hash: B256,
591    ) -> Result<ExecutionWitness, Eth::Error> {
592        let this = self.clone();
593        let block = this
594            .eth_api()
595            .recovered_block(hash.into())
596            .await?
597            .ok_or(EthApiError::HeaderNotFound(hash.into()))?;
598
599        self.debug_execution_witness_for_block(block).await
600    }
601
602    /// The `debug_executionWitness` method allows for re-execution of a block with the purpose of
603    /// generating an execution witness. The witness comprises of a map of all hashed trie nodes to
604    /// their preimages that were required during the execution of the block, including during state
605    /// root recomputation.
606    pub async fn debug_execution_witness(
607        &self,
608        block_id: BlockNumberOrTag,
609    ) -> Result<ExecutionWitness, Eth::Error> {
610        let this = self.clone();
611        let block = this
612            .eth_api()
613            .recovered_block(block_id.into())
614            .await?
615            .ok_or(EthApiError::HeaderNotFound(block_id.into()))?;
616
617        self.debug_execution_witness_for_block(block).await
618    }
619
620    /// Generates an execution witness, using the given recovered block.
621    pub async fn debug_execution_witness_for_block(
622        &self,
623        block: Arc<RecoveredBlock<ProviderBlock<Eth::Provider>>>,
624    ) -> Result<ExecutionWitness, Eth::Error> {
625        let this = self.clone();
626        let block_number = block.header().number();
627
628        let (mut exec_witness, lowest_block_number) = self
629            .eth_api()
630            .spawn_with_state_at_block(block.parent_hash().into(), move |state_provider| {
631                let db = StateProviderDatabase::new(&state_provider);
632                let block_executor = this.inner.evm_config.batch_executor(db);
633
634                let mut witness_record = ExecutionWitnessRecord::default();
635
636                let _ = block_executor
637                    .execute_with_state_closure(&(*block).clone(), |statedb: &State<_>| {
638                        witness_record.record_executed_state(statedb);
639                    })
640                    .map_err(|err| EthApiError::Internal(err.into()))?;
641
642                let ExecutionWitnessRecord { hashed_state, codes, keys, lowest_block_number } =
643                    witness_record;
644
645                let state = state_provider
646                    .witness(Default::default(), hashed_state)
647                    .map_err(EthApiError::from)?;
648                Ok((
649                    ExecutionWitness { state, codes, keys, ..Default::default() },
650                    lowest_block_number,
651                ))
652            })
653            .await?;
654
655        let smallest = match lowest_block_number {
656            Some(smallest) => smallest,
657            None => {
658                // Return only the parent header, if there were no calls to the
659                // BLOCKHASH opcode.
660                block_number.saturating_sub(1)
661            }
662        };
663
664        let range = smallest..block_number;
665        // TODO: Check if headers_range errors when one of the headers in the range is missing
666        exec_witness.headers = self
667            .provider()
668            .headers_range(range)
669            .map_err(EthApiError::from)?
670            .into_iter()
671            .map(|header| {
672                let mut serialized_header = Vec::new();
673                header.encode(&mut serialized_header);
674                serialized_header.into()
675            })
676            .collect();
677
678        Ok(exec_witness)
679    }
680
681    /// Returns the code associated with a given hash at the specified block ID. If no code is
682    /// found, it returns None. If no block ID is provided, it defaults to the latest block.
683    pub async fn debug_code_by_hash(
684        &self,
685        hash: B256,
686        block_id: Option<BlockId>,
687    ) -> Result<Option<Bytes>, Eth::Error> {
688        Ok(self
689            .provider()
690            .state_by_block_id(block_id.unwrap_or_default())
691            .map_err(Eth::Error::from_eth_err)?
692            .bytecode_by_hash(&hash)
693            .map_err(Eth::Error::from_eth_err)?
694            .map(|b| b.original_bytes()))
695    }
696
697    /// Executes the configured transaction with the environment on the given database.
698    ///
699    /// It optionally takes fused inspector ([`TracingInspector::fused`]) to avoid re-creating the
700    /// inspector for each transaction. This is useful when tracing multiple transactions in a
701    /// block. This is only useful for block tracing which uses the same tracer for all transactions
702    /// in the block.
703    ///
704    /// Caution: If the inspector is provided then `opts.tracer_config` is ignored.
705    ///
706    /// Returns the trace frame and the state that got updated after executing the transaction.
707    ///
708    /// Note: this does not apply any state overrides if they're configured in the `opts`.
709    ///
710    /// Caution: this is blocking and should be performed on a blocking task.
711    fn trace_transaction(
712        &self,
713        opts: &GethDebugTracingOptions,
714        evm_env: EvmEnvFor<Eth::Evm>,
715        tx_env: TxEnvFor<Eth::Evm>,
716        db: &mut StateCacheDb<'_>,
717        transaction_context: Option<TransactionContext>,
718        fused_inspector: &mut Option<TracingInspector>,
719    ) -> Result<(GethTrace, EvmState), Eth::Error> {
720        let GethDebugTracingOptions { config, tracer, tracer_config, .. } = opts;
721
722        let tx_info = TransactionInfo {
723            hash: transaction_context.as_ref().map(|c| c.tx_hash).unwrap_or_default(),
724            index: transaction_context
725                .as_ref()
726                .map(|c| c.tx_index.map(|i| i as u64))
727                .unwrap_or_default(),
728            block_hash: transaction_context.as_ref().map(|c| c.block_hash).unwrap_or_default(),
729            block_number: Some(evm_env.block_env.number),
730            base_fee: Some(evm_env.block_env.basefee),
731        };
732
733        if let Some(tracer) = tracer {
734            return match tracer {
735                GethDebugTracerType::BuiltInTracer(tracer) => match tracer {
736                    GethDebugBuiltInTracerType::FourByteTracer => {
737                        let mut inspector = FourByteInspector::default();
738                        let (res, _) =
739                            self.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?;
740                        return Ok((FourByteFrame::from(&inspector).into(), res.state))
741                    }
742                    GethDebugBuiltInTracerType::CallTracer => {
743                        let call_config = tracer_config
744                            .clone()
745                            .into_call_config()
746                            .map_err(|_| EthApiError::InvalidTracerConfig)?;
747
748                        let mut inspector = fused_inspector.get_or_insert_with(|| {
749                            TracingInspector::new(TracingInspectorConfig::from_geth_call_config(
750                                &call_config,
751                            ))
752                        });
753
754                        let (res, (_, tx_env)) =
755                            self.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?;
756
757                        inspector.set_transaction_gas_limit(tx_env.gas_limit());
758
759                        let frame = inspector
760                            .geth_builder()
761                            .geth_call_traces(call_config, res.result.gas_used());
762
763                        return Ok((frame.into(), res.state))
764                    }
765                    GethDebugBuiltInTracerType::PreStateTracer => {
766                        let prestate_config = tracer_config
767                            .clone()
768                            .into_pre_state_config()
769                            .map_err(|_| EthApiError::InvalidTracerConfig)?;
770
771                        let mut inspector = fused_inspector.get_or_insert_with(|| {
772                            TracingInspector::new(
773                                TracingInspectorConfig::from_geth_prestate_config(&prestate_config),
774                            )
775                        });
776                        let (res, (_, tx_env)) =
777                            self.eth_api().inspect(&mut *db, evm_env, tx_env, &mut inspector)?;
778
779                        inspector.set_transaction_gas_limit(tx_env.gas_limit());
780                        let frame = inspector
781                            .geth_builder()
782                            .geth_prestate_traces(&res, &prestate_config, db)
783                            .map_err(Eth::Error::from_eth_err)?;
784
785                        return Ok((frame.into(), res.state))
786                    }
787                    GethDebugBuiltInTracerType::NoopTracer => {
788                        Ok((NoopFrame::default().into(), Default::default()))
789                    }
790                    GethDebugBuiltInTracerType::MuxTracer => {
791                        let mux_config = tracer_config
792                            .clone()
793                            .into_mux_config()
794                            .map_err(|_| EthApiError::InvalidTracerConfig)?;
795
796                        let mut inspector = MuxInspector::try_from_config(mux_config)
797                            .map_err(Eth::Error::from_eth_err)?;
798
799                        let (res, _) =
800                            self.eth_api().inspect(&mut *db, evm_env, tx_env, &mut inspector)?;
801                        let frame = inspector
802                            .try_into_mux_frame(&res, db, tx_info)
803                            .map_err(Eth::Error::from_eth_err)?;
804                        return Ok((frame.into(), res.state))
805                    }
806                    GethDebugBuiltInTracerType::FlatCallTracer => {
807                        let flat_call_config = tracer_config
808                            .clone()
809                            .into_flat_call_config()
810                            .map_err(|_| EthApiError::InvalidTracerConfig)?;
811
812                        let mut inspector = TracingInspector::new(
813                            TracingInspectorConfig::from_flat_call_config(&flat_call_config),
814                        );
815
816                        let (res, (_, tx_env)) =
817                            self.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?;
818                        let frame: FlatCallFrame = inspector
819                            .with_transaction_gas_limit(tx_env.gas_limit())
820                            .into_parity_builder()
821                            .into_localized_transaction_traces(tx_info);
822
823                        return Ok((frame.into(), res.state));
824                    }
825                },
826                #[cfg(not(feature = "js-tracer"))]
827                GethDebugTracerType::JsTracer(_) => {
828                    Err(EthApiError::Unsupported("JS Tracer is not enabled").into())
829                }
830                #[cfg(feature = "js-tracer")]
831                GethDebugTracerType::JsTracer(code) => {
832                    let config = tracer_config.clone().into_json();
833                    let mut inspector =
834                        revm_inspectors::tracing::js::JsInspector::with_transaction_context(
835                            code.clone(),
836                            config,
837                            transaction_context.unwrap_or_default(),
838                        )
839                        .map_err(Eth::Error::from_eth_err)?;
840                    let (res, (evm_env, tx_env)) =
841                        self.eth_api().inspect(&mut *db, evm_env, tx_env, &mut inspector)?;
842
843                    let state = res.state.clone();
844                    let result = inspector
845                        .json_result(res, &tx_env, &evm_env.block_env, db)
846                        .map_err(Eth::Error::from_eth_err)?;
847                    Ok((GethTrace::JS(result), state))
848                }
849            }
850        }
851
852        // default structlog tracer
853        let mut inspector = fused_inspector.get_or_insert_with(|| {
854            let inspector_config = TracingInspectorConfig::from_geth_config(config);
855            TracingInspector::new(inspector_config)
856        });
857        let (res, (_, tx_env)) = self.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?;
858        let gas_used = res.result.gas_used();
859        let return_value = res.result.into_output().unwrap_or_default();
860        inspector.set_transaction_gas_limit(tx_env.gas_limit());
861        let frame = inspector.geth_builder().geth_traces(gas_used, return_value, *config);
862
863        Ok((frame.into(), res.state))
864    }
865}
866
867#[async_trait]
868impl<Eth, Evm> DebugApiServer for DebugApi<Eth, Evm>
869where
870    Eth: EthApiTypes + EthTransactions + TraceExt + 'static,
871    Evm: ConfigureEvm<Primitives: NodePrimitives<Block = ProviderBlock<Eth::Provider>>> + 'static,
872{
873    /// Handler for `debug_getRawHeader`
874    async fn raw_header(&self, block_id: BlockId) -> RpcResult<Bytes> {
875        let header = match block_id {
876            BlockId::Hash(hash) => self.provider().header(&hash.into()).to_rpc_result()?,
877            BlockId::Number(number_or_tag) => {
878                let number = self
879                    .provider()
880                    .convert_block_number(number_or_tag)
881                    .to_rpc_result()?
882                    .ok_or_else(|| {
883                    internal_rpc_err("Pending block not supported".to_string())
884                })?;
885                self.provider().header_by_number(number).to_rpc_result()?
886            }
887        };
888
889        let mut res = Vec::new();
890        if let Some(header) = header {
891            header.encode(&mut res);
892        }
893
894        Ok(res.into())
895    }
896
897    /// Handler for `debug_getRawBlock`
898    async fn raw_block(&self, block_id: BlockId) -> RpcResult<Bytes> {
899        let block = self
900            .provider()
901            .block_by_id(block_id)
902            .to_rpc_result()?
903            .ok_or(EthApiError::HeaderNotFound(block_id))?;
904        let mut res = Vec::new();
905        block.encode(&mut res);
906        Ok(res.into())
907    }
908
909    /// Handler for `debug_getRawTransaction`
910    ///
911    /// If this is a pooled EIP-4844 transaction, the blob sidecar is included.
912    ///
913    /// Returns the bytes of the transaction for the given hash.
914    async fn raw_transaction(&self, hash: B256) -> RpcResult<Option<Bytes>> {
915        self.eth_api().raw_transaction_by_hash(hash).await.map_err(Into::into)
916    }
917
918    /// Handler for `debug_getRawTransactions`
919    /// Returns the bytes of the transaction for the given hash.
920    async fn raw_transactions(&self, block_id: BlockId) -> RpcResult<Vec<Bytes>> {
921        let block = self
922            .provider()
923            .block_with_senders_by_id(block_id, TransactionVariant::NoHash)
924            .to_rpc_result()?
925            .unwrap_or_default();
926        Ok(block.into_transactions_recovered().map(|tx| tx.encoded_2718().into()).collect())
927    }
928
929    /// Handler for `debug_getRawReceipts`
930    async fn raw_receipts(&self, block_id: BlockId) -> RpcResult<Vec<Bytes>> {
931        Ok(self
932            .provider()
933            .receipts_by_block_id(block_id)
934            .to_rpc_result()?
935            .unwrap_or_default()
936            .into_iter()
937            .map(|receipt| ReceiptWithBloom::from(receipt).encoded_2718().into())
938            .collect())
939    }
940
941    /// Handler for `debug_getBadBlocks`
942    async fn bad_blocks(&self) -> RpcResult<Vec<RpcBlock>> {
943        Ok(vec![])
944    }
945
946    /// Handler for `debug_traceChain`
947    async fn debug_trace_chain(
948        &self,
949        _start_exclusive: BlockNumberOrTag,
950        _end_inclusive: BlockNumberOrTag,
951    ) -> RpcResult<Vec<BlockTraceResult>> {
952        Err(internal_rpc_err("unimplemented"))
953    }
954
955    /// Handler for `debug_traceBlock`
956    async fn debug_trace_block(
957        &self,
958        rlp_block: Bytes,
959        opts: Option<GethDebugTracingOptions>,
960    ) -> RpcResult<Vec<TraceResult>> {
961        let _permit = self.acquire_trace_permit().await;
962        Self::debug_trace_raw_block(self, rlp_block, opts.unwrap_or_default())
963            .await
964            .map_err(Into::into)
965    }
966
967    /// Handler for `debug_traceBlockByHash`
968    async fn debug_trace_block_by_hash(
969        &self,
970        block: B256,
971        opts: Option<GethDebugTracingOptions>,
972    ) -> RpcResult<Vec<TraceResult>> {
973        let _permit = self.acquire_trace_permit().await;
974        Self::debug_trace_block(self, block.into(), opts.unwrap_or_default())
975            .await
976            .map_err(Into::into)
977    }
978
979    /// Handler for `debug_traceBlockByNumber`
980    async fn debug_trace_block_by_number(
981        &self,
982        block: BlockNumberOrTag,
983        opts: Option<GethDebugTracingOptions>,
984    ) -> RpcResult<Vec<TraceResult>> {
985        let _permit = self.acquire_trace_permit().await;
986        Self::debug_trace_block(self, block.into(), opts.unwrap_or_default())
987            .await
988            .map_err(Into::into)
989    }
990
991    /// Handler for `debug_traceTransaction`
992    async fn debug_trace_transaction(
993        &self,
994        tx_hash: B256,
995        opts: Option<GethDebugTracingOptions>,
996    ) -> RpcResult<GethTrace> {
997        let _permit = self.acquire_trace_permit().await;
998        Self::debug_trace_transaction(self, tx_hash, opts.unwrap_or_default())
999            .await
1000            .map_err(Into::into)
1001    }
1002
1003    /// Handler for `debug_traceCall`
1004    async fn debug_trace_call(
1005        &self,
1006        request: TransactionRequest,
1007        block_id: Option<BlockId>,
1008        opts: Option<GethDebugTracingCallOptions>,
1009    ) -> RpcResult<GethTrace> {
1010        let _permit = self.acquire_trace_permit().await;
1011        Self::debug_trace_call(self, request, block_id, opts.unwrap_or_default())
1012            .await
1013            .map_err(Into::into)
1014    }
1015
1016    async fn debug_trace_call_many(
1017        &self,
1018        bundles: Vec<Bundle>,
1019        state_context: Option<StateContext>,
1020        opts: Option<GethDebugTracingCallOptions>,
1021    ) -> RpcResult<Vec<Vec<GethTrace>>> {
1022        let _permit = self.acquire_trace_permit().await;
1023        Self::debug_trace_call_many(self, bundles, state_context, opts).await.map_err(Into::into)
1024    }
1025
1026    /// Handler for `debug_executionWitness`
1027    async fn debug_execution_witness(
1028        &self,
1029        block: BlockNumberOrTag,
1030    ) -> RpcResult<ExecutionWitness> {
1031        let _permit = self.acquire_trace_permit().await;
1032        Self::debug_execution_witness(self, block).await.map_err(Into::into)
1033    }
1034
1035    /// Handler for `debug_executionWitnessByBlockHash`
1036    async fn debug_execution_witness_by_block_hash(
1037        &self,
1038        hash: B256,
1039    ) -> RpcResult<ExecutionWitness> {
1040        let _permit = self.acquire_trace_permit().await;
1041        Self::debug_execution_witness_by_block_hash(self, hash).await.map_err(Into::into)
1042    }
1043
1044    async fn debug_backtrace_at(&self, _location: &str) -> RpcResult<()> {
1045        Ok(())
1046    }
1047
1048    async fn debug_account_range(
1049        &self,
1050        _block_number: BlockNumberOrTag,
1051        _start: Bytes,
1052        _max_results: u64,
1053        _nocode: bool,
1054        _nostorage: bool,
1055        _incompletes: bool,
1056    ) -> RpcResult<()> {
1057        Ok(())
1058    }
1059
1060    async fn debug_block_profile(&self, _file: String, _seconds: u64) -> RpcResult<()> {
1061        Ok(())
1062    }
1063
1064    async fn debug_chaindb_compact(&self) -> RpcResult<()> {
1065        Ok(())
1066    }
1067
1068    async fn debug_chain_config(&self) -> RpcResult<ChainConfig> {
1069        Ok(self.provider().chain_spec().genesis().config.clone())
1070    }
1071
1072    async fn debug_chaindb_property(&self, _property: String) -> RpcResult<()> {
1073        Ok(())
1074    }
1075
1076    async fn debug_code_by_hash(
1077        &self,
1078        hash: B256,
1079        block_id: Option<BlockId>,
1080    ) -> RpcResult<Option<Bytes>> {
1081        Self::debug_code_by_hash(self, hash, block_id).await.map_err(Into::into)
1082    }
1083
1084    async fn debug_cpu_profile(&self, _file: String, _seconds: u64) -> RpcResult<()> {
1085        Ok(())
1086    }
1087
1088    async fn debug_db_ancient(&self, _kind: String, _number: u64) -> RpcResult<()> {
1089        Ok(())
1090    }
1091
1092    async fn debug_db_ancients(&self) -> RpcResult<()> {
1093        Ok(())
1094    }
1095
1096    async fn debug_db_get(&self, _key: String) -> RpcResult<()> {
1097        Ok(())
1098    }
1099
1100    async fn debug_dump_block(&self, _number: BlockId) -> RpcResult<()> {
1101        Ok(())
1102    }
1103
1104    async fn debug_free_os_memory(&self) -> RpcResult<()> {
1105        Ok(())
1106    }
1107
1108    async fn debug_freeze_client(&self, _node: String) -> RpcResult<()> {
1109        Ok(())
1110    }
1111
1112    async fn debug_gc_stats(&self) -> RpcResult<()> {
1113        Ok(())
1114    }
1115
1116    async fn debug_get_accessible_state(
1117        &self,
1118        _from: BlockNumberOrTag,
1119        _to: BlockNumberOrTag,
1120    ) -> RpcResult<()> {
1121        Ok(())
1122    }
1123
1124    async fn debug_get_modified_accounts_by_hash(
1125        &self,
1126        _start_hash: B256,
1127        _end_hash: B256,
1128    ) -> RpcResult<()> {
1129        Ok(())
1130    }
1131
1132    async fn debug_get_modified_accounts_by_number(
1133        &self,
1134        _start_number: u64,
1135        _end_number: u64,
1136    ) -> RpcResult<()> {
1137        Ok(())
1138    }
1139
1140    async fn debug_go_trace(&self, _file: String, _seconds: u64) -> RpcResult<()> {
1141        Ok(())
1142    }
1143
1144    async fn debug_intermediate_roots(
1145        &self,
1146        _block_hash: B256,
1147        _opts: Option<GethDebugTracingCallOptions>,
1148    ) -> RpcResult<()> {
1149        Ok(())
1150    }
1151
1152    async fn debug_mem_stats(&self) -> RpcResult<()> {
1153        Ok(())
1154    }
1155
1156    async fn debug_mutex_profile(&self, _file: String, _nsec: u64) -> RpcResult<()> {
1157        Ok(())
1158    }
1159
1160    async fn debug_preimage(&self, _hash: B256) -> RpcResult<()> {
1161        Ok(())
1162    }
1163
1164    async fn debug_print_block(&self, _number: u64) -> RpcResult<()> {
1165        Ok(())
1166    }
1167
1168    async fn debug_seed_hash(&self, _number: u64) -> RpcResult<B256> {
1169        Ok(Default::default())
1170    }
1171
1172    async fn debug_set_block_profile_rate(&self, _rate: u64) -> RpcResult<()> {
1173        Ok(())
1174    }
1175
1176    async fn debug_set_gc_percent(&self, _v: i32) -> RpcResult<()> {
1177        Ok(())
1178    }
1179
1180    async fn debug_set_head(&self, _number: u64) -> RpcResult<()> {
1181        Ok(())
1182    }
1183
1184    async fn debug_set_mutex_profile_fraction(&self, _rate: i32) -> RpcResult<()> {
1185        Ok(())
1186    }
1187
1188    async fn debug_set_trie_flush_interval(&self, _interval: String) -> RpcResult<()> {
1189        Ok(())
1190    }
1191
1192    async fn debug_stacks(&self) -> RpcResult<()> {
1193        Ok(())
1194    }
1195
1196    async fn debug_standard_trace_bad_block_to_file(
1197        &self,
1198        _block: BlockNumberOrTag,
1199        _opts: Option<GethDebugTracingCallOptions>,
1200    ) -> RpcResult<()> {
1201        Ok(())
1202    }
1203
1204    async fn debug_standard_trace_block_to_file(
1205        &self,
1206        _block: BlockNumberOrTag,
1207        _opts: Option<GethDebugTracingCallOptions>,
1208    ) -> RpcResult<()> {
1209        Ok(())
1210    }
1211
1212    async fn debug_start_cpu_profile(&self, _file: String) -> RpcResult<()> {
1213        Ok(())
1214    }
1215
1216    async fn debug_start_go_trace(&self, _file: String) -> RpcResult<()> {
1217        Ok(())
1218    }
1219
1220    async fn debug_stop_cpu_profile(&self) -> RpcResult<()> {
1221        Ok(())
1222    }
1223
1224    async fn debug_stop_go_trace(&self) -> RpcResult<()> {
1225        Ok(())
1226    }
1227
1228    async fn debug_storage_range_at(
1229        &self,
1230        _block_hash: B256,
1231        _tx_idx: usize,
1232        _contract_address: Address,
1233        _key_start: B256,
1234        _max_result: u64,
1235    ) -> RpcResult<()> {
1236        Ok(())
1237    }
1238
1239    async fn debug_trace_bad_block(
1240        &self,
1241        _block_hash: B256,
1242        _opts: Option<GethDebugTracingCallOptions>,
1243    ) -> RpcResult<()> {
1244        Ok(())
1245    }
1246
1247    async fn debug_verbosity(&self, _level: usize) -> RpcResult<()> {
1248        Ok(())
1249    }
1250
1251    async fn debug_vmodule(&self, _pattern: String) -> RpcResult<()> {
1252        Ok(())
1253    }
1254
1255    async fn debug_write_block_profile(&self, _file: String) -> RpcResult<()> {
1256        Ok(())
1257    }
1258
1259    async fn debug_write_mem_profile(&self, _file: String) -> RpcResult<()> {
1260        Ok(())
1261    }
1262
1263    async fn debug_write_mutex_profile(&self, _file: String) -> RpcResult<()> {
1264        Ok(())
1265    }
1266}
1267
1268impl<Eth, Evm> std::fmt::Debug for DebugApi<Eth, Evm> {
1269    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
1270        f.debug_struct("DebugApi").finish_non_exhaustive()
1271    }
1272}
1273
1274impl<Eth, Evm> Clone for DebugApi<Eth, Evm> {
1275    fn clone(&self) -> Self {
1276        Self { inner: Arc::clone(&self.inner) }
1277    }
1278}
1279
1280struct DebugApiInner<Eth, Evm> {
1281    /// The implementation of `eth` API
1282    eth_api: Eth,
1283    // restrict the number of concurrent calls to blocking calls
1284    blocking_task_guard: BlockingTaskGuard,
1285    /// block executor for debug & trace apis
1286    evm_config: Evm,
1287}