feat: add js tracer (#3100)

Co-authored-by: Georgios Konstantopoulos <me@gakonst.com>
This commit is contained in:
Matthias Seitz
2023-06-14 14:42:00 +02:00
committed by GitHub
parent 76302d945c
commit cf10590e4a
11 changed files with 2364 additions and 237 deletions

View File

@ -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 {

View File

@ -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))
}

View File

@ -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 {