feat: re-use BlockExecutionStrategy in payload building (#14609)

This commit is contained in:
Arsenii Kulikov
2025-02-20 19:06:19 +04:00
committed by GitHub
parent 09cf07d523
commit 72210736ad
15 changed files with 352 additions and 492 deletions

View File

@ -6,7 +6,7 @@ use crate::{
};
use alloc::{boxed::Box, sync::Arc, vec::Vec};
use alloy_consensus::{
transaction::Recovered, BlockHeader, Eip658Value, Receipt, Transaction as _, TxReceipt,
transaction::Recovered, BlockHeader, Eip658Value, Header, Receipt, Transaction as _, TxReceipt,
};
use op_alloy_consensus::OpDepositReceipt;
use reth_evm::{
@ -22,9 +22,12 @@ use reth_execution_types::BlockExecutionResult;
use reth_optimism_chainspec::OpChainSpec;
use reth_optimism_forks::OpHardforks;
use reth_optimism_primitives::{transaction::signed::OpTransaction, DepositReceipt, OpPrimitives};
use reth_primitives_traits::{NodePrimitives, RecoveredBlock, SealedBlock, SignedTransaction};
use reth_primitives_traits::{
Block, NodePrimitives, RecoveredBlock, SealedBlock, SignedTransaction,
};
use revm::{context_interface::result::ResultAndState, DatabaseCommit};
use revm_database::State;
use revm_primitives::{Address, B256};
use tracing::trace;
/// Factory for [`OpExecutionStrategy`].
@ -89,19 +92,59 @@ where
DB: Database,
{
let evm = self.evm_config.evm_for_block(db, block.header());
OpExecutionStrategy::new(evm, block.sealed_block(), self)
OpExecutionStrategy::new(
evm,
block.sealed_block(),
&self.chain_spec,
&self.evm_config,
self.receipt_builder.as_ref(),
)
}
}
/// Input for block execution.
#[derive(Debug, Clone, Copy)]
pub struct OpBlockExecutionInput {
/// Block number.
pub number: u64,
/// Block timestamp.
pub timestamp: u64,
/// Parent block hash.
pub parent_hash: B256,
/// Block gas limit.
pub gas_limit: u64,
/// Parent beacon block root.
pub parent_beacon_block_root: Option<B256>,
/// Block beneficiary.
pub beneficiary: Address,
}
impl<'a, B: Block> From<&'a SealedBlock<B>> for OpBlockExecutionInput {
fn from(block: &'a SealedBlock<B>) -> Self {
Self {
number: block.header().number(),
timestamp: block.header().timestamp(),
parent_hash: block.header().parent_hash(),
gas_limit: block.header().gas_limit(),
parent_beacon_block_root: block.header().parent_beacon_block_root(),
beneficiary: block.header().beneficiary(),
}
}
}
/// Block execution strategy for Optimism.
#[derive(Debug, derive_more::Deref)]
#[derive(Debug)]
pub struct OpExecutionStrategy<'a, Evm, N: NodePrimitives, ChainSpec, EvmConfig: ConfigureEvm> {
/// Reference to the parent factory.
#[deref]
factory: &'a OpExecutionStrategyFactory<N, ChainSpec, EvmConfig>,
/// Chainspec.
chain_spec: ChainSpec,
/// How to configure the EVM.
evm_config: EvmConfig,
/// Receipt builder.
receipt_builder:
&'a dyn OpReceiptBuilder<N::SignedTx, HaltReasonFor<EvmConfig>, Receipt = N::Receipt>,
/// Block being executed.
block: &'a SealedBlock<N::Block>,
/// Input for block execution.
input: OpBlockExecutionInput,
/// The EVM used by strategy.
evm: Evm,
/// Receipts of executed transactions.
@ -111,7 +154,7 @@ pub struct OpExecutionStrategy<'a, Evm, N: NodePrimitives, ChainSpec, EvmConfig:
/// Whether Regolith hardfork is active.
is_regolith: bool,
/// Utility to call system smart contracts.
system_caller: SystemCaller<&'a ChainSpec>,
system_caller: SystemCaller<ChainSpec>,
}
impl<'a, Evm, N, ChainSpec, EvmConfig> OpExecutionStrategy<'a, Evm, N, ChainSpec, EvmConfig>
@ -123,17 +166,26 @@ where
/// Creates a new [`OpExecutionStrategy`]
pub fn new(
evm: Evm,
block: &'a SealedBlock<N::Block>,
factory: &'a OpExecutionStrategyFactory<N, ChainSpec, EvmConfig>,
input: impl Into<OpBlockExecutionInput>,
chain_spec: ChainSpec,
evm_config: EvmConfig,
receipt_builder: &'a dyn OpReceiptBuilder<
N::SignedTx,
HaltReasonFor<EvmConfig>,
Receipt = N::Receipt,
>,
) -> Self {
let input = input.into();
Self {
is_regolith: chain_spec.is_regolith_active_at_timestamp(input.timestamp),
evm,
factory,
block,
system_caller: SystemCaller::new(chain_spec.clone()),
chain_spec,
evm_config,
receipt_builder,
receipts: Vec::new(),
gas_used: 0,
is_regolith: factory.chain_spec.is_regolith_active_at_timestamp(block.timestamp()),
system_caller: SystemCaller::new(&factory.chain_spec),
input,
}
}
}
@ -153,28 +205,26 @@ where
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(self.block.number());
self.chain_spec.is_spurious_dragon_active_at_block(self.input.number);
self.evm.db_mut().set_state_clear_flag(state_clear_flag);
self.system_caller.apply_beacon_root_contract_call(
self.block.parent_beacon_block_root(),
&mut self.evm,
)?;
self.system_caller
.apply_beacon_root_contract_call(self.input.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(), self.block.timestamp(), self.evm.db_mut())
ensure_create2_deployer(self.chain_spec.clone(), self.input.timestamp, self.evm.db_mut())
.map_err(|_| OpBlockExecutionError::ForceCreate2DeployerFail)?;
Ok(())
}
fn execute_transaction(&mut self, tx: Recovered<&N::SignedTx>) -> Result<(), Self::Error> {
fn execute_transaction(&mut self, tx: Recovered<&N::SignedTx>) -> Result<u64, Self::Error> {
// 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 = self.block.gas_limit() - self.gas_used;
let block_available_gas = self.input.gas_limit - self.gas_used;
if tx.gas_limit() > block_available_gas && (self.is_regolith || !tx.is_deposit()) {
return Err(BlockValidationError::TransactionGasLimitMoreThanAvailableBlockGas {
transaction_gas_limit: tx.gas_limit(),
@ -215,8 +265,10 @@ where
let ResultAndState { result, state } = result_and_state;
self.evm.db_mut().commit(state);
let gas_used = result.gas_used();
// append gas used
self.gas_used += result.gas_used();
self.gas_used += gas_used;
self.receipts.push(
match self.receipt_builder.build_receipt(ReceiptBuilderCtx {
@ -243,22 +295,25 @@ where
// this is only set for post-Canyon deposit
// transactions.
deposit_receipt_version: (tx.is_deposit() &&
self.chain_spec
.is_canyon_active_at_timestamp(self.block.timestamp()))
self.chain_spec.is_canyon_active_at_timestamp(self.input.timestamp))
.then_some(1),
})
}
},
);
Ok(())
Ok(gas_used)
}
fn apply_post_execution_changes(
mut self,
) -> Result<BlockExecutionResult<N::Receipt>, Self::Error> {
let balance_increments =
post_block_balance_increments(&self.chain_spec.clone(), self.block);
let balance_increments = post_block_balance_increments::<Header>(
&self.chain_spec.clone(),
self.evm.block(),
&[],
None,
);
// increment balances
self.evm
.db_mut()

View File

@ -7,26 +7,24 @@ use crate::{
OpPayloadPrimitives,
};
use alloy_consensus::{
constants::EMPTY_WITHDRAWALS, Eip658Value, Header, Transaction, Typed2718,
EMPTY_OMMER_ROOT_HASH,
constants::EMPTY_WITHDRAWALS, Header, Transaction, Typed2718, EMPTY_OMMER_ROOT_HASH,
};
use alloy_eips::{eip4895::Withdrawals, merge::BEACON_NONCE};
use alloy_primitives::{Address, Bytes, B256, U256};
use alloy_rlp::Encodable;
use alloy_rpc_types_debug::ExecutionWitness;
use alloy_rpc_types_engine::PayloadId;
use op_alloy_consensus::OpDepositReceipt;
use op_alloy_rpc_types_engine::OpPayloadAttributes;
use reth_basic_payload_builder::*;
use reth_chain_state::{ExecutedBlock, ExecutedBlockWithTrieUpdates};
use reth_chainspec::{ChainSpecProvider, EthChainSpec, EthereumHardforks};
use reth_evm::{
system_calls::SystemCaller, ConfigureEvm, ConfigureEvmFor, Database, Evm, EvmEnv, EvmError,
HaltReasonFor, InvalidTxError, NextBlockEnvAttributes,
execute::{BlockExecutionError, BlockExecutionStrategy, BlockValidationError},
ConfigureEvm, ConfigureEvmFor, Database, EvmEnv, HaltReasonFor, NextBlockEnvAttributes,
};
use reth_execution_types::ExecutionOutcome;
use reth_optimism_consensus::calculate_receipt_root_no_memo_optimism;
use reth_optimism_evm::{OpReceiptBuilder, ReceiptBuilderCtx};
use reth_optimism_evm::{OpBlockExecutionInput, OpExecutionStrategy, OpReceiptBuilder};
use reth_optimism_forks::OpHardforks;
use reth_optimism_primitives::transaction::signed::OpTransaction;
use reth_optimism_storage::predeploys;
@ -38,8 +36,8 @@ use reth_primitives::{
};
use reth_primitives_traits::{block::Block as _, proofs, RecoveredBlock};
use reth_provider::{
HashedPostStateProvider, ProviderError, StateProofProvider, StateProviderFactory,
StateRootProvider, StorageRootProvider,
BlockExecutionResult, HashedPostStateProvider, ProviderError, StateProofProvider,
StateProviderFactory, StateRootProvider, StorageRootProvider,
};
use reth_revm::{
cancelled::CancelOnDrop,
@ -48,14 +46,8 @@ use reth_revm::{
witness::ExecutionWitnessRecord,
};
use reth_transaction_pool::{BestTransactionsAttributes, PoolTransaction, TransactionPool};
use revm::{
context_interface::{
result::{ExecutionResult, ResultAndState},
Block,
},
Database as _, DatabaseCommit,
};
use std::{fmt::Display, sync::Arc};
use revm::context_interface::Block;
use std::sync::Arc;
use tracing::{debug, trace, warn};
/// Optimism's payload builder
@ -366,21 +358,34 @@ impl<Txs> OpBuilder<'_, Txs> {
let Self { best } = self;
debug!(target: "payload_builder", id=%ctx.payload_id(), parent_header = ?ctx.parent().hash(), parent_number = ctx.parent().number, "building new payload");
let mut evm = ctx.evm_config.evm_with_env(&mut *state, ctx.evm_env.clone());
let mut strategy = OpExecutionStrategy::new(
ctx.evm_config.evm_with_env(&mut *state, ctx.evm_env.clone()),
OpBlockExecutionInput {
number: ctx.evm_env.block_env.number,
timestamp: ctx.evm_env.block_env.timestamp,
parent_hash: ctx.parent().hash(),
gas_limit: ctx.evm_env.block_env.gas_limit,
parent_beacon_block_root: ctx.attributes().parent_beacon_block_root(),
beneficiary: ctx.evm_env.block_env.beneficiary,
},
&ctx.chain_spec,
&ctx.evm_config,
ctx.receipt_builder.as_ref(),
);
// 1. apply eip-4788 pre block contract call
ctx.apply_pre_beacon_root_contract_call(&mut evm)?;
// 1. apply pre-execution changes
strategy.apply_pre_execution_changes().map_err(|err| {
warn!(target: "payload_builder", %err, "failed to apply pre-execution changes");
PayloadBuilderError::Internal(err.into())
})?;
// 2. ensure create2deployer is force deployed
ctx.ensure_create2_deployer(evm.db_mut())?;
// 2. execute sequencer transactions
let mut info = ctx.execute_sequencer_transactions(&mut strategy)?;
// 3. execute sequencer transactions
let mut info = ctx.execute_sequencer_transactions(&mut evm)?;
// 4. if mem pool transactions are requested we execute them
// 3. if mem pool transactions are requested we execute them
if !ctx.attributes().no_tx_pool {
let best_txs = best(ctx.best_transaction_attributes());
if ctx.execute_best_transactions(&mut info, &mut evm, best_txs)?.is_some() {
if ctx.execute_best_transactions(&mut info, &mut strategy, best_txs)?.is_some() {
return Ok(BuildOutcomeKind::Cancelled)
}
@ -391,7 +396,9 @@ impl<Txs> OpBuilder<'_, Txs> {
}
}
drop(evm);
let BlockExecutionResult { receipts, .. } = strategy
.apply_post_execution_changes()
.map_err(|err| PayloadBuilderError::Internal(err.into()))?;
// merge all transitions into bundle state, this would apply the withdrawal balance changes
// and 4788 contract call
@ -407,7 +414,7 @@ impl<Txs> OpBuilder<'_, Txs> {
None
};
let payload = ExecutedPayload { info, withdrawals_root };
let payload = ExecutedPayload { receipts, info, withdrawals_root };
Ok(BuildOutcomeKind::Better { payload })
}
@ -426,19 +433,16 @@ impl<Txs> OpBuilder<'_, Txs> {
DB: Database<Error = ProviderError> + AsRef<P>,
P: StateRootProvider + HashedPostStateProvider + StorageRootProvider,
{
let ExecutedPayload { info, withdrawals_root } = match self.execute(&mut state, &ctx)? {
BuildOutcomeKind::Better { payload } | BuildOutcomeKind::Freeze(payload) => payload,
BuildOutcomeKind::Cancelled => return Ok(BuildOutcomeKind::Cancelled),
BuildOutcomeKind::Aborted { fees } => return Ok(BuildOutcomeKind::Aborted { fees }),
};
let ExecutedPayload { receipts, info, withdrawals_root } =
match self.execute(&mut state, &ctx)? {
BuildOutcomeKind::Better { payload } | BuildOutcomeKind::Freeze(payload) => payload,
BuildOutcomeKind::Cancelled => return Ok(BuildOutcomeKind::Cancelled),
BuildOutcomeKind::Aborted { fees } => return Ok(BuildOutcomeKind::Aborted { fees }),
};
let block_number = ctx.block_number();
let execution_outcome = ExecutionOutcome::new(
state.take_bundle(),
vec![info.receipts],
block_number,
Vec::new(),
);
let execution_outcome =
ExecutionOutcome::new(state.take_bundle(), vec![receipts], block_number, Vec::new());
let receipts_root = execution_outcome
.generic_receipts_root_slow(block_number, |receipts| {
calculate_receipt_root_no_memo_optimism(
@ -588,6 +592,8 @@ pub struct ExecutedPayload<N: NodePrimitives> {
pub info: ExecutionInfo<N>,
/// Withdrawal hash.
pub withdrawals_root: Option<B256>,
/// The transaction receipts.
pub receipts: Vec<N::Receipt>,
}
/// This acts as the container for executed transactions and its byproducts (receipts, gas used)
@ -597,8 +603,6 @@ pub struct ExecutionInfo<N: NodePrimitives> {
pub executed_transactions: Vec<N::SignedTx>,
/// The recovered senders for the executed transactions.
pub executed_senders: Vec<Address>,
/// The transaction receipts
pub receipts: Vec<N::Receipt>,
/// All gas used so far
pub cumulative_gas_used: u64,
/// Estimated DA size
@ -613,7 +617,6 @@ impl<N: NodePrimitives> ExecutionInfo<N> {
Self {
executed_transactions: Vec::with_capacity(capacity),
executed_senders: Vec::with_capacity(capacity),
receipts: Vec::with_capacity(capacity),
cumulative_gas_used: 0,
cumulative_da_bytes_used: 0,
total_fees: U256::ZERO,
@ -787,26 +790,6 @@ where
pub fn is_better_payload(&self, total_fees: U256) -> bool {
is_better_payload(self.best_payload.as_ref(), total_fees)
}
/// 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.
pub fn ensure_create2_deployer<DB>(&self, db: &mut State<DB>) -> Result<(), PayloadBuilderError>
where
DB: Database,
DB::Error: Display,
{
reth_optimism_evm::ensure_create2_deployer(
self.chain_spec.clone(),
self.attributes().payload_attributes.timestamp,
db,
)
.map_err(|err| {
warn!(target: "payload_builder", %err, "missing create2 deployer, skipping block.");
PayloadBuilderError::other(OpPayloadBuilderError::ForceCreate2DeployerFail)
})
}
}
impl<EvmConfig, ChainSpec, N> OpPayloadBuilderCtx<EvmConfig, ChainSpec, N>
@ -815,73 +798,10 @@ where
ChainSpec: EthChainSpec + OpHardforks,
N: OpPayloadPrimitives,
{
/// apply eip-4788 pre block contract call
pub fn apply_pre_beacon_root_contract_call(
&self,
evm: &mut impl Evm<DB: DatabaseCommit>,
) -> Result<(), PayloadBuilderError> {
SystemCaller::new(&self.chain_spec)
.apply_beacon_root_contract_call(
self.attributes().payload_attributes.parent_beacon_block_root,
evm,
)
.map_err(|err| {
warn!(target: "payload_builder",
parent_header=%self.parent().hash(),
%err,
"failed to apply beacon root contract call for payload"
);
PayloadBuilderError::Internal(err.into())
})?;
Ok(())
}
/// Constructs a receipt for the given transaction.
fn build_receipt(
&self,
info: &ExecutionInfo<N>,
result: ExecutionResult<HaltReasonFor<EvmConfig>>,
deposit_nonce: Option<u64>,
tx: &N::SignedTx,
) -> N::Receipt {
match self.receipt_builder.build_receipt(ReceiptBuilderCtx {
tx,
result,
cumulative_gas_used: info.cumulative_gas_used,
}) {
Ok(receipt) => receipt,
Err(ctx) => {
let receipt = alloy_consensus::Receipt {
// Success flag was added in `EIP-658: Embedding transaction status code
// in receipts`.
status: Eip658Value::Eip658(ctx.result.is_success()),
cumulative_gas_used: ctx.cumulative_gas_used,
logs: ctx.result.into_logs(),
};
self.receipt_builder.build_deposit_receipt(OpDepositReceipt {
inner: receipt,
deposit_nonce,
// The deposit receipt version was introduced in Canyon to indicate an
// update to how receipt hashes should be computed
// when set. The state transition process ensures
// this is only set for post-Canyon deposit
// transactions.
deposit_receipt_version: self.is_canyon_active().then_some(1),
})
}
}
}
/// Executes all sequencer transactions that are included in the payload attributes.
pub fn execute_sequencer_transactions(
&self,
evm: &mut impl Evm<
DB: Database<Error = ProviderError> + DatabaseCommit,
Tx = EvmConfig::TxEnv,
HaltReason = HaltReasonFor<EvmConfig>,
>,
strategy: &mut impl BlockExecutionStrategy<Primitives = N, Error = BlockExecutionError>,
) -> Result<ExecutionInfo<N>, PayloadBuilderError> {
let mut info = ExecutionInfo::with_capacity(self.attributes().transactions.len());
@ -901,51 +821,24 @@ where
PayloadBuilderError::other(OpPayloadBuilderError::TransactionEcRecoverFailed)
})?;
// Cache the depositor account prior to the state transition for the deposit nonce.
//
// 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_nonce = (self.is_regolith_active() && sequencer_tx.is_deposit())
.then(|| {
evm.db_mut()
.basic(sequencer_tx.signer())
.map(|acc| acc.unwrap_or_default().nonce)
})
.transpose()
.map_err(|_| {
PayloadBuilderError::other(OpPayloadBuilderError::AccountLoadFailed(
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,
let gas_used = match strategy.execute_transaction(sequencer_tx.as_recovered_ref()) {
Ok(gas_used) => gas_used,
Err(BlockExecutionError::Validation(BlockValidationError::InvalidTx {
error,
..
})) => {
trace!(target: "payload_builder", %error, ?sequencer_tx, "Error in sequencer transaction, skipping.");
continue
}
Err(err) => {
if err.is_invalid_tx_err() {
trace!(target: "payload_builder", %err, ?sequencer_tx, "Error in sequencer transaction, skipping.");
continue
}
// this is an error that we should treat as fatal for this attempt
return Err(PayloadBuilderError::EvmExecutionError(Box::new(err)))
}
};
let gas_used = result.gas_used();
// add gas used by the transaction to cumulative gas used, before creating the receipt
info.cumulative_gas_used += gas_used;
// Push transaction changeset and calculate header bloom filter for receipt.
info.receipts.push(self.build_receipt(
&info,
result,
depositor_nonce,
sequencer_tx.tx(),
));
// append sender and transaction to the respective lists
info.executed_senders.push(sequencer_tx.signer());
info.executed_transactions.push(sequencer_tx.into_tx());
@ -960,11 +853,7 @@ where
pub fn execute_best_transactions(
&self,
info: &mut ExecutionInfo<N>,
evm: &mut impl Evm<
DB: DatabaseCommit,
Tx = EvmConfig::TxEnv,
HaltReason = HaltReasonFor<EvmConfig>,
>,
strategy: &mut impl BlockExecutionStrategy<Primitives = N, Error = BlockExecutionError>,
mut best_txs: impl PayloadTransactions<
Transaction: PoolTransaction<Consensus = EvmConfig::Transaction>,
>,
@ -995,40 +884,34 @@ where
return Ok(Some(()))
}
// Configure the environment for the tx.
let tx_env = self.evm_config.tx_env(&tx);
let ResultAndState { result, state: _ } = match evm.transact_commit(tx_env) {
Ok(res) => res,
Err(err) => {
if let Some(err) = err.as_invalid_tx_err() {
if err.is_nonce_too_low() {
// if the nonce is too low, we can skip this transaction
trace!(target: "payload_builder", %err, ?tx, "skipping nonce too low transaction");
} else {
// if the transaction is invalid, we can skip it and all of its
// descendants
trace!(target: "payload_builder", %err, ?tx, "skipping invalid transaction and its descendants");
best_txs.mark_invalid(tx.signer(), tx.nonce());
}
continue
let gas_used = match strategy.execute_transaction(tx.as_recovered_ref()) {
Ok(gas_used) => gas_used,
Err(BlockExecutionError::Validation(BlockValidationError::InvalidTx {
error,
..
})) => {
if error.is_nonce_too_low() {
// if the nonce is too low, we can skip this transaction
trace!(target: "payload_builder", %error, ?tx, "skipping nonce too low transaction");
} else {
// if the transaction is invalid, we can skip it and all of its
// descendants
trace!(target: "payload_builder", %error, ?tx, "skipping invalid transaction and its descendants");
best_txs.mark_invalid(tx.signer(), tx.nonce());
}
continue
}
Err(err) => {
// this is an error that we should treat as fatal for this attempt
return Err(PayloadBuilderError::EvmExecutionError(Box::new(err)))
}
};
let gas_used = result.gas_used();
// add gas used by the transaction to cumulative gas used, before creating the
// receipt
info.cumulative_gas_used += gas_used;
info.cumulative_da_bytes_used += tx.length() as u64;
// Push transaction changeset and calculate header bloom filter for receipt.
info.receipts.push(self.build_receipt(info, result, None, &tx));
// update add to total fees
let miner_fee = tx
.effective_tip_per_gas(base_fee)