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