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

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