feat: relax bounds for eth_simulateV1 (#13232)

This commit is contained in:
Arsenii Kulikov
2024-12-09 23:08:49 +04:00
committed by GitHub
parent 3af2afe995
commit c7c84f2d3f
16 changed files with 172 additions and 200 deletions

1
Cargo.lock generated
View File

@ -9129,7 +9129,6 @@ dependencies = [
"reth-chainspec", "reth-chainspec",
"reth-errors", "reth-errors",
"reth-evm", "reth-evm",
"reth-execution-types",
"reth-network-api", "reth-network-api",
"reth-node-api", "reth-node-api",
"reth-primitives", "reth-primitives",

View File

@ -5,15 +5,15 @@ use alloy_rpc_types_eth::transaction::TransactionRequest;
use reth_evm::ConfigureEvm; use reth_evm::ConfigureEvm;
use reth_provider::ProviderHeader; use reth_provider::ProviderHeader;
use reth_rpc_eth_api::{ use reth_rpc_eth_api::{
helpers::{estimate::EstimateCall, Call, EthCall, LoadPendingBlock, LoadState, SpawnBlocking}, helpers::{estimate::EstimateCall, Call, EthCall, LoadBlock, LoadState, SpawnBlocking},
FromEthApiError, IntoEthApiError, FromEthApiError, FullEthApiTypes, IntoEthApiError,
}; };
use reth_rpc_eth_types::{revm_utils::CallFees, RpcInvalidTransactionError}; use reth_rpc_eth_types::{revm_utils::CallFees, RpcInvalidTransactionError};
use revm::primitives::{BlockEnv, OptimismFields, TxEnv}; use revm::primitives::{BlockEnv, OptimismFields, TxEnv};
impl<N> EthCall for OpEthApi<N> impl<N> EthCall for OpEthApi<N>
where where
Self: EstimateCall + LoadPendingBlock, Self: EstimateCall + LoadBlock + FullEthApiTypes,
N: OpNodeCore, N: OpNodeCore,
{ {
} }

View File

@ -21,7 +21,7 @@ use reth_rpc_eth_api::{
}; };
use reth_rpc_eth_types::{EthApiError, PendingBlock}; use reth_rpc_eth_types::{EthApiError, PendingBlock};
use reth_transaction_pool::{PoolTransaction, TransactionPool}; use reth_transaction_pool::{PoolTransaction, TransactionPool};
use revm::primitives::{BlockEnv, CfgEnvWithHandlerCfg, ExecutionResult, SpecId}; use revm::primitives::{BlockEnv, ExecutionResult};
impl<N> LoadPendingBlock for OpEthApi<N> impl<N> LoadPendingBlock for OpEthApi<N>
where where
@ -82,23 +82,26 @@ where
fn assemble_block( fn assemble_block(
&self, &self,
cfg: CfgEnvWithHandlerCfg, block_env: &BlockEnv,
block_env: BlockEnv,
parent_hash: B256, parent_hash: B256,
state_root: B256, state_root: B256,
transactions: Vec<ProviderTx<Self::Provider>>, transactions: Vec<ProviderTx<Self::Provider>>,
receipts: &[ProviderReceipt<Self::Provider>], receipts: &[ProviderReceipt<Self::Provider>],
) -> reth_provider::ProviderBlock<Self::Provider> { ) -> reth_provider::ProviderBlock<Self::Provider> {
let chain_spec = self.provider().chain_spec(); let chain_spec = self.provider().chain_spec();
let timestamp = block_env.timestamp.to::<u64>();
let transactions_root = calculate_transaction_root(&transactions); let transactions_root = calculate_transaction_root(&transactions);
let receipts_root = calculate_receipt_root_no_memo_optimism( let receipts_root = calculate_receipt_root_no_memo_optimism(
&receipts.iter().collect::<Vec<_>>(), &receipts.iter().collect::<Vec<_>>(),
&chain_spec, &chain_spec,
block_env.timestamp.to::<u64>(), timestamp,
); );
let logs_bloom = logs_bloom(receipts.iter().flat_map(|r| &r.logs)); let logs_bloom = logs_bloom(receipts.iter().flat_map(|r| &r.logs));
let is_cancun = chain_spec.is_cancun_active_at_timestamp(timestamp);
let is_prague = chain_spec.is_prague_active_at_timestamp(timestamp);
let is_shanghai = chain_spec.is_shanghai_active_at_timestamp(timestamp);
let header = Header { let header = Header {
parent_hash, parent_hash,
@ -107,10 +110,9 @@ where
state_root, state_root,
transactions_root, transactions_root,
receipts_root, receipts_root,
withdrawals_root: (cfg.handler_cfg.spec_id >= SpecId::SHANGHAI) withdrawals_root: (is_shanghai).then_some(EMPTY_WITHDRAWALS),
.then_some(EMPTY_WITHDRAWALS),
logs_bloom, logs_bloom,
timestamp: block_env.timestamp.to::<u64>(), timestamp,
mix_hash: block_env.prevrandao.unwrap_or_default(), mix_hash: block_env.prevrandao.unwrap_or_default(),
nonce: BEACON_NONCE.into(), nonce: BEACON_NONCE.into(),
base_fee_per_gas: Some(block_env.basefee.to::<u64>()), base_fee_per_gas: Some(block_env.basefee.to::<u64>()),
@ -118,15 +120,13 @@ where
gas_limit: block_env.gas_limit.to::<u64>(), gas_limit: block_env.gas_limit.to::<u64>(),
difficulty: U256::ZERO, difficulty: U256::ZERO,
gas_used: receipts.last().map(|r| r.cumulative_gas_used).unwrap_or_default(), gas_used: receipts.last().map(|r| r.cumulative_gas_used).unwrap_or_default(),
blob_gas_used: (cfg.handler_cfg.spec_id >= SpecId::CANCUN).then(|| { blob_gas_used: is_cancun.then(|| {
transactions.iter().map(|tx| tx.blob_gas_used().unwrap_or_default()).sum::<u64>() transactions.iter().map(|tx| tx.blob_gas_used().unwrap_or_default()).sum::<u64>()
}), }),
excess_blob_gas: block_env.get_blob_excess_gas().map(Into::into), excess_blob_gas: block_env.get_blob_excess_gas().map(Into::into),
extra_data: Default::default(), extra_data: Default::default(),
parent_beacon_block_root: (cfg.handler_cfg.spec_id >= SpecId::CANCUN) parent_beacon_block_root: is_cancun.then_some(B256::ZERO),
.then_some(B256::ZERO), requests_hash: is_prague.then_some(EMPTY_REQUESTS_HASH),
requests_hash: (cfg.handler_cfg.spec_id >= SpecId::PRAGUE)
.then_some(EMPTY_REQUESTS_HASH),
target_blobs_per_block: None, target_blobs_per_block: None,
}; };
@ -139,7 +139,7 @@ where
fn assemble_receipt( fn assemble_receipt(
&self, &self,
tx: &reth_primitives::RecoveredTx<ProviderTx<Self::Provider>>, tx: &ProviderTx<Self::Provider>,
result: ExecutionResult, result: ExecutionResult,
cumulative_gas_used: u64, cumulative_gas_used: u64,
) -> reth_provider::ProviderReceipt<Self::Provider> { ) -> reth_provider::ProviderReceipt<Self::Provider> {

View File

@ -1,7 +1,7 @@
//! Loads and formats OP transaction RPC response. //! Loads and formats OP transaction RPC response.
use alloy_consensus::{Signed, Transaction as _}; use alloy_consensus::{Signed, Transaction as _};
use alloy_primitives::{Bytes, Sealable, Sealed, B256}; use alloy_primitives::{Bytes, PrimitiveSignature as Signature, Sealable, Sealed, B256};
use alloy_rpc_types_eth::TransactionInfo; use alloy_rpc_types_eth::TransactionInfo;
use op_alloy_consensus::OpTxEnvelope; use op_alloy_consensus::OpTxEnvelope;
use op_alloy_rpc_types::Transaction; use op_alloy_rpc_types::Transaction;
@ -14,7 +14,7 @@ use reth_rpc_eth_api::{
helpers::{EthSigner, EthTransactions, LoadTransaction, SpawnBlocking}, helpers::{EthSigner, EthTransactions, LoadTransaction, SpawnBlocking},
FromEthApiError, FullEthApiTypes, RpcNodeCore, RpcNodeCoreExt, TransactionCompat, FromEthApiError, FullEthApiTypes, RpcNodeCore, RpcNodeCoreExt, TransactionCompat,
}; };
use reth_rpc_eth_types::utils::recover_raw_transaction; use reth_rpc_eth_types::{utils::recover_raw_transaction, EthApiError};
use reth_transaction_pool::{PoolTransaction, TransactionOrigin, TransactionPool}; use reth_transaction_pool::{PoolTransaction, TransactionOrigin, TransactionPool};
use crate::{eth::OpNodeCore, OpEthApi, OpEthApiError, SequencerClient}; use crate::{eth::OpNodeCore, OpEthApi, OpEthApiError, SequencerClient};
@ -151,6 +151,19 @@ where
}) })
} }
fn build_simulate_v1_transaction(
&self,
request: alloy_rpc_types_eth::TransactionRequest,
) -> Result<TransactionSigned, Self::Error> {
let Ok(tx) = request.build_typed_tx() else {
return Err(OpEthApiError::Eth(EthApiError::TransactionConversionError))
};
// Create an empty signature for the transaction.
let signature = Signature::new(Default::default(), Default::default(), false);
Ok(TransactionSigned::new_unhashed(tx.into(), signature))
}
fn otterscan_api_truncate_input(tx: &mut Self::Transaction) { fn otterscan_api_truncate_input(tx: &mut Self::Transaction) {
let input = match &mut tx.inner.inner { let input = match &mut tx.inner.inner {
OpTxEnvelope::Eip1559(tx) => &mut tx.tx_mut().input, OpTxEnvelope::Eip1559(tx) => &mut tx.tx_mut().input,

View File

@ -24,6 +24,9 @@ impl<T> FullBlock for T where
{ {
} }
/// Helper trait to access [`BlockBody::Transaction`] given a [`Block`].
pub type BlockTx<B> = <<B as Block>::Body as BlockBody>::Transaction;
/// Abstraction of block data type. /// Abstraction of block data type.
// todo: make sealable super-trait, depends on <https://github.com/paradigmxyz/reth/issues/11449> // todo: make sealable super-trait, depends on <https://github.com/paradigmxyz/reth/issues/11449>
// todo: make with senders extension trait, so block can be impl by block type already containing // todo: make with senders extension trait, so block can be impl by block type already containing

View File

@ -26,7 +26,6 @@ reth-rpc-types-compat.workspace = true
reth-tasks = { workspace = true, features = ["rayon"] } reth-tasks = { workspace = true, features = ["rayon"] }
reth-transaction-pool.workspace = true reth-transaction-pool.workspace = true
reth-chainspec.workspace = true reth-chainspec.workspace = true
reth-execution-types.workspace = true
reth-rpc-eth-types.workspace = true reth-rpc-eth-types.workspace = true
reth-rpc-server-types.workspace = true reth-rpc-server-types.workspace = true
reth-network-api.workspace = true reth-network-api.workspace = true

View File

@ -13,7 +13,6 @@ use alloy_rpc_types_eth::{
}; };
use alloy_serde::JsonStorageKey; use alloy_serde::JsonStorageKey;
use jsonrpsee::{core::RpcResult, proc_macros::rpc}; use jsonrpsee::{core::RpcResult, proc_macros::rpc};
use reth_provider::BlockReader;
use reth_rpc_server_types::{result::internal_rpc_err, ToRpcResult}; use reth_rpc_server_types::{result::internal_rpc_err, ToRpcResult};
use tracing::trace; use tracing::trace;
@ -372,12 +371,7 @@ impl<T>
RpcHeader<T::NetworkTypes>, RpcHeader<T::NetworkTypes>,
> for T > for T
where where
T: FullEthApi< T: FullEthApi,
Provider: BlockReader<
Header = alloy_consensus::Header,
Transaction = reth_primitives::TransactionSigned,
>,
>,
jsonrpsee_types::error::ErrorObject<'static>: From<T::Error>, jsonrpsee_types::error::ErrorObject<'static>: From<T::Error>,
{ {
/// Handler for: `eth_protocolVersion` /// Handler for: `eth_protocolVersion`

View File

@ -20,9 +20,7 @@ use reth_chainspec::EthChainSpec;
use reth_evm::{ConfigureEvm, ConfigureEvmEnv}; use reth_evm::{ConfigureEvm, ConfigureEvmEnv};
use reth_node_api::BlockBody; use reth_node_api::BlockBody;
use reth_primitives_traits::SignedTransaction; use reth_primitives_traits::SignedTransaction;
use reth_provider::{ use reth_provider::{BlockIdReader, ChainSpecProvider, HeaderProvider, ProviderHeader};
BlockIdReader, BlockReader, ChainSpecProvider, HeaderProvider, ProviderHeader,
};
use reth_revm::{ use reth_revm::{
database::StateProviderDatabase, database::StateProviderDatabase,
db::CacheDB, db::CacheDB,
@ -50,7 +48,7 @@ pub type SimulatedBlocksResult<N, E> = Result<Vec<SimulatedBlock<RpcBlock<N>>>,
/// Execution related functions for the [`EthApiServer`](crate::EthApiServer) trait in /// Execution related functions for the [`EthApiServer`](crate::EthApiServer) trait in
/// the `eth_` namespace. /// the `eth_` namespace.
pub trait EthCall: EstimateCall + Call + LoadPendingBlock { pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthApiTypes {
/// Estimate gas needed for execution of the `request` at the [`BlockId`]. /// Estimate gas needed for execution of the `request` at the [`BlockId`].
fn estimate_gas_at( fn estimate_gas_at(
&self, &self,
@ -70,15 +68,7 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock {
&self, &self,
payload: SimulatePayload, payload: SimulatePayload,
block: Option<BlockId>, block: Option<BlockId>,
) -> impl Future<Output = SimulatedBlocksResult<Self::NetworkTypes, Self::Error>> + Send ) -> impl Future<Output = SimulatedBlocksResult<Self::NetworkTypes, Self::Error>> + Send {
where
Self: LoadBlock<
Provider: BlockReader<
Header = alloy_consensus::Header,
Transaction = reth_primitives::TransactionSigned,
>,
> + FullEthApiTypes,
{
async move { async move {
if payload.block_state_calls.len() > self.max_simulate_blocks() as usize { if payload.block_state_calls.len() > self.max_simulate_blocks() as usize {
return Err(EthApiError::InvalidParams("too many blocks.".to_string()).into()) return Err(EthApiError::InvalidParams("too many blocks.".to_string()).into())
@ -171,9 +161,11 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock {
block_env.gas_limit.to(), block_env.gas_limit.to(),
cfg.chain_id, cfg.chain_id,
&mut db, &mut db,
this.tx_resp_builder(),
)?; )?;
let mut calls = calls.into_iter().peekable(); let mut calls = calls.into_iter().peekable();
let mut senders = Vec::with_capacity(transactions.len());
let mut results = Vec::with_capacity(calls.len()); let mut results = Vec::with_capacity(calls.len());
while let Some(tx) = calls.next() { while let Some(tx) = calls.next() {
@ -197,18 +189,27 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock {
db.commit(res.state); db.commit(res.state);
} }
results.push((env.tx.caller, res.result)); senders.push(env.tx.caller);
results.push(res.result);
} }
let (block, _) = this.assemble_block_and_receipts(
&block_env,
parent_hash,
// state root calculation is skipped for performance reasons
B256::ZERO,
transactions,
results.clone(),
);
let block: SimulatedBlock<RpcBlock<Self::NetworkTypes>> = let block: SimulatedBlock<RpcBlock<Self::NetworkTypes>> =
simulate::build_block( simulate::build_simulated_block(
senders,
results, results,
transactions,
&block_env,
parent_hash,
total_difficulty, total_difficulty,
return_full_transactions, return_full_transactions,
this.tx_resp_builder(), this.tx_resp_builder(),
block,
)?; )?;
parent_hash = block.inner.header.hash; parent_hash = block.inner.header.hash;
@ -245,10 +246,7 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock {
bundle: Bundle, bundle: Bundle,
state_context: Option<StateContext>, state_context: Option<StateContext>,
mut state_override: Option<StateOverride>, mut state_override: Option<StateOverride>,
) -> impl Future<Output = Result<Vec<EthCallResponse>, Self::Error>> + Send ) -> impl Future<Output = Result<Vec<EthCallResponse>, Self::Error>> + Send {
where
Self: LoadBlock,
{
async move { async move {
let Bundle { transactions, block_override } = bundle; let Bundle { transactions, block_override } = bundle;
if transactions.is_empty() { if transactions.is_empty() {
@ -608,7 +606,7 @@ pub trait Call:
f: F, f: F,
) -> impl Future<Output = Result<Option<R>, Self::Error>> + Send ) -> impl Future<Output = Result<Option<R>, Self::Error>> + Send
where where
Self: LoadBlock + LoadPendingBlock + LoadTransaction, Self: LoadBlock + LoadTransaction,
F: FnOnce(TransactionInfo, ResultAndState, StateCacheDb<'_>) -> Result<R, Self::Error> F: FnOnce(TransactionInfo, ResultAndState, StateCacheDb<'_>) -> Result<R, Self::Error>
+ Send + Send
+ 'static, + 'static,

View File

@ -42,12 +42,9 @@ pub use transaction::{EthTransactions, LoadTransaction};
use crate::FullEthApiTypes; use crate::FullEthApiTypes;
/// Extension trait that bundles traits needed for tracing transactions. /// Extension trait that bundles traits needed for tracing transactions.
pub trait TraceExt: pub trait TraceExt: LoadTransaction + LoadBlock + SpawnBlocking + Trace + Call {}
LoadTransaction + LoadBlock + LoadPendingBlock + SpawnBlocking + Trace + Call
{
}
impl<T> TraceExt for T where T: LoadTransaction + LoadBlock + LoadPendingBlock + Trace + Call {} impl<T> TraceExt for T where T: LoadTransaction + LoadBlock + Trace + Call {}
/// Helper trait to unify all `eth` rpc server building block traits, for simplicity. /// Helper trait to unify all `eth` rpc server building block traits, for simplicity.
/// ///

View File

@ -15,8 +15,7 @@ use reth_evm::{
state_change::post_block_withdrawals_balance_increments, system_calls::SystemCaller, state_change::post_block_withdrawals_balance_increments, system_calls::SystemCaller,
ConfigureEvm, ConfigureEvmEnv, NextBlockEnvAttributes, ConfigureEvm, ConfigureEvmEnv, NextBlockEnvAttributes,
}; };
use reth_execution_types::ExecutionOutcome; use reth_primitives::{BlockExt, InvalidTransactionError, SealedBlockWithSenders};
use reth_primitives::{BlockExt, InvalidTransactionError, RecoveredTx, SealedBlockWithSenders};
use reth_primitives_traits::receipt::ReceiptExt; use reth_primitives_traits::receipt::ReceiptExt;
use reth_provider::{ use reth_provider::{
BlockReader, BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, ProviderBlock, ProviderError, BlockReader, BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, ProviderBlock, ProviderError,
@ -199,7 +198,7 @@ pub trait LoadPendingBlock:
/// Assembles a receipt for a transaction, based on its [`ExecutionResult`]. /// Assembles a receipt for a transaction, based on its [`ExecutionResult`].
fn assemble_receipt( fn assemble_receipt(
&self, &self,
tx: &RecoveredTx<ProviderTx<Self::Provider>>, tx: &ProviderTx<Self::Provider>,
result: ExecutionResult, result: ExecutionResult,
cumulative_gas_used: u64, cumulative_gas_used: u64,
) -> ProviderReceipt<Self::Provider>; ) -> ProviderReceipt<Self::Provider>;
@ -207,14 +206,36 @@ pub trait LoadPendingBlock:
/// Assembles a pending block. /// Assembles a pending block.
fn assemble_block( fn assemble_block(
&self, &self,
cfg: CfgEnvWithHandlerCfg, block_env: &BlockEnv,
block_env: BlockEnv,
parent_hash: revm_primitives::B256, parent_hash: revm_primitives::B256,
state_root: revm_primitives::B256, state_root: revm_primitives::B256,
transactions: Vec<ProviderTx<Self::Provider>>, transactions: Vec<ProviderTx<Self::Provider>>,
receipts: &[ProviderReceipt<Self::Provider>], receipts: &[ProviderReceipt<Self::Provider>],
) -> ProviderBlock<Self::Provider>; ) -> ProviderBlock<Self::Provider>;
/// Helper to invoke both [`Self::assemble_block`] and [`Self::assemble_receipt`].
fn assemble_block_and_receipts(
&self,
block_env: &BlockEnv,
parent_hash: revm_primitives::B256,
state_root: revm_primitives::B256,
transactions: Vec<ProviderTx<Self::Provider>>,
results: Vec<ExecutionResult>,
) -> (ProviderBlock<Self::Provider>, Vec<ProviderReceipt<Self::Provider>>) {
let mut cumulative_gas_used = 0;
let mut receipts = Vec::with_capacity(results.len());
for (tx, outcome) in transactions.iter().zip(results) {
cumulative_gas_used += outcome.gas_used();
receipts.push(self.assemble_receipt(tx, outcome, cumulative_gas_used));
}
let block =
self.assemble_block(block_env, parent_hash, state_root, transactions, &receipts);
(block, receipts)
}
/// Builds a pending block using the configured provider and pool. /// Builds a pending block using the configured provider and pool.
/// ///
/// If the origin is the actual pending block, the block is built with withdrawals. /// If the origin is the actual pending block, the block is built with withdrawals.
@ -248,7 +269,6 @@ pub trait LoadPendingBlock:
let mut sum_blob_gas_used = 0; let mut sum_blob_gas_used = 0;
let block_gas_limit: u64 = block_env.gas_limit.to::<u64>(); let block_gas_limit: u64 = block_env.gas_limit.to::<u64>();
let base_fee = block_env.basefee.to::<u64>(); let base_fee = block_env.basefee.to::<u64>();
let block_number = block_env.number.to::<u64>();
let mut executed_txs = Vec::new(); let mut executed_txs = Vec::new();
let mut senders = Vec::new(); let mut senders = Vec::new();
@ -266,7 +286,7 @@ pub trait LoadPendingBlock:
.pre_block_blockhashes_contract_call(&mut db, &cfg, &block_env, parent_hash) .pre_block_blockhashes_contract_call(&mut db, &cfg, &block_env, parent_hash)
.map_err(|err| EthApiError::Internal(err.into()))?; .map_err(|err| EthApiError::Internal(err.into()))?;
let mut receipts = Vec::new(); let mut results = Vec::new();
while let Some(pool_tx) = best_txs.next() { while let Some(pool_tx) = best_txs.next() {
// ensure we still have capacity for this transaction // ensure we still have capacity for this transaction
@ -374,13 +394,11 @@ pub trait LoadPendingBlock:
// add gas used by the transaction to cumulative gas used, before creating the receipt // add gas used by the transaction to cumulative gas used, before creating the receipt
cumulative_gas_used += gas_used; cumulative_gas_used += gas_used;
// Push transaction changeset and calculate header bloom filter for receipt.
receipts.push(Some(self.assemble_receipt(&tx, result, cumulative_gas_used)));
// append transaction to the list of executed transactions // append transaction to the list of executed transactions
let (tx, sender) = tx.to_components(); let (tx, sender) = tx.to_components();
executed_txs.push(tx); executed_txs.push(tx);
senders.push(sender); senders.push(sender);
results.push(result);
} }
// executes the withdrawals and commits them to the Database and BundleState. // executes the withdrawals and commits them to the Database and BundleState.
@ -396,22 +414,19 @@ pub trait LoadPendingBlock:
// merge all transitions into bundle state. // merge all transitions into bundle state.
db.merge_transitions(BundleRetention::PlainState); db.merge_transitions(BundleRetention::PlainState);
let execution_outcome: ExecutionOutcome<ProviderReceipt<Self::Provider>> = let bundle_state = db.take_bundle();
ExecutionOutcome::new( let hashed_state = db.database.hashed_post_state(&bundle_state);
db.take_bundle(),
vec![receipts.clone()].into(),
block_number,
Vec::new(),
);
let hashed_state = db.database.hashed_post_state(execution_outcome.state());
// calculate the state root // calculate the state root
let state_root = db.database.state_root(hashed_state).map_err(Self::Error::from_eth_err)?; let state_root = db.database.state_root(hashed_state).map_err(Self::Error::from_eth_err)?;
// Convert Vec<Option<Receipt>> to Vec<Receipt> let (block, receipts) = self.assemble_block_and_receipts(
let receipts: Vec<_> = receipts.into_iter().flatten().collect(); &block_env,
let block = parent_hash,
self.assemble_block(cfg, block_env, parent_hash, state_root, executed_txs, &receipts); state_root,
executed_txs,
results,
);
Ok((SealedBlockWithSenders { block: block.seal_slow(), senders }, receipts)) Ok((SealedBlockWithSenders { block: block.seal_slow(), senders }, receipts))
} }

View File

@ -25,9 +25,7 @@ use reth_rpc_types_compat::transaction::{from_recovered, from_recovered_with_blo
use reth_transaction_pool::{PoolTransaction, TransactionOrigin, TransactionPool}; use reth_transaction_pool::{PoolTransaction, TransactionOrigin, TransactionPool};
use std::sync::Arc; use std::sync::Arc;
use super::{ use super::{EthApiSpec, EthSigner, LoadBlock, LoadReceipt, LoadState, SpawnBlocking};
EthApiSpec, EthSigner, LoadBlock, LoadPendingBlock, LoadReceipt, LoadState, SpawnBlocking,
};
use crate::{ use crate::{
helpers::estimate::EstimateCall, FromEthApiError, FullEthApiTypes, IntoEthApiError, helpers::estimate::EstimateCall, FromEthApiError, FullEthApiTypes, IntoEthApiError,
RpcNodeCore, RpcNodeCoreExt, RpcReceipt, RpcTransaction, RpcNodeCore, RpcNodeCoreExt, RpcReceipt, RpcTransaction,
@ -365,7 +363,7 @@ pub trait EthTransactions: LoadTransaction<Provider: BlockReaderIdExt> {
mut request: TransactionRequest, mut request: TransactionRequest,
) -> impl Future<Output = Result<B256, Self::Error>> + Send ) -> impl Future<Output = Result<B256, Self::Error>> + Send
where where
Self: EthApiSpec + LoadBlock + LoadPendingBlock + EstimateCall, Self: EthApiSpec + LoadBlock + EstimateCall,
{ {
async move { async move {
let from = match request.from { let from = match request.from {

View File

@ -1,21 +1,18 @@
//! Utilities for serving `eth_simulateV1` //! Utilities for serving `eth_simulateV1`
use alloy_consensus::{Transaction as _, TxType}; use alloy_consensus::{BlockHeader, Transaction as _, TxType};
use alloy_primitives::PrimitiveSignature as Signature;
use alloy_rpc_types_eth::{ use alloy_rpc_types_eth::{
simulate::{SimCallResult, SimulateError, SimulatedBlock}, simulate::{SimCallResult, SimulateError, SimulatedBlock},
transaction::TransactionRequest, transaction::TransactionRequest,
Block, BlockTransactionsKind, Block, BlockTransactionsKind, Header,
}; };
use jsonrpsee_types::ErrorObject; use jsonrpsee_types::ErrorObject;
use reth_primitives::{ use reth_primitives::BlockWithSenders;
proofs::{calculate_receipt_root, calculate_transaction_root}, use reth_primitives_traits::{block::BlockTx, BlockBody as _, SignedTransaction};
BlockBody, BlockWithSenders, Receipt, TransactionSigned,
};
use reth_rpc_server_types::result::rpc_err; use reth_rpc_server_types::result::rpc_err;
use reth_rpc_types_compat::{block::from_block, TransactionCompat}; use reth_rpc_types_compat::{block::from_block, TransactionCompat};
use revm::Database; use revm::Database;
use revm_primitives::{Address, BlockEnv, Bytes, ExecutionResult, TxKind, B256, U256}; use revm_primitives::{Address, Bytes, ExecutionResult, TxKind, U256};
use crate::{ use crate::{
error::{api::FromEthApiError, ToRpcError}, error::{api::FromEthApiError, ToRpcError},
@ -49,17 +46,18 @@ impl ToRpcError for EthSimulateError {
} }
/// Goes over the list of [`TransactionRequest`]s and populates missing fields trying to resolve /// Goes over the list of [`TransactionRequest`]s and populates missing fields trying to resolve
/// them into [`TransactionSigned`]. /// them into primitive transactions.
/// ///
/// If validation is enabled, the function will return error if any of the transactions can't be /// If validation is enabled, the function will return error if any of the transactions can't be
/// built right away. /// built right away.
pub fn resolve_transactions<DB: Database>( pub fn resolve_transactions<DB: Database, Tx, T: TransactionCompat<Tx>>(
txs: &mut [TransactionRequest], txs: &mut [TransactionRequest],
validation: bool, validation: bool,
block_gas_limit: u64, block_gas_limit: u64,
chain_id: u64, chain_id: u64,
db: &mut DB, db: &mut DB,
) -> Result<Vec<TransactionSigned>, EthApiError> tx_resp_builder: &T,
) -> Result<Vec<Tx>, EthApiError>
where where
EthApiError: From<DB::Error>, EthApiError: From<DB::Error>,
{ {
@ -125,49 +123,44 @@ where
} }
} }
let Ok(tx) = tx.clone().build_typed_tx() else { transactions.push(
return Err(EthApiError::TransactionConversionError) tx_resp_builder
}; .build_simulate_v1_transaction(tx.clone())
.map_err(|e| EthApiError::other(e.into()))?,
// Create an empty signature for the transaction. );
let signature = Signature::new(Default::default(), Default::default(), false);
let tx = TransactionSigned::new_unhashed(tx.into(), signature);
transactions.push(tx);
} }
Ok(transactions) Ok(transactions)
} }
/// Handles outputs of the calls execution and builds a [`SimulatedBlock`]. /// Handles outputs of the calls execution and builds a [`SimulatedBlock`].
pub fn build_block<T: TransactionCompat<Error: FromEthApiError>>( #[expect(clippy::type_complexity)]
results: Vec<(Address, ExecutionResult)>, pub fn build_simulated_block<T, B>(
transactions: Vec<TransactionSigned>, senders: Vec<Address>,
block_env: &BlockEnv, results: Vec<ExecutionResult>,
parent_hash: B256,
total_difficulty: U256, total_difficulty: U256,
full_transactions: bool, full_transactions: bool,
tx_resp_builder: &T, tx_resp_builder: &T,
) -> Result<SimulatedBlock<Block<T::Transaction>>, T::Error> { block: B,
) -> Result<SimulatedBlock<Block<T::Transaction, Header<B::Header>>>, T::Error>
where
T: TransactionCompat<BlockTx<B>, Error: FromEthApiError>,
B: reth_primitives_traits::Block,
{
let mut calls: Vec<SimCallResult> = Vec::with_capacity(results.len()); let mut calls: Vec<SimCallResult> = Vec::with_capacity(results.len());
let mut senders = Vec::with_capacity(results.len());
let mut receipts = Vec::with_capacity(results.len());
let mut log_index = 0; let mut log_index = 0;
for (transaction_index, ((sender, result), tx)) in for (index, (result, tx)) in results.iter().zip(block.body().transactions()).enumerate() {
results.into_iter().zip(transactions.iter()).enumerate()
{
senders.push(sender);
let call = match result { let call = match result {
ExecutionResult::Halt { reason, gas_used } => { ExecutionResult::Halt { reason, gas_used } => {
let error = RpcInvalidTransactionError::halt(reason, tx.gas_limit()); let error = RpcInvalidTransactionError::halt(*reason, tx.gas_limit());
SimCallResult { SimCallResult {
return_data: Bytes::new(), return_data: Bytes::new(),
error: Some(SimulateError { error: Some(SimulateError {
code: error.error_code(), code: error.error_code(),
message: error.to_string(), message: error.to_string(),
}), }),
gas_used, gas_used: *gas_used,
logs: Vec::new(), logs: Vec::new(),
status: false, status: false,
} }
@ -175,31 +168,31 @@ pub fn build_block<T: TransactionCompat<Error: FromEthApiError>>(
ExecutionResult::Revert { output, gas_used } => { ExecutionResult::Revert { output, gas_used } => {
let error = RevertError::new(output.clone()); let error = RevertError::new(output.clone());
SimCallResult { SimCallResult {
return_data: output, return_data: output.clone(),
error: Some(SimulateError { error: Some(SimulateError {
code: error.error_code(), code: error.error_code(),
message: error.to_string(), message: error.to_string(),
}), }),
gas_used, gas_used: *gas_used,
status: false, status: false,
logs: Vec::new(), logs: Vec::new(),
} }
} }
ExecutionResult::Success { output, gas_used, logs, .. } => SimCallResult { ExecutionResult::Success { output, gas_used, logs, .. } => SimCallResult {
return_data: output.into_data(), return_data: output.clone().into_data(),
error: None, error: None,
gas_used, gas_used: *gas_used,
logs: logs logs: logs
.into_iter() .iter()
.map(|log| { .map(|log| {
log_index += 1; log_index += 1;
alloy_rpc_types_eth::Log { alloy_rpc_types_eth::Log {
inner: log, inner: log.clone(),
log_index: Some(log_index - 1), log_index: Some(log_index - 1),
transaction_index: Some(transaction_index as u64), transaction_index: Some(index as u64),
transaction_hash: Some(tx.hash()), transaction_hash: Some(*tx.tx_hash()),
block_number: Some(block_env.number.to()), block_number: Some(block.header().number()),
block_timestamp: Some(block_env.timestamp.to()), block_timestamp: Some(block.header().timestamp()),
..Default::default() ..Default::default()
} }
}) })
@ -208,70 +201,10 @@ pub fn build_block<T: TransactionCompat<Error: FromEthApiError>>(
}, },
}; };
receipts.push(
#[allow(clippy::needless_update)]
Receipt {
tx_type: tx.tx_type(),
success: call.status,
cumulative_gas_used: call.gas_used + calls.iter().map(|c| c.gas_used).sum::<u64>(),
logs: call.logs.iter().map(|log| &log.inner).cloned().collect(),
..Default::default()
}
.with_bloom(),
);
calls.push(call); calls.push(call);
} }
// TODO: uncomment once performance cost is acceptable let block = BlockWithSenders { block, senders };
//
// let mut hashed_state = HashedPostState::default();
// for (address, account) in &db.accounts {
// let hashed_address = keccak256(address);
// hashed_state.accounts.insert(hashed_address, Some(account.info.clone().into()));
// let storage = hashed_state
// .storages
// .entry(hashed_address)
// .or_insert_with(|| HashedStorage::new(account.account_state.is_storage_cleared()));
// for (slot, value) in &account.storage {
// let slot = B256::from(*slot);
// let hashed_slot = keccak256(slot);
// storage.storage.insert(hashed_slot, *value);
// }
// }
// let state_root = db.db.state_root(hashed_state).map_err(T::Error::from_eth_err)?;
let state_root = B256::ZERO;
let header = alloy_consensus::Header {
beneficiary: block_env.coinbase,
difficulty: block_env.difficulty,
number: block_env.number.to(),
timestamp: block_env.timestamp.to(),
base_fee_per_gas: Some(block_env.basefee.to()),
gas_limit: block_env.gas_limit.to(),
gas_used: calls.iter().map(|c| c.gas_used).sum(),
blob_gas_used: Some(0),
parent_hash,
receipts_root: calculate_receipt_root(&receipts),
transactions_root: calculate_transaction_root(&transactions),
state_root,
logs_bloom: alloy_primitives::logs_bloom(
receipts.iter().flat_map(|r| r.receipt.logs.iter()),
),
mix_hash: block_env.prevrandao.unwrap_or_default(),
..Default::default()
};
let block = BlockWithSenders {
block: reth_primitives::Block {
header,
body: BlockBody { transactions, ..Default::default() },
},
senders,
};
let txs_kind = let txs_kind =
if full_transactions { BlockTransactionsKind::Full } else { BlockTransactionsKind::Hashes }; if full_transactions { BlockTransactionsKind::Full } else { BlockTransactionsKind::Hashes };

View File

@ -57,6 +57,10 @@ pub trait TransactionCompat<T = TransactionSigned>:
tx_inf: TransactionInfo, tx_inf: TransactionInfo,
) -> Result<Self::Transaction, Self::Error>; ) -> Result<Self::Transaction, Self::Error>;
/// Builds a fake transaction from a transaction request for inclusion into block built in
/// `eth_simulateV1`.
fn build_simulate_v1_transaction(&self, request: TransactionRequest) -> Result<T, Self::Error>;
/// Truncates the input of a transaction to only the first 4 bytes. /// Truncates the input of a transaction to only the first 4 bytes.
// todo: remove in favour of using constructor on `TransactionResponse` or similar // todo: remove in favour of using constructor on `TransactionResponse` or similar
// <https://github.com/alloy-rs/alloy/issues/1315>. // <https://github.com/alloy-rs/alloy/issues/1315>.

View File

@ -4,13 +4,14 @@ use crate::EthApi;
use alloy_consensus::Header; use alloy_consensus::Header;
use reth_evm::ConfigureEvm; use reth_evm::ConfigureEvm;
use reth_provider::{BlockReader, ProviderHeader}; use reth_provider::{BlockReader, ProviderHeader};
use reth_rpc_eth_api::helpers::{ use reth_rpc_eth_api::{
estimate::EstimateCall, Call, EthCall, LoadPendingBlock, LoadState, SpawnBlocking, helpers::{estimate::EstimateCall, Call, EthCall, LoadPendingBlock, LoadState, SpawnBlocking},
FullEthApiTypes,
}; };
impl<Provider, Pool, Network, EvmConfig> EthCall for EthApi<Provider, Pool, Network, EvmConfig> impl<Provider, Pool, Network, EvmConfig> EthCall for EthApi<Provider, Pool, Network, EvmConfig>
where where
Self: EstimateCall + LoadPendingBlock, Self: EstimateCall + LoadPendingBlock + FullEthApiTypes,
Provider: BlockReader, Provider: BlockReader,
{ {
} }

View File

@ -20,7 +20,7 @@ use reth_rpc_eth_api::{
}; };
use reth_rpc_eth_types::PendingBlock; use reth_rpc_eth_types::PendingBlock;
use reth_transaction_pool::{PoolTransaction, TransactionPool}; use reth_transaction_pool::{PoolTransaction, TransactionPool};
use revm_primitives::{BlockEnv, CfgEnvWithHandlerCfg, SpecId, B256}; use revm_primitives::{BlockEnv, B256};
use crate::EthApi; use crate::EthApi;
@ -56,18 +56,24 @@ where
fn assemble_block( fn assemble_block(
&self, &self,
cfg: CfgEnvWithHandlerCfg, block_env: &BlockEnv,
block_env: BlockEnv,
parent_hash: revm_primitives::B256, parent_hash: revm_primitives::B256,
state_root: revm_primitives::B256, state_root: revm_primitives::B256,
transactions: Vec<ProviderTx<Self::Provider>>, transactions: Vec<ProviderTx<Self::Provider>>,
receipts: &[ProviderReceipt<Self::Provider>], receipts: &[ProviderReceipt<Self::Provider>],
) -> reth_provider::ProviderBlock<Self::Provider> { ) -> reth_provider::ProviderBlock<Self::Provider> {
let chain_spec = self.provider().chain_spec();
let transactions_root = calculate_transaction_root(&transactions); let transactions_root = calculate_transaction_root(&transactions);
let receipts_root = calculate_receipt_root_no_memo(&receipts.iter().collect::<Vec<_>>()); let receipts_root = calculate_receipt_root_no_memo(&receipts.iter().collect::<Vec<_>>());
let logs_bloom = logs_bloom(receipts.iter().flat_map(|r| &r.logs)); let logs_bloom = logs_bloom(receipts.iter().flat_map(|r| &r.logs));
let timestamp = block_env.timestamp.to::<u64>();
let is_shanghai = chain_spec.is_shanghai_active_at_timestamp(timestamp);
let is_cancun = chain_spec.is_cancun_active_at_timestamp(timestamp);
let is_prague = chain_spec.is_prague_active_at_timestamp(timestamp);
let header = Header { let header = Header {
parent_hash, parent_hash,
ommers_hash: EMPTY_OMMER_ROOT_HASH, ommers_hash: EMPTY_OMMER_ROOT_HASH,
@ -75,8 +81,7 @@ where
state_root, state_root,
transactions_root, transactions_root,
receipts_root, receipts_root,
withdrawals_root: (cfg.handler_cfg.spec_id >= SpecId::SHANGHAI) withdrawals_root: is_shanghai.then_some(EMPTY_WITHDRAWALS),
.then_some(EMPTY_WITHDRAWALS),
logs_bloom, logs_bloom,
timestamp: block_env.timestamp.to::<u64>(), timestamp: block_env.timestamp.to::<u64>(),
mix_hash: block_env.prevrandao.unwrap_or_default(), mix_hash: block_env.prevrandao.unwrap_or_default(),
@ -86,15 +91,13 @@ where
gas_limit: block_env.gas_limit.to::<u64>(), gas_limit: block_env.gas_limit.to::<u64>(),
difficulty: U256::ZERO, difficulty: U256::ZERO,
gas_used: receipts.last().map(|r| r.cumulative_gas_used).unwrap_or_default(), gas_used: receipts.last().map(|r| r.cumulative_gas_used).unwrap_or_default(),
blob_gas_used: (cfg.handler_cfg.spec_id >= SpecId::CANCUN).then(|| { blob_gas_used: is_cancun.then(|| {
transactions.iter().map(|tx| tx.blob_gas_used().unwrap_or_default()).sum::<u64>() transactions.iter().map(|tx| tx.blob_gas_used().unwrap_or_default()).sum::<u64>()
}), }),
excess_blob_gas: block_env.get_blob_excess_gas().map(Into::into), excess_blob_gas: block_env.get_blob_excess_gas().map(Into::into),
extra_data: Default::default(), extra_data: Default::default(),
parent_beacon_block_root: (cfg.handler_cfg.spec_id >= SpecId::CANCUN) parent_beacon_block_root: is_cancun.then_some(B256::ZERO),
.then_some(B256::ZERO), requests_hash: is_prague.then_some(EMPTY_REQUESTS_HASH),
requests_hash: (cfg.handler_cfg.spec_id >= SpecId::PRAGUE)
.then_some(EMPTY_REQUESTS_HASH),
target_blobs_per_block: None, target_blobs_per_block: None,
}; };
@ -107,7 +110,7 @@ where
fn assemble_receipt( fn assemble_receipt(
&self, &self,
tx: &reth_primitives::RecoveredTx<ProviderTx<Self::Provider>>, tx: &ProviderTx<Self::Provider>,
result: revm_primitives::ExecutionResult, result: revm_primitives::ExecutionResult,
cumulative_gas_used: u64, cumulative_gas_used: u64,
) -> reth_provider::ProviderReceipt<Self::Provider> { ) -> reth_provider::ProviderReceipt<Self::Provider> {

View File

@ -2,6 +2,8 @@
use alloy_consensus::{Signed, Transaction as _, TxEip4844Variant, TxEnvelope}; use alloy_consensus::{Signed, Transaction as _, TxEip4844Variant, TxEnvelope};
use alloy_network::{Ethereum, Network}; use alloy_network::{Ethereum, Network};
use alloy_primitives::PrimitiveSignature as Signature;
use alloy_rpc_types::TransactionRequest;
use alloy_rpc_types_eth::{Transaction, TransactionInfo}; use alloy_rpc_types_eth::{Transaction, TransactionInfo};
use reth_primitives::{RecoveredTx, TransactionSigned}; use reth_primitives::{RecoveredTx, TransactionSigned};
use reth_rpc_eth_api::EthApiTypes; use reth_rpc_eth_api::EthApiTypes;
@ -84,6 +86,19 @@ where
}) })
} }
fn build_simulate_v1_transaction(
&self,
request: TransactionRequest,
) -> Result<TransactionSigned, Self::Error> {
let Ok(tx) = request.build_typed_tx() else {
return Err(EthApiError::TransactionConversionError)
};
// Create an empty signature for the transaction.
let signature = Signature::new(Default::default(), Default::default(), false);
Ok(TransactionSigned::new_unhashed(tx.into(), signature))
}
fn otterscan_api_truncate_input(tx: &mut Self::Transaction) { fn otterscan_api_truncate_input(tx: &mut Self::Transaction) {
let input = match &mut tx.inner { let input = match &mut tx.inner {
TxEnvelope::Eip1559(tx) => &mut tx.tx_mut().input, TxEnvelope::Eip1559(tx) => &mut tx.tx_mut().input,