From 1f1c8181428fd10329faac57adaf0e50778c792c Mon Sep 17 00:00:00 2001 From: Guillaume Potier Date: Thu, 9 Jan 2025 13:11:33 +0100 Subject: [PATCH] feat(rpc) Implement `Filecoin.EthTraceBlock` (#4991) --- CHANGELOG.md | 3 + Cargo.lock | 16 + Cargo.toml | 2 + src/eth/transaction.rs | 13 + src/rpc/methods/eth.rs | 132 ++++--- src/rpc/methods/eth/trace.rs | 624 ++++++++++++++++++++++++++++++++ src/rpc/methods/eth/types.rs | 78 ++++ src/rpc/methods/eth/utils.rs | 163 +++++++++ src/rpc/methods/state/types.rs | 14 +- src/rpc/mod.rs | 1 + src/shim/error.rs | 86 ++++- src/state_manager/mod.rs | 39 ++ src/tool/subcommands/api_cmd.rs | 4 + 13 files changed, 1108 insertions(+), 67 deletions(-) create mode 100644 src/rpc/methods/eth/trace.rs create mode 100644 src/rpc/methods/eth/utils.rs diff --git a/CHANGELOG.md b/CHANGELOG.md index 32b4797e5ad..6e79a9a2dbb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -29,6 +29,9 @@ ### Added +- [#4708](https://github.com/ChainSafe/forest/issues/4708) Add support for the + `Filecoin.EthTraceBlock` RPC method. + ### Changed ### Removed diff --git a/Cargo.lock b/Cargo.lock index c3c44d811dd..c68eb0085c6 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2582,6 +2582,21 @@ dependencies = [ "serde", ] +[[package]] +name = "fil_actor_eam_state" +version = "19.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f8a3553ead569004c9e3a9dac7802f9c7a1c5c47c8d6a832cb55725497a50a2" +dependencies = [ + "fil_actor_evm_state", + "fvm_ipld_encoding", + "fvm_shared 3.12.0", + "fvm_shared 4.5.3", + "num-derive", + "num-traits", + "serde", +] + [[package]] name = "fil_actor_evm_state" version = "19.0.0" @@ -3033,6 +3048,7 @@ dependencies = [ "fil_actor_account_state", "fil_actor_cron_state", "fil_actor_datacap_state", + "fil_actor_eam_state", "fil_actor_evm_state", "fil_actor_init_state", "fil_actor_market_state", diff --git a/Cargo.toml b/Cargo.toml index 959b9f0adc3..f428f4d9de8 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -63,6 +63,7 @@ ez-jsonrpc-types = "0.5" fil_actor_account_state = { version = "19" } fil_actor_cron_state = { version = "19" } fil_actor_datacap_state = { version = "19" } +fil_actor_eam_state = { version = "19" } fil_actor_evm_state = { version = "19" } fil_actor_init_state = { version = "19" } fil_actor_market_state = { version = "19" } @@ -227,6 +228,7 @@ ariadne = "0.5" assert_cmd = "2" bimap = "0.6" cargo_metadata = "0.19" +cbor4ii = { version = "0.2", default-features = false, features = ["serde1"] } criterion = { version = "0.5", features = ["async_tokio", "csv"] } cs_serde_bytes = "0.12" derive-quickcheck-arbitrary = "0.1" diff --git a/src/eth/transaction.rs b/src/eth/transaction.rs index 22539f5ecd9..1647199e620 100644 --- a/src/eth/transaction.rs +++ b/src/eth/transaction.rs @@ -9,7 +9,9 @@ use anyhow::{bail, ensure, Context}; use bytes::BufMut; use bytes::BytesMut; use cbor4ii::core::{dec::Decode as _, utils::SliceReader, Value}; +use fvm_shared4::METHOD_CONSTRUCTOR; use num::{bigint::Sign, BigInt, Signed as _}; +use num_derive::FromPrimitive; use num_traits::cast::ToPrimitive; use rlp::Rlp; @@ -32,13 +34,24 @@ use super::{ EthChainId, EIP_1559_TX_TYPE, EIP_2930_TX_TYPE, }; // As per `ref-fvm`, which hardcodes it as well. +#[derive(FromPrimitive)] #[repr(u64)] pub enum EAMMethod { + Constructor = METHOD_CONSTRUCTOR, + Create = 2, + Create2 = 3, CreateExternal = 4, } +#[derive(FromPrimitive)] #[repr(u64)] pub enum EVMMethod { + Constructor = METHOD_CONSTRUCTOR, + Resurrect = 2, + GetBytecode = 3, + GetBytecodeHash = 4, + GetStorageAt = 5, + InvokeContractDelegate = 6, // As per `ref-fvm`: // it is very unfortunate but the hasher creates a circular dependency, so we use the raw // number. diff --git a/src/rpc/methods/eth.rs b/src/rpc/methods/eth.rs index ae007ff4a23..7a798bb7121 100644 --- a/src/rpc/methods/eth.rs +++ b/src/rpc/methods/eth.rs @@ -3,7 +3,9 @@ mod eth_tx; pub mod filter; +mod trace; pub mod types; +mod utils; use self::eth_tx::*; use self::filter::hex_str_to_epoch; @@ -22,12 +24,14 @@ use crate::interpreter::VMTrace; use crate::lotus_json::{lotus_json_with_self, HasLotusJson}; use crate::message::{ChainMessage, Message as _, SignedMessage}; use crate::rpc::error::ServerError; +use crate::rpc::eth::types::EthBlockTrace; use crate::rpc::types::{ApiTipsetKey, EventEntry, MessageLookup}; use crate::rpc::EthEventHandler; use crate::rpc::{ApiPaths, Ctx, Permission, RpcMethod}; use crate::shim::actors::eam; use crate::shim::actors::evm; use crate::shim::actors::is_evm_actor; +use crate::shim::actors::system; use crate::shim::actors::EVMActorStateLoad as _; use crate::shim::address::{Address as FilecoinAddress, Protocol}; use crate::shim::crypto::Signature; @@ -44,8 +48,6 @@ use crate::utils::db::BlockstoreExt as _; use crate::utils::encoding::from_slice_with_fallback; use crate::utils::multihash::prelude::*; use anyhow::{anyhow, bail, Context, Error, Result}; -use cbor4ii::core::dec::Decode as _; -use cbor4ii::core::Value; use cid::Cid; use fvm_ipld_blockstore::Blockstore; use fvm_ipld_encoding::{RawBytes, CBOR, DAG_CBOR, IPLD_RAW}; @@ -56,6 +58,7 @@ use schemars::JsonSchema; use serde::{Deserialize, Serialize}; use std::str::FromStr; use std::{ops::Add, sync::Arc}; +use utils::{decode_payload, lookup_eth_address}; const MASKED_ID_PREFIX: [u8; 12] = [0xff, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0]; @@ -185,7 +188,7 @@ impl EthHash { let mh = MultihashCode::Blake2b256 .wrap(self.0.as_bytes()) .expect("should not fail"); - Cid::new_v1(fvm_ipld_encoding::DAG_CBOR, mh) + Cid::new_v1(DAG_CBOR, mh) } pub fn empty_uncles() -> Self { @@ -819,47 +822,6 @@ pub fn eth_tx_from_signed_eth_message( Ok((from, tx)) } -fn lookup_eth_address( - addr: &FilecoinAddress, - state: &StateTree, -) -> Result> { - // Attempt to convert directly, if it's an f4 address. - if let Ok(eth_addr) = EthAddress::from_filecoin_address(addr) { - if !eth_addr.is_masked_id() { - return Ok(Some(eth_addr)); - } - } - - // Otherwise, resolve the ID addr. - let id_addr = match state.lookup_id(addr)? { - Some(id) => id, - _ => return Ok(None), - }; - - // Lookup on the target actor and try to get an f410 address. - let result = state.get_actor(addr); - if let Ok(Some(actor_state)) = result { - if let Some(addr) = actor_state.delegated_address { - if let Ok(eth_addr) = EthAddress::from_filecoin_address(&addr.into()) { - if !eth_addr.is_masked_id() { - // Conversable into an eth address, use it. - return Ok(Some(eth_addr)); - } - } - } else { - // No delegated address -> use a masked ID address - } - } else if let Ok(None) = result { - // Not found -> use a masked ID address - } else { - // Any other error -> fail. - result?; - } - - // Otherwise, use the masked address. - Ok(Some(EthAddress::from_actor_id(id_addr))) -} - /// See /// for ABI specification fn encode_filecoin_params_as_abi( @@ -916,21 +878,6 @@ fn encode_as_abi_helper(param1: u64, param2: u64, data: &[u8]) -> Vec { buf } -/// Decodes the payload using the given codec. -fn decode_payload(payload: &fvm_ipld_encoding::RawBytes, codec: u64) -> Result { - match codec { - DAG_CBOR | CBOR => { - let mut reader = cbor4ii::core::utils::SliceReader::new(payload.bytes()); - match Value::decode(&mut reader) { - Ok(Value::Bytes(bytes)) => Ok(EthBytes(bytes)), - _ => bail!("failed to read params byte array"), - } - } - IPLD_RAW => Ok(EthBytes(payload.to_vec())), - _ => bail!("decode_payload: unsupported codec {codec}"), - } -} - /// Convert a native message to an eth transaction. /// /// - The state-tree must be from after the message was applied (ideally the following tipset). @@ -2655,6 +2602,73 @@ impl RpcMethod<1> for EthGetLogs { } } +pub enum EthTraceBlock {} +impl RpcMethod<1> for EthTraceBlock { + const NAME: &'static str = "Filecoin.EthTraceBlock"; + const NAME_ALIAS: Option<&'static str> = Some("eth_traceBlock"); + const N_REQUIRED_PARAMS: usize = 1; + const PARAM_NAMES: [&'static str; 1] = ["block_param"]; + const API_PATHS: ApiPaths = ApiPaths::V1; + const PERMISSION: Permission = Permission::Read; + type Params = (BlockNumberOrHash,); + type Ok = Vec; + async fn handle( + ctx: Ctx, + (block_param,): Self::Params, + ) -> Result { + let ts = tipset_by_block_number_or_hash(ctx.chain_store(), block_param)?; + + let (state_root, trace) = ctx.state_manager.execution_trace(&ts)?; + + let state = StateTree::new_from_root(ctx.store_owned(), &state_root)?; + + let cid = ts.key().cid()?; + + let block_hash: EthHash = cid.into(); + + let mut all_traces = vec![]; + let mut msg_idx = 0; + for ir in trace.into_iter() { + // ignore messages from system actor + if ir.msg.from == system::ADDRESS.into() { + continue; + } + + msg_idx += 1; + + let tx_hash = EthGetTransactionHashByCid::handle(ctx.clone(), (ir.msg_cid,)).await?; + + let tx_hash = tx_hash + .with_context(|| format!("cannot find transaction hash for cid {}", ir.msg_cid))?; + + let mut env = trace::base_environment(&state, &ir.msg.from) + .map_err(|e| format!("when processing message {}: {}", ir.msg_cid, e))?; + + if let Some(execution_trace) = ir.execution_trace { + trace::build_traces(&mut env, &[], execution_trace)?; + + for trace in env.traces { + all_traces.push(EthBlockTrace { + r#type: trace.r#type, + subtraces: trace.subtraces, + trace_address: trace.trace_address, + action: trace.action, + result: trace.result, + error: trace.error, + + block_hash: block_hash.clone(), + block_number: ts.epoch(), + transaction_hash: tx_hash.clone(), + transaction_position: msg_idx as i64, + }); + } + } + } + + Ok(all_traces) + } +} + #[cfg(test)] mod test { use super::*; diff --git a/src/rpc/methods/eth/trace.rs b/src/rpc/methods/eth/trace.rs new file mode 100644 index 00000000000..45959f3df4e --- /dev/null +++ b/src/rpc/methods/eth/trace.rs @@ -0,0 +1,624 @@ +// Copyright 2019-2025 ChainSafe Systems +// SPDX-License-Identifier: Apache-2.0, MIT + +use super::types::{ + EthAddress, EthBlockTrace, EthBytes, EthCallTraceAction, TraceAction, TraceResult, +}; +use super::utils::{decode_params, decode_return}; +use super::{ + decode_payload, encode_filecoin_params_as_abi, encode_filecoin_returns_as_abi, + EthCallTraceResult, EthCreateTraceAction, EthCreateTraceResult, +}; +use crate::eth::{EAMMethod, EVMMethod}; +use crate::rpc::methods::eth::lookup_eth_address; +use crate::rpc::methods::state::ExecutionTrace; +use crate::rpc::state::ActorTrace; +use crate::shim::fvm_shared_latest::METHOD_CONSTRUCTOR; +use crate::shim::{actors::is_evm_actor, address::Address, error::ExitCode, state_tree::StateTree}; +use fil_actor_eam_state::v12 as eam12; +use fil_actor_evm_state::v15 as evm12; +use fil_actor_init_state::v12::ExecReturn; +use fil_actor_init_state::v15::Method as InitMethod; +use fvm_ipld_blockstore::Blockstore; + +use anyhow::{bail, Context}; +use num::FromPrimitive; +use tracing::debug; + +#[derive(Default)] +pub struct Environment { + caller: EthAddress, + is_evm: bool, + subtrace_count: i64, + pub traces: Vec, + last_byte_code: Option, +} + +pub fn base_environment( + state: &StateTree, + from: &Address, +) -> anyhow::Result { + let sender = lookup_eth_address(from, state)? + .with_context(|| format!("top-level message sender {} s could not be found", from))?; + Ok(Environment { + caller: sender, + ..Environment::default() + }) +} + +fn trace_to_address(trace: &ActorTrace) -> EthAddress { + if let Some(addr) = trace.state.delegated_address { + if let Ok(eth_addr) = EthAddress::from_filecoin_address(&addr.into()) { + return eth_addr; + } + } + EthAddress::from_actor_id(trace.id) +} + +/// Returns true if the trace is a call to an EVM or EAM actor. +fn trace_is_evm_or_eam(trace: &ExecutionTrace) -> bool { + if let Some(invoked_actor) = &trace.invoked_actor { + is_evm_actor(&invoked_actor.state.code) + || invoked_actor.id != Address::ETHEREUM_ACCOUNT_MANAGER_ACTOR.id().unwrap() + } else { + false + } +} + +/// Returns true if the trace is a call to an EVM or EAM actor. +fn trace_err_msg(trace: &ExecutionTrace) -> Option { + let code = trace.msg_rct.exit_code; + + if code.is_success() { + return None; + } + + // EVM tools often expect this literal string. + if code == ExitCode::SYS_OUT_OF_GAS { + return Some("out of gas".into()); + } + + // indicate when we have a "system" error. + if code < ExitCode::FIRST_ACTOR_ERROR_CODE.into() { + return Some(format!("vm error: {}", code)); + } + + // handle special exit codes from the EVM/EAM. + if trace_is_evm_or_eam(trace) { + match code.into() { + evm12::EVM_CONTRACT_REVERTED => return Some("Reverted".into()), // capitalized for compatibility + evm12::EVM_CONTRACT_INVALID_INSTRUCTION => return Some("invalid instruction".into()), + evm12::EVM_CONTRACT_UNDEFINED_INSTRUCTION => { + return Some("undefined instruction".into()) + } + evm12::EVM_CONTRACT_STACK_UNDERFLOW => return Some("stack underflow".into()), + evm12::EVM_CONTRACT_STACK_OVERFLOW => return Some("stack overflow".into()), + evm12::EVM_CONTRACT_ILLEGAL_MEMORY_ACCESS => { + return Some("illegal memory access".into()) + } + evm12::EVM_CONTRACT_BAD_JUMPDEST => return Some("invalid jump destination".into()), + evm12::EVM_CONTRACT_SELFDESTRUCT_FAILED => return Some("self destruct failed".into()), + _ => (), + } + } + // everything else... + Some(format!("actor error: {}", code)) +} + +/// Recursively builds the traces for a given ExecutionTrace by walking the subcalls +pub fn build_traces( + env: &mut Environment, + address: &[i64], + trace: ExecutionTrace, +) -> anyhow::Result<()> { + let (trace, recurse_into) = build_trace(env, address, trace)?; + + let last_trace_idx = if let Some(trace) = trace { + let len = env.traces.len(); + env.traces.push(trace); + env.subtrace_count += 1; + Some(len) + } else { + None + }; + + // Skip if there's nothing more to do and/or `build_trace` told us to skip this one. + let (recurse_into, invoked_actor) = if let Some(trace) = recurse_into { + if let Some(invoked_actor) = &trace.invoked_actor { + let invoked_actor = invoked_actor.clone(); + (trace, invoked_actor) + } else { + return Ok(()); + } + } else { + return Ok(()); + }; + + let mut sub_env = Environment { + caller: trace_to_address(&invoked_actor), + is_evm: is_evm_actor(&invoked_actor.state.code), + traces: env.traces.clone(), + ..Environment::default() + }; + for subcall in recurse_into.subcalls.into_iter() { + let mut new_address = address.to_vec(); + new_address.push(sub_env.subtrace_count); + build_traces(&mut sub_env, &new_address, subcall)?; + } + env.traces = sub_env.traces; + if let Some(idx) = last_trace_idx { + env.traces.get_mut(idx).expect("Infallible").subtraces = sub_env.subtrace_count; + } + + Ok(()) +} + +// `build_trace` processes the passed execution trace and updates the environment, if necessary. +// +// On success, it returns a trace to add (or `None` to skip) and the trace to recurse into (or `None` to skip). +fn build_trace( + env: &mut Environment, + address: &[i64], + trace: ExecutionTrace, +) -> anyhow::Result<(Option, Option)> { + // This function first assumes that the call is a "native" call, then handles all the "not + // native" cases. If we get any unexpected results in any of these special cases, we just + // keep the "native" interpretation and move on. + // + // 1. If we're invoking a contract (even if the caller is a native account/actor), we + // attempt to decode the params/return value as a contract invocation. + // 2. If we're calling the EAM and/or init actor, we try to treat the call as a CREATE. + // 3. Finally, if the caller is an EVM smart contract and it's calling a "private" (1-1023) + // method, we know something special is going on. We look for calls related to + // DELEGATECALL and drop everything else (everything else includes calls triggered by, + // e.g., EXTCODEHASH). + + // If we don't have sufficient funds, or we have a fatal error, or we have some + // other syscall error: skip the entire trace to mimic Ethereum (Ethereum records + // traces _after_ checking things like this). + // + // NOTE: The FFI currently folds all unknown syscall errors into "sys assertion + // failed" which is turned into SysErrFatal. + if !address.is_empty() + && Into::::into(trace.msg_rct.exit_code) == ExitCode::SYS_INSUFFICIENT_FUNDS + { + return Ok((None, None)); + } + + // We may fail before we can even invoke the actor. In that case, we have no 100% reliable + // way of getting its address (e.g., due to reverts) so we're just going to drop the entire + // trace. This is OK (ish) because the call never really "happened". + if trace.invoked_actor.is_none() { + return Ok((None, None)); + } + + // Step 2: Decode as a contract invocation + // + // Normal EVM calls. We don't care if the caller/receiver are actually EVM actors, we only + // care if the call _looks_ like an EVM call. If we fail to decode it as an EVM call, we + // fallback on interpreting it as a native call. + let method = EVMMethod::from_u64(trace.msg.method); + if let Some(EVMMethod::InvokeContract) = method { + let (trace, exec_trace) = trace_evm_call(env, address, trace)?; + return Ok((Some(trace), Some(exec_trace))); + } + + // Step 3: Decode as a contract deployment + match trace.msg.to { + Address::INIT_ACTOR => { + let method = InitMethod::from_u64(trace.msg.method); + match method { + Some(InitMethod::Exec) | Some(InitMethod::Exec4) => { + return trace_native_create(env, address, &trace); + } + _ => (), + } + } + Address::ETHEREUM_ACCOUNT_MANAGER_ACTOR => { + let method = EAMMethod::from_u64(trace.msg.method); + match method { + Some(EAMMethod::Create) + | Some(EAMMethod::Create2) + | Some(EAMMethod::CreateExternal) => { + return trace_eth_create(env, address, &trace); + } + _ => (), + } + } + _ => (), + } + + // Step 4: Handle DELEGATECALL + // + // EVM contracts cannot call methods in the range 1-1023, only the EVM itself can. So, if we + // see a call in this range, we know it's an implementation detail of the EVM and not an + // explicit call by the user. + // + // While the EVM calls several methods in this range (some we've already handled above with + // respect to the EAM), we only care about the ones relevant DELEGATECALL and can _ignore_ + // all the others. + if env.is_evm && trace.msg.method > 0 && trace.msg.method < 1024 { + return trace_evm_private(env, address, &trace); + } + + Ok((Some(trace_native_call(env, address, &trace)?), Some(trace))) +} + +// Build an EthTrace for a "call" with the given input & output. +fn trace_call( + env: &mut Environment, + address: &[i64], + trace: &ExecutionTrace, + input: EthBytes, + output: EthBytes, +) -> anyhow::Result { + if let Some(invoked_actor) = &trace.invoked_actor { + let to = trace_to_address(invoked_actor); + let call_type: String = if trace.msg.read_only.unwrap_or_default() { + "staticcall" + } else { + "call" + } + .into(); + + let default = EthBlockTrace::default(); + Ok(EthBlockTrace { + r#type: "call".into(), + action: TraceAction::Call(EthCallTraceAction { + call_type, + from: env.caller.clone(), + to: Some(to), + gas: trace.msg.gas_limit.unwrap_or_default().into(), + value: trace.msg.value.clone().into(), + input, + }), + result: TraceResult::Call(EthCallTraceResult { + gas_used: trace.sum_gas().total_gas.into(), + output, + }), + trace_address: Vec::from(address), + error: trace_err_msg(trace), + ..default + }) + } else { + bail!("no invoked actor") + } +} + +// Build an EthTrace for a "call", parsing the inputs & outputs as a "native" FVM call. +fn trace_native_call( + env: &mut Environment, + address: &[i64], + trace: &ExecutionTrace, +) -> anyhow::Result { + trace_call( + env, + address, + trace, + encode_filecoin_params_as_abi(trace.msg.method, trace.msg.params_codec, &trace.msg.params)?, + EthBytes(encode_filecoin_returns_as_abi( + trace.msg_rct.exit_code.value().into(), + trace.msg_rct.return_codec, + &trace.msg_rct.r#return, + )), + ) +} + +// Build an EthTrace for a "call", parsing the inputs & outputs as an EVM call (falling back on +// treating it as a native call). +fn trace_evm_call( + env: &mut Environment, + address: &[i64], + trace: ExecutionTrace, +) -> anyhow::Result<(EthBlockTrace, ExecutionTrace)> { + let input = match decode_payload(&trace.msg.params, trace.msg.params_codec) { + Ok(value) => value, + Err(err) => { + debug!("failed to decode contract invocation payload: {err}"); + return Ok((trace_native_call(env, address, &trace)?, trace)); + } + }; + let output = match decode_payload(&trace.msg_rct.r#return, trace.msg_rct.return_codec) { + Ok(value) => value, + Err(err) => { + debug!("failed to decode contract invocation return: {err}"); + return Ok((trace_native_call(env, address, &trace)?, trace)); + } + }; + Ok((trace_call(env, address, &trace, input, output)?, trace)) +} + +// Build an EthTrace for a native "create" operation. This should only be called with an +// ExecutionTrace is an Exec or Exec4 method invocation on the Init actor. + +fn trace_native_create( + env: &mut Environment, + address: &[i64], + trace: &ExecutionTrace, +) -> anyhow::Result<(Option, Option)> { + if trace.msg.read_only.unwrap_or_default() { + // "create" isn't valid in a staticcall, so we just skip this trace + // (couldn't have created an actor anyways). + // This mimic's the EVM: it doesn't trace CREATE calls when in + // read-only mode. + return Ok((None, None)); + } + + let sub_trace = trace + .subcalls + .iter() + .find(|c| c.msg.method == METHOD_CONSTRUCTOR); + + let sub_trace = if let Some(sub_trace) = sub_trace { + sub_trace + } else { + // If we succeed in calling Exec/Exec4 but don't even try to construct + // something, we have a bug in our tracing logic or a mismatch between our + // tracing logic and the actors. + if trace.msg_rct.exit_code.is_success() { + bail!("successful Exec/Exec4 call failed to call a constructor"); + } + // Otherwise, this can happen if creation fails early (bad params, + // out of gas, contract already exists, etc.). The EVM wouldn't + // trace such cases, so we don't either. + // + // NOTE: It's actually impossible to run out of gas before calling + // initcode in the EVM (without running out of gas in the calling + // contract), but this is an equivalent edge-case to InvokedActor + // being nil, so we treat it the same way and skip the entire + // operation. + return Ok((None, None)); + }; + + // Native actors that aren't the EAM can attempt to call Exec4, but such + // call should fail immediately without ever attempting to construct an + // actor. I'm catching this here because it likely means that there's a bug + // in our trace-conversion logic. + if trace.msg.method == (InitMethod::Exec4 as u64) { + bail!("direct call to Exec4 successfully called a constructor!"); + } + + let mut output = EthBytes::default(); + let mut create_addr = EthAddress::default(); + if trace.msg_rct.exit_code.is_success() { + // We're supposed to put the "installed bytecode" here. But this + // isn't an EVM actor, so we just put some invalid bytecode (this is + // the answer you'd get if you called EXTCODECOPY on a native + // non-account actor, anyways). + output = EthBytes(vec![0xFE]); + + // Extract the address of the created actor from the return value. + let init_return: ExecReturn = decode_return(&trace.msg_rct)?; + let actor_id = init_return.id_address.id()?; + let eth_addr = EthAddress::from_actor_id(actor_id); + create_addr = eth_addr; + } + + Ok(( + Some(EthBlockTrace { + r#type: "create".into(), + action: TraceAction::Create(EthCreateTraceAction { + from: env.caller.clone(), + gas: trace.msg.gas_limit.unwrap_or_default().into(), + value: trace.msg.value.clone().into(), + // If we get here, this isn't a native EVM create. Those always go through + // the EAM. So we have no "real" initcode and must use the sentinel value + // for "invalid" initcode. + init: EthBytes(vec![0xFE]), + }), + result: TraceResult::Create(EthCreateTraceResult { + gas_used: trace.sum_gas().total_gas.into(), + address: Some(create_addr), + code: output, + }), + trace_address: Vec::from(address), + error: trace_err_msg(trace), + ..EthBlockTrace::default() + }), + Some(sub_trace.clone()), + )) +} + +// Decode the parameters and return value of an EVM smart contract creation through the EAM. This +// should only be called with an ExecutionTrace for a Create, Create2, or CreateExternal method +// invocation on the EAM. +fn decode_create_via_eam(trace: &ExecutionTrace) -> anyhow::Result<(Vec, EthAddress)> { + let init_code = match EAMMethod::from_u64(trace.msg.method) { + Some(EAMMethod::Create) => { + let params = decode_params::(&trace.msg)?; + params.initcode + } + Some(EAMMethod::Create2) => { + let params = decode_params::(&trace.msg)?; + params.initcode + } + Some(EAMMethod::CreateExternal) => { + decode_payload(&trace.msg.params, trace.msg.params_codec)?.into() + } + _ => bail!("unexpected CREATE method {}", trace.msg.method), + }; + let ret = decode_return::(&trace.msg_rct)?; + + Ok((init_code, ret.eth_address.0.into())) +} + +// Build an EthTrace for an EVM "create" operation. This should only be called with an +// ExecutionTrace for a Create, Create2, or CreateExternal method invocation on the EAM. +fn trace_eth_create( + env: &mut Environment, + address: &[i64], + trace: &ExecutionTrace, +) -> anyhow::Result<(Option, Option)> { + // Same as the Init actor case above, see the comment there. + if trace.msg.read_only.unwrap_or_default() { + return Ok((None, None)); + } + + // Look for a call to either a constructor or the EVM's resurrect method. + let sub_trace = trace + .subcalls + .iter() + .filter_map(|et| { + if et.msg.to == Address::INIT_ACTOR { + et.subcalls + .iter() + .find(|et| et.msg.method == METHOD_CONSTRUCTOR) + } else { + match EVMMethod::from_u64(et.msg.method) { + Some(EVMMethod::Resurrect) => Some(et), + _ => None, + } + } + }) + .next(); + + // Same as the Init actor case above, see the comment there. + let sub_trace = if let Some(sub_trace) = sub_trace { + sub_trace + } else { + if trace.msg_rct.exit_code.is_success() { + bail!("successful Create/Create2 call failed to call a constructor"); + } + return Ok((None, None)); + }; + + // Decode inputs & determine create type. + let (init_code, create_addr) = decode_create_via_eam(trace)?; + + // Handle the output. + let output = match trace.msg_rct.exit_code.value() { + 0 => { + // success + // We're _supposed_ to include the contracts bytecode here, but we + // can't do that reliably (e.g., if some part of the trace reverts). + // So we don't try and include a sentinel "impossible bytecode" + // value (the value specified by EIP-3541). + EthBytes(vec![0xFE]) + } + 33 => { + // Reverted, parse the revert message. + // If we managed to call the constructor, parse/return its revert message. If we + // fail, we just return no output. + decode_payload(&sub_trace.msg_rct.r#return, sub_trace.msg_rct.return_codec)? + } + _ => EthBytes::default(), + }; + + Ok(( + Some(EthBlockTrace { + r#type: "create".into(), + action: TraceAction::Create(EthCreateTraceAction { + from: env.caller.clone(), + gas: trace.msg.gas_limit.unwrap_or_default().into(), + value: trace.msg.value.clone().into(), + init: init_code.into(), + }), + result: TraceResult::Create(EthCreateTraceResult { + gas_used: trace.sum_gas().total_gas.into(), + address: Some(create_addr), + code: output, + }), + trace_address: Vec::from(address), + error: trace_err_msg(trace), + ..EthBlockTrace::default() + }), + Some(sub_trace.clone()), + )) +} + +// Build an EthTrace for a "private" method invocation from the EVM. This should only be called with +// an ExecutionTrace from an EVM instance and on a method between 1 and 1023 inclusive. +fn trace_evm_private( + env: &mut Environment, + address: &[i64], + trace: &ExecutionTrace, +) -> anyhow::Result<(Option, Option)> { + // The EVM actor implements DELEGATECALL by: + // + // 1. Asking the callee for its bytecode by calling it on the GetBytecode method. + // 2. Recursively invoking the currently executing contract on the + // InvokeContractDelegate method. + // + // The code below "reconstructs" that delegate call by: + // + // 1. Remembering the last contract on which we called GetBytecode. + // 2. Treating the contract invoked in step 1 as the DELEGATECALL receiver. + // + // Note, however: GetBytecode will be called, e.g., if the user invokes the + // EXTCODECOPY instruction. It's not an error to see multiple GetBytecode calls + // before we see an InvokeContractDelegate. + match EVMMethod::from_u64(trace.msg.method) { + Some(EVMMethod::GetBytecode) => { + // NOTE: I'm not checking anything about the receiver here. The EVM won't + // DELEGATECALL any non-EVM actor, but there's no need to encode that fact + // here in case we decide to loosen this up in the future. + env.last_byte_code = None; + if trace.msg_rct.exit_code.is_success() { + if let Option::Some(actor_trace) = &trace.invoked_actor { + let to = trace_to_address(actor_trace); + env.last_byte_code = Some(to); + } + } + Ok((None, None)) + } + Some(EVMMethod::InvokeContractDelegate) => { + // NOTE: We return errors in all the failure cases below instead of trying + // to continue because the caller is an EVM actor. If something goes wrong + // here, there's a bug in our EVM implementation. + + // Handle delegate calls + // + // 1) Look for trace from an EVM actor to itself on InvokeContractDelegate, + // method 6. + // 2) Check that the previous trace calls another actor on method 3 + // (GetByteCode) and they are at the same level (same parent) + // 3) Treat this as a delegate call to actor A. + if env.last_byte_code.is_none() { + bail!("unknown bytecode for delegate call"); + } + + if let Option::Some(actor_trace) = &trace.invoked_actor { + let to = trace_to_address(actor_trace); + if env.caller != to { + bail!( + "delegate-call not from address to self: {:?} != {:?}", + env.caller, + to + ); + } + } + + let dp = decode_params::(&trace.msg)?; + + let output = decode_payload(&trace.msg_rct.r#return, trace.msg_rct.return_codec) + .map_err(|e| anyhow::anyhow!("failed to decode delegate-call return: {}", e))?; + + Ok(( + Some(EthBlockTrace { + r#type: "call".into(), + action: TraceAction::Call(EthCallTraceAction { + call_type: "delegatecall".into(), + from: env.caller.clone(), + to: env.last_byte_code.clone(), + gas: trace.msg.gas_limit.unwrap_or_default().into(), + value: trace.msg.value.clone().into(), + input: dp.input.into(), + }), + result: TraceResult::Call(EthCallTraceResult { + gas_used: trace.sum_gas().total_gas.into(), + output, + }), + trace_address: Vec::from(address), + error: trace_err_msg(trace), + ..EthBlockTrace::default() + }), + Some(trace.clone()), + )) + } + _ => { + // We drop all other "private" calls from FEVM. We _forbid_ explicit calls between 0 and + // 1024 (exclusive), so any calls in this range must be implementation details. + Ok((None, None)) + } + } +} diff --git a/src/rpc/methods/eth/types.rs b/src/rpc/methods/eth/types.rs index 1b4c5e5c947..cac1efc27aa 100644 --- a/src/rpc/methods/eth/types.rs +++ b/src/rpc/methods/eth/types.rs @@ -447,6 +447,84 @@ pub enum EthFilterResult { } lotus_json_with_self!(EthFilterResult); +#[derive(PartialEq, Default, Serialize, Deserialize, Debug, Clone, JsonSchema)] +#[serde(rename_all = "camelCase")] +pub struct EthCallTraceAction { + pub call_type: String, + pub from: EthAddress, + pub to: Option, + pub gas: EthUint64, + pub value: EthBigInt, + pub input: EthBytes, +} + +#[derive(PartialEq, Default, Serialize, Deserialize, Debug, Clone, JsonSchema)] +#[serde(rename_all = "camelCase")] +pub struct EthCreateTraceAction { + pub from: EthAddress, + pub gas: EthUint64, + pub value: EthBigInt, + pub init: EthBytes, +} + +#[derive(PartialEq, Debug, Clone, Serialize, Deserialize, JsonSchema)] +#[serde(untagged)] +pub enum TraceAction { + Call(EthCallTraceAction), + Create(EthCreateTraceAction), +} + +impl Default for TraceAction { + fn default() -> Self { + TraceAction::Call(EthCallTraceAction::default()) + } +} + +#[derive(PartialEq, Default, Serialize, Deserialize, Debug, Clone, JsonSchema)] +#[serde(rename_all = "camelCase")] +pub struct EthCallTraceResult { + pub gas_used: EthUint64, + pub output: EthBytes, +} + +#[derive(PartialEq, Default, Serialize, Deserialize, Debug, Clone, JsonSchema)] +#[serde(rename_all = "camelCase")] +pub struct EthCreateTraceResult { + pub address: Option, + pub gas_used: EthUint64, + pub code: EthBytes, +} + +#[derive(PartialEq, Debug, Clone, Serialize, Deserialize, JsonSchema)] +#[serde(untagged)] +pub enum TraceResult { + Call(EthCallTraceResult), + Create(EthCreateTraceResult), +} + +impl Default for TraceResult { + fn default() -> Self { + TraceResult::Call(EthCallTraceResult::default()) + } +} + +#[derive(PartialEq, Default, Serialize, Deserialize, Debug, Clone, JsonSchema)] +#[serde(rename_all = "camelCase")] +pub struct EthBlockTrace { + pub r#type: String, + pub subtraces: i64, + pub trace_address: Vec, + pub action: TraceAction, + pub result: TraceResult, + #[serde(skip_serializing_if = "Option::is_none")] + pub error: Option, + pub block_hash: EthHash, + pub block_number: i64, + pub transaction_hash: EthHash, + pub transaction_position: i64, +} +lotus_json_with_self!(EthBlockTrace); + #[cfg(test)] mod tests { use super::*; diff --git a/src/rpc/methods/eth/utils.rs b/src/rpc/methods/eth/utils.rs new file mode 100644 index 00000000000..d0aa31c0bed --- /dev/null +++ b/src/rpc/methods/eth/utils.rs @@ -0,0 +1,163 @@ +// Copyright 2019-2025 ChainSafe Systems +// SPDX-License-Identifier: Apache-2.0, MIT + +use super::types::{EthAddress, EthBytes}; +use crate::rpc::state::{MessageTrace, ReturnTrace}; +use crate::shim::address::Address as FilecoinAddress; +use crate::shim::fvm_shared_latest::IDENTITY_HASH; +use crate::shim::state_tree::StateTree; + +use anyhow::{bail, Result}; +use cbor4ii::core::dec::Decode as _; +use cbor4ii::core::Value; +use fvm_ipld_blockstore::Blockstore; +use fvm_ipld_encoding::{RawBytes, CBOR, DAG_CBOR, IPLD_RAW}; +use serde::de; + +pub fn lookup_eth_address( + addr: &FilecoinAddress, + state: &StateTree, +) -> Result> { + // Attempt to convert directly, if it's an f4 address. + if let Ok(eth_addr) = EthAddress::from_filecoin_address(addr) { + if !eth_addr.is_masked_id() { + return Ok(Some(eth_addr)); + } + } + + // Otherwise, resolve the ID addr. + let id_addr = match state.lookup_id(addr)? { + Some(id) => id, + _ => return Ok(None), + }; + + // Lookup on the target actor and try to get an f410 address. + let result = state.get_actor(addr); + if let Ok(Some(actor_state)) = result { + if let Some(addr) = actor_state.delegated_address { + if let Ok(eth_addr) = EthAddress::from_filecoin_address(&addr.into()) { + if !eth_addr.is_masked_id() { + // Conversable into an eth address, use it. + return Ok(Some(eth_addr)); + } + } + } else { + // No delegated address -> use a masked ID address + } + } else if let Ok(None) = result { + // Not found -> use a masked ID address + } else { + // Any other error -> fail. + result?; + } + + // Otherwise, use the masked address. + Ok(Some(EthAddress::from_actor_id(id_addr))) +} + +/// Decodes the payload using the given codec. +pub fn decode_payload(payload: &RawBytes, codec: u64) -> Result { + match codec { + IDENTITY_HASH => Ok(EthBytes::default()), + DAG_CBOR | CBOR => { + let mut reader = cbor4ii::core::utils::SliceReader::new(payload.bytes()); + match Value::decode(&mut reader) { + Ok(Value::Bytes(bytes)) => Ok(EthBytes(bytes)), + _ => bail!("failed to read params byte array"), + } + } + IPLD_RAW => Ok(EthBytes(payload.to_vec())), + _ => bail!("decode_payload: unsupported codec {codec}"), + } +} + +/// Decodes the message trace params using the message trace codec. +pub fn decode_params<'a, T>(trace: &'a MessageTrace) -> anyhow::Result +where + T: de::Deserialize<'a>, +{ + let codec = trace.params_codec; + match codec { + DAG_CBOR | CBOR => fvm_ipld_encoding::from_slice(&trace.params) + .map_err(|e| anyhow::anyhow!("failed to decode params: {}", e)), + _ => bail!("Method called an unexpected codec {codec}"), + } +} + +/// Decodes the return bytes using the return trace codec. +pub fn decode_return<'a, T>(trace: &'a ReturnTrace) -> anyhow::Result +where + T: de::Deserialize<'a>, +{ + let codec = trace.return_codec; + match codec { + DAG_CBOR | CBOR => fvm_ipld_encoding::from_slice(trace.r#return.bytes()) + .map_err(|e| anyhow::anyhow!("failed to decode return value: {}", e)), + _ => bail!("Method returned an unexpected codec {codec}"), + } +} + +#[cfg(test)] +mod test { + use super::*; + use cbor4ii::core::{enc::Encode, utils::BufWriter}; + use cbor4ii::serde::Serializer; + + #[test] + fn test_decode_payload() { + // empty + let result = decode_payload(&RawBytes::default(), 0); + assert!(result.unwrap().0.is_empty()); + + // raw empty + let result = decode_payload(&RawBytes::default(), IPLD_RAW); + assert!(result.unwrap().0.is_empty()); + + // raw non-empty + let result = decode_payload(&RawBytes::new(vec![1]), IPLD_RAW); + assert_eq!(result.unwrap(), EthBytes(vec![1])); + + // invalid cbor bytes + let result = decode_payload(&RawBytes::default(), DAG_CBOR); + assert!(result.is_err()); + + // valid cbor bytes + let mut writer = BufWriter::new(Vec::new()); + Value::Bytes(vec![1]).encode(&mut writer).unwrap(); + let serializer = Serializer::new(writer); + let encoded = serializer.into_inner().into_inner(); + + let result = decode_payload(&RawBytes::new(encoded.clone()), DAG_CBOR); + assert_eq!(result.unwrap(), EthBytes(vec![1])); + + // regular cbor also works + let result = decode_payload(&RawBytes::new(encoded), CBOR); + assert_eq!(result.unwrap(), EthBytes(vec![1])); + + // random codec should fail + let result = decode_payload(&RawBytes::default(), 42); + assert!(result.is_err()); + + // some payload taken from calibnet + assert_eq!( + decode_payload( + &RawBytes::new( + hex::decode( + "58200000000000000000000000000000000000000000000000000000000000002710" + ) + .unwrap(), + ), + CBOR + ) + .unwrap(), + EthBytes(vec![ + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 39, 16, + ]) + ); + + // identity + let result = decode_payload(&RawBytes::new(vec![1]), IDENTITY_HASH); + assert!(result.unwrap().0.is_empty()); + } +} diff --git a/src/rpc/methods/state/types.rs b/src/rpc/methods/state/types.rs index 1f3f4a5a342..2315f678114 100644 --- a/src/rpc/methods/state/types.rs +++ b/src/rpc/methods/state/types.rs @@ -108,6 +108,18 @@ pub struct ExecutionTrace { pub subcalls: Vec, } +impl ExecutionTrace { + pub fn sum_gas(&self) -> GasTrace { + let mut out: GasTrace = GasTrace::default(); + for gc in self.gas_charges.iter() { + out.total_gas += gc.total_gas; + out.compute_gas += gc.compute_gas; + out.storage_gas += gc.storage_gas; + } + out + } +} + lotus_json_with_self!(ExecutionTrace); #[derive(Debug, Clone, PartialEq, Serialize, Deserialize, JsonSchema)] @@ -156,7 +168,7 @@ pub struct ReturnTrace { lotus_json_with_self!(ReturnTrace); -#[derive(Debug, Clone, Serialize, Deserialize, JsonSchema)] +#[derive(Default, Debug, Clone, Serialize, Deserialize, JsonSchema)] #[serde(rename_all = "PascalCase")] pub struct GasTrace { pub name: String, diff --git a/src/rpc/mod.rs b/src/rpc/mod.rs index 5536113e6c7..79d922f451f 100644 --- a/src/rpc/mod.rs +++ b/src/rpc/mod.rs @@ -103,6 +103,7 @@ macro_rules! for_each_method { $callback!(crate::rpc::eth::EthNewBlockFilter); $callback!(crate::rpc::eth::EthUninstallFilter); $callback!(crate::rpc::eth::EthSyncing); + $callback!(crate::rpc::eth::EthTraceBlock); $callback!(crate::rpc::eth::Web3ClientVersion); $callback!(crate::rpc::eth::EthSendRawTransaction); diff --git a/src/shim/error.rs b/src/shim/error.rs index f68db81f9b4..2678d50d27a 100644 --- a/src/shim/error.rs +++ b/src/shim/error.rs @@ -3,8 +3,11 @@ use fvm_shared2::error::ExitCode as ExitCodeV2; use fvm_shared3::error::ExitCode as ExitCodeV3; use fvm_shared4::error::ExitCode as ExitCodeV4; +use fvm_shared4::error::ExitCode as ExitCode_latest; use schemars::JsonSchema; use serde::{Deserialize, Serialize}; +use std::cmp::Ordering; +use std::fmt; /// `Newtype` wrapper for the FVM `ExitCode`. /// @@ -22,11 +25,76 @@ use serde::{Deserialize, Serialize}; /// assert_eq!(shim_from_v3, fvm3_success.into()); /// ``` #[derive(PartialEq, Eq, Debug, Clone, Copy, Serialize, Deserialize, JsonSchema)] -pub struct ExitCode(#[schemars(with = "u32")] ExitCodeV3); +pub struct ExitCode(#[schemars(with = "u32")] ExitCodeV4); + +impl PartialOrd for ExitCode { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.value().cmp(&other.value())) + } +} + +impl fmt::Display for ExitCode { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let name = match self.0 { + ExitCode_latest::SYS_SENDER_INVALID => Some("SysErrSenderInvalid"), + ExitCode_latest::SYS_SENDER_STATE_INVALID => Some("SysErrSenderStateInvalid"), + ExitCode_latest::SYS_ILLEGAL_INSTRUCTION => Some("SysErrIllegalInstruction"), + ExitCode_latest::SYS_INVALID_RECEIVER => Some("SysErrInvalidReceiver"), + ExitCode_latest::SYS_INSUFFICIENT_FUNDS => Some("SysErrInsufficientFunds"), + ExitCode_latest::SYS_OUT_OF_GAS => Some("SysErrOutOfGas"), + ExitCode_latest::SYS_ILLEGAL_EXIT_CODE => Some("SysErrIllegalExitCode"), + ExitCode_latest::SYS_ASSERTION_FAILED => Some("SysFatal"), + ExitCode_latest::SYS_MISSING_RETURN => Some("SysErrMissingReturn"), + + ExitCode_latest::USR_ILLEGAL_ARGUMENT => Some("ErrIllegalArgument"), + ExitCode_latest::USR_NOT_FOUND => Some("ErrNotFound"), + ExitCode_latest::USR_FORBIDDEN => Some("ErrForbidden"), + ExitCode_latest::USR_INSUFFICIENT_FUNDS => Some("ErrInsufficientFunds"), + ExitCode_latest::USR_ILLEGAL_STATE => Some("ErrIllegalState"), + ExitCode_latest::USR_SERIALIZATION => Some("ErrSerialization"), + ExitCode_latest::USR_UNHANDLED_MESSAGE => Some("ErrUnhandledMessage"), + ExitCode_latest::USR_UNSPECIFIED => Some("ErrUnspecified"), + ExitCode_latest::USR_ASSERTION_FAILED => Some("ErrAssertionFailed"), + ExitCode_latest::USR_READ_ONLY => Some("ErrReadOnly"), + ExitCode_latest::USR_NOT_PAYABLE => Some("ErrNotPayable"), + + _ => None, + }; + if let Some(name) = name { + write!(f, "{}({})", name, self.value()) + } else { + match self.value() { + code if code > ExitCode_latest::SYS_MISSING_RETURN.value() + && code < ExitCode_latest::FIRST_USER_EXIT_CODE => + { + // We want to match Lotus display exit codes + // See + write!( + f, + "SysErrReserved{}({})", + code - (ExitCode_latest::SYS_ASSERTION_FAILED.value()), + code + ) + } + _ => write!(f, "{}", self.value()), + } + } + } +} impl ExitCode { /// The lowest exit code that an actor may abort with. - pub const FIRST_USER_EXIT_CODE: u32 = ExitCodeV3::FIRST_USER_EXIT_CODE; + pub const FIRST_USER_EXIT_CODE: u32 = ExitCode_latest::FIRST_USER_EXIT_CODE; + + /// Message execution (including sub-calls) used more gas than the specified limit. + pub const SYS_OUT_OF_GAS: Self = Self::new(ExitCode_latest::SYS_OUT_OF_GAS); + + /// The message sender didn't have the requisite funds. + pub const SYS_INSUFFICIENT_FUNDS: Self = Self::new(ExitCode_latest::SYS_INSUFFICIENT_FUNDS); + + /// The initial range of exit codes is reserved for system errors. + /// Actors may define codes starting with this one. + pub const FIRST_ACTOR_ERROR_CODE: u32 = 16; pub fn value(&self) -> u32 { self.0.value() @@ -35,23 +103,27 @@ impl ExitCode { pub fn is_success(&self) -> bool { self.0.is_success() } + + pub const fn new(value: ExitCode_latest) -> Self { + Self(value) + } } impl From for ExitCode { fn from(value: u32) -> Self { - Self(ExitCodeV3::new(value)) + Self(ExitCodeV4::new(value)) } } impl From for ExitCode { fn from(value: ExitCodeV4) -> Self { - value.value().into() + Self(value) } } impl From for ExitCode { fn from(value: ExitCodeV3) -> Self { - Self(value) + value.value().into() } } @@ -69,12 +141,12 @@ impl From for ExitCodeV2 { impl From for ExitCodeV3 { fn from(value: ExitCode) -> Self { - value.0 + Self::new(value.0.value()) } } impl From for ExitCodeV4 { fn from(value: ExitCode) -> Self { - Self::new(value.0.value()) + value.0 } } diff --git a/src/state_manager/mod.rs b/src/state_manager/mod.rs index 49854d23afa..3218ccf4d60 100644 --- a/src/state_manager/mod.rs +++ b/src/state_manager/mod.rs @@ -1559,6 +1559,45 @@ where } } } + + pub fn execution_trace(&self, tipset: &Tipset) -> anyhow::Result<(Cid, Vec)> { + let mut invoc_trace = vec![]; + + let genesis_timestamp = self.chain_store().genesis_block_header().timestamp; + + let callback = |ctx: MessageCallbackCtx<'_>| { + match ctx.at { + CalledAt::Applied | CalledAt::Reward => { + invoc_trace.push(ApiInvocResult { + msg_cid: ctx.message.cid(), + msg: ctx.message.message().clone(), + msg_rct: Some(ctx.apply_ret.msg_receipt()), + error: ctx.apply_ret.failure_info().unwrap_or_default(), + duration: ctx.duration.as_nanos().clamp(0, u64::MAX as u128) as u64, + gas_cost: MessageGasCost::new(ctx.message.message(), ctx.apply_ret)?, + execution_trace: structured::parse_events(ctx.apply_ret.exec_trace()) + .unwrap_or_default(), + }); + Ok(()) + } + _ => Ok(()), // ignored + } + }; + + let StateOutput { state_root, .. } = apply_block_messages( + genesis_timestamp, + self.chain_store().chain_index.clone(), + self.chain_config().clone(), + self.beacon_schedule().clone(), + &self.engine, + Arc::new(tipset.clone()), + Some(callback), + VMTrace::Traced, + VMEvent::NotPushed, + )?; + + Ok((state_root, invoc_trace)) + } } pub fn validate_tipsets( diff --git a/src/tool/subcommands/api_cmd.rs b/src/tool/subcommands/api_cmd.rs index 4f110366945..651d2d1d443 100644 --- a/src/tool/subcommands/api_cmd.rs +++ b/src/tool/subcommands/api_cmd.rs @@ -1541,6 +1541,10 @@ fn eth_tests_with_tipset(store: &Arc, shared_tipset: &Tipset .unwrap(), ), RpcTest::identity(EthGetTransactionHashByCid::request((block_cid,)).unwrap()), + RpcTest::identity( + EthTraceBlock::request((BlockNumberOrHash::from_block_number(shared_tipset.epoch()),)) + .unwrap(), + ), ]; for block in shared_tipset.block_headers() {