mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: add debug_traceBlock test for entire chain (#6636)
This commit is contained in:
@ -5,7 +5,10 @@ use jsonrpsee::core::Error as RpcError;
|
||||
use reth_primitives::{BlockId, TxHash, B256};
|
||||
use reth_rpc_api::{clients::DebugApiClient, EthApiClient};
|
||||
use reth_rpc_types::{
|
||||
trace::geth::{GethDebugTracerType, GethDebugTracingOptions},
|
||||
trace::{
|
||||
common::TraceResult,
|
||||
geth::{GethDebugTracerType, GethDebugTracingOptions, GethTrace},
|
||||
},
|
||||
TransactionRequest,
|
||||
};
|
||||
use std::{
|
||||
@ -18,6 +21,10 @@ const JS_TRACER_TEMPLATE: &str = include_str!("../assets/tracer-template.js");
|
||||
/// A result type for the `debug_trace_transaction` method that also captures the requested hash.
|
||||
pub type TraceTransactionResult = Result<(serde_json::Value, TxHash), (RpcError, TxHash)>;
|
||||
|
||||
/// A result type for the `debug_trace_block` method that also captures the requested block.
|
||||
pub type DebugTraceBlockResult =
|
||||
Result<(Vec<TraceResult<GethTrace, String>>, BlockId), (RpcError, BlockId)>;
|
||||
|
||||
/// An extension trait for the Trace API.
|
||||
#[async_trait::async_trait]
|
||||
pub trait DebugApiExt {
|
||||
@ -39,6 +46,18 @@ pub trait DebugApiExt {
|
||||
) -> Result<DebugTraceTransactionsStream<'_>, jsonrpsee::core::Error>
|
||||
where
|
||||
B: Into<BlockId> + Send;
|
||||
|
||||
/// Trace all given blocks with the given tracing opts, returning a stream.
|
||||
fn debug_trace_block_buffered_unordered<I, B>(
|
||||
&self,
|
||||
params: I,
|
||||
opts: Option<GethDebugTracingOptions>,
|
||||
n: usize,
|
||||
) -> DebugTraceBlockStream<'_>
|
||||
where
|
||||
I: IntoIterator<Item = B>,
|
||||
B: Into<BlockId> + Send;
|
||||
|
||||
/// method for debug_traceCall
|
||||
async fn debug_trace_call_json(
|
||||
&self,
|
||||
@ -96,6 +115,37 @@ where
|
||||
|
||||
Ok(DebugTraceTransactionsStream { stream: Box::pin(stream) })
|
||||
}
|
||||
|
||||
fn debug_trace_block_buffered_unordered<I, B>(
|
||||
&self,
|
||||
params: I,
|
||||
opts: Option<GethDebugTracingOptions>,
|
||||
n: usize,
|
||||
) -> DebugTraceBlockStream<'_>
|
||||
where
|
||||
I: IntoIterator<Item = B>,
|
||||
B: Into<BlockId> + Send,
|
||||
{
|
||||
let blocks =
|
||||
params.into_iter().map(|block| (block.into(), opts.clone())).collect::<Vec<_>>();
|
||||
let stream =
|
||||
futures::stream::iter(blocks.into_iter().map(move |(block, opts)| async move {
|
||||
let trace_future = match block {
|
||||
BlockId::Hash(hash) => {
|
||||
self.debug_trace_block_by_hash(hash.block_hash, opts.clone())
|
||||
}
|
||||
BlockId::Number(tag) => self.debug_trace_block_by_number(tag, opts.clone()),
|
||||
};
|
||||
|
||||
match trace_future.await {
|
||||
Ok(result) => Ok((result, block)),
|
||||
Err(err) => Err((err, block)),
|
||||
}
|
||||
}))
|
||||
.buffer_unordered(n);
|
||||
DebugTraceBlockStream { stream: Box::pin(stream) }
|
||||
}
|
||||
|
||||
async fn debug_trace_call_json(
|
||||
&self,
|
||||
request: TransactionRequest,
|
||||
@ -127,16 +177,21 @@ pub struct JsTracerBuilder {
|
||||
/// `setup_body` is invoked once at the beginning, during the construction of a given
|
||||
/// transaction.
|
||||
setup_body: Option<String>,
|
||||
|
||||
/// `fault_body` is invoked when an error happens during the execution of an opcode which
|
||||
/// wasn't reported in step.
|
||||
fault_body: Option<String>,
|
||||
|
||||
/// `result_body` returns a JSON-serializable value to the RPC caller.
|
||||
result_body: Option<String>,
|
||||
|
||||
/// `enter_body` is invoked on stepping in of an internal call.
|
||||
enter_body: Option<String>,
|
||||
|
||||
/// `step_body` is called for each step of the EVM, or when an error occurs, as the specified
|
||||
/// transaction is traced.
|
||||
step_body: Option<String>,
|
||||
|
||||
/// `exit_body` is invoked on stepping out of an internal call.
|
||||
exit_body: Option<String>,
|
||||
}
|
||||
@ -262,6 +317,38 @@ impl<'a> std::fmt::Debug for DebugTraceTransactionsStream<'a> {
|
||||
}
|
||||
}
|
||||
|
||||
/// A stream that yields the `debug_` traces for the requested blocks.
|
||||
#[must_use = "streams do nothing unless polled"]
|
||||
pub struct DebugTraceBlockStream<'a> {
|
||||
stream: Pin<Box<dyn Stream<Item = DebugTraceBlockResult> + 'a>>,
|
||||
}
|
||||
|
||||
impl<'a> DebugTraceBlockStream<'a> {
|
||||
/// Returns the next error result of the stream.
|
||||
pub async fn next_err(&mut self) -> Option<(RpcError, BlockId)> {
|
||||
loop {
|
||||
match self.next().await? {
|
||||
Ok(_) => continue,
|
||||
Err(err) => return Some(err),
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a> Stream for DebugTraceBlockStream<'a> {
|
||||
type Item = DebugTraceBlockResult;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
self.stream.as_mut().poll_next(cx)
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a> std::fmt::Debug for DebugTraceBlockStream<'a> {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("DebugTraceBlockStream").finish_non_exhaustive()
|
||||
}
|
||||
}
|
||||
|
||||
/// A javascript tracer that does nothing
|
||||
#[derive(Debug, Clone, Copy, Default)]
|
||||
#[non_exhaustive]
|
||||
|
||||
@ -20,25 +20,28 @@ use std::{
|
||||
/// A type alias that represents the result of a raw transaction trace stream.
|
||||
type RawTransactionTraceResult<'a> =
|
||||
Pin<Box<dyn Stream<Item = Result<(TraceResults, Bytes), (RpcError, Bytes)>> + 'a>>;
|
||||
|
||||
/// A result type for the `trace_block` method that also captures the requested block.
|
||||
pub type TraceBlockResult = Result<(Vec<LocalizedTransactionTrace>, BlockId), (RpcError, BlockId)>;
|
||||
/// Type alias representing the result of replaying a transaction.
|
||||
|
||||
/// Type alias representing the result of replaying a transaction.
|
||||
pub type ReplayTransactionResult = Result<(TraceResults, TxHash), (RpcError, TxHash)>;
|
||||
|
||||
/// A type representing the result of calling `trace_call_many` method.
|
||||
|
||||
pub type CallManyTraceResult = Result<
|
||||
(Vec<TraceResults>, Vec<(TransactionRequest, HashSet<TraceType>)>),
|
||||
(RpcError, Vec<(TransactionRequest, HashSet<TraceType>)>),
|
||||
>;
|
||||
|
||||
/// Result type for the `trace_get` method that also captures the requested transaction hash and
|
||||
/// index.
|
||||
pub type TraceGetResult =
|
||||
Result<(Option<LocalizedTransactionTrace>, B256, Vec<Index>), (RpcError, B256, Vec<Index>)>;
|
||||
|
||||
/// Represents a result type for the `trace_filter` stream extension.
|
||||
pub type TraceFilterResult =
|
||||
Result<(Vec<LocalizedTransactionTrace>, TraceFilter), (RpcError, TraceFilter)>;
|
||||
|
||||
/// Represents the result of a single trace call.
|
||||
pub type TraceCallResult = Result<TraceResults, (RpcError, TraceCallRequest)>;
|
||||
|
||||
@ -82,9 +85,9 @@ pub trait TraceApiExt {
|
||||
trace_types: HashSet<TraceType>,
|
||||
block_id: Option<BlockId>,
|
||||
) -> RawTransactionTraceStream<'_>;
|
||||
|
||||
/// Creates a stream of results for multiple dependent transaction calls on top of the same
|
||||
/// block.
|
||||
|
||||
fn trace_call_many_stream<I>(
|
||||
&self,
|
||||
calls: I,
|
||||
@ -92,6 +95,7 @@ pub trait TraceApiExt {
|
||||
) -> CallManyTraceStream<'_>
|
||||
where
|
||||
I: IntoIterator<Item = (TransactionRequest, HashSet<TraceType>)>;
|
||||
|
||||
/// Returns a new stream that yields the traces for the given transaction hash and indices.
|
||||
fn trace_get_stream<I>(&self, hash: B256, indices: I) -> TraceGetStream<'_>
|
||||
where
|
||||
@ -101,15 +105,16 @@ pub trait TraceApiExt {
|
||||
fn trace_filter_stream<I>(&self, filters: I) -> TraceFilterStream<'_>
|
||||
where
|
||||
I: IntoIterator<Item = TraceFilter>;
|
||||
|
||||
/// Returns a new stream that yields the trace results for the given call requests.
|
||||
fn trace_call_stream(&self, request: TraceCallRequest) -> TraceCallStream<'_>;
|
||||
}
|
||||
/// `TraceCallStream` provides an asynchronous stream of tracing results.
|
||||
|
||||
#[must_use = "streams do nothing unless polled"]
|
||||
pub struct TraceCallStream<'a> {
|
||||
stream: Pin<Box<dyn Stream<Item = TraceCallResult> + 'a>>,
|
||||
}
|
||||
|
||||
impl<'a> Stream for TraceCallStream<'a> {
|
||||
type Item = TraceCallResult;
|
||||
|
||||
@ -145,14 +150,17 @@ impl<'a> std::fmt::Debug for TraceFilterStream<'a> {
|
||||
f.debug_struct("TraceFilterStream").finish_non_exhaustive()
|
||||
}
|
||||
}
|
||||
|
||||
/// A stream that asynchronously yields the results of the `trace_get` method for a given
|
||||
/// transaction hash and a series of indices.
|
||||
#[must_use = "streams do nothing unless polled"]
|
||||
pub struct TraceGetStream<'a> {
|
||||
stream: Pin<Box<dyn Stream<Item = TraceGetResult> + 'a>>,
|
||||
}
|
||||
|
||||
impl<'a> Stream for TraceGetStream<'a> {
|
||||
type Item = TraceGetResult;
|
||||
|
||||
/// Attempts to pull out the next item of the stream
|
||||
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
self.stream.as_mut().poll_next(cx)
|
||||
@ -175,8 +183,8 @@ pub struct CallManyTraceStream<'a> {
|
||||
|
||||
impl<'a> Stream for CallManyTraceStream<'a> {
|
||||
type Item = CallManyTraceResult;
|
||||
/// Polls for the next item from the stream.
|
||||
|
||||
/// Polls for the next item from the stream.
|
||||
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
self.stream.as_mut().poll_next(cx)
|
||||
}
|
||||
@ -189,7 +197,6 @@ impl<'a> std::fmt::Debug for CallManyTraceStream<'a> {
|
||||
}
|
||||
|
||||
/// A stream that traces the provided raw transaction data.
|
||||
|
||||
#[must_use = "streams do nothing unless polled"]
|
||||
pub struct RawTransactionTraceStream<'a> {
|
||||
stream: RawTransactionTraceResult<'a>,
|
||||
@ -214,6 +221,7 @@ impl<'a> std::fmt::Debug for RawTransactionTraceStream<'a> {
|
||||
pub struct ReplayTransactionStream<'a> {
|
||||
stream: Pin<Box<dyn Stream<Item = ReplayTransactionResult> + 'a>>,
|
||||
}
|
||||
|
||||
impl<'a> Stream for ReplayTransactionStream<'a> {
|
||||
type Item = ReplayTransactionResult;
|
||||
|
||||
@ -285,6 +293,7 @@ impl<T: TraceApiClient + Sync> TraceApiExt for T {
|
||||
.buffered(10);
|
||||
ReplayTransactionStream { stream: Box::pin(stream) }
|
||||
}
|
||||
|
||||
fn trace_raw_transaction_stream(
|
||||
&self,
|
||||
data: Bytes,
|
||||
|
||||
@ -1,10 +1,12 @@
|
||||
use futures::StreamExt;
|
||||
use jsonrpsee::http_client::HttpClientBuilder;
|
||||
use reth_rpc_api_testing_util::{trace::TraceApiExt, utils::parse_env_url};
|
||||
use reth_rpc_api::EthApiClient;
|
||||
use reth_rpc_api_testing_util::{debug::DebugApiExt, trace::TraceApiExt, utils::parse_env_url};
|
||||
use reth_rpc_types::trace::{
|
||||
filter::TraceFilter, parity::TraceType, tracerequest::TraceCallRequest,
|
||||
};
|
||||
use std::{collections::HashSet, time::Instant};
|
||||
|
||||
/// This is intended to be run locally against a running node.
|
||||
///
|
||||
/// This is a noop of env var `RETH_RPC_TEST_NODE_URL` is not set.
|
||||
@ -26,7 +28,6 @@ async fn trace_many_blocks() {
|
||||
}
|
||||
|
||||
/// Tests the replaying of transactions on a local Ethereum node.
|
||||
|
||||
#[tokio::test(flavor = "multi_thread")]
|
||||
#[ignore]
|
||||
async fn replay_transactions() {
|
||||
@ -49,7 +50,6 @@ async fn replay_transactions() {
|
||||
}
|
||||
|
||||
/// Tests the tracers filters on a local Ethereum node
|
||||
|
||||
#[tokio::test(flavor = "multi_thread")]
|
||||
#[ignore]
|
||||
async fn trace_filters() {
|
||||
@ -92,3 +92,26 @@ async fn trace_call() {
|
||||
|
||||
println!("Completed in {:?}", start_time.elapsed());
|
||||
}
|
||||
|
||||
/// This is intended to be run locally against a running node. This traces all blocks for a given
|
||||
/// chain.
|
||||
///
|
||||
/// This is a noop of env var `RETH_RPC_TEST_NODE_URL` is not set.
|
||||
#[tokio::test(flavor = "multi_thread")]
|
||||
async fn debug_trace_block_entire_chain() {
|
||||
let url = parse_env_url("RETH_RPC_TEST_NODE_URL");
|
||||
if url.is_err() {
|
||||
return
|
||||
}
|
||||
let url = url.unwrap();
|
||||
|
||||
let client = HttpClientBuilder::default().build(url).unwrap();
|
||||
let current_block: u64 = client.block_number().await.unwrap().try_into().unwrap();
|
||||
let range = 0..=current_block;
|
||||
let mut stream = client.debug_trace_block_buffered_unordered(range, None, 20);
|
||||
let now = Instant::now();
|
||||
while let Some((err, block)) = stream.next_err().await {
|
||||
eprintln!("Error tracing block {block:?}: {err}");
|
||||
}
|
||||
println!("Traced all blocks in {:?}", now.elapsed());
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user