refactor: BlockExecutionStrategy API (#14480)

This commit is contained in:
Arsenii Kulikov
2025-02-19 17:10:50 +04:00
committed by GitHub
parent 1ae733a7d1
commit 4f454c91d6
30 changed files with 423 additions and 555 deletions

View File

@ -8,7 +8,6 @@ use reth_evm::{
state_change::post_block_balance_increments, system_calls::SystemCaller, ConfigureEvmFor, Evm,
};
use reth_primitives::{NodePrimitives, RecoveredBlock, SealedHeader};
use reth_primitives_traits::{BlockBody, SignedTransaction};
use reth_provider::{BlockExecutionOutput, ChainSpecProvider, StateProviderFactory};
use reth_revm::{
database::StateProviderDatabase,
@ -85,10 +84,8 @@ where
// Re-execute all of the transactions in the block to load all touched accounts into
// the cache DB.
for tx in block.body().transactions() {
let signer =
tx.recover_signer().map_err(|_| eyre::eyre!("failed to recover sender"))?;
evm.transact_commit(self.evm_config.tx_env(tx, signer))?;
for tx in block.transactions_recovered() {
evm.transact_commit(self.evm_config.tx_env(tx))?;
}
drop(evm);

View File

@ -2775,7 +2775,7 @@ where
let mut evm = evm_config.evm_with_env(state_provider, evm_env);
// create the tx env and reset nonce
let tx_env = evm_config.tx_env(&tx, tx.signer());
let tx_env = evm_config.tx_env(&tx);
// exit early if execution is done
if cancel_execution.is_cancelled() {

View File

@ -312,7 +312,7 @@ where
// Configure the environment for the block.
let tx_recovered =
tx.try_clone_into_recovered().map_err(|_| ProviderError::SenderRecoveryError)?;
let tx_env = evm_config.tx_env(&tx_recovered, tx_recovered.signer());
let tx_env = evm_config.tx_env(tx_recovered);
let exec_result = match evm.transact(tx_env) {
Ok(result) => result,
Err(err) if err.is_invalid_tx_err() => {

View File

@ -12,13 +12,12 @@ workspace = true
[dependencies]
# Reth
reth-execution-types.workspace = true
reth-chainspec.workspace = true
reth-ethereum-forks.workspace = true
reth-revm.workspace = true
reth-evm.workspace = true
reth-primitives.workspace = true
reth-ethereum-consensus.workspace = true
reth-consensus.workspace = true
# Ethereum
reth-primitives-traits.workspace = true
@ -30,6 +29,8 @@ alloy-evm.workspace = true
alloy-sol-types.workspace = true
alloy-consensus.workspace = true
derive_more.workspace = true
[dev-dependencies]
reth-testing-utils.workspace = true
reth-evm = { workspace = true, features = ["test-utils"] }
@ -43,7 +44,6 @@ alloy-genesis.workspace = true
[features]
default = ["std"]
std = [
"reth-consensus/std",
"reth-primitives/std",
"reth-revm/std",
"alloy-consensus/std",
@ -58,4 +58,5 @@ std = [
"alloy-evm/std",
"reth-execution-types/std",
"reth-evm/std",
"derive_more/std",
]

View File

@ -8,19 +8,20 @@ use alloc::{boxed::Box, sync::Arc, vec::Vec};
use alloy_consensus::{BlockHeader, Transaction};
use alloy_eips::{eip6110, eip7685::Requests};
use reth_chainspec::{ChainSpec, EthereumHardfork, EthereumHardforks, MAINNET};
use reth_consensus::ConsensusError;
use reth_ethereum_consensus::validate_block_post_execution;
use reth_evm::{
execute::{
balance_increment_state, BasicBlockExecutorProvider, BlockExecutionError,
BlockExecutionStrategy, BlockExecutionStrategyFactory, BlockValidationError, ExecuteOutput,
BlockExecutionStrategy, BlockExecutionStrategyFactory, BlockValidationError,
},
state_change::post_block_balance_increments,
system_calls::{OnStateHook, StateChangePostBlockSource, StateChangeSource, SystemCaller},
ConfigureEvm, Database, Evm,
ConfigureEvm, ConfigureEvmEnv, Database, Evm,
};
use reth_primitives::{EthPrimitives, Receipt, RecoveredBlock};
use reth_primitives_traits::{BlockBody, SignedTransaction};
use reth_execution_types::BlockExecutionResult;
use reth_primitives::{
EthPrimitives, Receipt, Recovered, RecoveredBlock, SealedBlock, TransactionSigned,
};
use reth_primitives_traits::NodePrimitives;
use reth_revm::{context_interface::result::ResultAndState, db::State, DatabaseCommit};
/// Factory for [`EthExecutionStrategy`].
@ -46,7 +47,7 @@ impl EthExecutionStrategyFactory {
impl<EvmConfig> EthExecutionStrategyFactory<EvmConfig> {
/// Creates a new executor strategy factory.
pub const fn new(chain_spec: Arc<ChainSpec>, evm_config: EvmConfig) -> Self {
pub fn new(chain_spec: Arc<ChainSpec>, evm_config: EvmConfig) -> Self {
Self { chain_spec, evm_config }
}
}
@ -65,113 +66,108 @@ where
{
type Primitives = EthPrimitives;
type Strategy<DB: Database> = EthExecutionStrategy<DB, EvmConfig>;
fn create_strategy<DB>(&self, db: DB) -> Self::Strategy<DB>
fn create_strategy<'a, DB>(
&'a mut self,
db: &'a mut State<DB>,
block: &'a RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> impl BlockExecutionStrategy<Primitives = Self::Primitives, Error = BlockExecutionError> + 'a
where
DB: Database,
{
let state =
State::builder().with_database(db).with_bundle_update().without_state_clear().build();
EthExecutionStrategy::new(state, self.chain_spec.clone(), self.evm_config.clone())
let evm = self.evm_config.evm_for_block(db, block.header());
EthExecutionStrategy::new(evm, block.sealed_block(), self)
}
}
/// Block execution strategy for Ethereum.
#[allow(missing_debug_implementations)]
pub struct EthExecutionStrategy<DB, EvmConfig>
where
EvmConfig: Clone,
{
/// The chainspec
chain_spec: Arc<ChainSpec>,
/// How to create an EVM.
evm_config: EvmConfig,
/// Current state for block execution.
state: State<DB>,
#[derive(Debug, derive_more::Deref)]
pub struct EthExecutionStrategy<'a, Evm, EvmConfig> {
/// Reference to the parent factory providing access to [`ChainSpec`].
#[deref]
factory: &'a EthExecutionStrategyFactory<EvmConfig>,
/// Block being executed.
block: &'a SealedBlock,
/// The EVM used by strategy.
evm: Evm,
/// Receipts of executed transactions.
receipts: Vec<Receipt>,
/// Utility to call system smart contracts.
system_caller: SystemCaller<Arc<ChainSpec>>,
system_caller: SystemCaller<&'a ChainSpec>,
}
impl<DB, EvmConfig> EthExecutionStrategy<DB, EvmConfig>
where
EvmConfig: Clone,
{
impl<'a, Evm, EvmConfig> EthExecutionStrategy<'a, Evm, EvmConfig> {
/// Creates a new [`EthExecutionStrategy`]
pub fn new(state: State<DB>, chain_spec: Arc<ChainSpec>, evm_config: EvmConfig) -> Self {
let system_caller = SystemCaller::new(chain_spec.clone());
Self { state, chain_spec, evm_config, system_caller }
pub fn new(
evm: Evm,
block: &'a SealedBlock,
factory: &'a EthExecutionStrategyFactory<EvmConfig>,
) -> Self {
Self {
evm,
factory,
block,
receipts: Vec::new(),
system_caller: SystemCaller::new(&factory.chain_spec),
}
}
}
impl<DB, EvmConfig> BlockExecutionStrategy for EthExecutionStrategy<DB, EvmConfig>
impl<'db, DB, E, EvmConfig> BlockExecutionStrategy for EthExecutionStrategy<'_, E, EvmConfig>
where
DB: Database,
EvmConfig: ConfigureEvm<
Header = alloy_consensus::Header,
Transaction = reth_primitives::TransactionSigned,
>,
DB: Database + 'db,
E: Evm<DB = &'db mut State<DB>, Tx = EvmConfig::TxEnv>,
EvmConfig: ConfigureEvmEnv<Transaction = TransactionSigned>,
{
type DB = DB;
type Error = BlockExecutionError;
type Primitives = EthPrimitives;
fn apply_pre_execution_changes(
&mut self,
block: &RecoveredBlock<reth_primitives::Block>,
) -> Result<(), Self::Error> {
fn apply_pre_execution_changes(&mut self) -> Result<(), Self::Error> {
// Set state clear flag if the block is after the Spurious Dragon hardfork.
let state_clear_flag = self.chain_spec.is_spurious_dragon_active_at_block(block.number());
self.state.set_state_clear_flag(state_clear_flag);
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
self.system_caller.apply_pre_execution_changes(block.header(), &mut evm)?;
let state_clear_flag =
self.chain_spec.is_spurious_dragon_active_at_block(self.block.number());
self.evm.db_mut().set_state_clear_flag(state_clear_flag);
self.system_caller.apply_pre_execution_changes(self.block.header(), &mut self.evm)?;
Ok(())
}
fn execute_transactions(
fn execute_transactions<'a>(
&mut self,
block: &RecoveredBlock<reth_primitives::Block>,
) -> Result<ExecuteOutput<Receipt>, Self::Error> {
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
transactions: impl IntoIterator<Item = Recovered<&'a TransactionSigned>>,
) -> Result<(), Self::Error> {
let mut cumulative_gas_used = 0;
let mut receipts = Vec::with_capacity(block.body().transaction_count());
for (tx_index, (sender, transaction)) in block.transactions_with_sender().enumerate() {
for (tx_index, tx) in transactions.into_iter().enumerate() {
// The sum of the transaction's gas limit, Tg, and the gas utilized in this block prior,
// must be no greater than the block's gasLimit.
let block_available_gas = block.gas_limit() - cumulative_gas_used;
if transaction.gas_limit() > block_available_gas {
let block_available_gas = self.block.gas_limit() - cumulative_gas_used;
if tx.gas_limit() > block_available_gas {
return Err(BlockValidationError::TransactionGasLimitMoreThanAvailableBlockGas {
transaction_gas_limit: transaction.gas_limit(),
transaction_gas_limit: tx.gas_limit(),
block_available_gas,
}
.into())
}
let tx_env = self.evm_config.tx_env(transaction, *sender);
let tx_env = self.evm_config.tx_env(tx.clone());
let hash = tx.hash();
// Execute transaction.
let result_and_state = evm.transact(tx_env).map_err(move |err| {
let result_and_state = self.evm.transact(tx_env).map_err(move |err| {
// Ensure hash is calculated for error log, if not already done
BlockValidationError::EVM {
hash: transaction.recalculate_hash(),
error: Box::new(err),
}
BlockValidationError::EVM { hash: *hash, error: Box::new(err) }
})?;
self.system_caller
.on_state(StateChangeSource::Transaction(tx_index), &result_and_state.state);
let ResultAndState { result, state } = result_and_state;
evm.db_mut().commit(state);
self.evm.db_mut().commit(state);
// append gas used
cumulative_gas_used += result.gas_used();
// Push transaction changeset and calculate header bloom filter for receipt.
receipts.push(Receipt {
tx_type: transaction.tx_type(),
self.receipts.push(Receipt {
tx_type: tx.tx_type(),
// Success flag was added in `EIP-658: Embedding transaction status code in
// receipts`.
success: result.is_success(),
@ -179,20 +175,16 @@ where
logs: result.into_logs(),
});
}
Ok(ExecuteOutput { receipts, gas_used: cumulative_gas_used })
Ok(())
}
fn apply_post_execution_changes(
&mut self,
block: &RecoveredBlock<reth_primitives::Block>,
receipts: &[Receipt],
) -> Result<Requests, Self::Error> {
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
let requests = if self.chain_spec.is_prague_active_at_timestamp(block.timestamp) {
mut self,
) -> Result<BlockExecutionResult<Receipt>, Self::Error> {
let requests = if self.chain_spec.is_prague_active_at_timestamp(self.block.timestamp) {
// Collect all EIP-6110 deposits
let deposit_requests =
crate::eip6110::parse_deposits_from_receipts(&self.chain_spec, receipts)?;
crate::eip6110::parse_deposits_from_receipts(&self.chain_spec, &self.receipts)?;
let mut requests = Requests::default();
@ -200,20 +192,20 @@ where
requests.push_request_with_type(eip6110::DEPOSIT_REQUEST_TYPE, deposit_requests);
}
requests.extend(self.system_caller.apply_post_execution_changes(&mut evm)?);
requests.extend(self.system_caller.apply_post_execution_changes(&mut self.evm)?);
requests
} else {
Requests::default()
};
drop(evm);
let mut balance_increments = post_block_balance_increments(&self.chain_spec, block);
let mut balance_increments = post_block_balance_increments(&self.chain_spec, self.block);
// Irregular state change at Ethereum DAO hardfork
if self.chain_spec.fork(EthereumHardfork::Dao).transitions_at_block(block.number()) {
if self.chain_spec.fork(EthereumHardfork::Dao).transitions_at_block(self.block.number()) {
// drain balances from hardcoded addresses.
let drained_balance: u128 = self
.state
.evm
.db_mut()
.drain_balances(DAO_HARDFORK_ACCOUNTS)
.map_err(|_| BlockValidationError::IncrementBalanceFailed)?
.into_iter()
@ -223,43 +215,24 @@ where
*balance_increments.entry(DAO_HARDFORK_BENEFICIARY).or_default() += drained_balance;
}
// increment balances
self.state
self.evm
.db_mut()
.increment_balances(balance_increments.clone())
.map_err(|_| BlockValidationError::IncrementBalanceFailed)?;
// call state hook with changes due to balance increments.
let balance_state = balance_increment_state(&balance_increments, &mut self.state)?;
let balance_state = balance_increment_state(&balance_increments, self.evm.db_mut())?;
self.system_caller.on_state(
StateChangeSource::PostBlock(StateChangePostBlockSource::BalanceIncrements),
&balance_state,
);
Ok(requests)
}
fn state_ref(&self) -> &State<DB> {
&self.state
}
fn state_mut(&mut self) -> &mut State<DB> {
&mut self.state
}
fn into_state(self) -> State<Self::DB> {
self.state
let gas_used = self.receipts.last().map(|r| r.cumulative_gas_used).unwrap_or_default();
Ok(BlockExecutionResult { receipts: self.receipts, requests, gas_used })
}
fn with_state_hook(&mut self, hook: Option<Box<dyn OnStateHook>>) {
self.system_caller.with_state_hook(hook);
}
fn validate_block_post_execution(
&self,
block: &RecoveredBlock<reth_primitives::Block>,
receipts: &[Receipt],
requests: &Requests,
) -> Result<(), ConsensusError> {
validate_block_post_execution(block, &self.chain_spec.clone(), receipts, requests)
}
}
/// Helper type with backwards compatible methods to obtain Ethereum executor

View File

@ -17,11 +17,11 @@
extern crate alloc;
use alloc::sync::Arc;
use alloy_consensus::{BlockHeader, Header};
use alloc::{borrow::Borrow, sync::Arc};
use alloy_consensus::{transaction::Recovered, BlockHeader, Header};
pub use alloy_evm::EthEvm;
use alloy_evm::EthEvmFactory;
use alloy_primitives::{Address, U256};
use alloy_primitives::U256;
use core::{convert::Infallible, fmt::Debug};
use reth_chainspec::{ChainSpec, EthChainSpec, MAINNET};
use reth_evm::{ConfigureEvm, ConfigureEvmEnv, EvmEnv, NextBlockEnvAttributes};
@ -77,9 +77,14 @@ impl ConfigureEvmEnv for EthEvmConfig {
type TxEnv = TxEnv;
type Spec = SpecId;
fn tx_env(&self, transaction: &TransactionSigned, sender: Address) -> Self::TxEnv {
fn tx_env<T: Borrow<Self::Transaction>>(
&self,
transaction: impl Borrow<Recovered<T>>,
) -> Self::TxEnv {
let transaction = transaction.borrow();
let mut tx_env = TxEnv::default();
transaction.fill_tx_env(&mut tx_env, sender);
transaction.tx().borrow().fill_tx_env(&mut tx_env, transaction.signer());
tx_env
}

View File

@ -279,7 +279,7 @@ where
}
// Configure the environment for the tx.
let tx_env = evm_config.tx_env(tx.tx(), tx.signer());
let tx_env = evm_config.tx_env(&tx);
let ResultAndState { result, state } = match evm.transact(tx_env) {
Ok(res) => res,

View File

@ -13,7 +13,6 @@ workspace = true
[dependencies]
# reth
reth-chainspec.workspace = true
reth-consensus.workspace = true
reth-consensus-common.workspace = true
reth-execution-errors.workspace = true
reth-execution-types.workspace = true
@ -34,6 +33,7 @@ alloy-evm.workspace = true
alloy-consensus.workspace = true
auto_impl.workspace = true
derive_more.workspace = true
futures-util.workspace = true
metrics = { workspace = true, optional = true }
parking_lot = { workspace = true, optional = true }
@ -47,7 +47,6 @@ metrics-util = { workspace = true, features = ["debugging"] }
[features]
default = ["std"]
std = [
"reth-consensus/std",
"reth-primitives/std",
"reth-primitives-traits/std",
"alloy-eips/std",
@ -65,6 +64,7 @@ std = [
"reth-execution-types/std",
"reth-storage-errors/std",
"futures-util/std",
"derive_more/std",
]
metrics = [
"std",
@ -74,7 +74,6 @@ metrics = [
test-utils = [
"dep:parking_lot",
"reth-chainspec/test-utils",
"reth-consensus/test-utils",
"reth-ethereum-primitives/test-utils",
"reth-primitives/test-utils",
"reth-primitives-traits/test-utils",

View File

@ -3,7 +3,7 @@ use alloy_eips::eip7685::Requests;
use revm_database::BundleState;
/// The result of executing a block.
#[derive(Debug, Clone, PartialEq, Eq)]
#[derive(Debug, Clone, Default, PartialEq, Eq)]
pub struct BlockExecutionResult<T> {
/// All the receipts of the transactions in the block.
pub receipts: Vec<T>,

View File

@ -2,22 +2,19 @@
use alloy_consensus::BlockHeader;
// Re-export execution types
use crate::{system_calls::OnStateHook, Database};
use alloc::{boxed::Box, vec::Vec};
use alloy_primitives::{
map::{DefaultHashBuilder, HashMap},
Address,
};
pub use reth_execution_errors::{
BlockExecutionError, BlockValidationError, InternalBlockExecutionError,
};
use reth_execution_types::BlockExecutionResult;
pub use reth_execution_types::{BlockExecutionOutput, ExecutionOutcome};
use reth_primitives::{NodePrimitives, Receipt, Recovered, RecoveredBlock};
pub use reth_storage_errors::provider::ProviderError;
use crate::{system_calls::OnStateHook, Database};
use alloc::{boxed::Box, vec::Vec};
use alloy_eips::eip7685::Requests;
use alloy_primitives::{
map::{DefaultHashBuilder, HashMap},
Address,
};
use reth_consensus::ConsensusError;
use reth_primitives::{NodePrimitives, Receipt, RecoveredBlock};
use revm::state::{Account, AccountStatus, EvmState};
use revm_database::{states::bundle_state::BundleRetention, State};
@ -167,9 +164,6 @@ pub struct ExecuteOutput<R = Receipt> {
/// Defines the strategy for executing a single block.
pub trait BlockExecutionStrategy {
/// Database this strategy operates on.
type DB: revm::Database;
/// Primitive types used by the strategy.
type Primitives: NodePrimitives;
@ -177,45 +171,23 @@ pub trait BlockExecutionStrategy {
type Error: core::error::Error;
/// Applies any necessary changes before executing the block's transactions.
fn apply_pre_execution_changes(
&mut self,
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> Result<(), Self::Error>;
fn apply_pre_execution_changes(&mut self) -> Result<(), Self::Error>;
/// Executes all transactions in the block.
fn execute_transactions(
fn execute_transactions<'a>(
&mut self,
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> Result<ExecuteOutput<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>;
transactions: impl IntoIterator<
Item = Recovered<&'a <Self::Primitives as NodePrimitives>::SignedTx>,
>,
) -> Result<(), Self::Error>;
/// Applies any necessary changes after executing the block's transactions.
fn apply_post_execution_changes(
&mut self,
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
receipts: &[<Self::Primitives as NodePrimitives>::Receipt],
) -> Result<Requests, Self::Error>;
/// Returns a reference to the current state.
fn state_ref(&self) -> &State<Self::DB>;
/// Returns a mutable reference to the current state.
fn state_mut(&mut self) -> &mut State<Self::DB>;
/// Consumes the strategy and returns inner [`State`].
fn into_state(self) -> State<Self::DB>;
self,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>;
/// Sets a hook to be called after each state change during execution.
fn with_state_hook(&mut self, _hook: Option<Box<dyn OnStateHook>>) {}
/// Validate a block with regard to execution results.
fn validate_block_post_execution(
&self,
_block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
_receipts: &[<Self::Primitives as NodePrimitives>::Receipt],
_requests: &Requests,
) -> Result<(), ConsensusError> {
Ok(())
}
fn with_state_hook(&mut self, hook: Option<Box<dyn OnStateHook>>);
}
/// A strategy factory that can create block execution strategies.
@ -223,15 +195,12 @@ pub trait BlockExecutionStrategyFactory: Send + Sync + Clone + Unpin + 'static {
/// Primitive types used by the strategy.
type Primitives: NodePrimitives;
/// Associated strategy type.
type Strategy<DB: Database>: BlockExecutionStrategy<
DB = DB,
Primitives = Self::Primitives,
Error = BlockExecutionError,
>;
/// Creates a strategy using the give database.
fn create_strategy<DB>(&self, db: DB) -> Self::Strategy<DB>
/// Creates a strategy using the given database.
fn create_strategy<'a, DB>(
&'a mut self,
db: &'a mut State<DB>,
block: &'a RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> impl BlockExecutionStrategy<Primitives = Self::Primitives, Error = BlockExecutionError> + 'a
where
DB: Database;
}
@ -264,74 +233,85 @@ where
{
type Primitives = F::Primitives;
type Executor<DB: Database> = BasicBlockExecutor<F::Strategy<DB>>;
type Executor<DB: Database> = BasicBlockExecutor<F, DB>;
fn executor<DB>(&self, db: DB) -> Self::Executor<DB>
where
DB: Database,
{
let strategy = self.strategy_factory.create_strategy(db);
BasicBlockExecutor::new(strategy)
BasicBlockExecutor::new(self.strategy_factory.clone(), db)
}
}
/// A generic block executor that uses a [`BlockExecutionStrategy`] to
/// execute blocks.
#[allow(missing_debug_implementations, dead_code)]
pub struct BasicBlockExecutor<S> {
pub struct BasicBlockExecutor<F, DB> {
/// Block execution strategy.
pub(crate) strategy: S,
pub(crate) strategy_factory: F,
/// Database.
pub(crate) db: State<DB>,
}
impl<S> BasicBlockExecutor<S> {
impl<F, DB: Database> BasicBlockExecutor<F, DB> {
/// Creates a new `BasicBlockExecutor` with the given strategy.
pub const fn new(strategy: S) -> Self {
Self { strategy }
pub fn new(strategy_factory: F, db: DB) -> Self {
let db =
State::builder().with_database(db).with_bundle_update().without_state_clear().build();
Self { strategy_factory, db }
}
}
impl<S, DB> Executor<DB> for BasicBlockExecutor<S>
impl<F, DB> Executor<DB> for BasicBlockExecutor<F, DB>
where
S: BlockExecutionStrategy<DB = DB>,
F: BlockExecutionStrategyFactory,
DB: Database,
{
type Primitives = S::Primitives;
type Error = S::Error;
type Primitives = F::Primitives;
type Error = BlockExecutionError;
fn execute_one(
&mut self,
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
{
self.strategy.apply_pre_execution_changes(block)?;
let ExecuteOutput { receipts, gas_used } = self.strategy.execute_transactions(block)?;
let requests = self.strategy.apply_post_execution_changes(block, &receipts)?;
self.strategy.state_mut().merge_transitions(BundleRetention::Reverts);
let mut strategy = self.strategy_factory.create_strategy(&mut self.db, block);
Ok(BlockExecutionResult { receipts, requests, gas_used })
strategy.apply_pre_execution_changes()?;
strategy.execute_transactions(block.transactions_recovered())?;
let result = strategy.apply_post_execution_changes()?;
self.db.merge_transitions(BundleRetention::Reverts);
Ok(result)
}
fn execute_one_with_state_hook<F>(
fn execute_one_with_state_hook<H>(
&mut self,
block: &RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
state_hook: F,
state_hook: H,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
where
F: OnStateHook + 'static,
H: OnStateHook + 'static,
{
self.strategy.with_state_hook(Some(Box::new(state_hook)));
let result = self.execute_one(block);
self.strategy.with_state_hook(None);
let mut strategy = self.strategy_factory.create_strategy(&mut self.db, block);
strategy.with_state_hook(Some(Box::new(state_hook)));
result
strategy.apply_pre_execution_changes()?;
strategy.execute_transactions(block.transactions_recovered())?;
let result = strategy.apply_post_execution_changes()?;
self.db.merge_transitions(BundleRetention::Reverts);
Ok(result)
}
fn into_state(self) -> State<DB> {
self.strategy.into_state()
self.db
}
fn size_hint(&self) -> usize {
self.strategy.state_ref().bundle_state.size_hint()
self.db.bundle_state.size_hint()
}
}
@ -374,17 +354,14 @@ where
#[cfg(test)]
mod tests {
use super::*;
use alloy_primitives::U256;
use alloy_consensus::constants::KECCAK_EMPTY;
use alloy_eips::eip7685::Requests;
use alloy_primitives::{address, bytes, U256};
use core::marker::PhantomData;
use reth_chainspec::{ChainSpec, MAINNET};
use reth_ethereum_primitives::TransactionSigned;
use reth_primitives::EthPrimitives;
use revm::{
database_interface::EmptyDBTyped,
primitives::{address, bytes, KECCAK_EMPTY},
state::AccountInfo,
};
use revm_database::CacheDB;
use std::sync::Arc;
use revm::state::AccountInfo;
use revm_database::{CacheDB, EmptyDBTyped};
#[derive(Clone, Default)]
struct TestExecutorProvider;
@ -435,94 +412,55 @@ mod tests {
}
}
struct TestExecutorStrategy<DB, EvmConfig> {
// chain spec and evm config here only to illustrate how the strategy
// factory can use them in a real use case.
_chain_spec: Arc<ChainSpec>,
_evm_config: EvmConfig,
state: State<DB>,
execute_transactions_result: ExecuteOutput<Receipt>,
apply_post_execution_changes_result: Requests,
struct TestExecutorStrategy {
result: BlockExecutionResult<Receipt>,
}
#[derive(Clone)]
struct TestExecutorStrategyFactory {
execute_transactions_result: ExecuteOutput<Receipt>,
apply_post_execution_changes_result: Requests,
result: BlockExecutionResult<Receipt>,
}
impl BlockExecutionStrategyFactory for TestExecutorStrategyFactory {
type Primitives = EthPrimitives;
type Strategy<DB: Database> = TestExecutorStrategy<DB, TestEvmConfig>;
fn create_strategy<DB>(&self, db: DB) -> Self::Strategy<DB>
fn create_strategy<'a, DB>(
&'a mut self,
_db: &'a mut State<DB>,
_block: &'a RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> impl BlockExecutionStrategy<Primitives = Self::Primitives, Error = BlockExecutionError> + 'a
where
DB: Database,
{
let state = State::builder()
.with_database(db)
.with_bundle_update()
.without_state_clear()
.build();
TestExecutorStrategy {
_chain_spec: MAINNET.clone(),
_evm_config: TestEvmConfig {},
execute_transactions_result: self.execute_transactions_result.clone(),
apply_post_execution_changes_result: self
.apply_post_execution_changes_result
.clone(),
state,
}
TestExecutorStrategy { result: self.result.clone() }
}
}
impl<DB> BlockExecutionStrategy for TestExecutorStrategy<DB, TestEvmConfig>
where
DB: Database,
{
type DB = DB;
impl BlockExecutionStrategy for TestExecutorStrategy {
type Primitives = EthPrimitives;
type Error = BlockExecutionError;
fn apply_pre_execution_changes(
fn apply_pre_execution_changes(&mut self) -> Result<(), Self::Error> {
Ok(())
}
fn execute_transactions<'a>(
&mut self,
_block: &RecoveredBlock<reth_primitives::Block>,
_transactions: impl IntoIterator<Item = Recovered<&'a TransactionSigned>>,
) -> Result<(), Self::Error> {
Ok(())
}
fn execute_transactions(
&mut self,
_block: &RecoveredBlock<reth_primitives::Block>,
) -> Result<ExecuteOutput<Receipt>, Self::Error> {
Ok(self.execute_transactions_result.clone())
}
fn apply_post_execution_changes(
&mut self,
_block: &RecoveredBlock<reth_primitives::Block>,
_receipts: &[Receipt],
) -> Result<Requests, Self::Error> {
Ok(self.apply_post_execution_changes_result.clone())
self,
) -> Result<BlockExecutionResult<<Self::Primitives as NodePrimitives>::Receipt>, Self::Error>
{
Ok(self.result)
}
fn state_ref(&self) -> &State<DB> {
&self.state
}
fn state_mut(&mut self) -> &mut State<DB> {
&mut self.state
}
fn into_state(self) -> State<Self::DB> {
self.state
}
fn with_state_hook(&mut self, _hook: Option<Box<dyn OnStateHook>>) {}
}
#[derive(Clone)]
struct TestEvmConfig {}
#[test]
fn test_provider() {
let provider = TestExecutorProvider;
@ -533,19 +471,13 @@ mod tests {
#[test]
fn test_strategy() {
let expected_gas_used = 10;
let expected_receipts = vec![Receipt::default()];
let expected_execute_transactions_result = ExecuteOutput::<Receipt> {
receipts: expected_receipts.clone(),
gas_used: expected_gas_used,
let expected_result = BlockExecutionResult {
receipts: vec![Receipt::default()],
gas_used: 10,
requests: Requests::new(vec![bytes!("deadbeef")]),
};
let expected_apply_post_execution_changes_result = Requests::new(vec![bytes!("deadbeef")]);
let strategy_factory = TestExecutorStrategyFactory {
execute_transactions_result: expected_execute_transactions_result,
apply_post_execution_changes_result: expected_apply_post_execution_changes_result
.clone(),
};
let strategy_factory = TestExecutorStrategyFactory { result: expected_result.clone() };
let provider = BasicBlockExecutorProvider::new(strategy_factory);
let db = CacheDB::<EmptyDBTyped<ProviderError>>::default();
let executor = provider.executor(db);
@ -553,9 +485,7 @@ mod tests {
assert!(result.is_ok());
let block_execution_output = result.unwrap();
assert_eq!(block_execution_output.gas_used, expected_gas_used);
assert_eq!(block_execution_output.receipts, expected_receipts);
assert_eq!(block_execution_output.requests, expected_apply_post_execution_changes_result);
assert_eq!(block_execution_output.result, expected_result);
}
fn setup_state_with_account(

View File

@ -17,6 +17,7 @@
extern crate alloc;
use alloc::borrow::Borrow;
use alloy_consensus::transaction::Recovered;
use alloy_eips::eip2930::AccessList;
pub use alloy_evm::evm::EvmFactory;
@ -182,13 +183,10 @@ pub trait ConfigureEvmEnv: Send + Sync + Unpin + Clone + 'static {
type Spec: Debug + Copy + Send + Sync + 'static;
/// Returns a [`TxEnv`] from a transaction and [`Address`].
fn tx_env(&self, transaction: &Self::Transaction, signer: Address) -> Self::TxEnv;
/// Returns a [`TxEnv`] from a [`Recovered`] transaction.
fn tx_env_from_recovered(&self, tx: Recovered<&Self::Transaction>) -> Self::TxEnv {
let (tx, address) = tx.into_parts();
self.tx_env(tx, address)
}
fn tx_env<T: Borrow<Self::Transaction>>(
&self,
transaction: impl Borrow<Recovered<T>>,
) -> Self::TxEnv;
/// Creates a new [`EvmEnv`] for the given header.
fn evm_env(&self, header: &Self::Header) -> EvmEnv<Self::Spec>;

View File

@ -77,10 +77,11 @@ impl OnStateHook for NoopHook {
/// An ephemeral helper type for executing system calls.
///
/// This can be used to chain system transaction calls.
#[allow(missing_debug_implementations)]
#[derive(derive_more::Debug)]
pub struct SystemCaller<ChainSpec> {
chain_spec: ChainSpec,
/// Optional hook to be called after each state change.
#[debug(skip)]
hook: Option<Box<dyn OnStateHook>>,
}
@ -149,7 +150,7 @@ where
eip2935::transact_blockhashes_contract_call(&self.chain_spec, parent_block_hash, evm)?;
if let Some(res) = result_and_state {
if let Some(ref mut hook) = self.hook {
if let Some(hook) = &mut self.hook {
hook.on_state(
StateChangeSource::PreBlock(StateChangePreBlockSource::BlockHashesContract),
&res.state,
@ -174,7 +175,7 @@ where
)?;
if let Some(res) = result_and_state {
if let Some(ref mut hook) = self.hook {
if let Some(hook) = &mut self.hook {
hook.on_state(
StateChangeSource::PreBlock(StateChangePreBlockSource::BeaconRootContract),
&res.state,
@ -193,7 +194,7 @@ where
) -> Result<Bytes, BlockExecutionError> {
let result_and_state = eip7002::transact_withdrawal_requests_contract_call(evm)?;
if let Some(ref mut hook) = self.hook {
if let Some(ref mut hook) = &mut self.hook {
hook.on_state(
StateChangeSource::PostBlock(
StateChangePostBlockSource::WithdrawalRequestsContract,
@ -213,7 +214,7 @@ where
) -> Result<Bytes, BlockExecutionError> {
let result_and_state = eip7251::transact_consolidation_requests_contract_call(evm)?;
if let Some(ref mut hook) = self.hook {
if let Some(ref mut hook) = &mut self.hook {
hook.on_state(
StateChangeSource::PostBlock(
StateChangePostBlockSource::ConsolidationRequestsContract,
@ -228,7 +229,7 @@ where
/// Delegate to stored `OnStateHook`, noop if hook is `None`.
pub fn on_state(&mut self, source: StateChangeSource, state: &EvmState) {
if let Some(ref mut hook) = &mut self.hook {
if let Some(hook) = &mut self.hook {
hook.on_state(source, state);
}
}

View File

@ -1,10 +1,7 @@
//! Helpers for testing.
use crate::{
execute::{
BasicBlockExecutor, BlockExecutionOutput, BlockExecutionStrategy, BlockExecutorProvider,
Executor,
},
execute::{BasicBlockExecutor, BlockExecutionOutput, BlockExecutorProvider, Executor},
system_calls::OnStateHook,
Database,
};
@ -125,23 +122,20 @@ impl<DB: Database> Executor<DB> for MockExecutorProvider {
}
}
impl<S> BasicBlockExecutor<S>
where
S: BlockExecutionStrategy,
{
impl<Factory, DB> BasicBlockExecutor<Factory, DB> {
/// Provides safe read access to the state
pub fn with_state<F, R>(&self, f: F) -> R
where
F: FnOnce(&State<S::DB>) -> R,
F: FnOnce(&State<DB>) -> R,
{
f(self.strategy.state_ref())
f(&self.db)
}
/// Provides safe write access to the state
pub fn with_state_mut<F, R>(&mut self, f: F) -> R
where
F: FnOnce(&mut State<S::DB>) -> R,
F: FnOnce(&mut State<DB>) -> R,
{
f(self.strategy.state_mut())
f(&mut self.db)
}
}

View File

@ -5,24 +5,24 @@ use crate::{
OpReceiptBuilder, ReceiptBuilderCtx,
};
use alloc::{boxed::Box, sync::Arc, vec::Vec};
use alloy_consensus::{BlockHeader, Eip658Value, Receipt, Transaction as _};
use alloy_eips::eip7685::Requests;
use alloy_consensus::{
transaction::Recovered, BlockHeader, Eip658Value, Receipt, Transaction as _, TxReceipt,
};
use op_alloy_consensus::OpDepositReceipt;
use reth_consensus::ConsensusError;
use reth_evm::{
execute::{
balance_increment_state, BasicBlockExecutorProvider, BlockExecutionError,
BlockExecutionStrategy, BlockExecutionStrategyFactory, BlockValidationError, ExecuteOutput,
BlockExecutionStrategy, BlockExecutionStrategyFactory, BlockValidationError,
},
state_change::post_block_balance_increments,
system_calls::{OnStateHook, StateChangePostBlockSource, StateChangeSource, SystemCaller},
ConfigureEvm, ConfigureEvmFor, Database, Evm, HaltReasonFor,
};
use reth_execution_types::BlockExecutionResult;
use reth_optimism_chainspec::OpChainSpec;
use reth_optimism_consensus::validate_block_post_execution;
use reth_optimism_forks::OpHardforks;
use reth_optimism_primitives::{transaction::signed::OpTransaction, DepositReceipt, OpPrimitives};
use reth_primitives_traits::{BlockBody, NodePrimitives, RecoveredBlock, SignedTransaction};
use reth_primitives_traits::{NodePrimitives, RecoveredBlock, SealedBlock, SignedTransaction};
use revm::{context_interface::result::ResultAndState, DatabaseCommit};
use revm_database::State;
use tracing::trace;
@ -79,43 +79,38 @@ where
EvmConfig: ConfigureEvmFor<N> + Clone + Unpin + Sync + Send + 'static,
{
type Primitives = N;
type Strategy<DB: Database> = OpExecutionStrategy<DB, N, ChainSpec, EvmConfig>;
fn create_strategy<DB>(&self, db: DB) -> Self::Strategy<DB>
fn create_strategy<'a, DB>(
&'a mut self,
db: &'a mut State<DB>,
block: &'a RecoveredBlock<<Self::Primitives as NodePrimitives>::Block>,
) -> impl BlockExecutionStrategy<Primitives = Self::Primitives, Error = BlockExecutionError> + 'a
where
DB: Database,
{
let state =
State::builder().with_database(db).with_bundle_update().without_state_clear().build();
OpExecutionStrategy::new(
state,
self.chain_spec.clone(),
self.evm_config.clone(),
self.receipt_builder.clone(),
)
let evm = self.evm_config.evm_for_block(db, block.header());
OpExecutionStrategy::new(evm, block.sealed_block(), self)
}
}
/// Block execution strategy for Optimism.
#[allow(missing_debug_implementations)]
pub struct OpExecutionStrategy<DB, N: NodePrimitives, ChainSpec, EvmConfig>
where
EvmConfig: ConfigureEvm,
{
/// The chainspec
chain_spec: Arc<ChainSpec>,
/// How to create an EVM.
evm_config: EvmConfig,
/// Current state for block execution.
state: State<DB>,
#[derive(Debug, derive_more::Deref)]
pub struct OpExecutionStrategy<'a, Evm, N: NodePrimitives, ChainSpec, EvmConfig: ConfigureEvm> {
/// Reference to the parent factory.
#[deref]
factory: &'a OpExecutionStrategyFactory<N, ChainSpec, EvmConfig>,
/// Block being executed.
block: &'a SealedBlock<N::Block>,
/// The EVM used by strategy.
evm: Evm,
/// Receipts of executed transactions.
receipts: Vec<N::Receipt>,
/// Utility to call system smart contracts.
system_caller: SystemCaller<Arc<ChainSpec>>,
/// Receipt builder.
receipt_builder:
Arc<dyn OpReceiptBuilder<N::SignedTx, HaltReasonFor<EvmConfig>, Receipt = N::Receipt>>,
system_caller: SystemCaller<&'a ChainSpec>,
}
impl<DB, N, ChainSpec, EvmConfig> OpExecutionStrategy<DB, N, ChainSpec, EvmConfig>
impl<'a, Evm, N, ChainSpec, EvmConfig> OpExecutionStrategy<'a, Evm, N, ChainSpec, EvmConfig>
where
N: NodePrimitives,
ChainSpec: OpHardforks,
@ -123,73 +118,67 @@ where
{
/// Creates a new [`OpExecutionStrategy`]
pub fn new(
state: State<DB>,
chain_spec: Arc<ChainSpec>,
evm_config: EvmConfig,
receipt_builder: Arc<
dyn OpReceiptBuilder<N::SignedTx, HaltReasonFor<EvmConfig>, Receipt = N::Receipt>,
>,
evm: Evm,
block: &'a SealedBlock<N::Block>,
factory: &'a OpExecutionStrategyFactory<N, ChainSpec, EvmConfig>,
) -> Self {
let system_caller = SystemCaller::new(chain_spec.clone());
Self { state, chain_spec, evm_config, system_caller, receipt_builder }
Self {
evm,
factory,
block,
receipts: Vec::new(),
system_caller: SystemCaller::new(&factory.chain_spec),
}
}
}
impl<DB, N, ChainSpec, EvmConfig> BlockExecutionStrategy
for OpExecutionStrategy<DB, N, ChainSpec, EvmConfig>
impl<'db, DB, E, N, ChainSpec, EvmConfig> BlockExecutionStrategy
for OpExecutionStrategy<'_, E, N, ChainSpec, EvmConfig>
where
DB: Database,
DB: Database + 'db,
E: Evm<DB = &'db mut State<DB>, Tx = EvmConfig::TxEnv, HaltReason = HaltReasonFor<EvmConfig>>,
N: NodePrimitives<SignedTx: OpTransaction, Receipt: DepositReceipt>,
ChainSpec: OpHardforks,
EvmConfig: ConfigureEvmFor<N>,
{
type DB = DB;
type Primitives = N;
type Error = BlockExecutionError;
fn apply_pre_execution_changes(
&mut self,
block: &RecoveredBlock<N::Block>,
) -> Result<(), Self::Error> {
fn apply_pre_execution_changes(&mut self) -> Result<(), Self::Error> {
// Set state clear flag if the block is after the Spurious Dragon hardfork.
let state_clear_flag =
(*self.chain_spec).is_spurious_dragon_active_at_block(block.number());
self.state.set_state_clear_flag(state_clear_flag);
(*self.chain_spec).is_spurious_dragon_active_at_block(self.block.number());
self.evm.db_mut().set_state_clear_flag(state_clear_flag);
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
self.system_caller
.apply_beacon_root_contract_call(block.header().parent_beacon_block_root(), &mut evm)?;
self.system_caller.apply_beacon_root_contract_call(
self.block.parent_beacon_block_root(),
&mut self.evm,
)?;
// Ensure that the create2deployer is force-deployed at the canyon transition. Optimism
// blocks will always have at least a single transaction in them (the L1 info transaction),
// so we can safely assume that this will always be triggered upon the transition and that
// the above check for empty blocks will never be hit on OP chains.
ensure_create2_deployer(self.chain_spec.clone(), block.header().timestamp(), evm.db_mut())
ensure_create2_deployer(self.chain_spec.clone(), self.block.timestamp(), self.evm.db_mut())
.map_err(|_| OpBlockExecutionError::ForceCreate2DeployerFail)?;
Ok(())
}
fn execute_transactions(
fn execute_transactions<'a>(
&mut self,
block: &RecoveredBlock<N::Block>,
) -> Result<ExecuteOutput<N::Receipt>, Self::Error> {
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
let is_regolith = self.chain_spec.is_regolith_active_at_timestamp(block.timestamp());
transactions: impl IntoIterator<Item = Recovered<&'a N::SignedTx>>,
) -> Result<(), Self::Error> {
let is_regolith = self.chain_spec.is_regolith_active_at_timestamp(self.block.timestamp());
let mut cumulative_gas_used = 0;
let mut receipts = Vec::with_capacity(block.body().transaction_count());
for (tx_index, (sender, transaction)) in block.transactions_with_sender().enumerate() {
for (tx_index, tx) in transactions.into_iter().enumerate() {
// The sum of the transactions gas limit, Tg, and the gas utilized in this block prior,
// must be no greater than the blocks gasLimit.
let block_available_gas = block.gas_limit() - cumulative_gas_used;
if transaction.gas_limit() > block_available_gas &&
(is_regolith || !transaction.is_deposit())
{
let block_available_gas = self.block.gas_limit() - cumulative_gas_used;
if tx.gas_limit() > block_available_gas && (is_regolith || !tx.is_deposit()) {
return Err(BlockValidationError::TransactionGasLimitMoreThanAvailableBlockGas {
transaction_gas_limit: transaction.gas_limit(),
transaction_gas_limit: tx.gas_limit(),
block_available_gas,
}
.into())
@ -200,42 +189,41 @@ where
// Note that this *only* needs to be done post-regolith hardfork, as deposit nonces
// were not introduced in Bedrock. In addition, regular transactions don't have deposit
// nonces, so we don't need to touch the DB for those.
let depositor = (is_regolith && transaction.is_deposit())
let depositor = (is_regolith && tx.is_deposit())
.then(|| {
evm.db_mut()
.load_cache_account(*sender)
self.evm
.db_mut()
.load_cache_account(tx.signer())
.map(|acc| acc.account_info().unwrap_or_default())
})
.transpose()
.map_err(|_| OpBlockExecutionError::AccountLoadFailed(*sender))?;
.map_err(|_| OpBlockExecutionError::AccountLoadFailed(tx.signer()))?;
let tx_env = self.evm_config.tx_env(transaction, *sender);
let tx_env = self.evm_config.tx_env(&tx);
let hash = tx.tx_hash();
// Execute transaction.
let result_and_state = evm.transact(tx_env).map_err(move |err| {
let result_and_state = self.evm.transact(tx_env).map_err(move |err| {
// Ensure hash is calculated for error log, if not already done
BlockValidationError::EVM {
hash: transaction.recalculate_hash(),
error: Box::new(err),
}
BlockValidationError::EVM { hash: *hash, error: Box::new(err) }
})?;
trace!(
target: "evm",
?transaction,
?tx,
"Executed transaction"
);
self.system_caller
.on_state(StateChangeSource::Transaction(tx_index), &result_and_state.state);
let ResultAndState { result, state } = result_and_state;
evm.db_mut().commit(state);
self.evm.db_mut().commit(state);
// append gas used
cumulative_gas_used += result.gas_used();
receipts.push(
self.receipts.push(
match self.receipt_builder.build_receipt(ReceiptBuilderCtx {
tx: transaction,
tx: tx.tx(),
result,
cumulative_gas_used,
}) {
@ -257,9 +245,9 @@ where
// when set. The state transition process ensures
// this is only set for post-Canyon deposit
// transactions.
deposit_receipt_version: (transaction.is_deposit() &&
deposit_receipt_version: (tx.is_deposit() &&
self.chain_spec
.is_canyon_active_at_timestamp(block.timestamp()))
.is_canyon_active_at_timestamp(self.block.timestamp()))
.then_some(1),
})
}
@ -267,53 +255,33 @@ where
);
}
Ok(ExecuteOutput { receipts, gas_used: cumulative_gas_used })
Ok(())
}
fn apply_post_execution_changes(
&mut self,
block: &RecoveredBlock<N::Block>,
_receipts: &[N::Receipt],
) -> Result<Requests, Self::Error> {
let balance_increments = post_block_balance_increments(&self.chain_spec.clone(), block);
mut self,
) -> Result<BlockExecutionResult<N::Receipt>, Self::Error> {
let balance_increments =
post_block_balance_increments(&self.chain_spec.clone(), self.block);
// increment balances
self.state
self.evm
.db_mut()
.increment_balances(balance_increments.clone())
.map_err(|_| BlockValidationError::IncrementBalanceFailed)?;
// call state hook with changes due to balance increments.
let balance_state = balance_increment_state(&balance_increments, &mut self.state)?;
let balance_state = balance_increment_state(&balance_increments, self.evm.db_mut())?;
self.system_caller.on_state(
StateChangeSource::PostBlock(StateChangePostBlockSource::BalanceIncrements),
&balance_state,
);
Ok(Requests::default())
}
fn state_ref(&self) -> &State<DB> {
&self.state
}
fn state_mut(&mut self) -> &mut State<DB> {
&mut self.state
}
fn into_state(self) -> revm_database::State<Self::DB> {
self.state
let gas_used = self.receipts.last().map(|r| r.cumulative_gas_used()).unwrap_or_default();
Ok(BlockExecutionResult { receipts: self.receipts, requests: Default::default(), gas_used })
}
fn with_state_hook(&mut self, hook: Option<Box<dyn OnStateHook>>) {
self.system_caller.with_state_hook(hook);
}
fn validate_block_post_execution(
&self,
block: &RecoveredBlock<N::Block>,
receipts: &[N::Receipt],
_requests: &Requests,
) -> Result<(), ConsensusError> {
validate_block_post_execution(block.header(), self.chain_spec.clone(), receipts)
}
}
/// Helper type with backwards compatible methods to obtain executor providers.

View File

@ -11,10 +11,10 @@
extern crate alloc;
use alloc::sync::Arc;
use alloy_consensus::{BlockHeader, Header};
use alloy_consensus::{transaction::Recovered, BlockHeader, Header};
use alloy_op_evm::OpEvmFactory;
use alloy_primitives::{Address, U256};
use core::fmt::Debug;
use alloy_primitives::U256;
use core::{borrow::Borrow, fmt::Debug};
use op_alloy_consensus::EIP1559ParamError;
use reth_chainspec::EthChainSpec;
use reth_evm::{ConfigureEvm, ConfigureEvmEnv, EvmEnv, NextBlockEnvAttributes};
@ -74,9 +74,14 @@ impl<ChainSpec: EthChainSpec + OpHardforks + 'static> ConfigureEvmEnv for OpEvmC
type TxEnv = OpTransaction<TxEnv>;
type Spec = OpSpecId;
fn tx_env(&self, transaction: &Self::Transaction, signer: Address) -> Self::TxEnv {
fn tx_env<T: Borrow<Self::Transaction>>(
&self,
transaction: impl Borrow<Recovered<T>>,
) -> Self::TxEnv {
let transaction = transaction.borrow();
let mut tx_env = Default::default();
transaction.fill_tx_env(&mut tx_env, signer);
transaction.tx().borrow().fill_tx_env(&mut tx_env, transaction.signer());
tx_env
}

View File

@ -934,7 +934,7 @@ where
))
})?;
let tx_env = self.evm_config.tx_env(sequencer_tx.tx(), sequencer_tx.signer());
let tx_env = self.evm_config.tx_env(&sequencer_tx);
let ResultAndState { result, state: _ } = match evm.transact_commit(tx_env) {
Ok(res) => res,
@ -1011,7 +1011,7 @@ where
}
// Configure the environment for the tx.
let tx_env = self.evm_config.tx_env(tx.tx(), tx.signer());
let tx_env = self.evm_config.tx_env(&tx);
let ResultAndState { result, state: _ } = match evm.transact_commit(tx_env) {
Ok(res) => res,

View File

@ -2,8 +2,8 @@
use crate::OpEthApi;
use alloy_consensus::{
constants::EMPTY_WITHDRAWALS, proofs::calculate_transaction_root, Eip658Value, Header,
Transaction as _, TxReceipt, EMPTY_OMMER_ROOT_HASH,
constants::EMPTY_WITHDRAWALS, proofs::calculate_transaction_root, transaction::Recovered,
Eip658Value, Header, Transaction as _, TxReceipt, EMPTY_OMMER_ROOT_HASH,
};
use alloy_eips::{eip7685::EMPTY_REQUESTS_HASH, merge::BEACON_NONCE, BlockNumberOrTag};
use alloy_primitives::{B256, U256};
@ -100,7 +100,7 @@ where
block_env: &BlockEnv,
parent_hash: B256,
state_root: B256,
transactions: Vec<ProviderTx<Self::Provider>>,
transactions: Vec<Recovered<ProviderTx<Self::Provider>>>,
receipts: &[ProviderReceipt<Self::Provider>],
) -> reth_provider::ProviderBlock<Self::Provider> {
let chain_spec = self.provider().chain_spec();
@ -144,7 +144,11 @@ where
// seal the block
reth_primitives::Block {
header,
body: BlockBody { transactions, ommers: vec![], withdrawals: None },
body: BlockBody {
transactions: transactions.into_iter().map(|tx| tx.into_tx()).collect(),
ommers: vec![],
withdrawals: None,
},
}
}

View File

@ -7,7 +7,7 @@ use crate::{
};
use alloy_consensus::BlockHeader;
use alloy_eips::{eip1559::calc_next_block_base_fee, eip2930::AccessListResult};
use alloy_primitives::{Address, Bytes, B256, U256};
use alloy_primitives::{Bytes, B256, U256};
use alloy_rpc_types_eth::{
simulate::{SimBlock, SimulatePayload, SimulatedBlock},
state::{EvmOverrides, StateOverride},
@ -22,6 +22,7 @@ use reth_evm::{
TransactionEnv,
};
use reth_node_api::BlockBody;
use reth_primitives::Recovered;
use reth_primitives_traits::SignedTransaction;
use reth_provider::{BlockIdReader, ChainSpecProvider, ProviderHeader};
use reth_revm::{database::StateProviderDatabase, db::CacheDB, DatabaseRef};
@ -172,8 +173,6 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthA
let mut results = Vec::with_capacity(calls.len());
while let Some(call) = calls.next() {
let sender = call.from.unwrap_or_default();
// Resolve transaction, populate missing fields and enforce calls
// correctness.
let tx = simulate::resolve_transaction(
@ -185,7 +184,7 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthA
this.tx_resp_builder(),
)?;
let tx_env = this.evm_config().tx_env(&tx, sender);
let tx_env = this.evm_config().tx_env(&tx);
let (res, (_, tx_env)) = {
if trace_transfers {
@ -324,9 +323,9 @@ pub trait EthCall: EstimateCall + Call + LoadPendingBlock + LoadBlock + FullEthA
if replay_block_txs {
// only need to replay the transactions in the block if not all transactions are
// to be replayed
let transactions = block.transactions_with_sender().take(num_txs);
for (signer, tx) in transactions {
let tx_env = RpcNodeCore::evm_config(&this).tx_env(tx, *signer);
let transactions = block.transactions_recovered().take(num_txs);
for tx in transactions {
let tx_env = RpcNodeCore::evm_config(&this).tx_env(tx);
let (res, _) = this.transact(&mut db, evm_env.clone(), tx_env)?;
db.commit(res.state);
}
@ -671,12 +670,12 @@ pub trait Call:
let this = self.clone();
self.spawn_with_state_at_block(parent_block.into(), move |state| {
let mut db = CacheDB::new(StateProviderDatabase::new(state));
let block_txs = block.transactions_with_sender();
let block_txs = block.transactions_recovered();
// replay all transactions prior to the targeted transaction
this.replay_transactions_until(&mut db, evm_env.clone(), block_txs, *tx.tx_hash())?;
let tx_env = RpcNodeCore::evm_config(&this).tx_env(tx.tx(), tx.signer());
let tx_env = RpcNodeCore::evm_config(&this).tx_env(tx);
let (res, _) = this.transact(&mut db, evm_env, tx_env)?;
f(tx_info, res, db)
@ -702,18 +701,18 @@ pub trait Call:
) -> Result<usize, Self::Error>
where
DB: Database<Error = ProviderError> + DatabaseCommit,
I: IntoIterator<Item = (&'a Address, &'a <Self::Evm as ConfigureEvmEnv>::Transaction)>,
I: IntoIterator<Item = Recovered<&'a <Self::Evm as ConfigureEvmEnv>::Transaction>>,
<Self::Evm as ConfigureEvmEnv>::Transaction: SignedTransaction,
{
let mut evm = self.evm_config().evm_with_env(db, evm_env);
let mut index = 0;
for (sender, tx) in transactions {
for tx in transactions {
if *tx.tx_hash() == target_tx_hash {
// reached the target transaction
break
}
let tx_env = self.evm_config().tx_env(tx, *sender);
let tx_env = self.evm_config().tx_env(tx);
evm.transact_commit(tx_env).map_err(Self::Error::from_evm_err)?;
index += 1;
}

View File

@ -3,7 +3,7 @@
use super::SpawnBlocking;
use crate::{types::RpcTypes, EthApiTypes, FromEthApiError, FromEvmError, RpcNodeCore};
use alloy_consensus::{BlockHeader, Transaction};
use alloy_consensus::{transaction::Recovered, BlockHeader, Transaction};
use alloy_eips::eip4844::MAX_DATA_GAS_PER_BLOCK;
use alloy_primitives::B256;
use alloy_rpc_types_eth::BlockNumberOrTag;
@ -209,7 +209,7 @@ pub trait LoadPendingBlock:
block_env: &BlockEnv,
parent_hash: B256,
state_root: B256,
transactions: Vec<ProviderTx<Self::Provider>>,
transactions: Vec<Recovered<ProviderTx<Self::Provider>>>,
receipts: &[ProviderReceipt<Self::Provider>],
) -> ProviderBlock<Self::Provider>;
@ -219,7 +219,7 @@ pub trait LoadPendingBlock:
block_env: &BlockEnv,
parent_hash: B256,
state_root: B256,
transactions: Vec<ProviderTx<Self::Provider>>,
transactions: Vec<Recovered<ProviderTx<Self::Provider>>>,
results: Vec<ExecutionResult<HaltReasonFor<Self::Evm>>>,
) -> (ProviderBlock<Self::Provider>, Vec<ProviderReceipt<Self::Provider>>) {
let mut cumulative_gas_used = 0;
@ -267,7 +267,6 @@ pub trait LoadPendingBlock:
let base_fee = evm_env.block_env.basefee;
let mut executed_txs = Vec::new();
let mut senders = Vec::new();
let mut best_txs =
self.pool().best_transactions_with_attributes(BestTransactionsAttributes::new(
base_fee,
@ -336,7 +335,7 @@ pub trait LoadPendingBlock:
}
}
let tx_env = self.evm_config().tx_env(tx.tx(), tx.signer());
let tx_env = self.evm_config().tx_env(&tx);
let ResultAndState { result, state: _ } = match evm.transact_commit(tx_env) {
Ok(res) => res,
@ -377,9 +376,7 @@ pub trait LoadPendingBlock:
cumulative_gas_used += gas_used;
// append transaction to the list of executed transactions
let (tx, sender) = tx.into_parts();
executed_txs.push(tx);
senders.push(sender);
results.push(result);
}
@ -405,6 +402,8 @@ pub trait LoadPendingBlock:
// calculate the state root
let state_root = db.database.state_root(hashed_state).map_err(Self::Error::from_eth_err)?;
let senders = executed_txs.iter().map(|tx| tx.signer()).collect();
let (block, receipts) = self.assemble_block_and_receipts(
&evm_env.block_env,
parent_hash,

View File

@ -204,14 +204,14 @@ pub trait Trace:
let this = self.clone();
self.spawn_with_state_at_block(parent_block.into(), move |state| {
let mut db = CacheDB::new(StateProviderDatabase::new(state));
let block_txs = block.transactions_with_sender();
let block_txs = block.transactions_recovered();
this.apply_pre_execution_changes(&block, &mut db, &evm_env)?;
// replay all transactions prior to the targeted transaction
this.replay_transactions_until(&mut db, evm_env.clone(), block_txs, *tx.tx_hash())?;
let tx_env = this.evm_config().tx_env(tx.tx(), tx.signer());
let tx_env = this.evm_config().tx_env(tx);
let (res, _) = this.inspect(
StateCacheDbRefMutWrapper(&mut db),
evm_env,
@ -339,10 +339,10 @@ pub trait Trace:
let mut results = Vec::with_capacity(max_transactions);
let mut transactions = block
.transactions_with_sender()
.transactions_recovered()
.take(max_transactions)
.enumerate()
.map(|(idx, (signer, tx))| {
.map(|(idx, tx)| {
let tx_info = TransactionInfo {
hash: Some(*tx.tx_hash()),
index: Some(idx as u64),
@ -350,7 +350,7 @@ pub trait Trace:
block_number: Some(block_number),
base_fee: Some(base_fee),
};
let tx_env = this.evm_config().tx_env(tx, *signer);
let tx_env = this.evm_config().tx_env(tx);
(tx_info, tx_env)
})
.peekable();

View File

@ -7,7 +7,7 @@ use alloy_rpc_types_eth::{
Block, BlockTransactionsKind, Header,
};
use jsonrpsee_types::ErrorObject;
use reth_primitives::RecoveredBlock;
use reth_primitives::{Recovered, RecoveredBlock};
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};
@ -60,7 +60,7 @@ pub fn resolve_transaction<DB: Database, Tx, T: TransactionCompat<Tx>>(
chain_id: u64,
db: &mut DB,
tx_resp_builder: &T,
) -> Result<Tx, EthApiError>
) -> Result<Recovered<Tx>, EthApiError>
where
EthApiError: From<DB::Error>,
{
@ -108,7 +108,11 @@ where
}
}
tx_resp_builder.build_simulate_v1_transaction(tx).map_err(|e| EthApiError::other(e.into()))
let tx = tx_resp_builder
.build_simulate_v1_transaction(tx)
.map_err(|e| EthApiError::other(e.into()))?;
Ok(Recovered::new_unchecked(tx, from))
}
/// Handles outputs of the calls execution and builds a [`SimulatedBlock`].

View File

@ -109,12 +109,12 @@ where
this.eth_api().apply_pre_execution_changes(&block, &mut db, &evm_env)?;
let mut transactions = block.transactions_with_sender().enumerate().peekable();
let mut transactions = block.transactions_recovered().enumerate().peekable();
let mut inspector = None;
while let Some((index, (signer, tx))) = transactions.next() {
while let Some((index, tx)) = transactions.next() {
let tx_hash = *tx.tx_hash();
let tx_env = this.eth_api().evm_config().tx_env(tx, *signer);
let tx_env = this.eth_api().evm_config().tx_env(tx);
let (result, state_changes) = this.trace_transaction(
&opts,
@ -229,7 +229,7 @@ where
let this = self.clone();
self.eth_api()
.spawn_with_state_at_block(state_at, move |state| {
let block_txs = block.transactions_with_sender();
let block_txs = block.transactions_recovered();
// configure env for the target transaction
let tx = transaction.into_recovered();
@ -246,7 +246,7 @@ where
*tx.tx_hash(),
)?;
let tx_env = this.eth_api().evm_config().tx_env(tx.tx(), tx.signer());
let tx_env = this.eth_api().evm_config().tx_env(&tx);
this.trace_transaction(
&opts,
@ -530,11 +530,11 @@ where
if replay_block_txs {
// only need to replay the transactions in the block if not all transactions are
// to be replayed
let transactions = block.transactions_with_sender().take(num_txs);
let transactions = block.transactions_recovered().take(num_txs);
// Execute all transactions until index
for (signer, tx) in transactions {
let tx_env = this.eth_api().evm_config().tx_env(tx, *signer);
for tx in transactions {
let tx_env = this.eth_api().evm_config().tx_env(tx);
let (res, _) = this.eth_api().transact(&mut db, evm_env.clone(), tx_env)?;
db.commit(res.state);
}

View File

@ -176,7 +176,7 @@ where
hasher.update(*tx.tx_hash());
let gas_price = tx.effective_gas_price(basefee);
let ResultAndState { result, state } = evm
.transact(eth_api.evm_config().tx_env(&tx, signer))
.transact(eth_api.evm_config().tx_env(&tx))
.map_err(Eth::Error::from_evm_err)?;
let gas_used = result.gas_used();

View File

@ -1,6 +1,9 @@
//! Support for building a pending block with transactions from local view of mempool.
use alloy_consensus::{constants::EMPTY_WITHDRAWALS, Header, Transaction, EMPTY_OMMER_ROOT_HASH};
use alloy_consensus::{
constants::EMPTY_WITHDRAWALS, transaction::Recovered, Header, Transaction,
EMPTY_OMMER_ROOT_HASH,
};
use alloy_eips::{eip7685::EMPTY_REQUESTS_HASH, merge::BEACON_NONCE};
use alloy_primitives::U256;
use reth_chainspec::{EthChainSpec, EthereumHardforks};
@ -61,7 +64,7 @@ where
block_env: &BlockEnv,
parent_hash: revm_primitives::B256,
state_root: revm_primitives::B256,
transactions: Vec<ProviderTx<Self::Provider>>,
transactions: Vec<Recovered<ProviderTx<Self::Provider>>>,
receipts: &[ProviderReceipt<Self::Provider>],
) -> reth_provider::ProviderBlock<Self::Provider> {
let chain_spec = self.provider().chain_spec();
@ -105,7 +108,11 @@ where
// seal the block
reth_primitives::Block {
header,
body: BlockBody { transactions, ommers: vec![], withdrawals: None },
body: BlockBody {
transactions: transactions.into_iter().map(|tx| tx.into_tx()).collect(),
ommers: vec![],
withdrawals: None,
},
}
}

View File

@ -10,6 +10,7 @@ use alloy_rpc_types_mev::{
};
use jsonrpsee::core::RpcResult;
use reth_evm::{ConfigureEvm, ConfigureEvmEnv, Evm};
use reth_primitives::Recovered;
use reth_provider::ProviderTx;
use reth_revm::{database::StateProviderDatabase, db::CacheDB};
use reth_rpc_api::MevSimApiServer;
@ -21,9 +22,8 @@ use reth_rpc_eth_types::{
revm_utils::apply_block_overrides, utils::recover_raw_transaction, EthApiError,
};
use reth_tasks::pool::BlockingTaskGuard;
use reth_transaction_pool::{PoolConsensusTx, PoolPooledTx, PoolTransaction, TransactionPool};
use reth_transaction_pool::{PoolPooledTx, PoolTransaction, TransactionPool};
use revm::{context_interface::result::ResultAndState, DatabaseCommit, DatabaseRef};
use revm_primitives::Address;
use std::{sync::Arc, time::Duration};
use tracing::info;
@ -46,9 +46,7 @@ const SBUNDLE_PAYOUT_MAX_COST: u64 = 30_000;
#[derive(Clone, Debug)]
pub struct FlattenedBundleItem<T> {
/// The signed transaction
pub tx: T,
/// The address that signed the transaction
pub signer: Address,
pub tx: Recovered<T>,
/// Whether the transaction is allowed to revert
pub can_revert: bool,
/// Item-level inclusion constraints
@ -169,10 +167,10 @@ where
while idx < body.len() {
match &body[idx] {
BundleItem::Tx { tx, can_revert } => {
let recovered_tx = recover_raw_transaction::<PoolPooledTx<Eth::Pool>>(tx)?;
let (tx, signer) = recovered_tx.into_parts();
let tx: PoolConsensusTx<Eth::Pool> =
<Eth::Pool as TransactionPool>::Transaction::pooled_into_consensus(tx);
let tx = recover_raw_transaction::<PoolPooledTx<Eth::Pool>>(tx)?;
let tx = tx.map_transaction(
<Eth::Pool as TransactionPool>::Transaction::pooled_into_consensus,
);
let refund_percent =
validity.as_ref().and_then(|v| v.refund.as_ref()).and_then(|refunds| {
@ -186,7 +184,6 @@ where
// Create FlattenedBundleItem with current inclusion, validity, and privacy
let flattened_item = FlattenedBundleItem {
tx,
signer,
can_revert: *can_revert,
inclusion: inclusion.clone(),
validity: validity.clone(),
@ -282,7 +279,7 @@ where
}
let ResultAndState { result, state } = evm
.transact(eth_api.evm_config().tx_env(&item.tx, item.signer))
.transact(eth_api.evm_config().tx_env(&item.tx))
.map_err(Eth::Error::from_evm_err)?;
if !result.is_success() && !item.can_revert {
@ -330,7 +327,7 @@ where
if let Some(refund_percent) = item.refund_percent {
// Get refund configurations
let refund_configs = item.refund_configs.clone().unwrap_or_else(|| {
vec![RefundConfig { address: item.signer, percent: 100 }]
vec![RefundConfig { address: item.tx.signer(), percent: 100 }]
});
// Calculate payout transaction fee

View File

@ -113,7 +113,7 @@ where
.map_transaction(<Eth::Pool as TransactionPool>::Transaction::pooled_into_consensus);
let (evm_env, at) = self.eth_api().evm_env_at(block_id.unwrap_or_default()).await?;
let tx_env = self.eth_api().evm_config().tx_env(tx.tx(), tx.signer());
let tx_env = self.eth_api().evm_config().tx_env(tx);
let config = TracingInspectorConfig::from_parity_config(&trace_types);