mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
refactor: BlockExecutionStrategy API (#14480)
This commit is contained in:
@ -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 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 &&
|
||||
(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.
|
||||
|
||||
Reference in New Issue
Block a user