reth_rpc/
trace.rs

1use alloy_consensus::BlockHeader as _;
2use alloy_eips::BlockId;
3use alloy_primitives::{map::HashSet, Bytes, B256, U256};
4use alloy_rpc_types_eth::{
5    state::{EvmOverrides, StateOverride},
6    transaction::TransactionRequest,
7    BlockOverrides, Index,
8};
9use alloy_rpc_types_trace::{
10    filter::TraceFilter,
11    opcode::{BlockOpcodeGas, TransactionOpcodeGas},
12    parity::*,
13    tracerequest::TraceCallRequest,
14};
15use async_trait::async_trait;
16use jsonrpsee::core::RpcResult;
17use reth_chainspec::EthereumHardforks;
18use reth_consensus_common::calc::{
19    base_block_reward, base_block_reward_pre_merge, block_reward, ommer_reward,
20};
21use reth_evm::ConfigureEvmEnv;
22use reth_primitives_traits::{BlockBody, BlockHeader};
23use reth_provider::{BlockNumReader, BlockReader, ChainSpecProvider, HeaderProvider};
24use reth_revm::database::StateProviderDatabase;
25use reth_rpc_api::TraceApiServer;
26use reth_rpc_eth_api::{helpers::TraceExt, FromEthApiError, RpcNodeCore};
27use reth_rpc_eth_types::{error::EthApiError, utils::recover_raw_transaction};
28use reth_tasks::pool::BlockingTaskGuard;
29use reth_transaction_pool::{PoolPooledTx, PoolTransaction, TransactionPool};
30use revm::{
31    db::{CacheDB, DatabaseCommit},
32    primitives::EnvWithHandlerCfg,
33};
34use revm_inspectors::{
35    opcode::OpcodeGasInspector,
36    tracing::{parity::populate_state_diff, TracingInspector, TracingInspectorConfig},
37};
38use std::sync::Arc;
39use tokio::sync::{AcquireError, OwnedSemaphorePermit};
40
41/// `trace` API implementation.
42///
43/// This type provides the functionality for handling `trace` related requests.
44pub struct TraceApi<Eth> {
45    inner: Arc<TraceApiInner<Eth>>,
46}
47
48// === impl TraceApi ===
49
50impl<Eth> TraceApi<Eth> {
51    /// Create a new instance of the [`TraceApi`]
52    pub fn new(eth_api: Eth, blocking_task_guard: BlockingTaskGuard) -> Self {
53        let inner = Arc::new(TraceApiInner { eth_api, blocking_task_guard });
54        Self { inner }
55    }
56
57    /// Acquires a permit to execute a tracing call.
58    async fn acquire_trace_permit(
59        &self,
60    ) -> std::result::Result<OwnedSemaphorePermit, AcquireError> {
61        self.inner.blocking_task_guard.clone().acquire_owned().await
62    }
63
64    /// Access the underlying `Eth` API.
65    pub fn eth_api(&self) -> &Eth {
66        &self.inner.eth_api
67    }
68}
69
70impl<Eth: RpcNodeCore> TraceApi<Eth> {
71    /// Access the underlying provider.
72    pub fn provider(&self) -> &Eth::Provider {
73        self.inner.eth_api.provider()
74    }
75}
76
77// === impl TraceApi ===
78
79impl<Eth> TraceApi<Eth>
80where
81    Eth: TraceExt + 'static,
82{
83    /// Executes the given call and returns a number of possible traces for it.
84    pub async fn trace_call(
85        &self,
86        trace_request: TraceCallRequest,
87    ) -> Result<TraceResults, Eth::Error> {
88        let at = trace_request.block_id.unwrap_or_default();
89        let config = TracingInspectorConfig::from_parity_config(&trace_request.trace_types);
90        let overrides =
91            EvmOverrides::new(trace_request.state_overrides, trace_request.block_overrides);
92        let mut inspector = TracingInspector::new(config);
93        let this = self.clone();
94        self.eth_api()
95            .spawn_with_call_at(trace_request.call, at, overrides, move |db, env| {
96                // wrapper is hack to get around 'higher-ranked lifetime error', see
97                // <https://github.com/rust-lang/rust/issues/100013>
98                let db = db.0;
99
100                let (res, _) = this.eth_api().inspect(&mut *db, env, &mut inspector)?;
101                let trace_res = inspector
102                    .into_parity_builder()
103                    .into_trace_results_with_state(&res, &trace_request.trace_types, &db)
104                    .map_err(Eth::Error::from_eth_err)?;
105                Ok(trace_res)
106            })
107            .await
108    }
109
110    /// Traces a call to `eth_sendRawTransaction` without making the call, returning the traces.
111    pub async fn trace_raw_transaction(
112        &self,
113        tx: Bytes,
114        trace_types: HashSet<TraceType>,
115        block_id: Option<BlockId>,
116    ) -> Result<TraceResults, Eth::Error> {
117        let tx = recover_raw_transaction::<PoolPooledTx<Eth::Pool>>(&tx)?
118            .map_transaction(<Eth::Pool as TransactionPool>::Transaction::pooled_into_consensus);
119
120        let (cfg, block, at) = self.eth_api().evm_env_at(block_id.unwrap_or_default()).await?;
121
122        let env = EnvWithHandlerCfg::new_with_cfg_env(
123            cfg,
124            block,
125            self.eth_api()
126                .evm_config()
127                .tx_env(tx.as_signed(), tx.signer())
128                .map_err(|_| EthApiError::FailedToDecodeSignedTransaction)?,
129        );
130
131        let config = TracingInspectorConfig::from_parity_config(&trace_types);
132
133        let trace = self
134            .eth_api()
135            .spawn_trace_at_with_state(env, config, at, move |inspector, res, db| {
136                inspector
137                    .into_parity_builder()
138                    .into_trace_results_with_state(&res, &trace_types, &db)
139                    .map_err(Eth::Error::from_eth_err)
140            })
141            .await?;
142        Ok(trace.shield_inputs())
143    }
144
145    /// Performs multiple call traces on top of the same block. i.e. transaction n will be executed
146    /// on top of a pending block with all n-1 transactions applied (traced) first.
147    ///
148    /// Note: Allows tracing dependent transactions, hence all transactions are traced in sequence
149    pub async fn trace_call_many(
150        &self,
151        calls: Vec<(TransactionRequest, HashSet<TraceType>)>,
152        block_id: Option<BlockId>,
153    ) -> Result<Vec<TraceResults>, Eth::Error> {
154        let at = block_id.unwrap_or(BlockId::pending());
155        let (cfg, block_env, at) = self.eth_api().evm_env_at(at).await?;
156
157        let this = self.clone();
158        // execute all transactions on top of each other and record the traces
159        self.eth_api()
160            .spawn_with_state_at_block(at, move |state| {
161                let mut results = Vec::with_capacity(calls.len());
162                let mut db = CacheDB::new(StateProviderDatabase::new(state));
163
164                let mut calls = calls.into_iter().peekable();
165
166                while let Some((call, trace_types)) = calls.next() {
167                    let env = this.eth_api().prepare_call_env(
168                        cfg.clone(),
169                        block_env.clone(),
170                        call,
171                        &mut db,
172                        Default::default(),
173                    )?;
174                    let config = TracingInspectorConfig::from_parity_config(&trace_types);
175                    let mut inspector = TracingInspector::new(config);
176                    let (res, _) = this.eth_api().inspect(&mut db, env, &mut inspector)?;
177
178                    let trace_res = inspector
179                        .into_parity_builder()
180                        .into_trace_results_with_state(&res, &trace_types, &db)
181                        .map_err(Eth::Error::from_eth_err)?;
182
183                    results.push(trace_res);
184
185                    // need to apply the state changes of this call before executing the
186                    // next call
187                    if calls.peek().is_some() {
188                        // need to apply the state changes of this call before executing
189                        // the next call
190                        db.commit(res.state)
191                    }
192                }
193
194                Ok(results)
195            })
196            .await
197    }
198
199    /// Replays a transaction, returning the traces.
200    pub async fn replay_transaction(
201        &self,
202        hash: B256,
203        trace_types: HashSet<TraceType>,
204    ) -> Result<TraceResults, Eth::Error> {
205        let config = TracingInspectorConfig::from_parity_config(&trace_types);
206        let trace = self
207            .eth_api()
208            .spawn_trace_transaction_in_block(hash, config, move |_, inspector, res, db| {
209                let trace_res = inspector
210                    .into_parity_builder()
211                    .into_trace_results_with_state(&res, &trace_types, &db)
212                    .map_err(Eth::Error::from_eth_err)?;
213                Ok(trace_res)
214            })
215            .await
216            .transpose()
217            .ok_or(EthApiError::TransactionNotFound)?;
218        Ok(trace?.shield_inputs())
219    }
220
221    /// Returns transaction trace objects at the given index
222    ///
223    /// Note: For compatibility reasons this only supports 1 single index, since this method is
224    /// supposed to return a single trace. See also: <https://github.com/ledgerwatch/erigon/blob/862faf054b8a0fa15962a9c73839b619886101eb/turbo/jsonrpc/trace_filtering.go#L114-L133>
225    ///
226    /// This returns `None` if `indices` is empty
227    pub async fn trace_get(
228        &self,
229        hash: B256,
230        indices: Vec<usize>,
231    ) -> Result<Option<LocalizedTransactionTrace>, Eth::Error> {
232        if indices.len() != 1 {
233            // The OG impl failed if it gets more than a single index
234            return Ok(None)
235        }
236        let trace = self.trace_get_index(hash, indices[0]).await?;
237        Ok(trace.map(|trace| trace.shield_inputs()))
238    }
239
240    /// Returns transaction trace object at the given index.
241    ///
242    /// Returns `None` if the trace object at that index does not exist
243    pub async fn trace_get_index(
244        &self,
245        hash: B256,
246        index: usize,
247    ) -> Result<Option<LocalizedTransactionTrace>, Eth::Error> {
248        Ok(self.trace_transaction(hash).await?.and_then(|traces| traces.into_iter().nth(index)))
249    }
250
251    /// Returns all transaction traces that match the given filter.
252    ///
253    /// This is similar to [`Self::trace_block`] but only returns traces for transactions that match
254    /// the filter.
255    pub async fn trace_filter(
256        &self,
257        filter: TraceFilter,
258    ) -> Result<Vec<LocalizedTransactionTrace>, Eth::Error> {
259        // We'll reuse the matcher across multiple blocks that are traced in parallel
260        let matcher = Arc::new(filter.matcher());
261        let TraceFilter { from_block, to_block, after, count, .. } = filter;
262        let start = from_block.unwrap_or(0);
263        let end = if let Some(to_block) = to_block {
264            to_block
265        } else {
266            self.provider().best_block_number().map_err(Eth::Error::from_eth_err)?
267        };
268
269        if start > end {
270            return Err(EthApiError::InvalidParams(
271                "invalid parameters: fromBlock cannot be greater than toBlock".to_string(),
272            )
273            .into())
274        }
275
276        // ensure that the range is not too large, since we need to fetch all blocks in the range
277        let distance = end.saturating_sub(start);
278        if distance > 100 {
279            return Err(EthApiError::InvalidParams(
280                "Block range too large; currently limited to 100 blocks".to_string(),
281            )
282            .into())
283        }
284
285        // fetch all blocks in that range
286        let blocks = self
287            .provider()
288            .sealed_block_with_senders_range(start..=end)
289            .map_err(Eth::Error::from_eth_err)?
290            .into_iter()
291            .map(Arc::new)
292            .collect::<Vec<_>>();
293
294        // trace all blocks
295        let mut block_traces = Vec::with_capacity(blocks.len());
296        for block in &blocks {
297            let matcher = matcher.clone();
298            let traces = self.eth_api().trace_block_until(
299                block.hash().into(),
300                Some(block.clone()),
301                None,
302                TracingInspectorConfig::default_parity(),
303                move |tx_info, inspector, _, _, _| {
304                    let mut traces =
305                        inspector.into_parity_builder().into_localized_transaction_traces(tx_info);
306                    traces.retain(|trace| matcher.matches(&trace.trace));
307                    Ok(Some(traces))
308                },
309            );
310            block_traces.push(traces);
311        }
312
313        let block_traces = futures::future::try_join_all(block_traces).await?;
314        let mut all_traces = block_traces
315            .into_iter()
316            .flatten()
317            .flat_map(|traces| traces.into_iter().flatten().flat_map(|traces| traces.into_iter()))
318            .collect::<Vec<_>>();
319
320        // add reward traces for all blocks
321        for block in &blocks {
322            if let Some(base_block_reward) =
323                self.calculate_base_block_reward(block.header.header())?
324            {
325                all_traces.extend(
326                    self.extract_reward_traces(
327                        block.header.header(),
328                        block.body.ommers(),
329                        base_block_reward,
330                    )
331                    .into_iter()
332                    .filter(|trace| matcher.matches(&trace.trace)),
333                );
334            } else {
335                // no block reward, means we're past the Paris hardfork and don't expect any rewards
336                // because the blocks in ascending order
337                break
338            }
339        }
340
341        // Skips the first `after` number of matching traces.
342        // If `after` is greater than or equal to the number of matched traces, it returns an empty
343        // array.
344        if let Some(after) = after.map(|a| a as usize) {
345            if after < all_traces.len() {
346                all_traces.drain(..after);
347            } else {
348                return Ok(vec![])
349            }
350        }
351
352        // Return at most `count` of traces
353        if let Some(count) = count {
354            let count = count as usize;
355            if count < all_traces.len() {
356                all_traces.truncate(count);
357            }
358        };
359
360        Ok(all_traces.into_iter().map(|trace| trace.shield_inputs()).collect())
361    }
362
363    /// Returns all traces for the given transaction hash
364    pub async fn trace_transaction(
365        &self,
366        hash: B256,
367    ) -> Result<Option<Vec<LocalizedTransactionTrace>>, Eth::Error> {
368        self.eth_api()
369            .spawn_trace_transaction_in_block(
370                hash,
371                TracingInspectorConfig::default_parity(),
372                move |tx_info, inspector, _, _| {
373                    let traces =
374                        inspector.into_parity_builder().into_localized_transaction_traces(tx_info);
375                    Ok(traces)
376                },
377            )
378            .await
379    }
380
381    /// Returns traces created at given block.
382    pub async fn trace_block(
383        &self,
384        block_id: BlockId,
385    ) -> Result<Option<Vec<LocalizedTransactionTrace>>, Eth::Error> {
386        let traces = self.eth_api().trace_block_with(
387            block_id,
388            None,
389            TracingInspectorConfig::default_parity(),
390            |tx_info, inspector, _, _, _| {
391                let traces =
392                    inspector.into_parity_builder().into_localized_transaction_traces(tx_info);
393                Ok(traces)
394            },
395        );
396
397        let block = self.eth_api().block_with_senders(block_id);
398        let (maybe_traces, maybe_block) = futures::try_join!(traces, block)?;
399
400        let mut maybe_traces =
401            maybe_traces.map(|traces| traces.into_iter().flatten().collect::<Vec<_>>());
402
403        if let (Some(block), Some(traces)) = (maybe_block, maybe_traces.as_mut()) {
404            if let Some(base_block_reward) =
405                self.calculate_base_block_reward(block.header.header())?
406            {
407                traces.extend(self.extract_reward_traces(
408                    block.block.header(),
409                    block.body.ommers(),
410                    base_block_reward,
411                ));
412            }
413        }
414
415        Ok(maybe_traces)
416    }
417
418    /// Replays all transactions in a block
419    pub async fn replay_block_transactions(
420        &self,
421        block_id: BlockId,
422        trace_types: HashSet<TraceType>,
423    ) -> Result<Option<Vec<TraceResultsWithTransactionHash>>, Eth::Error> {
424        self.eth_api()
425            .trace_block_with(
426                block_id,
427                None,
428                TracingInspectorConfig::from_parity_config(&trace_types),
429                move |tx_info, inspector, res, state, db| {
430                    let mut full_trace =
431                        inspector.into_parity_builder().into_trace_results(&res, &trace_types);
432
433                    // If statediffs were requested, populate them with the account balance and
434                    // nonce from pre-state
435                    if let Some(ref mut state_diff) = full_trace.state_diff {
436                        populate_state_diff(state_diff, db, state.iter())
437                            .map_err(Eth::Error::from_eth_err)?;
438                    }
439
440                    let trace = TraceResultsWithTransactionHash {
441                        transaction_hash: tx_info.hash.expect("tx hash is set"),
442                        full_trace,
443                    };
444                    Ok(trace)
445                },
446            )
447            .await
448    }
449
450    /// Returns all opcodes with their count and combined gas usage for the given transaction in no
451    /// particular order.
452    pub async fn trace_transaction_opcode_gas(
453        &self,
454        tx_hash: B256,
455    ) -> Result<Option<TransactionOpcodeGas>, Eth::Error> {
456        self.eth_api()
457            .spawn_trace_transaction_in_block_with_inspector(
458                tx_hash,
459                OpcodeGasInspector::default(),
460                move |_tx_info, inspector, _res, _| {
461                    let trace = TransactionOpcodeGas {
462                        transaction_hash: tx_hash,
463                        opcode_gas: inspector.opcode_gas_iter().collect(),
464                    };
465                    Ok(trace)
466                },
467            )
468            .await
469    }
470
471    /// Returns the opcodes of all transactions in the given block.
472    ///
473    /// This is the same as [`Self::trace_transaction_opcode_gas`] but for all transactions in a
474    /// block.
475    pub async fn trace_block_opcode_gas(
476        &self,
477        block_id: BlockId,
478    ) -> Result<Option<BlockOpcodeGas>, Eth::Error> {
479        let res = self
480            .eth_api()
481            .trace_block_inspector(
482                block_id,
483                None,
484                OpcodeGasInspector::default,
485                move |tx_info, inspector, _res, _, _| {
486                    let trace = TransactionOpcodeGas {
487                        transaction_hash: tx_info.hash.expect("tx hash is set"),
488                        opcode_gas: inspector.opcode_gas_iter().collect(),
489                    };
490                    Ok(trace)
491                },
492            )
493            .await?;
494
495        let Some(transactions) = res else { return Ok(None) };
496
497        let Some(block) = self.eth_api().block_with_senders(block_id).await? else {
498            return Ok(None)
499        };
500
501        Ok(Some(BlockOpcodeGas {
502            block_hash: block.hash(),
503            block_number: block.header.number(),
504            transactions,
505        }))
506    }
507
508    /// Calculates the base block reward for the given block:
509    ///
510    /// - if Paris hardfork is activated, no block rewards are given
511    /// - if Paris hardfork is not activated, calculate block rewards with block number only
512    /// - if Paris hardfork is unknown, calculate block rewards with block number and ttd
513    fn calculate_base_block_reward<H: BlockHeader>(
514        &self,
515        header: &H,
516    ) -> Result<Option<u128>, Eth::Error> {
517        let chain_spec = self.provider().chain_spec();
518        let is_paris_activated = chain_spec.is_paris_active_at_block(header.number());
519
520        Ok(match is_paris_activated {
521            Some(true) => None,
522            Some(false) => Some(base_block_reward_pre_merge(&chain_spec, header.number())),
523            None => {
524                // if Paris hardfork is unknown, we need to fetch the total difficulty at the
525                // block's height and check if it is pre-merge to calculate the base block reward
526                if let Some(header_td) = self
527                    .provider()
528                    .header_td_by_number(header.number())
529                    .map_err(Eth::Error::from_eth_err)?
530                {
531                    base_block_reward(
532                        chain_spec.as_ref(),
533                        header.number(),
534                        header.difficulty(),
535                        header_td,
536                    )
537                } else {
538                    None
539                }
540            }
541        })
542    }
543
544    /// Extracts the reward traces for the given block:
545    ///  - block reward
546    ///  - uncle rewards
547    fn extract_reward_traces<H: BlockHeader>(
548        &self,
549        header: &H,
550        ommers: Option<&[H]>,
551        base_block_reward: u128,
552    ) -> Vec<LocalizedTransactionTrace> {
553        let ommers_cnt = ommers.map(|o| o.len()).unwrap_or_default();
554        let mut traces = Vec::with_capacity(ommers_cnt + 1);
555
556        let block_reward = block_reward(base_block_reward, ommers_cnt);
557        traces.push(reward_trace(
558            header,
559            RewardAction {
560                author: header.beneficiary(),
561                reward_type: RewardType::Block,
562                value: U256::from(block_reward),
563            },
564        ));
565
566        let Some(ommers) = ommers else { return traces };
567
568        for uncle in ommers {
569            let uncle_reward = ommer_reward(base_block_reward, header.number(), uncle.number());
570            traces.push(reward_trace(
571                header,
572                RewardAction {
573                    author: uncle.beneficiary(),
574                    reward_type: RewardType::Uncle,
575                    value: U256::from(uncle_reward),
576                },
577            ));
578        }
579        traces
580    }
581}
582
583#[async_trait]
584impl<Eth> TraceApiServer for TraceApi<Eth>
585where
586    Eth: TraceExt + 'static,
587{
588    /// Executes the given call and returns a number of possible traces for it.
589    ///
590    /// Handler for `trace_call`
591    async fn trace_call(
592        &self,
593        call: TransactionRequest,
594        trace_types: HashSet<TraceType>,
595        block_id: Option<BlockId>,
596        state_overrides: Option<StateOverride>,
597        block_overrides: Option<Box<BlockOverrides>>,
598    ) -> RpcResult<TraceResults> {
599        let _permit = self.acquire_trace_permit().await;
600        let request =
601            TraceCallRequest { call, trace_types, block_id, state_overrides, block_overrides };
602        Ok(Self::trace_call(self, request).await.map_err(Into::into)?)
603    }
604
605    /// Handler for `trace_callMany`
606    async fn trace_call_many(
607        &self,
608        calls: Vec<(TransactionRequest, HashSet<TraceType>)>,
609        block_id: Option<BlockId>,
610    ) -> RpcResult<Vec<TraceResults>> {
611        let _permit = self.acquire_trace_permit().await;
612        Ok(Self::trace_call_many(self, calls, block_id).await.map_err(Into::into)?)
613    }
614
615    /// Handler for `trace_rawTransaction`
616    async fn trace_raw_transaction(
617        &self,
618        data: Bytes,
619        trace_types: HashSet<TraceType>,
620        block_id: Option<BlockId>,
621    ) -> RpcResult<TraceResults> {
622        let _permit = self.acquire_trace_permit().await;
623        Ok(Self::trace_raw_transaction(self, data, trace_types, block_id)
624            .await
625            .map_err(Into::into)?)
626    }
627
628    /// Handler for `trace_replayBlockTransactions`
629    async fn replay_block_transactions(
630        &self,
631        block_id: BlockId,
632        trace_types: HashSet<TraceType>,
633    ) -> RpcResult<Option<Vec<TraceResultsWithTransactionHash>>> {
634        let _permit = self.acquire_trace_permit().await;
635        Ok(Self::replay_block_transactions(self, block_id, trace_types)
636            .await
637            .map_err(Into::into)?)
638    }
639
640    /// Handler for `trace_replayTransaction`
641    async fn replay_transaction(
642        &self,
643        transaction: B256,
644        trace_types: HashSet<TraceType>,
645    ) -> RpcResult<TraceResults> {
646        let _permit = self.acquire_trace_permit().await;
647        Ok(Self::replay_transaction(self, transaction, trace_types).await.map_err(Into::into)?)
648    }
649
650    /// Handler for `trace_block`
651    async fn trace_block(
652        &self,
653        block_id: BlockId,
654    ) -> RpcResult<Option<Vec<LocalizedTransactionTrace>>> {
655        let _permit = self.acquire_trace_permit().await;
656        Ok(Self::trace_block(self, block_id).await.map_err(Into::into)?)
657    }
658
659    /// Handler for `trace_filter`
660    ///
661    /// This is similar to `eth_getLogs` but for traces.
662    ///
663    /// # Limitations
664    /// This currently requires block filter fields, since reth does not have address indices yet.
665    async fn trace_filter(&self, filter: TraceFilter) -> RpcResult<Vec<LocalizedTransactionTrace>> {
666        Ok(Self::trace_filter(self, filter).await.map_err(Into::into)?)
667    }
668
669    /// Returns transaction trace at given index.
670    /// Handler for `trace_get`
671    async fn trace_get(
672        &self,
673        hash: B256,
674        indices: Vec<Index>,
675    ) -> RpcResult<Option<LocalizedTransactionTrace>> {
676        let _permit = self.acquire_trace_permit().await;
677        Ok(Self::trace_get(self, hash, indices.into_iter().map(Into::into).collect())
678            .await
679            .map_err(Into::into)?)
680    }
681
682    /// Handler for `trace_transaction`
683    async fn trace_transaction(
684        &self,
685        hash: B256,
686    ) -> RpcResult<Option<Vec<LocalizedTransactionTrace>>> {
687        let _permit = self.acquire_trace_permit().await;
688        Ok(Self::trace_transaction(self, hash).await.map_err(Into::into)?)
689    }
690
691    /// Handler for `trace_transactionOpcodeGas`
692    async fn trace_transaction_opcode_gas(
693        &self,
694        tx_hash: B256,
695    ) -> RpcResult<Option<TransactionOpcodeGas>> {
696        let _permit = self.acquire_trace_permit().await;
697        Ok(Self::trace_transaction_opcode_gas(self, tx_hash).await.map_err(Into::into)?)
698    }
699
700    /// Handler for `trace_blockOpcodeGas`
701    async fn trace_block_opcode_gas(&self, block_id: BlockId) -> RpcResult<Option<BlockOpcodeGas>> {
702        let _permit = self.acquire_trace_permit().await;
703        Ok(Self::trace_block_opcode_gas(self, block_id).await.map_err(Into::into)?)
704    }
705}
706
707impl<Eth> std::fmt::Debug for TraceApi<Eth> {
708    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
709        f.debug_struct("TraceApi").finish_non_exhaustive()
710    }
711}
712impl<Eth> Clone for TraceApi<Eth> {
713    fn clone(&self) -> Self {
714        Self { inner: Arc::clone(&self.inner) }
715    }
716}
717
718struct TraceApiInner<Eth> {
719    /// Access to commonly used code of the `eth` namespace
720    eth_api: Eth,
721    // restrict the number of concurrent calls to `trace_*`
722    blocking_task_guard: BlockingTaskGuard,
723}
724
725/// Helper to construct a [`LocalizedTransactionTrace`] that describes a reward to the block
726/// beneficiary.
727fn reward_trace<H: BlockHeader>(header: &H, reward: RewardAction) -> LocalizedTransactionTrace {
728    LocalizedTransactionTrace {
729        block_hash: Some(header.hash_slow()),
730        block_number: Some(header.number()),
731        transaction_hash: None,
732        transaction_position: None,
733        trace: TransactionTrace {
734            trace_address: vec![],
735            subtraces: 0,
736            action: Action::Reward(reward),
737            error: None,
738            result: None,
739        },
740    }
741}