feat: add receipt builder for OpExecutionStrategy (#13792)

This commit is contained in:
Arsenii Kulikov
2025-01-14 17:11:18 +04:00
committed by GitHub
parent feccf3595b
commit b4610a04e6
12 changed files with 278 additions and 107 deletions

View File

@ -1,11 +1,14 @@
//! Optimism block execution strategy.
use crate::{l1::ensure_create2_deployer, OpBlockExecutionError, OpEvmConfig};
use crate::{
l1::ensure_create2_deployer, BasicOpReceiptBuilder, OpBlockExecutionError, OpEvmConfig,
OpReceiptBuilder, ReceiptBuilderCtx,
};
use alloc::{boxed::Box, sync::Arc, vec::Vec};
use alloy_consensus::{Eip658Value, Receipt, Transaction as _};
use alloy_eips::eip7685::Requests;
use core::fmt::Display;
use op_alloy_consensus::{OpDepositReceipt, OpTxType};
use op_alloy_consensus::{DepositTransaction, OpDepositReceipt};
use reth_chainspec::EthereumHardforks;
use reth_consensus::ConsensusError;
use reth_evm::{
@ -21,48 +24,63 @@ use reth_evm::{
use reth_optimism_chainspec::OpChainSpec;
use reth_optimism_consensus::validate_block_post_execution;
use reth_optimism_forks::OpHardfork;
use reth_optimism_primitives::{OpBlock, OpPrimitives, OpReceipt, OpTransactionSigned};
use reth_primitives::BlockWithSenders;
use reth_primitives_traits::SignedTransaction;
use reth_optimism_primitives::{DepositReceipt, OpPrimitives, OpReceipt};
use reth_primitives::{BlockWithSenders, NodePrimitives};
use reth_primitives_traits::{Block, BlockBody, SignedTransaction};
use reth_revm::{Database, State};
use revm_primitives::{db::DatabaseCommit, ResultAndState};
use tracing::trace;
/// Factory for [`OpExecutionStrategy`].
#[derive(Debug, Clone)]
pub struct OpExecutionStrategyFactory<EvmConfig = OpEvmConfig> {
pub struct OpExecutionStrategyFactory<N: NodePrimitives = OpPrimitives, EvmConfig = OpEvmConfig> {
/// The chainspec
chain_spec: Arc<OpChainSpec>,
/// How to create an EVM.
evm_config: EvmConfig,
/// Receipt builder.
receipt_builder: Arc<dyn OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>>,
}
impl OpExecutionStrategyFactory {
impl OpExecutionStrategyFactory<OpPrimitives> {
/// Creates a new default optimism executor strategy factory.
pub fn optimism(chain_spec: Arc<OpChainSpec>) -> Self {
Self::new(chain_spec.clone(), OpEvmConfig::new(chain_spec))
Self::new(
chain_spec.clone(),
OpEvmConfig::new(chain_spec),
BasicOpReceiptBuilder::default(),
)
}
}
impl<EvmConfig> OpExecutionStrategyFactory<EvmConfig> {
impl<N: NodePrimitives, EvmConfig> OpExecutionStrategyFactory<N, EvmConfig> {
/// Creates a new executor strategy factory.
pub const fn new(chain_spec: Arc<OpChainSpec>, evm_config: EvmConfig) -> Self {
Self { chain_spec, evm_config }
pub fn new(
chain_spec: Arc<OpChainSpec>,
evm_config: EvmConfig,
receipt_builder: impl OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>,
) -> Self {
Self { chain_spec, evm_config, receipt_builder: Arc::new(receipt_builder) }
}
}
impl<EvmConfig> BlockExecutionStrategyFactory for OpExecutionStrategyFactory<EvmConfig>
impl<N, EvmConfig> BlockExecutionStrategyFactory for OpExecutionStrategyFactory<N, EvmConfig>
where
N: NodePrimitives<
BlockHeader = alloy_consensus::Header,
Receipt = OpReceipt,
SignedTx: DepositTransaction,
>,
EvmConfig: Clone
+ Unpin
+ Sync
+ Send
+ 'static
+ ConfigureEvm<Header = alloy_consensus::Header, Transaction = OpTransactionSigned>,
+ ConfigureEvm<Header = N::BlockHeader, Transaction = N::SignedTx>,
{
type Primitives = OpPrimitives;
type Primitives = N;
type Strategy<DB: Database<Error: Into<ProviderError> + Display>> =
OpExecutionStrategy<DB, EvmConfig>;
OpExecutionStrategy<DB, N, EvmConfig>;
fn create_strategy<DB>(&self, db: DB) -> Self::Strategy<DB>
where
@ -70,13 +88,18 @@ where
{
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())
OpExecutionStrategy::new(
state,
self.chain_spec.clone(),
self.evm_config.clone(),
self.receipt_builder.clone(),
)
}
}
/// Block execution strategy for Optimism.
#[allow(missing_debug_implementations)]
pub struct OpExecutionStrategy<DB, EvmConfig>
pub struct OpExecutionStrategy<DB, N: NodePrimitives, EvmConfig>
where
EvmConfig: Clone,
{
@ -90,26 +113,46 @@ where
state: State<DB>,
/// Utility to call system smart contracts.
system_caller: SystemCaller<EvmConfig, OpChainSpec>,
/// Receipt builder.
receipt_builder: Arc<dyn OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>>,
}
impl<DB, EvmConfig> OpExecutionStrategy<DB, EvmConfig>
impl<DB, N, EvmConfig> OpExecutionStrategy<DB, N, EvmConfig>
where
N: NodePrimitives,
EvmConfig: Clone,
{
/// Creates a new [`OpExecutionStrategy`]
pub fn new(state: State<DB>, chain_spec: Arc<OpChainSpec>, evm_config: EvmConfig) -> Self {
pub fn new(
state: State<DB>,
chain_spec: Arc<OpChainSpec>,
evm_config: EvmConfig,
receipt_builder: Arc<dyn OpReceiptBuilder<N::SignedTx, Receipt = N::Receipt>>,
) -> Self {
let system_caller = SystemCaller::new(evm_config.clone(), chain_spec.clone());
Self { state, chain_spec, evm_config, system_caller, tx_env_overrides: None }
Self {
state,
chain_spec,
evm_config,
system_caller,
tx_env_overrides: None,
receipt_builder,
}
}
}
impl<DB, EvmConfig> BlockExecutionStrategy for OpExecutionStrategy<DB, EvmConfig>
impl<DB, N, EvmConfig> BlockExecutionStrategy for OpExecutionStrategy<DB, N, EvmConfig>
where
DB: Database<Error: Into<ProviderError> + Display>,
EvmConfig: ConfigureEvm<Header = alloy_consensus::Header, Transaction = OpTransactionSigned>,
N: NodePrimitives<
BlockHeader = alloy_consensus::Header,
SignedTx: DepositTransaction,
Receipt: DepositReceipt,
>,
EvmConfig: ConfigureEvm<Header = N::BlockHeader, Transaction = N::SignedTx>,
{
type DB = DB;
type Primitives = OpPrimitives;
type Primitives = N;
type Error = BlockExecutionError;
fn init(&mut self, tx_env_overrides: Box<dyn TxEnvOverrides>) {
@ -118,19 +161,19 @@ where
fn apply_pre_execution_changes(
&mut self,
block: &BlockWithSenders<OpBlock>,
block: &BlockWithSenders<N::Block>,
) -> 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.header.number);
(*self.chain_spec).is_spurious_dragon_active_at_block(block.header().number);
self.state.set_state_clear_flag(state_clear_flag);
let mut evm = self.evm_config.evm_for_block(&mut self.state, &block.header);
let mut evm = self.evm_config.evm_for_block(&mut self.state, block.header());
self.system_caller.apply_beacon_root_contract_call(
block.timestamp,
block.number,
block.parent_beacon_block_root,
block.header().timestamp,
block.header().number,
block.header().parent_beacon_block_root,
&mut evm,
)?;
@ -138,7 +181,7 @@ where
// 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.timestamp, evm.db_mut())
ensure_create2_deployer(self.chain_spec.clone(), block.header().timestamp, evm.db_mut())
.map_err(|_| OpBlockExecutionError::ForceCreate2DeployerFail)?;
Ok(())
@ -146,19 +189,21 @@ where
fn execute_transactions(
&mut self,
block: &BlockWithSenders<OpBlock>,
) -> Result<ExecuteOutput<OpReceipt>, Self::Error> {
let mut evm = self.evm_config.evm_for_block(&mut self.state, &block.header);
block: &BlockWithSenders<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.fork(OpHardfork::Regolith).active_at_timestamp(block.timestamp);
let is_regolith = self
.chain_spec
.fork(OpHardfork::Regolith)
.active_at_timestamp(block.header().timestamp);
let mut cumulative_gas_used = 0;
let mut receipts = Vec::with_capacity(block.body.transactions.len());
let mut receipts = Vec::with_capacity(block.body().transactions().len());
for (sender, transaction) in block.transactions_with_sender() {
// 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.header.gas_limit - cumulative_gas_used;
let block_available_gas = block.header().gas_limit - cumulative_gas_used;
if transaction.gas_limit() > block_available_gas &&
(is_regolith || !transaction.is_deposit())
{
@ -211,33 +256,41 @@ where
// append gas used
cumulative_gas_used += result.gas_used();
let receipt = Receipt {
// Success flag was added in `EIP-658: Embedding transaction status code in
// receipts`.
status: Eip658Value::Eip658(result.is_success()),
cumulative_gas_used,
logs: result.into_logs(),
};
receipts.push(
match self.receipt_builder.build_receipt(ReceiptBuilderCtx {
header: block.header(),
tx: transaction,
result,
cumulative_gas_used,
}) {
Ok(receipt) => receipt,
Err(ctx) => {
let receipt = Receipt {
// Success flag was added in `EIP-658: Embedding transaction status code
// in receipts`.
status: Eip658Value::Eip658(ctx.result.is_success()),
cumulative_gas_used,
logs: ctx.result.into_logs(),
};
// Push transaction changeset and calculate header bloom filter for receipt.
receipts.push(match transaction.tx_type() {
OpTxType::Legacy => OpReceipt::Legacy(receipt),
OpTxType::Eip2930 => OpReceipt::Eip2930(receipt),
OpTxType::Eip1559 => OpReceipt::Eip1559(receipt),
OpTxType::Eip7702 => OpReceipt::Eip7702(receipt),
OpTxType::Deposit => OpReceipt::Deposit(OpDepositReceipt {
inner: receipt,
deposit_nonce: depositor.map(|account| account.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: (transaction.is_deposit() &&
self.chain_spec
.is_fork_active_at_timestamp(OpHardfork::Canyon, block.timestamp))
.then_some(1),
}),
});
self.receipt_builder.build_deposit_receipt(OpDepositReceipt {
inner: receipt,
deposit_nonce: depositor.map(|account| account.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: (transaction.is_deposit() &&
self.chain_spec.is_fork_active_at_timestamp(
OpHardfork::Canyon,
block.header().timestamp,
))
.then_some(1),
})
}
},
);
}
Ok(ExecuteOutput { receipts, gas_used: cumulative_gas_used })
@ -245,8 +298,8 @@ where
fn apply_post_execution_changes(
&mut self,
block: &BlockWithSenders<OpBlock>,
_receipts: &[OpReceipt],
block: &BlockWithSenders<N::Block>,
_receipts: &[N::Receipt],
) -> Result<Requests, Self::Error> {
let balance_increments =
post_block_balance_increments(&self.chain_spec.clone(), &block.block);
@ -275,11 +328,11 @@ where
fn validate_block_post_execution(
&self,
block: &BlockWithSenders<OpBlock>,
receipts: &[OpReceipt],
block: &BlockWithSenders<N::Block>,
receipts: &[N::Receipt],
_requests: &Requests,
) -> Result<(), ConsensusError> {
validate_block_post_execution(block, &self.chain_spec.clone(), receipts)
validate_block_post_execution(block.header(), &self.chain_spec.clone(), receipts)
}
}
@ -291,7 +344,7 @@ impl OpExecutorProvider {
/// Creates a new default optimism executor strategy factory.
pub fn optimism(
chain_spec: Arc<OpChainSpec>,
) -> BasicBlockExecutorProvider<OpExecutionStrategyFactory> {
) -> BasicBlockExecutorProvider<OpExecutionStrategyFactory<OpPrimitives>> {
BasicBlockExecutorProvider::new(OpExecutionStrategyFactory::optimism(chain_spec))
}
}
@ -308,6 +361,7 @@ mod tests {
use reth_chainspec::MIN_TRANSACTION_GAS;
use reth_evm::execute::{BasicBlockExecutorProvider, BatchExecutor, BlockExecutorProvider};
use reth_optimism_chainspec::OpChainSpecBuilder;
use reth_optimism_primitives::OpTransactionSigned;
use reth_primitives::{Account, Block, BlockBody};
use reth_revm::{
database::StateProviderDatabase, test_utils::StateProviderTest, L1_BLOCK_CONTRACT,
@ -344,8 +398,7 @@ mod tests {
fn executor_provider(
chain_spec: Arc<OpChainSpec>,
) -> BasicBlockExecutorProvider<OpExecutionStrategyFactory> {
let strategy_factory =
OpExecutionStrategyFactory::new(chain_spec.clone(), OpEvmConfig::new(chain_spec));
let strategy_factory = OpExecutionStrategyFactory::optimism(chain_spec);
BasicBlockExecutorProvider::new(strategy_factory)
}

View File

@ -33,6 +33,8 @@ mod execute;
pub use execute::*;
pub mod l1;
pub use l1::*;
mod receipts;
pub use receipts::*;
mod error;
pub use error::OpBlockExecutionError;

View File

@ -0,0 +1,75 @@
use alloy_consensus::{Eip658Value, Header, Receipt};
use core::fmt;
use op_alloy_consensus::{OpDepositReceipt, OpTxType};
use reth_optimism_primitives::{OpReceipt, OpTransactionSigned};
use revm_primitives::ExecutionResult;
/// Context for building a receipt.
#[derive(Debug)]
pub struct ReceiptBuilderCtx<'a, T> {
/// Block header.
pub header: &'a Header,
/// Transaction
pub tx: &'a T,
/// Result of transaction execution.
pub result: ExecutionResult,
/// Cumulative gas used.
pub cumulative_gas_used: u64,
}
/// Type that knows how to build a receipt based on execution result.
pub trait OpReceiptBuilder<T>: fmt::Debug + Send + Sync + Unpin + 'static {
/// Receipt type.
type Receipt: Send + Sync + Clone + Unpin + 'static;
/// Builds a receipt given a transaction and the result of the execution.
///
/// Note: this method should return `Err` if the transaction is a deposit transaction. In that
/// case, the `build_deposit_receipt` method will be called.
fn build_receipt<'a>(
&self,
ctx: ReceiptBuilderCtx<'a, T>,
) -> Result<Self::Receipt, ReceiptBuilderCtx<'a, T>>;
/// Builds receipt for a deposit transaction.
fn build_deposit_receipt(&self, inner: OpDepositReceipt) -> Self::Receipt;
}
/// Basic builder for receipts of [`OpTransactionSigned`].
#[derive(Debug, Default, Clone, Copy)]
#[non_exhaustive]
pub struct BasicOpReceiptBuilder;
impl OpReceiptBuilder<OpTransactionSigned> for BasicOpReceiptBuilder {
type Receipt = OpReceipt;
fn build_receipt<'a>(
&self,
ctx: ReceiptBuilderCtx<'a, OpTransactionSigned>,
) -> Result<Self::Receipt, ReceiptBuilderCtx<'a, OpTransactionSigned>> {
match ctx.tx.tx_type() {
OpTxType::Deposit => Err(ctx),
ty => {
let receipt = 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(),
};
Ok(match ty {
OpTxType::Legacy => OpReceipt::Legacy(receipt),
OpTxType::Eip1559 => OpReceipt::Eip1559(receipt),
OpTxType::Eip2930 => OpReceipt::Eip2930(receipt),
OpTxType::Eip7702 => OpReceipt::Eip7702(receipt),
OpTxType::Deposit => unreachable!(),
})
}
}
}
fn build_deposit_receipt(&self, inner: OpDepositReceipt) -> Self::Receipt {
OpReceipt::Deposit(inner)
}
}