feat: generic TxEnv (#13957)

This commit is contained in:
Arsenii Kulikov
2025-01-24 14:48:44 +04:00
committed by GitHub
parent 1296bacb87
commit 006eea0c34
18 changed files with 271 additions and 244 deletions

View File

@ -19,7 +19,7 @@ use reth_evm::{
}, },
state_change::post_block_balance_increments, state_change::post_block_balance_increments,
system_calls::{OnStateHook, SystemCaller}, system_calls::{OnStateHook, SystemCaller},
ConfigureEvm, Evm, TxEnvOverrides, ConfigureEvm, Evm,
}; };
use reth_primitives::{EthPrimitives, Receipt, RecoveredBlock}; use reth_primitives::{EthPrimitives, Receipt, RecoveredBlock};
use reth_primitives_traits::{BlockBody, SignedTransaction}; use reth_primitives_traits::{BlockBody, SignedTransaction};
@ -94,8 +94,6 @@ where
chain_spec: Arc<ChainSpec>, chain_spec: Arc<ChainSpec>,
/// How to create an EVM. /// How to create an EVM.
evm_config: EvmConfig, evm_config: EvmConfig,
/// Optional overrides for the transactions environment.
tx_env_overrides: Option<Box<dyn TxEnvOverrides>>,
/// Current state for block execution. /// Current state for block execution.
state: State<DB>, state: State<DB>,
/// Utility to call system smart contracts. /// Utility to call system smart contracts.
@ -109,7 +107,7 @@ where
/// Creates a new [`EthExecutionStrategy`] /// Creates a new [`EthExecutionStrategy`]
pub fn new(state: State<DB>, chain_spec: Arc<ChainSpec>, evm_config: EvmConfig) -> Self { pub fn new(state: State<DB>, chain_spec: Arc<ChainSpec>, evm_config: EvmConfig) -> Self {
let system_caller = SystemCaller::new(evm_config.clone(), chain_spec.clone()); let system_caller = SystemCaller::new(evm_config.clone(), chain_spec.clone());
Self { state, chain_spec, evm_config, system_caller, tx_env_overrides: None } Self { state, chain_spec, evm_config, system_caller }
} }
} }
@ -123,13 +121,8 @@ where
{ {
type DB = DB; type DB = DB;
type Error = BlockExecutionError; type Error = BlockExecutionError;
type Primitives = EthPrimitives; type Primitives = EthPrimitives;
fn init(&mut self, tx_env_overrides: Box<dyn TxEnvOverrides>) {
self.tx_env_overrides = Some(tx_env_overrides);
}
fn apply_pre_execution_changes( fn apply_pre_execution_changes(
&mut self, &mut self,
block: &RecoveredBlock<reth_primitives::Block>, block: &RecoveredBlock<reth_primitives::Block>,
@ -166,11 +159,7 @@ where
.into()) .into())
} }
let mut tx_env = self.evm_config.tx_env(transaction, *sender); let tx_env = self.evm_config.tx_env(transaction, *sender);
if let Some(tx_env_overrides) = &mut self.tx_env_overrides {
tx_env_overrides.apply(&mut tx_env);
}
// Execute transaction. // Execute transaction.
let result_and_state = evm.transact(tx_env).map_err(move |err| { let result_and_state = evm.transact(tx_env).map_err(move |err| {

View File

@ -150,6 +150,7 @@ impl ConfigureEvmEnv for EthEvmConfig {
type Header = Header; type Header = Header;
type Transaction = TransactionSigned; type Transaction = TransactionSigned;
type Error = Infallible; type Error = Infallible;
type TxEnv = TxEnv;
fn fill_tx_env(&self, tx_env: &mut TxEnv, transaction: &TransactionSigned, sender: Address) { fn fill_tx_env(&self, tx_env: &mut TxEnv, transaction: &TransactionSigned, sender: Address) {
transaction.fill_tx_env(tx_env, sender); transaction.fill_tx_env(tx_env, sender);

View File

@ -6,7 +6,6 @@ use crate::{
execute::{BatchExecutor, BlockExecutorProvider, Executor}, execute::{BatchExecutor, BlockExecutorProvider, Executor},
system_calls::OnStateHook, system_calls::OnStateHook,
}; };
use alloc::boxed::Box;
use alloy_primitives::BlockNumber; use alloy_primitives::BlockNumber;
use reth_prune_types::PruneModes; use reth_prune_types::PruneModes;
use reth_storage_errors::provider::ProviderError; use reth_storage_errors::provider::ProviderError;
@ -60,13 +59,6 @@ where
type Output = A::Output; type Output = A::Output;
type Error = A::Error; type Error = A::Error;
fn init(&mut self, tx_env_overrides: Box<dyn crate::TxEnvOverrides>) {
match self {
Self::Left(a) => a.init(tx_env_overrides),
Self::Right(b) => b.init(tx_env_overrides),
}
}
fn execute(self, input: Self::Input<'_>) -> Result<Self::Output, Self::Error> { fn execute(self, input: Self::Input<'_>) -> Result<Self::Output, Self::Error> {
match self { match self {
Self::Left(a) => a.execute(input), Self::Left(a) => a.execute(input),

View File

@ -8,7 +8,7 @@ pub use reth_execution_errors::{
pub use reth_execution_types::{BlockExecutionOutput, ExecutionOutcome}; pub use reth_execution_types::{BlockExecutionOutput, ExecutionOutcome};
pub use reth_storage_errors::provider::ProviderError; pub use reth_storage_errors::provider::ProviderError;
use crate::{system_calls::OnStateHook, TxEnvOverrides}; use crate::system_calls::OnStateHook;
use alloc::{boxed::Box, vec::Vec}; use alloc::{boxed::Box, vec::Vec};
use alloy_eips::eip7685::Requests; use alloy_eips::eip7685::Requests;
use alloy_primitives::{ use alloy_primitives::{
@ -38,9 +38,6 @@ pub trait Executor<DB> {
/// The error type returned by the executor. /// The error type returned by the executor.
type Error; type Error;
/// Initialize the executor with the given transaction environment overrides.
fn init(&mut self, _tx_env_overrides: Box<dyn TxEnvOverrides>) {}
/// Consumes the type and executes the block. /// Consumes the type and executes the block.
/// ///
/// # Note /// # Note
@ -199,9 +196,6 @@ pub trait BlockExecutionStrategy {
/// The error type returned by this strategy's methods. /// The error type returned by this strategy's methods.
type Error: From<ProviderError> + core::error::Error; type Error: From<ProviderError> + core::error::Error;
/// Initialize the strategy with the given transaction environment overrides.
fn init(&mut self, _tx_env_overrides: Box<dyn TxEnvOverrides>) {}
/// Applies any necessary changes before executing the block's transactions. /// Applies any necessary changes before executing the block's transactions.
fn apply_pre_execution_changes( fn apply_pre_execution_changes(
&mut self, &mut self,
@ -341,10 +335,6 @@ where
type Output = BlockExecutionOutput<<S::Primitives as NodePrimitives>::Receipt>; type Output = BlockExecutionOutput<<S::Primitives as NodePrimitives>::Receipt>;
type Error = S::Error; type Error = S::Error;
fn init(&mut self, env_overrides: Box<dyn TxEnvOverrides>) {
self.strategy.init(env_overrides);
}
fn execute(mut self, block: Self::Input<'_>) -> Result<Self::Output, Self::Error> { fn execute(mut self, block: Self::Input<'_>) -> Result<Self::Output, Self::Error> {
self.strategy.apply_pre_execution_changes(block)?; self.strategy.apply_pre_execution_changes(block)?;
let ExecuteOutput { receipts, gas_used } = self.strategy.execute_transactions(block)?; let ExecuteOutput { receipts, gas_used } = self.strategy.execute_transactions(block)?;
@ -518,7 +508,7 @@ mod tests {
use reth_chainspec::{ChainSpec, MAINNET}; use reth_chainspec::{ChainSpec, MAINNET};
use reth_primitives::EthPrimitives; use reth_primitives::EthPrimitives;
use revm::db::{CacheDB, EmptyDBTyped}; use revm::db::{CacheDB, EmptyDBTyped};
use revm_primitives::{address, bytes, AccountInfo, TxEnv, KECCAK_EMPTY}; use revm_primitives::{address, bytes, AccountInfo, KECCAK_EMPTY};
use std::sync::Arc; use std::sync::Arc;
#[derive(Clone, Default)] #[derive(Clone, Default)]
@ -734,29 +724,6 @@ mod tests {
assert_eq!(block_execution_output.state, expected_finish_result); assert_eq!(block_execution_output.state, expected_finish_result);
} }
#[test]
fn test_tx_env_overrider() {
let strategy_factory = TestExecutorStrategyFactory {
execute_transactions_result: ExecuteOutput {
receipts: vec![Receipt::default()],
gas_used: 10,
},
apply_post_execution_changes_result: Requests::new(vec![bytes!("deadbeef")]),
finish_result: BundleState::default(),
};
let provider = BasicBlockExecutorProvider::new(strategy_factory);
let db = CacheDB::<EmptyDBTyped<ProviderError>>::default();
// if we want to apply tx env overrides the executor must be mut.
let mut executor = provider.executor(db);
// execute consumes the executor, so we can only call it once.
executor.init(Box::new(|tx_env: &mut TxEnv| {
tx_env.nonce.take();
}));
let result = executor.execute(&Default::default());
assert!(result.is_ok());
}
fn setup_state_with_account( fn setup_state_with_account(
addr: Address, addr: Address,
balance: u128, balance: u128,

View File

@ -17,11 +17,16 @@
extern crate alloc; extern crate alloc;
use core::fmt::Debug;
use alloy_consensus::BlockHeader as _; use alloy_consensus::BlockHeader as _;
use alloy_eips::eip2930::AccessList;
use alloy_primitives::{Address, Bytes, B256, U256}; use alloy_primitives::{Address, Bytes, B256, U256};
use reth_primitives_traits::{BlockHeader, SignedTransaction}; use reth_primitives_traits::{BlockHeader, SignedTransaction};
use revm::{Database, DatabaseCommit, GetInspector}; use revm::{Database, DatabaseCommit, GetInspector};
use revm_primitives::{BlockEnv, CfgEnvWithHandlerCfg, EVMError, ResultAndState, SpecId, TxEnv}; use revm_primitives::{
BlockEnv, CfgEnvWithHandlerCfg, EVMError, ResultAndState, SpecId, TxEnv, TxKind,
};
pub mod either; pub mod either;
/// EVM environment configuration. /// EVM environment configuration.
@ -87,7 +92,11 @@ pub trait Evm {
/// Trait for configuring the EVM for executing full blocks. /// Trait for configuring the EVM for executing full blocks.
pub trait ConfigureEvm: ConfigureEvmEnv { pub trait ConfigureEvm: ConfigureEvmEnv {
/// The EVM implementation. /// The EVM implementation.
type Evm<'a, DB: Database + 'a, I: 'a>: Evm<Tx = TxEnv, DB = DB, Error = EVMError<DB::Error>>; type Evm<'a, DB: Database + 'a, I: 'a>: Evm<
Tx = Self::TxEnv,
DB = DB,
Error = EVMError<DB::Error>,
>;
/// Returns a new EVM with the given database configured with the given environment settings, /// Returns a new EVM with the given database configured with the given environment settings,
/// including the spec id and transaction environment. /// including the spec id and transaction environment.
@ -127,7 +136,7 @@ pub trait ConfigureEvm: ConfigureEvmEnv {
impl<'b, T> ConfigureEvm for &'b T impl<'b, T> ConfigureEvm for &'b T
where where
T: ConfigureEvm, T: ConfigureEvm,
&'b T: ConfigureEvmEnv<Header = T::Header>, &'b T: ConfigureEvmEnv<Header = T::Header, TxEnv = T::TxEnv>,
{ {
type Evm<'a, DB: Database + 'a, I: 'a> = T::Evm<'a, DB, I>; type Evm<'a, DB: Database + 'a, I: 'a> = T::Evm<'a, DB, I>;
@ -165,18 +174,26 @@ pub trait ConfigureEvmEnv: Send + Sync + Unpin + Clone + 'static {
/// The transaction type. /// The transaction type.
type Transaction: SignedTransaction; type Transaction: SignedTransaction;
/// Transaction environment used by EVM.
type TxEnv: TransactionEnv;
/// The error type that is returned by [`Self::next_cfg_and_block_env`]. /// The error type that is returned by [`Self::next_cfg_and_block_env`].
type Error: core::error::Error + Send + Sync; type Error: core::error::Error + Send + Sync;
/// Returns a [`TxEnv`] from a transaction and [`Address`]. /// Returns a [`TxEnv`] from a transaction and [`Address`].
fn tx_env(&self, transaction: &Self::Transaction, signer: Address) -> TxEnv { fn tx_env(&self, transaction: &Self::Transaction, signer: Address) -> Self::TxEnv {
let mut tx_env = TxEnv::default(); let mut tx_env = Default::default();
self.fill_tx_env(&mut tx_env, transaction, signer); self.fill_tx_env(&mut tx_env, transaction, signer);
tx_env tx_env
} }
/// Fill transaction environment from a transaction and the given sender address. /// Fill transaction environment from a transaction and the given sender address.
fn fill_tx_env(&self, tx_env: &mut TxEnv, transaction: &Self::Transaction, sender: Address); fn fill_tx_env(
&self,
tx_env: &mut Self::TxEnv,
transaction: &Self::Transaction,
sender: Address,
);
/// Returns a [`CfgEnvWithHandlerCfg`] for the given header. /// Returns a [`CfgEnvWithHandlerCfg`] for the given header.
fn cfg_env(&self, header: &Self::Header) -> CfgEnvWithHandlerCfg { fn cfg_env(&self, header: &Self::Header) -> CfgEnvWithHandlerCfg {
@ -262,17 +279,77 @@ pub struct NextBlockEnvAttributes {
pub gas_limit: u64, pub gas_limit: u64,
} }
/// Function hook that allows to modify a transaction environment. /// Abstraction over transaction environment.
pub trait TxEnvOverrides { pub trait TransactionEnv:
/// Apply the overrides by modifying the given `TxEnv`. Into<revm_primitives::TxEnv> + Debug + Default + Clone + Send + Sync + 'static
fn apply(&mut self, env: &mut TxEnv); {
/// Returns configured gas limit.
fn gas_limit(&self) -> u64;
/// Set the gas limit.
fn set_gas_limit(&mut self, gas_limit: u64);
/// Set the gas limit.
fn with_gas_limit(mut self, gas_limit: u64) -> Self {
self.set_gas_limit(gas_limit);
self
} }
impl<F> TxEnvOverrides for F /// Returns configured gas price.
where fn gas_price(&self) -> U256;
F: FnMut(&mut TxEnv),
{ /// Returns configured value.
fn apply(&mut self, env: &mut TxEnv) { fn value(&self) -> U256;
self(env)
/// Caller of the transaction.
fn caller(&self) -> Address;
/// Set access list.
fn set_access_list(&mut self, access_list: AccessList);
/// Set access list.
fn with_access_list(mut self, access_list: AccessList) -> Self {
self.set_access_list(access_list);
self
}
/// Returns calldata for the transaction.
fn input(&self) -> &Bytes;
/// Returns [`TxKind`] of the transaction.
fn kind(&self) -> TxKind;
}
impl TransactionEnv for TxEnv {
fn gas_limit(&self) -> u64 {
self.gas_limit
}
fn set_gas_limit(&mut self, gas_limit: u64) {
self.gas_limit = gas_limit;
}
fn gas_price(&self) -> U256 {
self.gas_price.to()
}
fn value(&self) -> U256 {
self.value
}
fn caller(&self) -> Address {
self.caller
}
fn set_access_list(&mut self, access_list: AccessList) {
self.access_list = access_list.to_vec();
}
fn input(&self) -> &Bytes {
&self.data
}
fn kind(&self) -> TxKind {
self.transact_to
} }
} }

View File

@ -19,7 +19,7 @@ use reth_evm::{
}, },
state_change::post_block_balance_increments, state_change::post_block_balance_increments,
system_calls::{OnStateHook, SystemCaller}, system_calls::{OnStateHook, SystemCaller},
ConfigureEvmFor, Evm, TxEnvOverrides, ConfigureEvmFor, Evm,
}; };
use reth_optimism_chainspec::OpChainSpec; use reth_optimism_chainspec::OpChainSpec;
use reth_optimism_consensus::validate_block_post_execution; use reth_optimism_consensus::validate_block_post_execution;
@ -104,8 +104,6 @@ where
chain_spec: Arc<OpChainSpec>, chain_spec: Arc<OpChainSpec>,
/// How to create an EVM. /// How to create an EVM.
evm_config: EvmConfig, evm_config: EvmConfig,
/// Optional overrides for the transactions environment.
tx_env_overrides: Option<Box<dyn TxEnvOverrides>>,
/// Current state for block execution. /// Current state for block execution.
state: State<DB>, state: State<DB>,
/// Utility to call system smart contracts. /// Utility to call system smart contracts.
@ -127,14 +125,7 @@ where
receipt_builder: Arc<dyn OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>>, receipt_builder: Arc<dyn OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>>,
) -> Self { ) -> Self {
let system_caller = SystemCaller::new(evm_config.clone(), chain_spec.clone()); let system_caller = SystemCaller::new(evm_config.clone(), chain_spec.clone());
Self { Self { state, chain_spec, evm_config, system_caller, receipt_builder }
state,
chain_spec,
evm_config,
system_caller,
tx_env_overrides: None,
receipt_builder,
}
} }
} }
@ -152,10 +143,6 @@ where
type Primitives = N; type Primitives = N;
type Error = BlockExecutionError; type Error = BlockExecutionError;
fn init(&mut self, tx_env_overrides: Box<dyn TxEnvOverrides>) {
self.tx_env_overrides = Some(tx_env_overrides);
}
fn apply_pre_execution_changes( fn apply_pre_execution_changes(
&mut self, &mut self,
block: &RecoveredBlock<N::Block>, block: &RecoveredBlock<N::Block>,
@ -223,11 +210,7 @@ where
.transpose() .transpose()
.map_err(|_| OpBlockExecutionError::AccountLoadFailed(*sender))?; .map_err(|_| OpBlockExecutionError::AccountLoadFailed(*sender))?;
let mut tx_env = self.evm_config.tx_env(transaction, *sender); let tx_env = self.evm_config.tx_env(transaction, *sender);
if let Some(tx_env_overrides) = &mut self.tx_env_overrides {
tx_env_overrides.apply(&mut tx_env);
}
// Execute transaction. // Execute transaction.
let result_and_state = evm.transact(tx_env).map_err(move |err| { let result_and_state = evm.transact(tx_env).map_err(move |err| {

View File

@ -157,6 +157,7 @@ impl ConfigureEvmEnv for OpEvmConfig {
type Header = Header; type Header = Header;
type Transaction = OpTransactionSigned; type Transaction = OpTransactionSigned;
type Error = EIP1559ParamError; type Error = EIP1559ParamError;
type TxEnv = TxEnv;
fn fill_tx_env(&self, tx_env: &mut TxEnv, transaction: &OpTransactionSigned, sender: Address) { fn fill_tx_env(&self, tx_env: &mut TxEnv, transaction: &OpTransactionSigned, sender: Address) {
transaction.fill_tx_env(tx_env, sender); transaction.fill_tx_env(tx_env, sender);

View File

@ -9,7 +9,7 @@ use crate::{
use op_alloy_consensus::OpPooledTransaction; use op_alloy_consensus::OpPooledTransaction;
use reth_basic_payload_builder::{BasicPayloadJobGenerator, BasicPayloadJobGeneratorConfig}; use reth_basic_payload_builder::{BasicPayloadJobGenerator, BasicPayloadJobGeneratorConfig};
use reth_chainspec::{EthChainSpec, Hardforks}; use reth_chainspec::{EthChainSpec, Hardforks};
use reth_evm::{execute::BasicBlockExecutorProvider, ConfigureEvmFor}; use reth_evm::{execute::BasicBlockExecutorProvider, ConfigureEvmEnv, ConfigureEvmFor};
use reth_network::{NetworkConfig, NetworkHandle, NetworkManager, NetworkPrimitives, PeersInfo}; use reth_network::{NetworkConfig, NetworkHandle, NetworkManager, NetworkPrimitives, PeersInfo};
use reth_node_api::{AddOnsContext, FullNodeComponents, NodeAddOns, PrimitivesTy, TxTy}; use reth_node_api::{AddOnsContext, FullNodeComponents, NodeAddOns, PrimitivesTy, TxTy};
use reth_node_builder::{ use reth_node_builder::{
@ -43,6 +43,7 @@ use reth_transaction_pool::{
TransactionValidationTaskExecutor, TransactionValidationTaskExecutor,
}; };
use reth_trie_db::MerklePatriciaTrie; use reth_trie_db::MerklePatriciaTrie;
use revm::primitives::TxEnv;
use std::sync::Arc; use std::sync::Arc;
/// Storage implementation for Optimism. /// Storage implementation for Optimism.
@ -190,6 +191,7 @@ where
Storage = OpStorage, Storage = OpStorage,
Engine = OpEngineTypes, Engine = OpEngineTypes,
>, >,
Evm: ConfigureEvmEnv<TxEnv = TxEnv>,
>, >,
{ {
type Handle = RpcHandle<N, OpEthApi<N>>; type Handle = RpcHandle<N, OpEthApi<N>>;
@ -239,6 +241,7 @@ where
Storage = OpStorage, Storage = OpStorage,
Engine = OpEngineTypes, Engine = OpEngineTypes,
>, >,
Evm: ConfigureEvmEnv<TxEnv = TxEnv>,
>, >,
{ {
type EthApi = OpEthApi<N>; type EthApi = OpEthApi<N>;

View File

@ -28,7 +28,8 @@ where
impl<N> Call for OpEthApi<N> impl<N> Call for OpEthApi<N>
where where
Self: LoadState<Evm: ConfigureEvm<Header = ProviderHeader<Self::Provider>>> + SpawnBlocking, Self: LoadState<Evm: ConfigureEvm<Header = ProviderHeader<Self::Provider>, TxEnv = TxEnv>>
+ SpawnBlocking,
Self::Error: From<OpEthApiError>, Self::Error: From<OpEthApiError>,
N: OpNodeCore, N: OpNodeCore,
{ {

View File

@ -19,6 +19,7 @@
//! use reth_consensus::{ConsensusError, FullConsensus}; //! use reth_consensus::{ConsensusError, FullConsensus};
//! use reth_engine_primitives::PayloadValidator; //! use reth_engine_primitives::PayloadValidator;
//! use reth_evm::{execute::BlockExecutorProvider, ConfigureEvm}; //! use reth_evm::{execute::BlockExecutorProvider, ConfigureEvm};
//! use reth_evm_ethereum::EthEvmConfig;
//! use reth_network_api::{NetworkInfo, Peers}; //! use reth_network_api::{NetworkInfo, Peers};
//! use reth_primitives::{Header, PooledTransaction, TransactionSigned}; //! use reth_primitives::{Header, PooledTransaction, TransactionSigned};
//! use reth_provider::{AccountReader, CanonStateSubscriptions, ChangeSetReader, FullRpcProvider}; //! use reth_provider::{AccountReader, CanonStateSubscriptions, ChangeSetReader, FullRpcProvider};
@ -30,21 +31,12 @@
//! use reth_transaction_pool::{PoolTransaction, TransactionPool}; //! use reth_transaction_pool::{PoolTransaction, TransactionPool};
//! use std::sync::Arc; //! use std::sync::Arc;
//! //!
//! pub async fn launch< //! pub async fn launch<Provider, Pool, Network, Events, BlockExecutor, Consensus, Validator>(
//! Provider,
//! Pool,
//! Network,
//! Events,
//! EvmConfig,
//! BlockExecutor,
//! Consensus,
//! Validator,
//! >(
//! provider: Provider, //! provider: Provider,
//! pool: Pool, //! pool: Pool,
//! network: Network, //! network: Network,
//! events: Events, //! events: Events,
//! evm_config: EvmConfig, //! evm_config: EthEvmConfig,
//! block_executor: BlockExecutor, //! block_executor: BlockExecutor,
//! consensus: Consensus, //! consensus: Consensus,
//! validator: Validator, //! validator: Validator,
@ -66,7 +58,6 @@
//! Network: NetworkInfo + Peers + Clone + 'static, //! Network: NetworkInfo + Peers + Clone + 'static,
//! Events: //! Events:
//! CanonStateSubscriptions<Primitives = reth_primitives::EthPrimitives> + Clone + 'static, //! CanonStateSubscriptions<Primitives = reth_primitives::EthPrimitives> + Clone + 'static,
//! EvmConfig: ConfigureEvm<Header = Header, Transaction = TransactionSigned>,
//! BlockExecutor: BlockExecutorProvider<Primitives = Events::Primitives>, //! BlockExecutor: BlockExecutorProvider<Primitives = Events::Primitives>,
//! Consensus: FullConsensus<Error = ConsensusError> + Clone + 'static, //! Consensus: FullConsensus<Error = ConsensusError> + Clone + 'static,
//! Validator: PayloadValidator<Block = reth_primitives::Block>, //! Validator: PayloadValidator<Block = reth_primitives::Block>,
@ -103,6 +94,7 @@
//! use reth_consensus::{ConsensusError, FullConsensus}; //! use reth_consensus::{ConsensusError, FullConsensus};
//! use reth_engine_primitives::{EngineTypes, PayloadValidator}; //! use reth_engine_primitives::{EngineTypes, PayloadValidator};
//! use reth_evm::{execute::BlockExecutorProvider, ConfigureEvm}; //! use reth_evm::{execute::BlockExecutorProvider, ConfigureEvm};
//! use reth_evm_ethereum::EthEvmConfig;
//! use reth_network_api::{NetworkInfo, Peers}; //! use reth_network_api::{NetworkInfo, Peers};
//! use reth_primitives::{Header, PooledTransaction, TransactionSigned}; //! use reth_primitives::{Header, PooledTransaction, TransactionSigned};
//! use reth_provider::{AccountReader, CanonStateSubscriptions, ChangeSetReader, FullRpcProvider}; //! use reth_provider::{AccountReader, CanonStateSubscriptions, ChangeSetReader, FullRpcProvider};
@ -125,7 +117,6 @@
//! Events, //! Events,
//! EngineApi, //! EngineApi,
//! EngineT, //! EngineT,
//! EvmConfig,
//! BlockExecutor, //! BlockExecutor,
//! Consensus, //! Consensus,
//! Validator, //! Validator,
@ -135,7 +126,7 @@
//! network: Network, //! network: Network,
//! events: Events, //! events: Events,
//! engine_api: EngineApi, //! engine_api: EngineApi,
//! evm_config: EvmConfig, //! evm_config: EthEvmConfig,
//! block_executor: BlockExecutor, //! block_executor: BlockExecutor,
//! consensus: Consensus, //! consensus: Consensus,
//! validator: Validator, //! validator: Validator,
@ -159,7 +150,6 @@
//! CanonStateSubscriptions<Primitives = reth_primitives::EthPrimitives> + Clone + 'static, //! CanonStateSubscriptions<Primitives = reth_primitives::EthPrimitives> + Clone + 'static,
//! EngineApi: EngineApiServer<EngineT>, //! EngineApi: EngineApiServer<EngineT>,
//! EngineT: EngineTypes, //! EngineT: EngineTypes,
//! EvmConfig: ConfigureEvm<Header = Header, Transaction = TransactionSigned>,
//! BlockExecutor: BlockExecutorProvider<Primitives = Events::Primitives>, //! BlockExecutor: BlockExecutorProvider<Primitives = Events::Primitives>,
//! Consensus: FullConsensus<Error = ConsensusError> + Clone + 'static, //! Consensus: FullConsensus<Error = ConsensusError> + Clone + 'static,
//! Validator: PayloadValidator<Block = reth_primitives::Block>, //! Validator: PayloadValidator<Block = reth_primitives::Block>,

View File

@ -3,8 +3,8 @@
use super::{LoadBlock, LoadPendingBlock, LoadState, LoadTransaction, SpawnBlocking, Trace}; use super::{LoadBlock, LoadPendingBlock, LoadState, LoadTransaction, SpawnBlocking, Trace};
use crate::{ use crate::{
helpers::estimate::EstimateCall, FromEthApiError, FromEvmError, FullEthApiTypes, helpers::estimate::EstimateCall, FromEthApiError, FromEvmError, FullEthApiTypes, RpcBlock,
IntoEthApiError, RpcBlock, RpcNodeCore, RpcNodeCore,
}; };
use alloy_consensus::BlockHeader; use alloy_consensus::BlockHeader;
use alloy_eips::{eip1559::calc_next_block_base_fee, eip2930::AccessListResult}; use alloy_eips::{eip1559::calc_next_block_base_fee, eip2930::AccessListResult};
@ -17,14 +17,14 @@ use alloy_rpc_types_eth::{
}; };
use futures::Future; use futures::Future;
use reth_chainspec::EthChainSpec; use reth_chainspec::EthChainSpec;
use reth_evm::{env::EvmEnv, ConfigureEvm, ConfigureEvmEnv, Evm}; use reth_evm::{env::EvmEnv, ConfigureEvm, ConfigureEvmEnv, Evm, TransactionEnv};
use reth_node_api::BlockBody; use reth_node_api::BlockBody;
use reth_primitives_traits::SignedTransaction; use reth_primitives_traits::SignedTransaction;
use reth_provider::{BlockIdReader, ChainSpecProvider, ProviderHeader}; use reth_provider::{BlockIdReader, ChainSpecProvider, ProviderHeader};
use reth_revm::{ use reth_revm::{
database::StateProviderDatabase, database::StateProviderDatabase,
db::CacheDB, db::CacheDB,
primitives::{BlockEnv, ExecutionResult, ResultAndState, TxEnv}, primitives::{BlockEnv, ExecutionResult, ResultAndState},
DatabaseRef, DatabaseRef,
}; };
use reth_rpc_eth_types::{ use reth_rpc_eth_types::{
@ -32,7 +32,6 @@ use reth_rpc_eth_types::{
error::ensure_success, error::ensure_success,
revm_utils::{ revm_utils::{
apply_block_overrides, apply_state_overrides, caller_gas_allowance, get_precompiles, apply_block_overrides, apply_state_overrides, caller_gas_allowance, get_precompiles,
CallFees,
}, },
simulate::{self, EthSimulateError}, simulate::{self, EthSimulateError},
EthApiError, RevertError, RpcInvalidTransactionError, StateCacheDb, EthApiError, RevertError, RpcInvalidTransactionError, StateCacheDb,
@ -206,7 +205,7 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthA
} }
transactions.push(tx); transactions.push(tx);
senders.push(tx_env.caller); senders.push(tx_env.caller());
results.push(res.result); results.push(res.result);
} }
@ -410,10 +409,10 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthA
let mut db = CacheDB::new(StateProviderDatabase::new(state)); let mut db = CacheDB::new(StateProviderDatabase::new(state));
if request.gas.is_none() && tx_env.gas_price > U256::ZERO { if request.gas.is_none() && tx_env.gas_price() > U256::ZERO {
let cap = caller_gas_allowance(&mut db, &tx_env)?; let cap = caller_gas_allowance(&mut db, &tx_env)?;
// no gas limit was provided in the request, so we need to cap the request's gas limit // no gas limit was provided in the request, so we need to cap the request's gas limit
tx_env.gas_limit = cap.min(evm_env.block_env.gas_limit).saturating_to(); tx_env.set_gas_limit(cap.min(evm_env.block_env.gas_limit).saturating_to());
} }
let from = request.from.unwrap_or_default(); let from = request.from.unwrap_or_default();
@ -434,11 +433,11 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthA
let (result, (evm_env, mut tx_env)) = let (result, (evm_env, mut tx_env)) =
self.inspect(&mut db, evm_env, tx_env, &mut inspector)?; self.inspect(&mut db, evm_env, tx_env, &mut inspector)?;
let access_list = inspector.into_access_list(); let access_list = inspector.into_access_list();
tx_env.access_list = access_list.to_vec(); tx_env.set_access_list(access_list.clone());
match result.result { match result.result {
ExecutionResult::Halt { reason, gas_used } => { ExecutionResult::Halt { reason, gas_used } => {
let error = let error =
Some(RpcInvalidTransactionError::halt(reason, tx_env.gas_limit).to_string()); Some(RpcInvalidTransactionError::halt(reason, tx_env.gas_limit()).to_string());
return Ok(AccessListResult { access_list, gas_used: U256::from(gas_used), error }) return Ok(AccessListResult { access_list, gas_used: U256::from(gas_used), error })
} }
ExecutionResult::Revert { output, gas_used } => { ExecutionResult::Revert { output, gas_used } => {
@ -453,7 +452,7 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthA
let res = match result.result { let res = match result.result {
ExecutionResult::Halt { reason, gas_used } => { ExecutionResult::Halt { reason, gas_used } => {
let error = let error =
Some(RpcInvalidTransactionError::halt(reason, tx_env.gas_limit).to_string()); Some(RpcInvalidTransactionError::halt(reason, tx_env.gas_limit()).to_string());
AccessListResult { access_list, gas_used: U256::from(gas_used), error } AccessListResult { access_list, gas_used: U256::from(gas_used), error }
} }
ExecutionResult::Revert { output, gas_used } => { ExecutionResult::Revert { output, gas_used } => {
@ -490,14 +489,15 @@ pub trait Call:
f(StateProviderTraitObjWrapper(&state)) f(StateProviderTraitObjWrapper(&state))
} }
/// Executes the [`TxEnv`] against the given [Database] without committing state /// Executes the `TxEnv` against the given [Database] without committing state
/// changes. /// changes.
#[expect(clippy::type_complexity)]
fn transact<DB>( fn transact<DB>(
&self, &self,
db: DB, db: DB,
evm_env: EvmEnv, evm_env: EvmEnv,
tx_env: TxEnv, tx_env: <Self::Evm as ConfigureEvmEnv>::TxEnv,
) -> Result<(ResultAndState, (EvmEnv, TxEnv)), Self::Error> ) -> Result<(ResultAndState, (EvmEnv, <Self::Evm as ConfigureEvmEnv>::TxEnv)), Self::Error>
where where
DB: Database, DB: Database,
EthApiError: From<DB::Error>, EthApiError: From<DB::Error>,
@ -511,13 +511,14 @@ pub trait Call:
/// Executes the [`EvmEnv`] against the given [Database] without committing state /// Executes the [`EvmEnv`] against the given [Database] without committing state
/// changes. /// changes.
#[expect(clippy::type_complexity)]
fn transact_with_inspector<DB>( fn transact_with_inspector<DB>(
&self, &self,
db: DB, db: DB,
evm_env: EvmEnv, evm_env: EvmEnv,
tx_env: TxEnv, tx_env: <Self::Evm as ConfigureEvmEnv>::TxEnv,
inspector: impl GetInspector<DB>, inspector: impl GetInspector<DB>,
) -> Result<(ResultAndState, (EvmEnv, TxEnv)), Self::Error> ) -> Result<(ResultAndState, (EvmEnv, <Self::Evm as ConfigureEvmEnv>::TxEnv)), Self::Error>
where where
DB: Database, DB: Database,
EthApiError: From<DB::Error>, EthApiError: From<DB::Error>,
@ -530,12 +531,18 @@ pub trait Call:
} }
/// Executes the call request at the given [`BlockId`]. /// Executes the call request at the given [`BlockId`].
#[expect(clippy::type_complexity)]
fn transact_call_at( fn transact_call_at(
&self, &self,
request: TransactionRequest, request: TransactionRequest,
at: BlockId, at: BlockId,
overrides: EvmOverrides, overrides: EvmOverrides,
) -> impl Future<Output = Result<(ResultAndState, (EvmEnv, TxEnv)), Self::Error>> + Send ) -> impl Future<
Output = Result<
(ResultAndState, (EvmEnv, <Self::Evm as ConfigureEvmEnv>::TxEnv)),
Self::Error,
>,
> + Send
where where
Self: LoadPendingBlock, Self: LoadPendingBlock,
{ {
@ -585,7 +592,11 @@ pub trait Call:
) -> impl Future<Output = Result<R, Self::Error>> + Send ) -> impl Future<Output = Result<R, Self::Error>> + Send
where where
Self: LoadPendingBlock, Self: LoadPendingBlock,
F: FnOnce(StateCacheDbRefMutWrapper<'_, '_>, EvmEnv, TxEnv) -> Result<R, Self::Error> F: FnOnce(
StateCacheDbRefMutWrapper<'_, '_>,
EvmEnv,
<Self::Evm as ConfigureEvmEnv>::TxEnv,
) -> Result<R, Self::Error>
+ Send + Send
+ 'static, + 'static,
R: Send + 'static, R: Send + 'static,
@ -694,84 +705,15 @@ pub trait Call:
Ok(index) Ok(index)
} }
/// Configures a new [`TxEnv`] for the [`TransactionRequest`] /// Configures a new `TxEnv` for the [`TransactionRequest`]
/// ///
/// All [`TxEnv`] fields are derived from the given [`TransactionRequest`], if fields are /// All `TxEnv` fields are derived from the given [`TransactionRequest`], if fields are
/// `None`, they fall back to the [`BlockEnv`]'s settings. /// `None`, they fall back to the [`BlockEnv`]'s settings.
fn create_txn_env( fn create_txn_env(
&self, &self,
block_env: &BlockEnv, block_env: &BlockEnv,
request: TransactionRequest, request: TransactionRequest,
) -> Result<TxEnv, Self::Error> { ) -> Result<<Self::Evm as ConfigureEvmEnv>::TxEnv, Self::Error>;
// Ensure that if versioned hashes are set, they're not empty
if request.blob_versioned_hashes.as_ref().is_some_and(|hashes| hashes.is_empty()) {
return Err(RpcInvalidTransactionError::BlobTransactionMissingBlobHashes.into_eth_err())
}
let TransactionRequest {
from,
to,
gas_price,
max_fee_per_gas,
max_priority_fee_per_gas,
gas,
value,
input,
nonce,
access_list,
chain_id,
blob_versioned_hashes,
max_fee_per_blob_gas,
authorization_list,
transaction_type: _,
sidecar: _,
} = request;
let CallFees { max_priority_fee_per_gas, gas_price, max_fee_per_blob_gas } =
CallFees::ensure_fees(
gas_price.map(U256::from),
max_fee_per_gas.map(U256::from),
max_priority_fee_per_gas.map(U256::from),
block_env.basefee,
blob_versioned_hashes.as_deref(),
max_fee_per_blob_gas.map(U256::from),
block_env.get_blob_gasprice().map(U256::from),
)?;
let gas_limit = gas.unwrap_or_else(|| {
// Use maximum allowed gas limit. The reason for this
// is that both Erigon and Geth use pre-configured gas cap even if
// it's possible to derive the gas limit from the block:
// <https://github.com/ledgerwatch/erigon/blob/eae2d9a79cb70dbe30b3a6b79c436872e4605458/cmd/rpcdaemon/commands/trace_adhoc.go#L956
// https://github.com/ledgerwatch/erigon/blob/eae2d9a79cb70dbe30b3a6b79c436872e4605458/eth/ethconfig/config.go#L94>
block_env.gas_limit.saturating_to()
});
#[allow(clippy::needless_update)]
let env = TxEnv {
gas_limit,
nonce,
caller: from.unwrap_or_default(),
gas_price,
gas_priority_fee: max_priority_fee_per_gas,
transact_to: to.unwrap_or(TxKind::Create),
value: value.unwrap_or_default(),
data: input
.try_into_unique_input()
.map_err(Self::Error::from_eth_err)?
.unwrap_or_default(),
chain_id,
access_list: access_list.unwrap_or_default().into(),
// EIP-4844 fields
blob_hashes: blob_versioned_hashes.unwrap_or_default(),
max_fee_per_blob_gas,
// EIP-7702 fields
authorization_list: authorization_list.map(Into::into),
..Default::default()
};
Ok(env)
}
/// Prepares the [`EvmEnv`] for execution of calls. /// Prepares the [`EvmEnv`] for execution of calls.
/// ///
@ -792,7 +734,7 @@ pub trait Call:
mut request: TransactionRequest, mut request: TransactionRequest,
db: &mut CacheDB<DB>, db: &mut CacheDB<DB>,
overrides: EvmOverrides, overrides: EvmOverrides,
) -> Result<(EvmEnv, TxEnv), Self::Error> ) -> Result<(EvmEnv, <Self::Evm as ConfigureEvmEnv>::TxEnv), Self::Error>
where where
DB: DatabaseRef, DB: DatabaseRef,
EthApiError: From<<DB as DatabaseRef>::Error>, EthApiError: From<<DB as DatabaseRef>::Error>,
@ -831,12 +773,12 @@ pub trait Call:
if request_gas.is_none() { if request_gas.is_none() {
// No gas limit was provided in the request, so we need to cap the transaction gas limit // No gas limit was provided in the request, so we need to cap the transaction gas limit
if tx_env.gas_price > U256::ZERO { if tx_env.gas_price() > U256::ZERO {
// If gas price is specified, cap transaction gas limit with caller allowance // If gas price is specified, cap transaction gas limit with caller allowance
trace!(target: "rpc::eth::call", ?tx_env, "Applying gas limit cap with caller allowance"); trace!(target: "rpc::eth::call", ?tx_env, "Applying gas limit cap with caller allowance");
let cap = caller_gas_allowance(db, &tx_env)?; let cap = caller_gas_allowance(db, &tx_env)?;
// ensure we cap gas_limit to the block's // ensure we cap gas_limit to the block's
tx_env.gas_limit = cap.min(evm_env.block_env.gas_limit).saturating_to(); tx_env.set_gas_limit(cap.min(evm_env.block_env.gas_limit).saturating_to());
} }
} }

View File

@ -6,19 +6,19 @@ use alloy_primitives::U256;
use alloy_rpc_types_eth::{state::StateOverride, transaction::TransactionRequest, BlockId}; use alloy_rpc_types_eth::{state::StateOverride, transaction::TransactionRequest, BlockId};
use futures::Future; use futures::Future;
use reth_chainspec::MIN_TRANSACTION_GAS; use reth_chainspec::MIN_TRANSACTION_GAS;
use reth_evm::env::EvmEnv; use reth_evm::{env::EvmEnv, ConfigureEvmEnv, TransactionEnv};
use reth_provider::StateProvider; use reth_provider::StateProvider;
use reth_revm::{ use reth_revm::{
database::StateProviderDatabase, database::StateProviderDatabase,
db::CacheDB, db::CacheDB,
primitives::{ExecutionResult, HaltReason, TransactTo}, primitives::{ExecutionResult, HaltReason},
}; };
use reth_rpc_eth_types::{ use reth_rpc_eth_types::{
revm_utils::{apply_state_overrides, caller_gas_allowance}, revm_utils::{apply_state_overrides, caller_gas_allowance},
EthApiError, RevertError, RpcInvalidTransactionError, EthApiError, RevertError, RpcInvalidTransactionError,
}; };
use reth_rpc_server_types::constants::gas_oracle::{CALL_STIPEND_GAS, ESTIMATE_GAS_ERROR_RATIO}; use reth_rpc_server_types::constants::gas_oracle::{CALL_STIPEND_GAS, ESTIMATE_GAS_ERROR_RATIO};
use revm_primitives::{db::Database, TxEnv}; use revm_primitives::{db::Database, TxKind};
use tracing::trace; use tracing::trace;
/// Gas execution estimates /// Gas execution estimates
@ -84,8 +84,8 @@ pub trait EstimateCall: Call {
} }
// Optimize for simple transfer transactions, potentially reducing the gas estimate. // Optimize for simple transfer transactions, potentially reducing the gas estimate.
if tx_env.data.is_empty() { if tx_env.input().is_empty() {
if let TransactTo::Call(to) = tx_env.transact_to { if let TxKind::Call(to) = tx_env.kind() {
if let Ok(code) = db.db.account_code(&to) { if let Ok(code) = db.db.account_code(&to) {
let no_code_callee = code.map(|code| code.is_empty()).unwrap_or(true); let no_code_callee = code.map(|code| code.is_empty()).unwrap_or(true);
if no_code_callee { if no_code_callee {
@ -95,7 +95,7 @@ pub trait EstimateCall: Call {
// field combos that bump the price up, so we try executing the function // field combos that bump the price up, so we try executing the function
// with the minimum gas limit to make sure. // with the minimum gas limit to make sure.
let mut tx_env = tx_env.clone(); let mut tx_env = tx_env.clone();
tx_env.gas_limit = MIN_TRANSACTION_GAS; tx_env.set_gas_limit(MIN_TRANSACTION_GAS);
if let Ok((res, _)) = self.transact(&mut db, evm_env.clone(), tx_env) { if let Ok((res, _)) = self.transact(&mut db, evm_env.clone(), tx_env) {
if res.result.is_success() { if res.result.is_success() {
return Ok(U256::from(MIN_TRANSACTION_GAS)) return Ok(U256::from(MIN_TRANSACTION_GAS))
@ -109,7 +109,7 @@ pub trait EstimateCall: Call {
// Check funds of the sender (only useful to check if transaction gas price is more than 0). // Check funds of the sender (only useful to check if transaction gas price is more than 0).
// //
// The caller allowance is check by doing `(account.balance - tx.value) / tx.gas_price` // The caller allowance is check by doing `(account.balance - tx.value) / tx.gas_price`
if tx_env.gas_price > U256::ZERO { if tx_env.gas_price() > U256::ZERO {
// cap the highest gas limit by max gas caller can afford with given gas price // cap the highest gas limit by max gas caller can afford with given gas price
highest_gas_limit = highest_gas_limit highest_gas_limit = highest_gas_limit
.min(caller_gas_allowance(&mut db, &tx_env).map_err(Self::Error::from_eth_err)?); .min(caller_gas_allowance(&mut db, &tx_env).map_err(Self::Error::from_eth_err)?);
@ -119,7 +119,7 @@ pub trait EstimateCall: Call {
let mut highest_gas_limit = highest_gas_limit.saturating_to::<u64>(); let mut highest_gas_limit = highest_gas_limit.saturating_to::<u64>();
// If the provided gas limit is less than computed cap, use that // If the provided gas limit is less than computed cap, use that
tx_env.gas_limit = tx_env.gas_limit.min(highest_gas_limit); tx_env.set_gas_limit(tx_env.gas_limit().min(highest_gas_limit));
trace!(target: "rpc::eth::estimate", ?evm_env, ?tx_env, "Starting gas estimation"); trace!(target: "rpc::eth::estimate", ?evm_env, ?tx_env, "Starting gas estimation");
@ -169,7 +169,7 @@ pub trait EstimateCall: Call {
// we know the tx succeeded with the configured gas limit, so we can use that as the // we know the tx succeeded with the configured gas limit, so we can use that as the
// highest, in case we applied a gas cap due to caller allowance above // highest, in case we applied a gas cap due to caller allowance above
highest_gas_limit = tx_env.gas_limit; highest_gas_limit = tx_env.gas_limit();
// NOTE: this is the gas the transaction used, which is less than the // NOTE: this is the gas the transaction used, which is less than the
// transaction requires to succeed. // transaction requires to succeed.
@ -186,7 +186,7 @@ pub trait EstimateCall: Call {
let optimistic_gas_limit = (gas_used + gas_refund + CALL_STIPEND_GAS) * 64 / 63; let optimistic_gas_limit = (gas_used + gas_refund + CALL_STIPEND_GAS) * 64 / 63;
if optimistic_gas_limit < highest_gas_limit { if optimistic_gas_limit < highest_gas_limit {
// Set the transaction's gas limit to the calculated optimistic gas limit. // Set the transaction's gas limit to the calculated optimistic gas limit.
tx_env.gas_limit = optimistic_gas_limit; tx_env.set_gas_limit(optimistic_gas_limit);
// Re-execute the transaction with the new gas limit and update the result and // Re-execute the transaction with the new gas limit and update the result and
// environment. // environment.
(res, (evm_env, tx_env)) = self.transact(&mut db, evm_env, tx_env)?; (res, (evm_env, tx_env)) = self.transact(&mut db, evm_env, tx_env)?;
@ -221,7 +221,7 @@ pub trait EstimateCall: Call {
break break
}; };
tx_env.gas_limit = mid_gas_limit; tx_env.set_gas_limit(mid_gas_limit);
// Execute transaction and handle potential gas errors, adjusting limits accordingly. // Execute transaction and handle potential gas errors, adjusting limits accordingly.
match self.transact(&mut db, evm_env.clone(), tx_env.clone()) { match self.transact(&mut db, evm_env.clone(), tx_env.clone()) {
@ -282,15 +282,15 @@ pub trait EstimateCall: Call {
&self, &self,
env_gas_limit: U256, env_gas_limit: U256,
evm_env: EvmEnv, evm_env: EvmEnv,
mut tx_env: TxEnv, mut tx_env: <Self::Evm as ConfigureEvmEnv>::TxEnv,
db: &mut DB, db: &mut DB,
) -> Self::Error ) -> Self::Error
where where
DB: Database, DB: Database,
EthApiError: From<DB::Error>, EthApiError: From<DB::Error>,
{ {
let req_gas_limit = tx_env.gas_limit; let req_gas_limit = tx_env.gas_limit();
tx_env.gas_limit = env_gas_limit.try_into().unwrap_or(u64::MAX); tx_env.set_gas_limit(env_gas_limit.try_into().unwrap_or(u64::MAX));
let (res, _) = match self.transact(db, evm_env, tx_env) { let (res, _) = match self.transact(db, evm_env, tx_env) {
Ok(res) => res, Ok(res) => res,
Err(err) => return err, Err(err) => return err,

View File

@ -18,7 +18,7 @@ use reth_rpc_eth_types::{
}; };
use revm::{db::CacheDB, Database, DatabaseCommit, GetInspector, Inspector}; use revm::{db::CacheDB, Database, DatabaseCommit, GetInspector, Inspector};
use revm_inspectors::tracing::{TracingInspector, TracingInspectorConfig}; use revm_inspectors::tracing::{TracingInspector, TracingInspectorConfig};
use revm_primitives::{EvmState, ExecutionResult, ResultAndState, TxEnv}; use revm_primitives::{EvmState, ExecutionResult, ResultAndState};
use std::{fmt::Display, sync::Arc}; use std::{fmt::Display, sync::Arc};
/// Executes CPU heavy tasks. /// Executes CPU heavy tasks.
@ -33,13 +33,14 @@ pub trait Trace:
{ {
/// Executes the [`EvmEnv`] against the given [Database] without committing state /// Executes the [`EvmEnv`] against the given [Database] without committing state
/// changes. /// changes.
#[expect(clippy::type_complexity)]
fn inspect<DB, I>( fn inspect<DB, I>(
&self, &self,
db: DB, db: DB,
evm_env: EvmEnv, evm_env: EvmEnv,
tx_env: TxEnv, tx_env: <Self::Evm as ConfigureEvmEnv>::TxEnv,
inspector: I, inspector: I,
) -> Result<(ResultAndState, (EvmEnv, TxEnv)), Self::Error> ) -> Result<(ResultAndState, (EvmEnv, <Self::Evm as ConfigureEvmEnv>::TxEnv)), Self::Error>
where where
DB: Database, DB: Database,
EthApiError: From<DB::Error>, EthApiError: From<DB::Error>,
@ -61,7 +62,7 @@ pub trait Trace:
fn trace_at<F, R>( fn trace_at<F, R>(
&self, &self,
evm_env: EvmEnv, evm_env: EvmEnv,
tx_env: TxEnv, tx_env: <Self::Evm as ConfigureEvmEnv>::TxEnv,
config: TracingInspectorConfig, config: TracingInspectorConfig,
at: BlockId, at: BlockId,
f: F, f: F,
@ -88,7 +89,7 @@ pub trait Trace:
fn spawn_trace_at_with_state<F, R>( fn spawn_trace_at_with_state<F, R>(
&self, &self,
evm_env: EvmEnv, evm_env: EvmEnv,
tx_env: TxEnv, tx_env: <Self::Evm as ConfigureEvmEnv>::TxEnv,
config: TracingInspectorConfig, config: TracingInspectorConfig,
at: BlockId, at: BlockId,
f: F, f: F,

View File

@ -5,10 +5,11 @@ use alloy_rpc_types_eth::{
state::{AccountOverride, StateOverride}, state::{AccountOverride, StateOverride},
BlockOverrides, BlockOverrides,
}; };
use reth_evm::TransactionEnv;
use revm::{ use revm::{
db::CacheDB, db::CacheDB,
precompile::{PrecompileSpecId, Precompiles}, precompile::{PrecompileSpecId, Precompiles},
primitives::{db::DatabaseRef, Bytecode, SpecId, TxEnv}, primitives::{db::DatabaseRef, Bytecode, SpecId},
Database, Database,
}; };
use revm_primitives::BlockEnv; use revm_primitives::BlockEnv;
@ -32,26 +33,26 @@ pub fn get_precompiles(spec_id: SpecId) -> impl IntoIterator<Item = Address> {
/// ///
/// Note: this takes the mut [Database] trait because the loaded sender can be reused for the /// Note: this takes the mut [Database] trait because the loaded sender can be reused for the
/// following operation like `eth_call`. /// following operation like `eth_call`.
pub fn caller_gas_allowance<DB>(db: &mut DB, env: &TxEnv) -> EthResult<U256> pub fn caller_gas_allowance<DB>(db: &mut DB, env: &impl TransactionEnv) -> EthResult<U256>
where where
DB: Database, DB: Database,
EthApiError: From<<DB as Database>::Error>, EthApiError: From<<DB as Database>::Error>,
{ {
// Get the caller account. // Get the caller account.
let caller = db.basic(env.caller)?; let caller = db.basic(env.caller())?;
// Get the caller balance. // Get the caller balance.
let balance = caller.map(|acc| acc.balance).unwrap_or_default(); let balance = caller.map(|acc| acc.balance).unwrap_or_default();
// Get transaction value. // Get transaction value.
let value = env.value; let value = env.value();
// Subtract transferred value from the caller balance. Return error if the caller has // Subtract transferred value from the caller balance. Return error if the caller has
// insufficient funds. // insufficient funds.
let balance = balance let balance = balance
.checked_sub(env.value) .checked_sub(env.value())
.ok_or_else(|| RpcInvalidTransactionError::InsufficientFunds { cost: value, balance })?; .ok_or_else(|| RpcInvalidTransactionError::InsufficientFunds { cost: value, balance })?;
Ok(balance Ok(balance
// Calculate the amount of gas the caller can afford with the specified gas price. // Calculate the amount of gas the caller can afford with the specified gas price.
.checked_div(env.gas_price) .checked_div(env.gas_price())
// This will be 0 if gas price is 0. It is fine, because we check it before. // This will be 0 if gas price is 0. It is fine, because we check it before.
.unwrap_or_default()) .unwrap_or_default())
} }

View File

@ -18,7 +18,7 @@ use reth_chainspec::EthereumHardforks;
use reth_evm::{ use reth_evm::{
env::EvmEnv, env::EvmEnv,
execute::{BlockExecutorProvider, Executor}, execute::{BlockExecutorProvider, Executor},
ConfigureEvmEnv, ConfigureEvmEnv, TransactionEnv,
}; };
use reth_primitives::{NodePrimitives, ReceiptWithBloom, RecoveredBlock}; use reth_primitives::{NodePrimitives, ReceiptWithBloom, RecoveredBlock};
use reth_primitives_traits::{Block as _, BlockBody, SignedTransaction}; use reth_primitives_traits::{Block as _, BlockBody, SignedTransaction};
@ -42,7 +42,6 @@ use revm::{
use revm_inspectors::tracing::{ use revm_inspectors::tracing::{
FourByteInspector, MuxInspector, TracingInspector, TracingInspectorConfig, TransactionContext, FourByteInspector, MuxInspector, TracingInspector, TracingInspectorConfig, TransactionContext,
}; };
use revm_primitives::TxEnv;
use std::sync::Arc; use std::sync::Arc;
use tokio::sync::{AcquireError, OwnedSemaphorePermit}; use tokio::sync::{AcquireError, OwnedSemaphorePermit};
@ -311,7 +310,7 @@ where
let (res, (_, tx_env)) = let (res, (_, tx_env)) =
this.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?; this.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?;
let frame = inspector let frame = inspector
.with_transaction_gas_limit(tx_env.gas_limit) .with_transaction_gas_limit(tx_env.gas_limit())
.into_geth_builder() .into_geth_builder()
.geth_call_traces(call_config, res.result.gas_used()); .geth_call_traces(call_config, res.result.gas_used());
Ok(frame.into()) Ok(frame.into())
@ -341,7 +340,7 @@ where
&mut inspector, &mut inspector,
)?; )?;
let frame = inspector let frame = inspector
.with_transaction_gas_limit(tx_env.gas_limit) .with_transaction_gas_limit(tx_env.gas_limit())
.into_geth_builder() .into_geth_builder()
.geth_prestate_traces(&res, &prestate_config, db) .geth_prestate_traces(&res, &prestate_config, db)
.map_err(Eth::Error::from_eth_err)?; .map_err(Eth::Error::from_eth_err)?;
@ -410,7 +409,7 @@ where
this.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?; this.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?;
let tx_info = TransactionInfo::default(); let tx_info = TransactionInfo::default();
let frame: FlatCallFrame = inspector let frame: FlatCallFrame = inspector
.with_transaction_gas_limit(tx_env.gas_limit) .with_transaction_gas_limit(tx_env.gas_limit())
.into_parity_builder() .into_parity_builder()
.into_localized_transaction_traces(tx_info); .into_localized_transaction_traces(tx_info);
Ok(frame) Ok(frame)
@ -449,7 +448,7 @@ where
let env = revm_primitives::Env::boxed( let env = revm_primitives::Env::boxed(
evm_env.cfg_env_with_handler_cfg.cfg_env, evm_env.cfg_env_with_handler_cfg.cfg_env,
evm_env.block_env, evm_env.block_env,
tx_env, tx_env.into(),
); );
inspector.json_result(res, &env, db).map_err(Eth::Error::from_eth_err) inspector.json_result(res, &env, db).map_err(Eth::Error::from_eth_err)
}) })
@ -470,7 +469,7 @@ where
.spawn_with_call_at(call, at, overrides, move |db, evm_env, tx_env| { .spawn_with_call_at(call, at, overrides, move |db, evm_env, tx_env| {
let (res, (_, tx_env)) = let (res, (_, tx_env)) =
this.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?; this.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?;
Ok((res, tx_env.gas_limit, inspector)) Ok((res, tx_env.gas_limit(), inspector))
}) })
.await?; .await?;
let gas_used = res.result.gas_used(); let gas_used = res.result.gas_used();
@ -652,7 +651,7 @@ where
&self, &self,
opts: &GethDebugTracingOptions, opts: &GethDebugTracingOptions,
evm_env: EvmEnv, evm_env: EvmEnv,
tx_env: TxEnv, tx_env: <Eth::Evm as ConfigureEvmEnv>::TxEnv,
db: &mut StateCacheDb<'_>, db: &mut StateCacheDb<'_>,
transaction_context: Option<TransactionContext>, transaction_context: Option<TransactionContext>,
fused_inspector: &mut Option<TracingInspector>, fused_inspector: &mut Option<TracingInspector>,
@ -694,7 +693,7 @@ where
let (res, (_, tx_env)) = let (res, (_, tx_env)) =
self.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?; self.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?;
inspector.set_transaction_gas_limit(tx_env.gas_limit); inspector.set_transaction_gas_limit(tx_env.gas_limit());
let frame = inspector let frame = inspector
.geth_builder() .geth_builder()
@ -716,7 +715,7 @@ where
let (res, (_, tx_env)) = let (res, (_, tx_env)) =
self.eth_api().inspect(&mut *db, evm_env, tx_env, &mut inspector)?; self.eth_api().inspect(&mut *db, evm_env, tx_env, &mut inspector)?;
inspector.set_transaction_gas_limit(tx_env.gas_limit); inspector.set_transaction_gas_limit(tx_env.gas_limit());
let frame = inspector let frame = inspector
.geth_builder() .geth_builder()
.geth_prestate_traces(&res, &prestate_config, db) .geth_prestate_traces(&res, &prestate_config, db)
@ -756,7 +755,7 @@ where
let (res, (_, tx_env)) = let (res, (_, tx_env)) =
self.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?; self.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?;
let frame: FlatCallFrame = inspector let frame: FlatCallFrame = inspector
.with_transaction_gas_limit(tx_env.gas_limit) .with_transaction_gas_limit(tx_env.gas_limit())
.into_parity_builder() .into_parity_builder()
.into_localized_transaction_traces(tx_info); .into_localized_transaction_traces(tx_info);
@ -784,7 +783,7 @@ where
let env = revm_primitives::Env::boxed( let env = revm_primitives::Env::boxed(
evm_env.cfg_env_with_handler_cfg.cfg_env, evm_env.cfg_env_with_handler_cfg.cfg_env,
evm_env.block_env, evm_env.block_env,
tx_env, tx_env.into(),
); );
let result = let result =
inspector.json_result(res, &env, db).map_err(Eth::Error::from_eth_err)?; inspector.json_result(res, &env, db).map_err(Eth::Error::from_eth_err)?;
@ -801,7 +800,7 @@ where
let (res, (_, tx_env)) = self.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?; let (res, (_, tx_env)) = self.eth_api().inspect(db, evm_env, tx_env, &mut inspector)?;
let gas_used = res.result.gas_used(); let gas_used = res.result.gas_used();
let return_value = res.result.into_output().unwrap_or_default(); let return_value = res.result.into_output().unwrap_or_default();
inspector.set_transaction_gas_limit(tx_env.gas_limit); inspector.set_transaction_gas_limit(tx_env.gas_limit());
let frame = inspector.geth_builder().geth_traces(gas_used, return_value, *config); let frame = inspector.geth_builder().geth_traces(gas_used, return_value, *config);
Ok((frame.into(), res.state)) Ok((frame.into(), res.state))

View File

@ -2,12 +2,15 @@
use crate::EthApi; use crate::EthApi;
use alloy_consensus::Header; use alloy_consensus::Header;
use alloy_rpc_types::TransactionRequest;
use reth_evm::ConfigureEvm; use reth_evm::ConfigureEvm;
use reth_provider::{BlockReader, ProviderHeader}; use reth_provider::{BlockReader, ProviderHeader};
use reth_rpc_eth_api::{ use reth_rpc_eth_api::{
helpers::{estimate::EstimateCall, Call, EthCall, LoadPendingBlock, LoadState, SpawnBlocking}, helpers::{estimate::EstimateCall, Call, EthCall, LoadPendingBlock, LoadState, SpawnBlocking},
FullEthApiTypes, FromEthApiError, FullEthApiTypes, IntoEthApiError,
}; };
use reth_rpc_eth_types::{revm_utils::CallFees, RpcInvalidTransactionError};
use revm_primitives::{BlockEnv, TxEnv, TxKind, U256};
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
@ -18,7 +21,8 @@ where
impl<Provider, Pool, Network, EvmConfig> Call for EthApi<Provider, Pool, Network, EvmConfig> impl<Provider, Pool, Network, EvmConfig> Call for EthApi<Provider, Pool, Network, EvmConfig>
where where
Self: LoadState<Evm: ConfigureEvm<Header = ProviderHeader<Self::Provider>>> + SpawnBlocking, Self: LoadState<Evm: ConfigureEvm<TxEnv = TxEnv, Header = ProviderHeader<Self::Provider>>>
+ SpawnBlocking,
EvmConfig: ConfigureEvm<Header = Header>, EvmConfig: ConfigureEvm<Header = Header>,
Provider: BlockReader, Provider: BlockReader,
{ {
@ -31,6 +35,81 @@ where
fn max_simulate_blocks(&self) -> u64 { fn max_simulate_blocks(&self) -> u64 {
self.inner.max_simulate_blocks() self.inner.max_simulate_blocks()
} }
fn create_txn_env(
&self,
block_env: &BlockEnv,
request: TransactionRequest,
) -> Result<TxEnv, Self::Error> {
// Ensure that if versioned hashes are set, they're not empty
if request.blob_versioned_hashes.as_ref().is_some_and(|hashes| hashes.is_empty()) {
return Err(RpcInvalidTransactionError::BlobTransactionMissingBlobHashes.into_eth_err())
}
let TransactionRequest {
from,
to,
gas_price,
max_fee_per_gas,
max_priority_fee_per_gas,
gas,
value,
input,
nonce,
access_list,
chain_id,
blob_versioned_hashes,
max_fee_per_blob_gas,
authorization_list,
transaction_type: _,
sidecar: _,
} = request;
let CallFees { max_priority_fee_per_gas, gas_price, max_fee_per_blob_gas } =
CallFees::ensure_fees(
gas_price.map(U256::from),
max_fee_per_gas.map(U256::from),
max_priority_fee_per_gas.map(U256::from),
block_env.basefee,
blob_versioned_hashes.as_deref(),
max_fee_per_blob_gas.map(U256::from),
block_env.get_blob_gasprice().map(U256::from),
)?;
let gas_limit = gas.unwrap_or_else(|| {
// Use maximum allowed gas limit. The reason for this
// is that both Erigon and Geth use pre-configured gas cap even if
// it's possible to derive the gas limit from the block:
// <https://github.com/ledgerwatch/erigon/blob/eae2d9a79cb70dbe30b3a6b79c436872e4605458/cmd/rpcdaemon/commands/trace_adhoc.go#L956
// https://github.com/ledgerwatch/erigon/blob/eae2d9a79cb70dbe30b3a6b79c436872e4605458/eth/ethconfig/config.go#L94>
block_env.gas_limit.saturating_to()
});
#[allow(clippy::needless_update)]
let env = TxEnv {
gas_limit,
nonce,
caller: from.unwrap_or_default(),
gas_price,
gas_priority_fee: max_priority_fee_per_gas,
transact_to: to.unwrap_or(TxKind::Create),
value: value.unwrap_or_default(),
data: input
.try_into_unique_input()
.map_err(Self::Error::from_eth_err)?
.unwrap_or_default(),
chain_id,
access_list: access_list.unwrap_or_default().into(),
// EIP-4844 fields
blob_hashes: blob_versioned_hashes.unwrap_or_default(),
max_fee_per_blob_gas,
// EIP-7702 fields
authorization_list: authorization_list.map(Into::into),
..Default::default()
};
Ok(env)
}
} }
impl<Provider, Pool, Network, EvmConfig> EstimateCall for EthApi<Provider, Pool, Network, EvmConfig> impl<Provider, Pool, Network, EvmConfig> EstimateCall for EthApi<Provider, Pool, Network, EvmConfig>

View File

@ -86,8 +86,8 @@ impl MyEvmConfig {
impl ConfigureEvmEnv for MyEvmConfig { impl ConfigureEvmEnv for MyEvmConfig {
type Header = Header; type Header = Header;
type Transaction = TransactionSigned; type Transaction = TransactionSigned;
type Error = Infallible; type Error = Infallible;
type TxEnv = TxEnv;
fn fill_tx_env(&self, tx_env: &mut TxEnv, transaction: &TransactionSigned, sender: Address) { fn fill_tx_env(&self, tx_env: &mut TxEnv, transaction: &TransactionSigned, sender: Address) {
self.inner.fill_tx_env(tx_env, transaction, sender); self.inner.fill_tx_env(tx_env, transaction, sender);

View File

@ -150,6 +150,7 @@ impl ConfigureEvmEnv for MyEvmConfig {
type Header = Header; type Header = Header;
type Transaction = TransactionSigned; type Transaction = TransactionSigned;
type Error = Infallible; type Error = Infallible;
type TxEnv = TxEnv;
fn fill_tx_env(&self, tx_env: &mut TxEnv, transaction: &TransactionSigned, sender: Address) { fn fill_tx_env(&self, tx_env: &mut TxEnv, transaction: &TransactionSigned, sender: Address) {
self.inner.fill_tx_env(tx_env, transaction, sender) self.inner.fill_tx_env(tx_env, transaction, sender)