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

@ -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,