mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 19:09:54 +00:00
feat: relax bounds for eth_simulateV1 (#13232)
This commit is contained in:
1
Cargo.lock
generated
1
Cargo.lock
generated
@ -9129,7 +9129,6 @@ dependencies = [
|
||||
"reth-chainspec",
|
||||
"reth-errors",
|
||||
"reth-evm",
|
||||
"reth-execution-types",
|
||||
"reth-network-api",
|
||||
"reth-node-api",
|
||||
"reth-primitives",
|
||||
|
||||
@ -5,15 +5,15 @@ use alloy_rpc_types_eth::transaction::TransactionRequest;
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_provider::ProviderHeader;
|
||||
use reth_rpc_eth_api::{
|
||||
helpers::{estimate::EstimateCall, Call, EthCall, LoadPendingBlock, LoadState, SpawnBlocking},
|
||||
FromEthApiError, IntoEthApiError,
|
||||
helpers::{estimate::EstimateCall, Call, EthCall, LoadBlock, LoadState, SpawnBlocking},
|
||||
FromEthApiError, FullEthApiTypes, IntoEthApiError,
|
||||
};
|
||||
use reth_rpc_eth_types::{revm_utils::CallFees, RpcInvalidTransactionError};
|
||||
use revm::primitives::{BlockEnv, OptimismFields, TxEnv};
|
||||
|
||||
impl<N> EthCall for OpEthApi<N>
|
||||
where
|
||||
Self: EstimateCall + LoadPendingBlock,
|
||||
Self: EstimateCall + LoadBlock + FullEthApiTypes,
|
||||
N: OpNodeCore,
|
||||
{
|
||||
}
|
||||
|
||||
@ -21,7 +21,7 @@ use reth_rpc_eth_api::{
|
||||
};
|
||||
use reth_rpc_eth_types::{EthApiError, PendingBlock};
|
||||
use reth_transaction_pool::{PoolTransaction, TransactionPool};
|
||||
use revm::primitives::{BlockEnv, CfgEnvWithHandlerCfg, ExecutionResult, SpecId};
|
||||
use revm::primitives::{BlockEnv, ExecutionResult};
|
||||
|
||||
impl<N> LoadPendingBlock for OpEthApi<N>
|
||||
where
|
||||
@ -82,23 +82,26 @@ where
|
||||
|
||||
fn assemble_block(
|
||||
&self,
|
||||
cfg: CfgEnvWithHandlerCfg,
|
||||
block_env: BlockEnv,
|
||||
block_env: &BlockEnv,
|
||||
parent_hash: B256,
|
||||
state_root: B256,
|
||||
transactions: Vec<ProviderTx<Self::Provider>>,
|
||||
receipts: &[ProviderReceipt<Self::Provider>],
|
||||
) -> reth_provider::ProviderBlock<Self::Provider> {
|
||||
let chain_spec = self.provider().chain_spec();
|
||||
let timestamp = block_env.timestamp.to::<u64>();
|
||||
|
||||
let transactions_root = calculate_transaction_root(&transactions);
|
||||
let receipts_root = calculate_receipt_root_no_memo_optimism(
|
||||
&receipts.iter().collect::<Vec<_>>(),
|
||||
&chain_spec,
|
||||
block_env.timestamp.to::<u64>(),
|
||||
timestamp,
|
||||
);
|
||||
|
||||
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 {
|
||||
parent_hash,
|
||||
@ -107,10 +110,9 @@ where
|
||||
state_root,
|
||||
transactions_root,
|
||||
receipts_root,
|
||||
withdrawals_root: (cfg.handler_cfg.spec_id >= SpecId::SHANGHAI)
|
||||
.then_some(EMPTY_WITHDRAWALS),
|
||||
withdrawals_root: (is_shanghai).then_some(EMPTY_WITHDRAWALS),
|
||||
logs_bloom,
|
||||
timestamp: block_env.timestamp.to::<u64>(),
|
||||
timestamp,
|
||||
mix_hash: block_env.prevrandao.unwrap_or_default(),
|
||||
nonce: BEACON_NONCE.into(),
|
||||
base_fee_per_gas: Some(block_env.basefee.to::<u64>()),
|
||||
@ -118,15 +120,13 @@ where
|
||||
gas_limit: block_env.gas_limit.to::<u64>(),
|
||||
difficulty: U256::ZERO,
|
||||
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>()
|
||||
}),
|
||||
excess_blob_gas: block_env.get_blob_excess_gas().map(Into::into),
|
||||
extra_data: Default::default(),
|
||||
parent_beacon_block_root: (cfg.handler_cfg.spec_id >= SpecId::CANCUN)
|
||||
.then_some(B256::ZERO),
|
||||
requests_hash: (cfg.handler_cfg.spec_id >= SpecId::PRAGUE)
|
||||
.then_some(EMPTY_REQUESTS_HASH),
|
||||
parent_beacon_block_root: is_cancun.then_some(B256::ZERO),
|
||||
requests_hash: is_prague.then_some(EMPTY_REQUESTS_HASH),
|
||||
target_blobs_per_block: None,
|
||||
};
|
||||
|
||||
@ -139,7 +139,7 @@ where
|
||||
|
||||
fn assemble_receipt(
|
||||
&self,
|
||||
tx: &reth_primitives::RecoveredTx<ProviderTx<Self::Provider>>,
|
||||
tx: &ProviderTx<Self::Provider>,
|
||||
result: ExecutionResult,
|
||||
cumulative_gas_used: u64,
|
||||
) -> reth_provider::ProviderReceipt<Self::Provider> {
|
||||
|
||||
@ -1,7 +1,7 @@
|
||||
//! Loads and formats OP transaction RPC response.
|
||||
|
||||
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 op_alloy_consensus::OpTxEnvelope;
|
||||
use op_alloy_rpc_types::Transaction;
|
||||
@ -14,7 +14,7 @@ use reth_rpc_eth_api::{
|
||||
helpers::{EthSigner, EthTransactions, LoadTransaction, SpawnBlocking},
|
||||
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 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) {
|
||||
let input = match &mut tx.inner.inner {
|
||||
OpTxEnvelope::Eip1559(tx) => &mut tx.tx_mut().input,
|
||||
|
||||
@ -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.
|
||||
// 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
|
||||
|
||||
@ -26,7 +26,6 @@ reth-rpc-types-compat.workspace = true
|
||||
reth-tasks = { workspace = true, features = ["rayon"] }
|
||||
reth-transaction-pool.workspace = true
|
||||
reth-chainspec.workspace = true
|
||||
reth-execution-types.workspace = true
|
||||
reth-rpc-eth-types.workspace = true
|
||||
reth-rpc-server-types.workspace = true
|
||||
reth-network-api.workspace = true
|
||||
|
||||
@ -13,7 +13,6 @@ use alloy_rpc_types_eth::{
|
||||
};
|
||||
use alloy_serde::JsonStorageKey;
|
||||
use jsonrpsee::{core::RpcResult, proc_macros::rpc};
|
||||
use reth_provider::BlockReader;
|
||||
use reth_rpc_server_types::{result::internal_rpc_err, ToRpcResult};
|
||||
use tracing::trace;
|
||||
|
||||
@ -372,12 +371,7 @@ impl<T>
|
||||
RpcHeader<T::NetworkTypes>,
|
||||
> for T
|
||||
where
|
||||
T: FullEthApi<
|
||||
Provider: BlockReader<
|
||||
Header = alloy_consensus::Header,
|
||||
Transaction = reth_primitives::TransactionSigned,
|
||||
>,
|
||||
>,
|
||||
T: FullEthApi,
|
||||
jsonrpsee_types::error::ErrorObject<'static>: From<T::Error>,
|
||||
{
|
||||
/// Handler for: `eth_protocolVersion`
|
||||
|
||||
@ -20,9 +20,7 @@ use reth_chainspec::EthChainSpec;
|
||||
use reth_evm::{ConfigureEvm, ConfigureEvmEnv};
|
||||
use reth_node_api::BlockBody;
|
||||
use reth_primitives_traits::SignedTransaction;
|
||||
use reth_provider::{
|
||||
BlockIdReader, BlockReader, ChainSpecProvider, HeaderProvider, ProviderHeader,
|
||||
};
|
||||
use reth_provider::{BlockIdReader, ChainSpecProvider, HeaderProvider, ProviderHeader};
|
||||
use reth_revm::{
|
||||
database::StateProviderDatabase,
|
||||
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
|
||||
/// 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`].
|
||||
fn estimate_gas_at(
|
||||
&self,
|
||||
@ -70,15 +68,7 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock {
|
||||
&self,
|
||||
payload: SimulatePayload,
|
||||
block: Option<BlockId>,
|
||||
) -> impl Future<Output = SimulatedBlocksResult<Self::NetworkTypes, Self::Error>> + Send
|
||||
where
|
||||
Self: LoadBlock<
|
||||
Provider: BlockReader<
|
||||
Header = alloy_consensus::Header,
|
||||
Transaction = reth_primitives::TransactionSigned,
|
||||
>,
|
||||
> + FullEthApiTypes,
|
||||
{
|
||||
) -> impl Future<Output = SimulatedBlocksResult<Self::NetworkTypes, Self::Error>> + Send {
|
||||
async move {
|
||||
if payload.block_state_calls.len() > self.max_simulate_blocks() as usize {
|
||||
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(),
|
||||
cfg.chain_id,
|
||||
&mut db,
|
||||
this.tx_resp_builder(),
|
||||
)?;
|
||||
|
||||
let mut calls = calls.into_iter().peekable();
|
||||
let mut senders = Vec::with_capacity(transactions.len());
|
||||
let mut results = Vec::with_capacity(calls.len());
|
||||
|
||||
while let Some(tx) = calls.next() {
|
||||
@ -197,18 +189,27 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock {
|
||||
db.commit(res.state);
|
||||
}
|
||||
|
||||
results.push((env.tx.caller, res.result));
|
||||
senders.push(env.tx.caller);
|
||||
results.push(res.result);
|
||||
}
|
||||
|
||||
let block: SimulatedBlock<RpcBlock<Self::NetworkTypes>> =
|
||||
simulate::build_block(
|
||||
results,
|
||||
transactions,
|
||||
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>> =
|
||||
simulate::build_simulated_block(
|
||||
senders,
|
||||
results,
|
||||
total_difficulty,
|
||||
return_full_transactions,
|
||||
this.tx_resp_builder(),
|
||||
block,
|
||||
)?;
|
||||
|
||||
parent_hash = block.inner.header.hash;
|
||||
@ -245,10 +246,7 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock {
|
||||
bundle: Bundle,
|
||||
state_context: Option<StateContext>,
|
||||
mut state_override: Option<StateOverride>,
|
||||
) -> impl Future<Output = Result<Vec<EthCallResponse>, Self::Error>> + Send
|
||||
where
|
||||
Self: LoadBlock,
|
||||
{
|
||||
) -> impl Future<Output = Result<Vec<EthCallResponse>, Self::Error>> + Send {
|
||||
async move {
|
||||
let Bundle { transactions, block_override } = bundle;
|
||||
if transactions.is_empty() {
|
||||
@ -608,7 +606,7 @@ pub trait Call:
|
||||
f: F,
|
||||
) -> impl Future<Output = Result<Option<R>, Self::Error>> + Send
|
||||
where
|
||||
Self: LoadBlock + LoadPendingBlock + LoadTransaction,
|
||||
Self: LoadBlock + LoadTransaction,
|
||||
F: FnOnce(TransactionInfo, ResultAndState, StateCacheDb<'_>) -> Result<R, Self::Error>
|
||||
+ Send
|
||||
+ 'static,
|
||||
|
||||
@ -42,12 +42,9 @@ pub use transaction::{EthTransactions, LoadTransaction};
|
||||
use crate::FullEthApiTypes;
|
||||
|
||||
/// Extension trait that bundles traits needed for tracing transactions.
|
||||
pub trait TraceExt:
|
||||
LoadTransaction + LoadBlock + LoadPendingBlock + SpawnBlocking + Trace + Call
|
||||
{
|
||||
}
|
||||
pub trait TraceExt: LoadTransaction + LoadBlock + 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.
|
||||
///
|
||||
|
||||
@ -15,8 +15,7 @@ use reth_evm::{
|
||||
state_change::post_block_withdrawals_balance_increments, system_calls::SystemCaller,
|
||||
ConfigureEvm, ConfigureEvmEnv, NextBlockEnvAttributes,
|
||||
};
|
||||
use reth_execution_types::ExecutionOutcome;
|
||||
use reth_primitives::{BlockExt, InvalidTransactionError, RecoveredTx, SealedBlockWithSenders};
|
||||
use reth_primitives::{BlockExt, InvalidTransactionError, SealedBlockWithSenders};
|
||||
use reth_primitives_traits::receipt::ReceiptExt;
|
||||
use reth_provider::{
|
||||
BlockReader, BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, ProviderBlock, ProviderError,
|
||||
@ -199,7 +198,7 @@ pub trait LoadPendingBlock:
|
||||
/// Assembles a receipt for a transaction, based on its [`ExecutionResult`].
|
||||
fn assemble_receipt(
|
||||
&self,
|
||||
tx: &RecoveredTx<ProviderTx<Self::Provider>>,
|
||||
tx: &ProviderTx<Self::Provider>,
|
||||
result: ExecutionResult,
|
||||
cumulative_gas_used: u64,
|
||||
) -> ProviderReceipt<Self::Provider>;
|
||||
@ -207,14 +206,36 @@ pub trait LoadPendingBlock:
|
||||
/// Assembles a pending block.
|
||||
fn assemble_block(
|
||||
&self,
|
||||
cfg: CfgEnvWithHandlerCfg,
|
||||
block_env: BlockEnv,
|
||||
block_env: &BlockEnv,
|
||||
parent_hash: revm_primitives::B256,
|
||||
state_root: revm_primitives::B256,
|
||||
transactions: Vec<ProviderTx<Self::Provider>>,
|
||||
receipts: &[ProviderReceipt<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.
|
||||
///
|
||||
/// 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 block_gas_limit: u64 = block_env.gas_limit.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 senders = Vec::new();
|
||||
@ -266,7 +286,7 @@ pub trait LoadPendingBlock:
|
||||
.pre_block_blockhashes_contract_call(&mut db, &cfg, &block_env, parent_hash)
|
||||
.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() {
|
||||
// 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
|
||||
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
|
||||
let (tx, sender) = tx.to_components();
|
||||
executed_txs.push(tx);
|
||||
senders.push(sender);
|
||||
results.push(result);
|
||||
}
|
||||
|
||||
// executes the withdrawals and commits them to the Database and BundleState.
|
||||
@ -396,22 +414,19 @@ pub trait LoadPendingBlock:
|
||||
// merge all transitions into bundle state.
|
||||
db.merge_transitions(BundleRetention::PlainState);
|
||||
|
||||
let execution_outcome: ExecutionOutcome<ProviderReceipt<Self::Provider>> =
|
||||
ExecutionOutcome::new(
|
||||
db.take_bundle(),
|
||||
vec![receipts.clone()].into(),
|
||||
block_number,
|
||||
Vec::new(),
|
||||
);
|
||||
let hashed_state = db.database.hashed_post_state(execution_outcome.state());
|
||||
let bundle_state = db.take_bundle();
|
||||
let hashed_state = db.database.hashed_post_state(&bundle_state);
|
||||
|
||||
// calculate the state root
|
||||
let state_root = db.database.state_root(hashed_state).map_err(Self::Error::from_eth_err)?;
|
||||
|
||||
// Convert Vec<Option<Receipt>> to Vec<Receipt>
|
||||
let receipts: Vec<_> = receipts.into_iter().flatten().collect();
|
||||
let block =
|
||||
self.assemble_block(cfg, block_env, parent_hash, state_root, executed_txs, &receipts);
|
||||
let (block, receipts) = self.assemble_block_and_receipts(
|
||||
&block_env,
|
||||
parent_hash,
|
||||
state_root,
|
||||
executed_txs,
|
||||
results,
|
||||
);
|
||||
|
||||
Ok((SealedBlockWithSenders { block: block.seal_slow(), senders }, receipts))
|
||||
}
|
||||
|
||||
@ -25,9 +25,7 @@ use reth_rpc_types_compat::transaction::{from_recovered, from_recovered_with_blo
|
||||
use reth_transaction_pool::{PoolTransaction, TransactionOrigin, TransactionPool};
|
||||
use std::sync::Arc;
|
||||
|
||||
use super::{
|
||||
EthApiSpec, EthSigner, LoadBlock, LoadPendingBlock, LoadReceipt, LoadState, SpawnBlocking,
|
||||
};
|
||||
use super::{EthApiSpec, EthSigner, LoadBlock, LoadReceipt, LoadState, SpawnBlocking};
|
||||
use crate::{
|
||||
helpers::estimate::EstimateCall, FromEthApiError, FullEthApiTypes, IntoEthApiError,
|
||||
RpcNodeCore, RpcNodeCoreExt, RpcReceipt, RpcTransaction,
|
||||
@ -365,7 +363,7 @@ pub trait EthTransactions: LoadTransaction<Provider: BlockReaderIdExt> {
|
||||
mut request: TransactionRequest,
|
||||
) -> impl Future<Output = Result<B256, Self::Error>> + Send
|
||||
where
|
||||
Self: EthApiSpec + LoadBlock + LoadPendingBlock + EstimateCall,
|
||||
Self: EthApiSpec + LoadBlock + EstimateCall,
|
||||
{
|
||||
async move {
|
||||
let from = match request.from {
|
||||
|
||||
@ -1,21 +1,18 @@
|
||||
//! Utilities for serving `eth_simulateV1`
|
||||
|
||||
use alloy_consensus::{Transaction as _, TxType};
|
||||
use alloy_primitives::PrimitiveSignature as Signature;
|
||||
use alloy_consensus::{BlockHeader, Transaction as _, TxType};
|
||||
use alloy_rpc_types_eth::{
|
||||
simulate::{SimCallResult, SimulateError, SimulatedBlock},
|
||||
transaction::TransactionRequest,
|
||||
Block, BlockTransactionsKind,
|
||||
Block, BlockTransactionsKind, Header,
|
||||
};
|
||||
use jsonrpsee_types::ErrorObject;
|
||||
use reth_primitives::{
|
||||
proofs::{calculate_receipt_root, calculate_transaction_root},
|
||||
BlockBody, BlockWithSenders, Receipt, TransactionSigned,
|
||||
};
|
||||
use reth_primitives::BlockWithSenders;
|
||||
use reth_primitives_traits::{block::BlockTx, BlockBody as _, SignedTransaction};
|
||||
use reth_rpc_server_types::result::rpc_err;
|
||||
use reth_rpc_types_compat::{block::from_block, TransactionCompat};
|
||||
use revm::Database;
|
||||
use revm_primitives::{Address, BlockEnv, Bytes, ExecutionResult, TxKind, B256, U256};
|
||||
use revm_primitives::{Address, Bytes, ExecutionResult, TxKind, U256};
|
||||
|
||||
use crate::{
|
||||
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
|
||||
/// them into [`TransactionSigned`].
|
||||
/// them into primitive transactions.
|
||||
///
|
||||
/// If validation is enabled, the function will return error if any of the transactions can't be
|
||||
/// built right away.
|
||||
pub fn resolve_transactions<DB: Database>(
|
||||
pub fn resolve_transactions<DB: Database, Tx, T: TransactionCompat<Tx>>(
|
||||
txs: &mut [TransactionRequest],
|
||||
validation: bool,
|
||||
block_gas_limit: u64,
|
||||
chain_id: u64,
|
||||
db: &mut DB,
|
||||
) -> Result<Vec<TransactionSigned>, EthApiError>
|
||||
tx_resp_builder: &T,
|
||||
) -> Result<Vec<Tx>, EthApiError>
|
||||
where
|
||||
EthApiError: From<DB::Error>,
|
||||
{
|
||||
@ -125,49 +123,44 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
let Ok(tx) = tx.clone().build_typed_tx() else {
|
||||
return Err(EthApiError::TransactionConversionError)
|
||||
};
|
||||
|
||||
// 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);
|
||||
transactions.push(
|
||||
tx_resp_builder
|
||||
.build_simulate_v1_transaction(tx.clone())
|
||||
.map_err(|e| EthApiError::other(e.into()))?,
|
||||
);
|
||||
}
|
||||
|
||||
Ok(transactions)
|
||||
}
|
||||
|
||||
/// Handles outputs of the calls execution and builds a [`SimulatedBlock`].
|
||||
pub fn build_block<T: TransactionCompat<Error: FromEthApiError>>(
|
||||
results: Vec<(Address, ExecutionResult)>,
|
||||
transactions: Vec<TransactionSigned>,
|
||||
block_env: &BlockEnv,
|
||||
parent_hash: B256,
|
||||
#[expect(clippy::type_complexity)]
|
||||
pub fn build_simulated_block<T, B>(
|
||||
senders: Vec<Address>,
|
||||
results: Vec<ExecutionResult>,
|
||||
total_difficulty: U256,
|
||||
full_transactions: bool,
|
||||
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 senders = Vec::with_capacity(results.len());
|
||||
let mut receipts = Vec::with_capacity(results.len());
|
||||
|
||||
let mut log_index = 0;
|
||||
for (transaction_index, ((sender, result), tx)) in
|
||||
results.into_iter().zip(transactions.iter()).enumerate()
|
||||
{
|
||||
senders.push(sender);
|
||||
|
||||
for (index, (result, tx)) in results.iter().zip(block.body().transactions()).enumerate() {
|
||||
let call = match result {
|
||||
ExecutionResult::Halt { reason, gas_used } => {
|
||||
let error = RpcInvalidTransactionError::halt(reason, tx.gas_limit());
|
||||
let error = RpcInvalidTransactionError::halt(*reason, tx.gas_limit());
|
||||
SimCallResult {
|
||||
return_data: Bytes::new(),
|
||||
error: Some(SimulateError {
|
||||
code: error.error_code(),
|
||||
message: error.to_string(),
|
||||
}),
|
||||
gas_used,
|
||||
gas_used: *gas_used,
|
||||
logs: Vec::new(),
|
||||
status: false,
|
||||
}
|
||||
@ -175,31 +168,31 @@ pub fn build_block<T: TransactionCompat<Error: FromEthApiError>>(
|
||||
ExecutionResult::Revert { output, gas_used } => {
|
||||
let error = RevertError::new(output.clone());
|
||||
SimCallResult {
|
||||
return_data: output,
|
||||
return_data: output.clone(),
|
||||
error: Some(SimulateError {
|
||||
code: error.error_code(),
|
||||
message: error.to_string(),
|
||||
}),
|
||||
gas_used,
|
||||
gas_used: *gas_used,
|
||||
status: false,
|
||||
logs: Vec::new(),
|
||||
}
|
||||
}
|
||||
ExecutionResult::Success { output, gas_used, logs, .. } => SimCallResult {
|
||||
return_data: output.into_data(),
|
||||
return_data: output.clone().into_data(),
|
||||
error: None,
|
||||
gas_used,
|
||||
gas_used: *gas_used,
|
||||
logs: logs
|
||||
.into_iter()
|
||||
.iter()
|
||||
.map(|log| {
|
||||
log_index += 1;
|
||||
alloy_rpc_types_eth::Log {
|
||||
inner: log,
|
||||
inner: log.clone(),
|
||||
log_index: Some(log_index - 1),
|
||||
transaction_index: Some(transaction_index as u64),
|
||||
transaction_hash: Some(tx.hash()),
|
||||
block_number: Some(block_env.number.to()),
|
||||
block_timestamp: Some(block_env.timestamp.to()),
|
||||
transaction_index: Some(index as u64),
|
||||
transaction_hash: Some(*tx.tx_hash()),
|
||||
block_number: Some(block.header().number()),
|
||||
block_timestamp: Some(block.header().timestamp()),
|
||||
..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);
|
||||
}
|
||||
|
||||
// TODO: uncomment once performance cost is acceptable
|
||||
//
|
||||
// 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 block = BlockWithSenders { block, senders };
|
||||
|
||||
let txs_kind =
|
||||
if full_transactions { BlockTransactionsKind::Full } else { BlockTransactionsKind::Hashes };
|
||||
|
||||
@ -57,6 +57,10 @@ pub trait TransactionCompat<T = TransactionSigned>:
|
||||
tx_inf: TransactionInfo,
|
||||
) -> 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.
|
||||
// todo: remove in favour of using constructor on `TransactionResponse` or similar
|
||||
// <https://github.com/alloy-rs/alloy/issues/1315>.
|
||||
|
||||
@ -4,13 +4,14 @@ use crate::EthApi;
|
||||
use alloy_consensus::Header;
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_provider::{BlockReader, ProviderHeader};
|
||||
use reth_rpc_eth_api::helpers::{
|
||||
estimate::EstimateCall, Call, EthCall, LoadPendingBlock, LoadState, SpawnBlocking,
|
||||
use reth_rpc_eth_api::{
|
||||
helpers::{estimate::EstimateCall, Call, EthCall, LoadPendingBlock, LoadState, SpawnBlocking},
|
||||
FullEthApiTypes,
|
||||
};
|
||||
|
||||
impl<Provider, Pool, Network, EvmConfig> EthCall for EthApi<Provider, Pool, Network, EvmConfig>
|
||||
where
|
||||
Self: EstimateCall + LoadPendingBlock,
|
||||
Self: EstimateCall + LoadPendingBlock + FullEthApiTypes,
|
||||
Provider: BlockReader,
|
||||
{
|
||||
}
|
||||
|
||||
@ -20,7 +20,7 @@ use reth_rpc_eth_api::{
|
||||
};
|
||||
use reth_rpc_eth_types::PendingBlock;
|
||||
use reth_transaction_pool::{PoolTransaction, TransactionPool};
|
||||
use revm_primitives::{BlockEnv, CfgEnvWithHandlerCfg, SpecId, B256};
|
||||
use revm_primitives::{BlockEnv, B256};
|
||||
|
||||
use crate::EthApi;
|
||||
|
||||
@ -56,18 +56,24 @@ where
|
||||
|
||||
fn assemble_block(
|
||||
&self,
|
||||
cfg: CfgEnvWithHandlerCfg,
|
||||
block_env: BlockEnv,
|
||||
block_env: &BlockEnv,
|
||||
parent_hash: revm_primitives::B256,
|
||||
state_root: revm_primitives::B256,
|
||||
transactions: Vec<ProviderTx<Self::Provider>>,
|
||||
receipts: &[ProviderReceipt<Self::Provider>],
|
||||
) -> reth_provider::ProviderBlock<Self::Provider> {
|
||||
let chain_spec = self.provider().chain_spec();
|
||||
|
||||
let transactions_root = calculate_transaction_root(&transactions);
|
||||
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 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 {
|
||||
parent_hash,
|
||||
ommers_hash: EMPTY_OMMER_ROOT_HASH,
|
||||
@ -75,8 +81,7 @@ where
|
||||
state_root,
|
||||
transactions_root,
|
||||
receipts_root,
|
||||
withdrawals_root: (cfg.handler_cfg.spec_id >= SpecId::SHANGHAI)
|
||||
.then_some(EMPTY_WITHDRAWALS),
|
||||
withdrawals_root: is_shanghai.then_some(EMPTY_WITHDRAWALS),
|
||||
logs_bloom,
|
||||
timestamp: block_env.timestamp.to::<u64>(),
|
||||
mix_hash: block_env.prevrandao.unwrap_or_default(),
|
||||
@ -86,15 +91,13 @@ where
|
||||
gas_limit: block_env.gas_limit.to::<u64>(),
|
||||
difficulty: U256::ZERO,
|
||||
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>()
|
||||
}),
|
||||
excess_blob_gas: block_env.get_blob_excess_gas().map(Into::into),
|
||||
extra_data: Default::default(),
|
||||
parent_beacon_block_root: (cfg.handler_cfg.spec_id >= SpecId::CANCUN)
|
||||
.then_some(B256::ZERO),
|
||||
requests_hash: (cfg.handler_cfg.spec_id >= SpecId::PRAGUE)
|
||||
.then_some(EMPTY_REQUESTS_HASH),
|
||||
parent_beacon_block_root: is_cancun.then_some(B256::ZERO),
|
||||
requests_hash: is_prague.then_some(EMPTY_REQUESTS_HASH),
|
||||
target_blobs_per_block: None,
|
||||
};
|
||||
|
||||
@ -107,7 +110,7 @@ where
|
||||
|
||||
fn assemble_receipt(
|
||||
&self,
|
||||
tx: &reth_primitives::RecoveredTx<ProviderTx<Self::Provider>>,
|
||||
tx: &ProviderTx<Self::Provider>,
|
||||
result: revm_primitives::ExecutionResult,
|
||||
cumulative_gas_used: u64,
|
||||
) -> reth_provider::ProviderReceipt<Self::Provider> {
|
||||
|
||||
@ -2,6 +2,8 @@
|
||||
|
||||
use alloy_consensus::{Signed, Transaction as _, TxEip4844Variant, TxEnvelope};
|
||||
use alloy_network::{Ethereum, Network};
|
||||
use alloy_primitives::PrimitiveSignature as Signature;
|
||||
use alloy_rpc_types::TransactionRequest;
|
||||
use alloy_rpc_types_eth::{Transaction, TransactionInfo};
|
||||
use reth_primitives::{RecoveredTx, TransactionSigned};
|
||||
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) {
|
||||
let input = match &mut tx.inner {
|
||||
TxEnvelope::Eip1559(tx) => &mut tx.tx_mut().input,
|
||||
|
||||
Reference in New Issue
Block a user