reth_rpc/
debug.rs

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