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