mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: add js tracer (#3100)
Co-authored-by: Georgios Konstantopoulos <me@gakonst.com>
This commit is contained in:
@ -100,6 +100,7 @@ pub enum GethTraceFrame {
|
||||
FourByteTracer(FourByteFrame),
|
||||
CallTracer(CallFrame),
|
||||
PreStateTracer(PreStateFrame),
|
||||
JS(serde_json::Value),
|
||||
}
|
||||
|
||||
impl From<DefaultFrame> for GethTraceFrame {
|
||||
@ -241,6 +242,14 @@ impl GethDebugTracerConfig {
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the json config if this config is a JS tracer.
|
||||
pub fn into_js_config(self) -> Option<serde_json::Value> {
|
||||
match self {
|
||||
GethDebugTracerConfig::JsTracer(cfg) => Some(cfg),
|
||||
_ => None,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the [PreStateConfig] if it is a call config.
|
||||
pub fn into_pre_state_config(self) -> Option<PreStateConfig> {
|
||||
match self {
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
use crate::{
|
||||
eth::{
|
||||
error::{EthApiError, EthResult},
|
||||
revm_utils::{inspect, replay_transactions_until, EvmOverrides},
|
||||
revm_utils::{inspect, prepare_call_env, replay_transactions_until, EvmOverrides},
|
||||
EthTransactions, TransactionSource,
|
||||
},
|
||||
result::{internal_rpc_err, ToRpcResult},
|
||||
@ -10,11 +10,16 @@ use crate::{
|
||||
use async_trait::async_trait;
|
||||
use jsonrpsee::core::RpcResult;
|
||||
use reth_primitives::{Block, BlockId, BlockNumberOrTag, Bytes, TransactionSigned, H256};
|
||||
use reth_provider::{BlockProviderIdExt, HeaderProvider, ReceiptProviderIdExt, StateProviderBox};
|
||||
use reth_provider::{
|
||||
BlockProviderIdExt, HeaderProvider, ReceiptProviderIdExt, StateProvider, StateProviderBox,
|
||||
};
|
||||
use reth_revm::{
|
||||
database::{State, SubState},
|
||||
env::tx_env_with_recovered,
|
||||
tracing::{FourByteInspector, TracingInspector, TracingInspectorConfig},
|
||||
tracing::{
|
||||
js::{JsDbRequest, JsInspector},
|
||||
FourByteInspector, TracingInspector, TracingInspectorConfig,
|
||||
},
|
||||
};
|
||||
use reth_rlp::{Decodable, Encodable};
|
||||
use reth_rpc_api::DebugApiServer;
|
||||
@ -30,7 +35,8 @@ use reth_tasks::TaskSpawner;
|
||||
use revm::primitives::Env;
|
||||
use revm_primitives::{db::DatabaseCommit, BlockEnv, CfgEnv};
|
||||
use std::{future::Future, sync::Arc};
|
||||
use tokio::sync::{oneshot, AcquireError, OwnedSemaphorePermit};
|
||||
use tokio::sync::{mpsc, oneshot, AcquireError, OwnedSemaphorePermit};
|
||||
use tokio_stream::{wrappers::ReceiverStream, StreamExt};
|
||||
|
||||
/// `debug` API implementation.
|
||||
///
|
||||
@ -94,6 +100,7 @@ where
|
||||
opts: GethDebugTracingOptions,
|
||||
) -> EthResult<Vec<TraceResult>> {
|
||||
// replay all transactions of the block
|
||||
let this = self.clone();
|
||||
self.inner.eth_api.with_state_at_block(at, move |state| {
|
||||
let mut results = Vec::with_capacity(transactions.len());
|
||||
let mut db = SubState::new(State::new(state));
|
||||
@ -103,7 +110,8 @@ where
|
||||
let tx = tx.into_ecrecovered().ok_or(BlockError::InvalidSignature)?;
|
||||
let tx = tx_env_with_recovered(&tx);
|
||||
let env = Env { cfg: cfg.clone(), block: block_env.clone(), tx };
|
||||
let (result, state_changes) = trace_transaction(opts.clone(), env, &mut db)?;
|
||||
let (result, state_changes) =
|
||||
this.trace_transaction(opts.clone(), env, at, &mut db)?;
|
||||
results.push(TraceResult::Success { result });
|
||||
|
||||
if transactions.peek().is_some() {
|
||||
@ -204,11 +212,11 @@ where
|
||||
|
||||
// we need to get the state of the parent block because we're essentially replaying the
|
||||
// block the transaction is included in
|
||||
let state_at = block.parent_hash;
|
||||
let state_at: BlockId = block.parent_hash.into();
|
||||
let block_txs = block.body;
|
||||
|
||||
self.on_blocking_task(|this| async move {
|
||||
this.inner.eth_api.with_state_at_block(state_at.into(), |state| {
|
||||
this.inner.eth_api.with_state_at_block(state_at, |state| {
|
||||
// configure env for the target transaction
|
||||
let tx = transaction.into_recovered();
|
||||
|
||||
@ -223,7 +231,7 @@ where
|
||||
)?;
|
||||
|
||||
let env = Env { cfg, block: block_env, tx: tx_env_with_recovered(&tx) };
|
||||
trace_transaction(opts, env, &mut db).map(|(trace, _)| trace)
|
||||
this.trace_transaction(opts, env, state_at, &mut db).map(|(trace, _)| trace)
|
||||
})
|
||||
})
|
||||
.await
|
||||
@ -302,8 +310,23 @@ where
|
||||
}
|
||||
GethDebugBuiltInTracerType::NoopTracer => Ok(NoopFrame::default().into()),
|
||||
},
|
||||
GethDebugTracerType::JsTracer(_) => {
|
||||
Err(EthApiError::Unsupported("javascript tracers are unsupported."))
|
||||
GethDebugTracerType::JsTracer(code) => {
|
||||
let config = tracer_config.and_then(|c| c.into_js_config()).unwrap_or_default();
|
||||
|
||||
// for JS tracing we need to setup all async work before we can start tracing
|
||||
// because JSTracer and all JS types are not Send
|
||||
let (cfg, block_env, at) = self.inner.eth_api.evm_env_at(at).await?;
|
||||
let state = self.inner.eth_api.state_at(at)?;
|
||||
let mut db = SubState::new(State::new(state));
|
||||
let env = prepare_call_env(cfg, block_env, call, &mut db, overrides)?;
|
||||
|
||||
let to_db_service = self.spawn_js_trace_service(at)?;
|
||||
|
||||
let mut inspector = JsInspector::new(code, config, to_db_service)?;
|
||||
let (res, env) = inspect(db, env, &mut inspector)?;
|
||||
|
||||
let result = inspector.json_result(res, &env)?;
|
||||
Ok(GethTraceFrame::JS(result))
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -321,6 +344,149 @@ where
|
||||
|
||||
Ok(frame.into())
|
||||
}
|
||||
|
||||
/// Executes the configured transaction with the environment on the given database.
|
||||
///
|
||||
/// Returns the trace frame and the state that got updated after executing the transaction.
|
||||
///
|
||||
/// Note: this does not apply any state overrides if they're configured in the `opts`.
|
||||
fn trace_transaction(
|
||||
&self,
|
||||
opts: GethDebugTracingOptions,
|
||||
env: Env,
|
||||
at: BlockId,
|
||||
db: &mut SubState<StateProviderBox<'_>>,
|
||||
) -> EthResult<(GethTraceFrame, revm_primitives::State)> {
|
||||
let GethDebugTracingOptions { config, tracer, tracer_config, .. } = opts;
|
||||
|
||||
if let Some(tracer) = tracer {
|
||||
// valid matching config
|
||||
if let Some(ref config) = tracer_config {
|
||||
if !config.matches_tracer(&tracer) {
|
||||
return Err(EthApiError::InvalidTracerConfig)
|
||||
}
|
||||
}
|
||||
|
||||
return match tracer {
|
||||
GethDebugTracerType::BuiltInTracer(tracer) => match tracer {
|
||||
GethDebugBuiltInTracerType::FourByteTracer => {
|
||||
let mut inspector = FourByteInspector::default();
|
||||
let (res, _) = inspect(db, env, &mut inspector)?;
|
||||
return Ok((FourByteFrame::from(inspector).into(), res.state))
|
||||
}
|
||||
GethDebugBuiltInTracerType::CallTracer => {
|
||||
// we validated the config above
|
||||
let call_config =
|
||||
tracer_config.and_then(|c| c.into_call_config()).unwrap_or_default();
|
||||
|
||||
let mut inspector = TracingInspector::new(
|
||||
TracingInspectorConfig::from_geth_config(&config),
|
||||
);
|
||||
|
||||
let (res, _) = inspect(db, env, &mut inspector)?;
|
||||
|
||||
let frame = inspector.into_geth_builder().geth_call_traces(call_config);
|
||||
|
||||
return Ok((frame.into(), res.state))
|
||||
}
|
||||
GethDebugBuiltInTracerType::PreStateTracer => {
|
||||
Err(EthApiError::Unsupported("prestate tracer is unimplemented yet."))
|
||||
}
|
||||
GethDebugBuiltInTracerType::NoopTracer => {
|
||||
Ok((NoopFrame::default().into(), Default::default()))
|
||||
}
|
||||
},
|
||||
GethDebugTracerType::JsTracer(code) => {
|
||||
let config = tracer_config.and_then(|c| c.into_js_config()).unwrap_or_default();
|
||||
|
||||
// we spawn the database service that will be used by the JS tracer
|
||||
// TODO(mattsse) this is not quite accurate when tracing a block inside a
|
||||
// transaction because the service needs access to the committed state changes
|
||||
let to_db_service = self.spawn_js_trace_service(at)?;
|
||||
|
||||
let mut inspector = JsInspector::new(code, config, to_db_service)?;
|
||||
let (res, env) = inspect(db, env, &mut inspector)?;
|
||||
|
||||
let state = res.state.clone();
|
||||
let result = inspector.json_result(res, &env)?;
|
||||
Ok((GethTraceFrame::JS(result), state))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// default structlog tracer
|
||||
let inspector_config = TracingInspectorConfig::from_geth_config(&config);
|
||||
|
||||
let mut inspector = TracingInspector::new(inspector_config);
|
||||
|
||||
let (res, _) = inspect(db, env, &mut inspector)?;
|
||||
let gas_used = res.result.gas_used();
|
||||
|
||||
let frame = inspector.into_geth_builder().geth_traces(gas_used, config);
|
||||
|
||||
Ok((frame.into(), res.state))
|
||||
}
|
||||
|
||||
/// Spawns [Self::js_trace_db_service_task] on a new task and returns a channel to send requests
|
||||
/// to it.
|
||||
///
|
||||
/// Note: This blocks until the service is ready to receive requests.
|
||||
fn spawn_js_trace_service(&self, at: BlockId) -> EthResult<mpsc::Sender<JsDbRequest>> {
|
||||
let (to_db_service, rx) = mpsc::channel(1);
|
||||
let (ready_tx, ready_rx) = std::sync::mpsc::channel();
|
||||
let this = self.clone();
|
||||
self.inner
|
||||
.task_spawner
|
||||
.spawn(Box::pin(async move { this.js_trace_db_service_task(at, rx, ready_tx).await }));
|
||||
// wait for initialization
|
||||
ready_rx.recv().map_err(|_| {
|
||||
EthApiError::InternalJsTracerError("js tracer initialization failed".to_string())
|
||||
})??;
|
||||
Ok(to_db_service)
|
||||
}
|
||||
|
||||
/// A services that handles database requests issued from inside the JavaScript tracing engine.
|
||||
async fn js_trace_db_service_task(
|
||||
self,
|
||||
at: BlockId,
|
||||
rx: mpsc::Receiver<JsDbRequest>,
|
||||
on_ready: std::sync::mpsc::Sender<EthResult<()>>,
|
||||
) {
|
||||
let state = match self.inner.eth_api.state_at(at) {
|
||||
Ok(state) => {
|
||||
let _ = on_ready.send(Ok(()));
|
||||
state
|
||||
}
|
||||
Err(err) => {
|
||||
let _ = on_ready.send(Err(err));
|
||||
return
|
||||
}
|
||||
};
|
||||
|
||||
let mut stream = ReceiverStream::new(rx);
|
||||
while let Some(req) = stream.next().await {
|
||||
match req {
|
||||
JsDbRequest::Basic { address, resp } => {
|
||||
let acc = state.basic_account(address).map_err(|err| err.to_string());
|
||||
let _ = resp.send(acc);
|
||||
}
|
||||
JsDbRequest::Code { code_hash, resp } => {
|
||||
let code = state
|
||||
.bytecode_by_hash(code_hash)
|
||||
.map(|code| code.map(|c| c.bytecode.clone()).unwrap_or_default())
|
||||
.map_err(|err| err.to_string());
|
||||
let _ = resp.send(code);
|
||||
}
|
||||
JsDbRequest::StorageAt { address, index, resp } => {
|
||||
let value = state
|
||||
.storage(address, index)
|
||||
.map(|val| val.unwrap_or_default())
|
||||
.map_err(|err| err.to_string());
|
||||
let _ = resp.send(value);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
@ -486,70 +652,3 @@ struct DebugApiInner<Provider, Eth> {
|
||||
/// The type that can spawn tasks which would otherwise block.
|
||||
task_spawner: Box<dyn TaskSpawner>,
|
||||
}
|
||||
|
||||
/// Executes the configured transaction with the environment on the given database.
|
||||
///
|
||||
/// Returns the trace frame and the state that got updated after executing the transaction.
|
||||
///
|
||||
/// Note: this does not apply any state overrides if they're configured in the `opts`.
|
||||
fn trace_transaction(
|
||||
opts: GethDebugTracingOptions,
|
||||
env: Env,
|
||||
db: &mut SubState<StateProviderBox<'_>>,
|
||||
) -> EthResult<(GethTraceFrame, revm_primitives::State)> {
|
||||
let GethDebugTracingOptions { config, tracer, tracer_config, .. } = opts;
|
||||
|
||||
if let Some(tracer) = tracer {
|
||||
// valid matching config
|
||||
if let Some(ref config) = tracer_config {
|
||||
if !config.matches_tracer(&tracer) {
|
||||
return Err(EthApiError::InvalidTracerConfig)
|
||||
}
|
||||
}
|
||||
|
||||
return match tracer {
|
||||
GethDebugTracerType::BuiltInTracer(tracer) => match tracer {
|
||||
GethDebugBuiltInTracerType::FourByteTracer => {
|
||||
let mut inspector = FourByteInspector::default();
|
||||
let (res, _) = inspect(db, env, &mut inspector)?;
|
||||
return Ok((FourByteFrame::from(inspector).into(), res.state))
|
||||
}
|
||||
GethDebugBuiltInTracerType::CallTracer => {
|
||||
// we validated the config above
|
||||
let call_config =
|
||||
tracer_config.and_then(|c| c.into_call_config()).unwrap_or_default();
|
||||
|
||||
let mut inspector =
|
||||
TracingInspector::new(TracingInspectorConfig::from_geth_config(&config));
|
||||
|
||||
let (res, _) = inspect(db, env, &mut inspector)?;
|
||||
|
||||
let frame = inspector.into_geth_builder().geth_call_traces(call_config);
|
||||
|
||||
return Ok((frame.into(), res.state))
|
||||
}
|
||||
GethDebugBuiltInTracerType::PreStateTracer => {
|
||||
Err(EthApiError::Unsupported("prestate tracer is unimplemented yet."))
|
||||
}
|
||||
GethDebugBuiltInTracerType::NoopTracer => {
|
||||
Ok((NoopFrame::default().into(), Default::default()))
|
||||
}
|
||||
},
|
||||
GethDebugTracerType::JsTracer(_) => {
|
||||
Err(EthApiError::Unsupported("javascript tracers are unsupported."))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// default structlog tracer
|
||||
let inspector_config = TracingInspectorConfig::from_geth_config(&config);
|
||||
|
||||
let mut inspector = TracingInspector::new(inspector_config);
|
||||
|
||||
let (res, _) = inspect(db, env, &mut inspector)?;
|
||||
let gas_used = res.result.gas_used();
|
||||
|
||||
let frame = inspector.into_geth_builder().geth_traces(gas_used, config);
|
||||
|
||||
Ok((frame.into(), res.state))
|
||||
}
|
||||
|
||||
@ -3,6 +3,7 @@
|
||||
use crate::result::{internal_rpc_err, invalid_params_rpc_err, rpc_err, rpc_error_with_code};
|
||||
use jsonrpsee::{core::Error as RpcError, types::ErrorObject};
|
||||
use reth_primitives::{abi::decode_revert_reason, Address, Bytes, U256};
|
||||
use reth_revm::tracing::js::JsInspectorError;
|
||||
use reth_rpc_types::{error::EthRpcErrorCode, BlockError};
|
||||
use reth_transaction_pool::error::{InvalidPoolTransactionError, PoolError};
|
||||
use revm::primitives::{EVMError, ExecutionResult, Halt, OutOfGasError};
|
||||
@ -57,6 +58,9 @@ pub enum EthApiError {
|
||||
/// Some feature is unsupported
|
||||
#[error("unsupported")]
|
||||
Unsupported(&'static str),
|
||||
/// General purpose error for invalid params
|
||||
#[error("{0}")]
|
||||
InvalidParams(String),
|
||||
/// When tracer config does not match the tracer
|
||||
#[error("invalid tracer config")]
|
||||
InvalidTracerConfig,
|
||||
@ -69,6 +73,9 @@ pub enum EthApiError {
|
||||
/// Error thrown when a spawned blocking task failed to deliver an anticipated response.
|
||||
#[error("internal eth error")]
|
||||
InternalEthError,
|
||||
/// Internal Error thrown by the javascript tracer
|
||||
#[error("{0}")]
|
||||
InternalJsTracerError(String),
|
||||
}
|
||||
|
||||
impl From<EthApiError> for ErrorObject<'static> {
|
||||
@ -92,6 +99,8 @@ impl From<EthApiError> for ErrorObject<'static> {
|
||||
rpc_error_with_code(EthRpcErrorCode::ResourceNotFound.code(), error.to_string())
|
||||
}
|
||||
EthApiError::Unsupported(msg) => internal_rpc_err(msg),
|
||||
EthApiError::InternalJsTracerError(msg) => internal_rpc_err(msg),
|
||||
EthApiError::InvalidParams(msg) => invalid_params_rpc_err(msg),
|
||||
EthApiError::InvalidRewardPercentile(msg) => internal_rpc_err(msg.to_string()),
|
||||
err @ EthApiError::InternalTracingError => internal_rpc_err(err.to_string()),
|
||||
err @ EthApiError::InternalEthError => internal_rpc_err(err.to_string()),
|
||||
@ -104,6 +113,16 @@ impl From<EthApiError> for RpcError {
|
||||
RpcError::Call(error.into())
|
||||
}
|
||||
}
|
||||
impl From<JsInspectorError> for EthApiError {
|
||||
fn from(error: JsInspectorError) -> Self {
|
||||
match error {
|
||||
err @ JsInspectorError::JsError(_) => {
|
||||
EthApiError::InternalJsTracerError(err.to_string())
|
||||
}
|
||||
err => EthApiError::InvalidParams(err.to_string()),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl From<reth_interfaces::Error> for EthApiError {
|
||||
fn from(error: reth_interfaces::Error) -> Self {
|
||||
|
||||
Reference in New Issue
Block a user