mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat: add receipt builder for OpExecutionStrategy (#13792)
This commit is contained in:
@ -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 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.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)
|
||||
}
|
||||
|
||||
@ -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;
|
||||
|
||||
75
crates/optimism/evm/src/receipts.rs
Normal file
75
crates/optimism/evm/src/receipts.rs
Normal 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)
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user