mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: support nonce and balance changes in state diff tracers (#3199)
This commit is contained in:
@ -1,4 +1,7 @@
|
||||
//! Builder types for building traces
|
||||
|
||||
pub(crate) mod geth;
|
||||
pub(crate) mod parity;
|
||||
/// Geth style trace builders for `debug_` namespace
|
||||
pub mod geth;
|
||||
|
||||
/// Parity style trace builders for `trace_` namespace
|
||||
pub mod parity;
|
||||
|
||||
@ -1,7 +1,10 @@
|
||||
use crate::tracing::{types::CallTraceNode, TracingInspectorConfig};
|
||||
use reth_primitives::Address;
|
||||
use reth_primitives::{Address, U64};
|
||||
use reth_rpc_types::{trace::parity::*, TransactionInfo};
|
||||
use revm::primitives::ExecutionResult;
|
||||
use revm::{
|
||||
db::DatabaseRef,
|
||||
primitives::{AccountInfo, ExecutionResult, ResultAndState},
|
||||
};
|
||||
use std::collections::HashSet;
|
||||
|
||||
/// A type for creating parity style traces
|
||||
@ -91,6 +94,12 @@ impl ParityTraceBuilder {
|
||||
|
||||
/// Consumes the inspector and returns the trace results according to the configured trace
|
||||
/// types.
|
||||
///
|
||||
/// Warning: If `trace_types` contains [TraceType::StateDiff] the returned [StateDiff] will only
|
||||
/// contain accounts with changed state, not including their balance changes because this is not
|
||||
/// tracked during inspection and requires the State map returned after inspection. Use
|
||||
/// [ParityTraceBuilder::into_trace_results_with_state] to populate the balance and nonce
|
||||
/// changes for the [StateDiff] using the [DatabaseRef].
|
||||
pub fn into_trace_results(
|
||||
self,
|
||||
res: ExecutionResult,
|
||||
@ -107,6 +116,36 @@ impl ParityTraceBuilder {
|
||||
TraceResults { output: output.into(), trace, vm_trace, state_diff }
|
||||
}
|
||||
|
||||
/// Consumes the inspector and returns the trace results according to the configured trace
|
||||
/// types.
|
||||
///
|
||||
/// This also takes the [DatabaseRef] to populate the balance and nonce changes for the
|
||||
/// [StateDiff].
|
||||
///
|
||||
/// Note: this is considered a convenience method that takes the state map of
|
||||
/// [ResultAndState] after inspecting a transaction
|
||||
/// with the [TracingInspector](crate::tracing::TracingInspector).
|
||||
pub fn into_trace_results_with_state<DB>(
|
||||
self,
|
||||
res: ResultAndState,
|
||||
trace_types: &HashSet<TraceType>,
|
||||
db: DB,
|
||||
) -> Result<TraceResults, DB::Error>
|
||||
where
|
||||
DB: DatabaseRef,
|
||||
{
|
||||
let ResultAndState { result, state } = res;
|
||||
let mut trace_res = self.into_trace_results(result, trace_types);
|
||||
if let Some(ref mut state_diff) = trace_res.state_diff {
|
||||
populate_account_balance_nonce_diffs(
|
||||
state_diff,
|
||||
&db,
|
||||
state.into_iter().map(|(addr, acc)| (addr, acc.info)),
|
||||
)?;
|
||||
}
|
||||
Ok(trace_res)
|
||||
}
|
||||
|
||||
/// Returns the tracing types that are configured in the set
|
||||
pub fn into_trace_type_traces(
|
||||
self,
|
||||
@ -172,3 +211,39 @@ fn vm_trace(nodes: &[CallTraceNode]) -> VmTrace {
|
||||
|
||||
VmTrace { code: nodes[0].trace.data.clone().into(), ops: vec![] }
|
||||
}
|
||||
|
||||
/// Loops over all state accounts in the accounts diff that contains all accounts that are included
|
||||
/// in the [ExecutionResult] state map and compares the balance and nonce against what's in the
|
||||
/// `db`, which should point to the beginning of the transaction.
|
||||
///
|
||||
/// It's expected that `DB` is a [CacheDB](revm::db::CacheDB) which at this point already contains
|
||||
/// all the accounts that are in the state map and never has to fetch them from disk.
|
||||
pub fn populate_account_balance_nonce_diffs<DB, I>(
|
||||
state_diff: &mut StateDiff,
|
||||
db: DB,
|
||||
account_diffs: I,
|
||||
) -> Result<(), DB::Error>
|
||||
where
|
||||
I: IntoIterator<Item = (Address, AccountInfo)>,
|
||||
DB: DatabaseRef,
|
||||
{
|
||||
for (addr, changed_acc) in account_diffs.into_iter() {
|
||||
let entry = state_diff.entry(addr).or_default();
|
||||
let db_acc = db.basic(addr)?.unwrap_or_default();
|
||||
entry.balance = if db_acc.balance == changed_acc.balance {
|
||||
Delta::Unchanged
|
||||
} else {
|
||||
Delta::Changed(ChangedType { from: db_acc.balance, to: changed_acc.balance })
|
||||
};
|
||||
entry.nonce = if db_acc.nonce == changed_acc.nonce {
|
||||
Delta::Unchanged
|
||||
} else {
|
||||
Delta::Changed(ChangedType {
|
||||
from: U64::from(db_acc.nonce),
|
||||
to: U64::from(changed_acc.nonce),
|
||||
})
|
||||
};
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@ -25,7 +25,10 @@ use crate::tracing::{
|
||||
arena::PushTraceKind,
|
||||
types::{CallTraceNode, StorageChange},
|
||||
};
|
||||
pub use builder::{geth::GethTraceBuilder, parity::ParityTraceBuilder};
|
||||
pub use builder::{
|
||||
geth::{self, GethTraceBuilder},
|
||||
parity::{self, ParityTraceBuilder},
|
||||
};
|
||||
pub use config::TracingInspectorConfig;
|
||||
pub use fourbyte::FourByteInspector;
|
||||
pub use opcount::OpcodeCountInspector;
|
||||
|
||||
@ -268,8 +268,6 @@ impl CallTraceNode {
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: track nonce and balance changes
|
||||
|
||||
// iterate over all storage diffs
|
||||
for change in self.trace.steps.iter().filter_map(|s| s.storage_change) {
|
||||
let StorageChange { key, value, had_value } = change;
|
||||
|
||||
@ -173,6 +173,23 @@ pub trait EthTransactions: Send + Sync {
|
||||
where
|
||||
F: FnOnce(TracingInspector, ResultAndState) -> EthResult<R>;
|
||||
|
||||
/// Same as [Self::trace_at] but also provides the used database to the callback.
|
||||
///
|
||||
/// Executes the transaction on top of the given [BlockId] with a tracer configured by the
|
||||
/// config.
|
||||
///
|
||||
/// The callback is then called with the [TracingInspector] and the [ResultAndState] after the
|
||||
/// configured [Env] was inspected.
|
||||
fn trace_at_with_state<F, R>(
|
||||
&self,
|
||||
env: Env,
|
||||
config: TracingInspectorConfig,
|
||||
at: BlockId,
|
||||
f: F,
|
||||
) -> EthResult<R>
|
||||
where
|
||||
F: for<'a> FnOnce(TracingInspector, ResultAndState, StateCacheDB<'a>) -> EthResult<R>;
|
||||
|
||||
/// Fetches the transaction and the transaction's block
|
||||
async fn transaction_and_block(
|
||||
&self,
|
||||
@ -182,7 +199,9 @@ pub trait EthTransactions: Send + Sync {
|
||||
/// Retrieves the transaction if it exists and returns its trace.
|
||||
///
|
||||
/// Before the transaction is traced, all previous transaction in the block are applied to the
|
||||
/// state by executing them first
|
||||
/// state by executing them first.
|
||||
/// The callback `f` is invoked with the [ResultAndState] after the transaction was executed and
|
||||
/// the database that points to the beginning of the transaction.
|
||||
async fn trace_transaction_in_block<F, R>(
|
||||
&self,
|
||||
hash: H256,
|
||||
@ -190,7 +209,13 @@ pub trait EthTransactions: Send + Sync {
|
||||
f: F,
|
||||
) -> EthResult<Option<R>>
|
||||
where
|
||||
F: FnOnce(TransactionInfo, TracingInspector, ResultAndState) -> EthResult<R> + Send;
|
||||
F: for<'a> FnOnce(
|
||||
TransactionInfo,
|
||||
TracingInspector,
|
||||
ResultAndState,
|
||||
StateCacheDB<'a>,
|
||||
) -> EthResult<R>
|
||||
+ Send;
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
@ -541,6 +566,25 @@ where
|
||||
})
|
||||
}
|
||||
|
||||
fn trace_at_with_state<F, R>(
|
||||
&self,
|
||||
env: Env,
|
||||
config: TracingInspectorConfig,
|
||||
at: BlockId,
|
||||
f: F,
|
||||
) -> EthResult<R>
|
||||
where
|
||||
F: for<'a> FnOnce(TracingInspector, ResultAndState, StateCacheDB<'a>) -> EthResult<R>,
|
||||
{
|
||||
self.with_state_at_block(at, |state| {
|
||||
let db = SubState::new(State::new(state));
|
||||
let mut inspector = TracingInspector::new(config);
|
||||
let (res, _, db) = inspect_and_return_db(db, env, &mut inspector)?;
|
||||
|
||||
f(inspector, res, db)
|
||||
})
|
||||
}
|
||||
|
||||
async fn transaction_and_block(
|
||||
&self,
|
||||
hash: H256,
|
||||
@ -566,7 +610,13 @@ where
|
||||
f: F,
|
||||
) -> EthResult<Option<R>>
|
||||
where
|
||||
F: FnOnce(TransactionInfo, TracingInspector, ResultAndState) -> EthResult<R> + Send,
|
||||
F: for<'a> FnOnce(
|
||||
TransactionInfo,
|
||||
TracingInspector,
|
||||
ResultAndState,
|
||||
StateCacheDB<'a>,
|
||||
) -> EthResult<R>
|
||||
+ Send,
|
||||
{
|
||||
let (transaction, block) = match self.transaction_and_block(hash).await? {
|
||||
None => return Ok(None),
|
||||
@ -590,8 +640,8 @@ where
|
||||
let env = Env { cfg, block: block_env, tx: tx_env_with_recovered(&tx) };
|
||||
|
||||
let mut inspector = TracingInspector::new(config);
|
||||
let (res, _) = inspect(db, env, &mut inspector)?;
|
||||
f(tx_info, inspector, res)
|
||||
let (res, _, db) = inspect_and_return_db(db, env, &mut inspector)?;
|
||||
f(tx_info, inspector, res, db)
|
||||
})
|
||||
.map(Some)
|
||||
}
|
||||
|
||||
@ -12,11 +12,13 @@ use crate::{
|
||||
use async_trait::async_trait;
|
||||
use jsonrpsee::core::RpcResult as Result;
|
||||
use reth_primitives::{BlockId, BlockNumberOrTag, Bytes, H256};
|
||||
use reth_provider::{BlockProvider, EvmEnvProvider, StateProviderFactory};
|
||||
use reth_provider::{BlockProvider, EvmEnvProvider, StateProviderBox, StateProviderFactory};
|
||||
use reth_revm::{
|
||||
database::{State, SubState},
|
||||
env::tx_env_with_recovered,
|
||||
tracing::{TracingInspector, TracingInspectorConfig},
|
||||
tracing::{
|
||||
parity::populate_account_balance_nonce_diffs, TracingInspector, TracingInspectorConfig,
|
||||
},
|
||||
};
|
||||
use reth_rpc_api::TraceApiServer;
|
||||
use reth_rpc_types::{
|
||||
@ -25,8 +27,8 @@ use reth_rpc_types::{
|
||||
BlockError, BlockOverrides, CallRequest, Index, TransactionInfo,
|
||||
};
|
||||
use reth_tasks::TaskSpawner;
|
||||
use revm::primitives::Env;
|
||||
use revm_primitives::{db::DatabaseCommit, ExecutionResult};
|
||||
use revm::{db::CacheDB, primitives::Env};
|
||||
use revm_primitives::{db::DatabaseCommit, ExecutionResult, ResultAndState};
|
||||
use std::{collections::HashSet, future::Future, sync::Arc};
|
||||
use tokio::sync::{oneshot, AcquireError, OwnedSemaphorePermit};
|
||||
|
||||
@ -127,11 +129,18 @@ where
|
||||
let config = tracing_config(&trace_types);
|
||||
let mut inspector = TracingInspector::new(config);
|
||||
|
||||
let (res, _) =
|
||||
self.inner.eth_api.inspect_call_at(call, at, overrides, &mut inspector).await?;
|
||||
let (res, _, db) = self
|
||||
.inner
|
||||
.eth_api
|
||||
.inspect_call_at_and_return_state(call, at, overrides, &mut inspector)
|
||||
.await?;
|
||||
|
||||
let trace_res = inspector.into_parity_builder().into_trace_results_with_state(
|
||||
res,
|
||||
&trace_types,
|
||||
&db,
|
||||
)?;
|
||||
|
||||
let trace_res =
|
||||
inspector.into_parity_builder().into_trace_results(res.result, &trace_types);
|
||||
Ok(trace_res)
|
||||
}
|
||||
|
||||
@ -155,10 +164,12 @@ where
|
||||
let config = tracing_config(&trace_types);
|
||||
|
||||
self.on_blocking_task(|this| async move {
|
||||
this.inner.eth_api.trace_at(env, config, at, |inspector, res| {
|
||||
let trace_res =
|
||||
inspector.into_parity_builder().into_trace_results(res.result, &trace_types);
|
||||
Ok(trace_res)
|
||||
this.inner.eth_api.trace_at_with_state(env, config, at, |inspector, res, db| {
|
||||
Ok(inspector.into_parity_builder().into_trace_results_with_state(
|
||||
res,
|
||||
&trace_types,
|
||||
&db,
|
||||
)?)
|
||||
})
|
||||
})
|
||||
.await
|
||||
@ -193,9 +204,11 @@ where
|
||||
let config = tracing_config(&trace_types);
|
||||
let mut inspector = TracingInspector::new(config);
|
||||
let (res, _) = inspect(&mut db, env, &mut inspector)?;
|
||||
let trace_res = inspector
|
||||
.into_parity_builder()
|
||||
.into_trace_results(res.result, &trace_types);
|
||||
let trace_res = inspector.into_parity_builder().into_trace_results_with_state(
|
||||
res,
|
||||
&trace_types,
|
||||
&db,
|
||||
)?;
|
||||
results.push(trace_res);
|
||||
}
|
||||
|
||||
@ -215,10 +228,12 @@ where
|
||||
self.on_blocking_task(|this| async move {
|
||||
this.inner
|
||||
.eth_api
|
||||
.trace_transaction_in_block(hash, config, |_, inspector, res| {
|
||||
let trace_res = inspector
|
||||
.into_parity_builder()
|
||||
.into_trace_results(res.result, &trace_types);
|
||||
.trace_transaction_in_block(hash, config, |_, inspector, res, db| {
|
||||
let trace_res = inspector.into_parity_builder().into_trace_results_with_state(
|
||||
res,
|
||||
&trace_types,
|
||||
&db,
|
||||
)?;
|
||||
Ok(trace_res)
|
||||
})
|
||||
.await
|
||||
@ -255,7 +270,7 @@ where
|
||||
.trace_transaction_in_block(
|
||||
hash,
|
||||
TracingInspectorConfig::default_parity(),
|
||||
|tx_info, inspector, _| {
|
||||
|tx_info, inspector, _, _| {
|
||||
let traces = inspector
|
||||
.into_parity_builder()
|
||||
.into_localized_transaction_traces(tx_info);
|
||||
@ -268,6 +283,14 @@ where
|
||||
}
|
||||
|
||||
/// Executes all transactions of a block and returns a list of callback results.
|
||||
///
|
||||
/// This
|
||||
/// 1. fetches all transactions of the block
|
||||
/// 2. configures the EVM evn
|
||||
/// 3. loops over all transactions and executes them
|
||||
/// 4. calls the callback with the transaction info, the execution result, the changed state
|
||||
/// _after_ the transaction [State] and the database that points to the state right _before_ the
|
||||
/// transaction.
|
||||
async fn trace_block_with<F, R>(
|
||||
&self,
|
||||
block_id: BlockId,
|
||||
@ -275,7 +298,16 @@ where
|
||||
f: F,
|
||||
) -> EthResult<Option<Vec<R>>>
|
||||
where
|
||||
F: Fn(TransactionInfo, TracingInspector, ExecutionResult) -> EthResult<R> + Send + 'static,
|
||||
// This is the callback that's invoked for each transaction with
|
||||
F: for<'a> Fn(
|
||||
TransactionInfo,
|
||||
TracingInspector,
|
||||
ExecutionResult,
|
||||
&'a revm_primitives::State,
|
||||
&'a CacheDB<State<StateProviderBox<'a>>>,
|
||||
) -> EthResult<R>
|
||||
+ Send
|
||||
+ 'static,
|
||||
R: Send + 'static,
|
||||
{
|
||||
let ((cfg, block_env, _), block) = futures::try_join!(
|
||||
@ -320,14 +352,15 @@ where
|
||||
|
||||
let mut inspector = TracingInspector::new(config);
|
||||
let (res, _) = inspect(&mut db, env, &mut inspector)?;
|
||||
results.push(f(tx_info, inspector, res.result)?);
|
||||
let ResultAndState { result, state } = res;
|
||||
results.push(f(tx_info, inspector, result, &state, &db)?);
|
||||
|
||||
// need to apply the state changes of this transaction before executing the
|
||||
// next transaction
|
||||
if transactions.peek().is_some() {
|
||||
// need to apply the state changes of this transaction before executing
|
||||
// the next transaction
|
||||
db.commit(res.state)
|
||||
db.commit(state)
|
||||
}
|
||||
}
|
||||
|
||||
@ -347,7 +380,7 @@ where
|
||||
.trace_block_with(
|
||||
block_id,
|
||||
TracingInspectorConfig::default_parity(),
|
||||
|tx_info, inspector, _| {
|
||||
|tx_info, inspector, _, _, _| {
|
||||
let traces =
|
||||
inspector.into_parity_builder().into_localized_transaction_traces(tx_info);
|
||||
Ok(traces)
|
||||
@ -367,9 +400,17 @@ where
|
||||
self.trace_block_with(
|
||||
block_id,
|
||||
tracing_config(&trace_types),
|
||||
move |tx_info, inspector, res| {
|
||||
let full_trace =
|
||||
move |tx_info, inspector, res, state, db| {
|
||||
let mut full_trace =
|
||||
inspector.into_parity_builder().into_trace_results(res, &trace_types);
|
||||
if let Some(ref mut state_diff) = full_trace.state_diff {
|
||||
populate_account_balance_nonce_diffs(
|
||||
state_diff,
|
||||
db,
|
||||
state.iter().map(|(addr, acc)| (*addr, acc.info.clone())),
|
||||
)?;
|
||||
}
|
||||
|
||||
let trace = TraceResultsWithTransactionHash {
|
||||
transaction_hash: tx_info.hash.expect("tx hash is set"),
|
||||
full_trace,
|
||||
|
||||
Reference in New Issue
Block a user