From 1f1df6ce4aee0756065ba54edc9d61f743789363 Mon Sep 17 00:00:00 2001 From: Arsenii Kulikov Date: Sun, 2 Feb 2025 02:48:21 +0400 Subject: [PATCH 1/5] wip --- Cargo.lock | 1 + .../ethereum/engine-primitives/src/payload.rs | 16 - crates/optimism/consensus/src/proof.rs | 12 +- crates/optimism/evm/src/execute.rs | 1 - crates/optimism/evm/src/receipts.rs | 4 +- crates/optimism/node/src/engine.rs | 4 +- crates/optimism/node/src/node.rs | 13 +- crates/optimism/node/src/utils.rs | 5 +- crates/optimism/payload/src/builder.rs | 303 +++++++++++------- crates/optimism/payload/src/payload.rs | 118 ++++--- crates/optimism/rpc/src/witness.rs | 36 ++- crates/payload/primitives/Cargo.toml | 1 + crates/payload/primitives/src/traits.rs | 1 + 13 files changed, 300 insertions(+), 215 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ac47e2d9d181..5e2e29a734f9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8530,6 +8530,7 @@ dependencies = [ "alloy-primitives", "alloy-rpc-types-engine", "assert_matches", + "auto_impl", "op-alloy-rpc-types-engine", "reth-chain-state", "reth-chainspec", diff --git a/crates/ethereum/engine-primitives/src/payload.rs b/crates/ethereum/engine-primitives/src/payload.rs index e4a945558e21..68e18eadb729 100644 --- a/crates/ethereum/engine-primitives/src/payload.rs +++ b/crates/ethereum/engine-primitives/src/payload.rs @@ -98,22 +98,6 @@ impl BuiltPayload for EthBuiltPayload { } } -impl BuiltPayload for &EthBuiltPayload { - type Primitives = EthPrimitives; - - fn block(&self) -> &SealedBlock { - (**self).block() - } - - fn fees(&self) -> U256 { - (**self).fees() - } - - fn requests(&self) -> Option { - self.requests.clone() - } -} - // V1 engine_getPayloadV1 response impl From for ExecutionPayloadV1 { fn from(value: EthBuiltPayload) -> Self { diff --git a/crates/optimism/consensus/src/proof.rs b/crates/optimism/consensus/src/proof.rs index 0b1e6c74f9f9..d6ee80679cc2 100644 --- a/crates/optimism/consensus/src/proof.rs +++ b/crates/optimism/consensus/src/proof.rs @@ -1,13 +1,12 @@ //! Helper function for Receipt root calculation for Optimism hardforks. use alloc::vec::Vec; -use alloy_consensus::TxReceipt; use alloy_eips::eip2718::Encodable2718; use alloy_primitives::B256; use alloy_trie::root::ordered_trie_root_with_encoder; use reth_chainspec::ChainSpec; use reth_optimism_forks::{OpHardfork, OpHardforks}; -use reth_optimism_primitives::{DepositReceipt, OpReceipt}; +use reth_optimism_primitives::DepositReceipt; use reth_primitives::ReceiptWithBloom; /// Calculates the receipt root for a header. @@ -44,8 +43,8 @@ pub(crate) fn calculate_receipt_root_optimism( /// Calculates the receipt root for a header for the reference type of [`OpReceipt`]. /// /// NOTE: Prefer calculate receipt root optimism if you have log blooms memoized. -pub fn calculate_receipt_root_no_memo_optimism( - receipts: &[OpReceipt], +pub fn calculate_receipt_root_no_memo_optimism( + receipts: &[R], chain_spec: impl OpHardforks, timestamp: u64, ) -> B256 { @@ -61,8 +60,8 @@ pub fn calculate_receipt_root_no_memo_optimism( .iter() .map(|r| { let mut r = (*r).clone(); - if let OpReceipt::Deposit(r) = &mut r { - r.deposit_nonce = None; + if let Some(receipt) = r.as_deposit_receipt_mut() { + receipt.deposit_nonce = None; } r }) @@ -85,6 +84,7 @@ mod tests { use alloy_primitives::{b256, bloom, hex, Address, Bloom, Bytes, Log, LogData}; use op_alloy_consensus::OpDepositReceipt; use reth_optimism_chainspec::BASE_SEPOLIA; + use reth_optimism_primitives::OpReceipt; use reth_primitives::ReceiptWithBloom; /// Tests that the receipt root is computed correctly for the regolith block. diff --git a/crates/optimism/evm/src/execute.rs b/crates/optimism/evm/src/execute.rs index 69bd395729de..c7841205cc80 100644 --- a/crates/optimism/evm/src/execute.rs +++ b/crates/optimism/evm/src/execute.rs @@ -232,7 +232,6 @@ where receipts.push( match self.receipt_builder.build_receipt(ReceiptBuilderCtx { - header: block.header(), tx: transaction, result, cumulative_gas_used, diff --git a/crates/optimism/evm/src/receipts.rs b/crates/optimism/evm/src/receipts.rs index a2f6228d29f2..2ced9c7b7c6c 100644 --- a/crates/optimism/evm/src/receipts.rs +++ b/crates/optimism/evm/src/receipts.rs @@ -1,4 +1,4 @@ -use alloy_consensus::{Eip658Value, Header, Receipt}; +use alloy_consensus::{Eip658Value, Receipt}; use core::fmt; use op_alloy_consensus::{OpDepositReceipt, OpTxType}; use reth_optimism_primitives::{OpReceipt, OpTransactionSigned}; @@ -7,8 +7,6 @@ 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. diff --git a/crates/optimism/node/src/engine.rs b/crates/optimism/node/src/engine.rs index 2f07ff3603e5..b85624d1f4bb 100644 --- a/crates/optimism/node/src/engine.rs +++ b/crates/optimism/node/src/engine.rs @@ -18,7 +18,7 @@ use reth_node_api::{ use reth_optimism_chainspec::OpChainSpec; use reth_optimism_forks::{OpHardfork, OpHardforks}; use reth_optimism_payload_builder::{OpBuiltPayload, OpPayloadBuilderAttributes}; -use reth_optimism_primitives::OpBlock; +use reth_optimism_primitives::{OpBlock, OpTransactionSigned}; use reth_payload_validator::ExecutionPayloadValidator; use reth_primitives::SealedBlock; use std::sync::Arc; @@ -66,7 +66,7 @@ pub struct OpPayloadTypes; impl PayloadTypes for OpPayloadTypes { type BuiltPayload = OpBuiltPayload; type PayloadAttributes = OpPayloadAttributes; - type PayloadBuilderAttributes = OpPayloadBuilderAttributes; + type PayloadBuilderAttributes = OpPayloadBuilderAttributes; } /// Validator for Optimism engine API. diff --git a/crates/optimism/node/src/node.rs b/crates/optimism/node/src/node.rs index 3fa2ac6852b3..a25f9bf56b89 100644 --- a/crates/optimism/node/src/node.rs +++ b/crates/optimism/node/src/node.rs @@ -25,7 +25,7 @@ use reth_node_builder::{ }; use reth_optimism_chainspec::OpChainSpec; use reth_optimism_consensus::OpBeaconConsensus; -use reth_optimism_evm::{OpEvmConfig, OpExecutionStrategyFactory}; +use reth_optimism_evm::{BasicOpReceiptBuilder, OpEvmConfig, OpExecutionStrategyFactory}; use reth_optimism_payload_builder::{ builder::OpPayloadTransactions, config::{OpBuilderConfig, OpDAConfig}, @@ -205,11 +205,16 @@ where ctx: reth_node_api::AddOnsContext<'_, N>, ) -> eyre::Result { let Self { rpc_add_ons, da_config } = self; + + let builder = reth_optimism_payload_builder::OpPayloadBuilder::new( + ctx.node.evm_config().clone(), + BasicOpReceiptBuilder::default(), + ); // install additional OP specific rpc methods let debug_ext = OpDebugWitnessApi::new( ctx.node.provider().clone(), - ctx.node.evm_config().clone(), Box::new(ctx.node.task_executor().clone()), + builder, ); let miner_ext = OpMinerExtApi::new(da_config); @@ -493,9 +498,11 @@ where + Unpin + 'static, Evm: ConfigureEvmFor>, + Txs: OpPayloadTransactions>, { let payload_builder = reth_optimism_payload_builder::OpPayloadBuilder::with_builder_config( evm_config, + BasicOpReceiptBuilder::default(), OpBuilderConfig { da_config: self.da_config }, ) .with_transactions(self.best_transactions) @@ -535,7 +542,7 @@ where Pool: TransactionPool>> + Unpin + 'static, - Txs: OpPayloadTransactions, + Txs: OpPayloadTransactions>, { async fn spawn_payload_service( self, diff --git a/crates/optimism/node/src/utils.rs b/crates/optimism/node/src/utils.rs index 95875a767366..b92466aaf762 100644 --- a/crates/optimism/node/src/utils.rs +++ b/crates/optimism/node/src/utils.rs @@ -7,6 +7,7 @@ use reth_e2e_test_utils::{ }; use reth_node_api::NodeTypesWithDBAdapter; use reth_optimism_chainspec::OpChainSpecBuilder; +use reth_optimism_primitives::OpTransactionSigned; use reth_payload_builder::EthPayloadBuilderAttributes; use reth_provider::providers::BlockchainProvider; use reth_tasks::TaskManager; @@ -35,7 +36,7 @@ pub async fn advance_chain( length: usize, node: &mut OpNode, wallet: Arc>, -) -> eyre::Result> { +) -> eyre::Result)>> { node.advance(length as u64, |_| { let wallet = wallet.clone(); Box::pin(async move { @@ -53,7 +54,7 @@ pub async fn advance_chain( } /// Helper function to create a new eth payload attributes -pub fn optimism_payload_attributes(timestamp: u64) -> OpPayloadBuilderAttributes { +pub fn optimism_payload_attributes(timestamp: u64) -> OpPayloadBuilderAttributes { let attributes = PayloadAttributes { timestamp, prev_randao: B256::ZERO, diff --git a/crates/optimism/payload/src/builder.rs b/crates/optimism/payload/src/builder.rs index 4a447bdfc58c..f9f1f3e912df 100644 --- a/crates/optimism/payload/src/builder.rs +++ b/crates/optimism/payload/src/builder.rs @@ -11,27 +11,30 @@ 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, OpTxType}; +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, EthereumHardforks}; use reth_evm::{ - env::EvmEnv, system_calls::SystemCaller, ConfigureEvm, ConfigureEvmEnv, Database, Evm, + env::EvmEnv, system_calls::SystemCaller, ConfigureEvmEnv, ConfigureEvmFor, Database, Evm, EvmError, InvalidTxError, NextBlockEnvAttributes, }; use reth_execution_types::ExecutionOutcome; use reth_optimism_chainspec::OpChainSpec; use reth_optimism_consensus::calculate_receipt_root_no_memo_optimism; +use reth_optimism_evm::{OpReceiptBuilder, ReceiptBuilderCtx}; use reth_optimism_forks::OpHardforks; -use reth_optimism_primitives::{OpPrimitives, OpReceipt, OpTransactionSigned}; +use reth_optimism_primitives::{ + transaction::signed::OpTransaction, DepositReceipt, OpTransactionSigned, +}; use reth_payload_builder_primitives::PayloadBuilderError; use reth_payload_primitives::PayloadBuilderAttributes; use reth_payload_util::{NoopPayloadTransactions, PayloadTransactions}; use reth_primitives::{ - transaction::SignedTransactionIntoRecoveredExt, Block, BlockBody, SealedHeader, + transaction::SignedTransactionIntoRecoveredExt, BlockBody, NodePrimitives, SealedHeader, }; -use reth_primitives_traits::{block::Block as _, proofs, RecoveredBlock}; +use reth_primitives_traits::{block::Block as _, proofs, RecoveredBlock, SignedTransaction}; use reth_provider::{ HashedPostStateProvider, ProviderError, StateProofProvider, StateProviderFactory, StateRootProvider, @@ -42,15 +45,41 @@ use reth_transaction_pool::{ }; use revm::{ db::{states::bundle_state::BundleRetention, State}, - primitives::ResultAndState, + primitives::{ExecutionResult, ResultAndState}, DatabaseCommit, }; use std::{fmt::Display, sync::Arc}; use tracing::{debug, trace, warn}; +/// Helper trait to encapsulate common bounds on [`NodePrimitives`] for OP payload builder. +pub trait OpPayloadPrimitives: + NodePrimitives< + Receipt: DepositReceipt, + SignedTx = Self::_TX, + BlockHeader = Header, + BlockBody = BlockBody, +> +{ + /// Helper AT to bound [`NodePrimitives::Block`] type without causing bound cycle. + type _TX: SignedTransaction + OpTransaction; +} + +impl OpPayloadPrimitives for T +where + Tx: SignedTransaction + OpTransaction, + T: NodePrimitives< + SignedTx = Tx, + Receipt: DepositReceipt, + BlockHeader = Header, + BlockBody = BlockBody, + >, +{ + type _TX = Tx; +} + /// Optimism's payload builder #[derive(Debug, Clone)] -pub struct OpPayloadBuilder { +pub struct OpPayloadBuilder { /// The rollup's compute pending block configuration option. // TODO(clabby): Implement this feature. pub compute_pending_block: bool, @@ -61,23 +90,38 @@ pub struct OpPayloadBuilder { /// The type responsible for yielding the best transactions for the payload if mempool /// transactions are allowed. pub best_transactions: Txs, + /// Node primitive types. + pub receipt_builder: Arc>, } -impl OpPayloadBuilder { +impl OpPayloadBuilder { /// `OpPayloadBuilder` constructor. /// /// Configures the builder with the default settings. - pub fn new(evm_config: EvmConfig) -> Self { - Self::with_builder_config(evm_config, Default::default()) + pub fn new( + evm_config: EvmConfig, + receipt_builder: impl OpReceiptBuilder, + ) -> Self { + Self::with_builder_config(evm_config, receipt_builder, Default::default()) } /// Configures the builder with the given [`OpBuilderConfig`]. - pub const fn with_builder_config(evm_config: EvmConfig, config: OpBuilderConfig) -> Self { - Self { compute_pending_block: true, evm_config, config, best_transactions: () } + pub fn with_builder_config( + evm_config: EvmConfig, + receipt_builder: impl OpReceiptBuilder, + config: OpBuilderConfig, + ) -> Self { + Self { + compute_pending_block: true, + receipt_builder: Arc::new(receipt_builder), + evm_config, + config, + best_transactions: (), + } } } -impl OpPayloadBuilder { +impl OpPayloadBuilder { /// Sets the rollup's compute pending block configuration option. pub const fn set_compute_pending_block(mut self, compute_pending_block: bool) -> Self { self.compute_pending_block = compute_pending_block; @@ -89,9 +133,15 @@ impl OpPayloadBuilder { pub fn with_transactions( self, best_transactions: T, - ) -> OpPayloadBuilder { - let Self { compute_pending_block, evm_config, config, .. } = self; - OpPayloadBuilder { compute_pending_block, evm_config, best_transactions, config } + ) -> OpPayloadBuilder { + let Self { compute_pending_block, evm_config, config, receipt_builder, .. } = self; + OpPayloadBuilder { + compute_pending_block, + evm_config, + best_transactions, + config, + receipt_builder, + } } /// Enables the rollup's compute pending block configuration option. @@ -104,9 +154,10 @@ impl OpPayloadBuilder { self.compute_pending_block } } -impl OpPayloadBuilder +impl OpPayloadBuilder where - EvmConfig: ConfigureEvm
, + N: OpPayloadPrimitives, + EvmConfig: ConfigureEvmFor, { /// Constructs an Optimism payload from the transactions sent via the /// Payload attributes by the sequencer. If the `no_tx_pool` argument is passed in @@ -118,12 +169,17 @@ where /// a result indicating success with the payload or an error in case of failure. fn build_payload<'a, Client, Pool, Txs>( &self, - args: BuildArguments, + args: BuildArguments< + Pool, + Client, + OpPayloadBuilderAttributes, + OpBuiltPayload, + >, best: impl FnOnce(BestTransactionsAttributes) -> Txs + Send + Sync + 'a, - ) -> Result, PayloadBuilderError> + ) -> Result>, PayloadBuilderError> where Client: StateProviderFactory + ChainSpecProvider, - Txs: PayloadTransactions, + Txs: PayloadTransactions, { let evm_env = self .evm_env(&args.config.attributes, &args.config.parent_header) @@ -140,6 +196,7 @@ where evm_env, cancel, best_payload, + receipt_builder: self.receipt_builder.clone(), }; let builder = OpBuilder::new(best); @@ -165,7 +222,7 @@ where /// (that has the `parent` as its parent). pub fn evm_env( &self, - attributes: &OpPayloadBuilderAttributes, + attributes: &OpPayloadBuilderAttributes, parent: &Header, ) -> Result, EvmConfig::Error> { let next_attributes = NextBlockEnvAttributes { @@ -193,7 +250,7 @@ where let evm_env = self.evm_env(&attributes, &parent).map_err(PayloadBuilderError::other)?; let config = PayloadConfig { parent_header: Arc::new(parent), attributes }; - let ctx = OpPayloadBuilderCtx { + let ctx: OpPayloadBuilderCtx = OpPayloadBuilderCtx { evm_config: self.evm_config.clone(), da_config: self.config.da_config.clone(), chain_spec: client.chain_spec(), @@ -201,6 +258,7 @@ where evm_env, cancel: Default::default(), best_payload: Default::default(), + receipt_builder: self.receipt_builder.clone(), }; let state_provider = client.state_by_block_hash(ctx.parent().hash())?; @@ -213,27 +271,29 @@ where } /// Implementation of the [`PayloadBuilder`] trait for [`OpPayloadBuilder`]. -impl PayloadBuilder for OpPayloadBuilder +impl PayloadBuilder + for OpPayloadBuilder where Client: StateProviderFactory + ChainSpecProvider, - Pool: TransactionPool>, - EvmConfig: ConfigureEvm
, - Txs: OpPayloadTransactions, + N: OpPayloadPrimitives, + Pool: TransactionPool>, + EvmConfig: ConfigureEvmFor, + Txs: OpPayloadTransactions, { - type Attributes = OpPayloadBuilderAttributes; - type BuiltPayload = OpBuiltPayload; + type Attributes = OpPayloadBuilderAttributes; + type BuiltPayload = OpBuiltPayload; fn try_build( &self, - args: BuildArguments, - ) -> Result, PayloadBuilderError> { + args: BuildArguments, + ) -> Result, PayloadBuilderError> { let pool = args.pool.clone(); self.build_payload(args, |attrs| self.best_transactions.best_transactions(pool, attrs)) } fn on_missing_payload( &self, - _args: BuildArguments, + _args: BuildArguments, ) -> MissingPayloadBehaviour { // we want to await the job that's already in progress because that should be returned as // is, there's no benefit in racing another job @@ -246,7 +306,7 @@ where &self, client: &Client, config: PayloadConfig, - ) -> Result { + ) -> Result { let args = BuildArguments { client, config, @@ -292,16 +352,18 @@ impl<'a, Txs> OpBuilder<'a, Txs> { impl OpBuilder<'_, Txs> where - Txs: PayloadTransactions, + Txs: PayloadTransactions, { /// Executes the payload and returns the outcome. - pub fn execute( + pub fn execute( self, state: &mut State, - ctx: &OpPayloadBuilderCtx, - ) -> Result, PayloadBuilderError> + ctx: &OpPayloadBuilderCtx, + ) -> Result>, PayloadBuilderError> where - EvmConfig: ConfigureEvm
, + N: OpPayloadPrimitives, + Txs: PayloadTransactions, + EvmConfig: ConfigureEvmFor, DB: Database, { let Self { best } = self; @@ -340,13 +402,15 @@ where } /// Builds the payload on top of the state. - pub fn build( + pub fn build( self, mut state: State, - ctx: OpPayloadBuilderCtx, - ) -> Result, PayloadBuilderError> + ctx: OpPayloadBuilderCtx, + ) -> Result>, PayloadBuilderError> where - EvmConfig: ConfigureEvm
, + EvmConfig: ConfigureEvmFor, + N: OpPayloadPrimitives, + Txs: PayloadTransactions, DB: Database + AsRef

, P: StateRootProvider + HashedPostStateProvider, { @@ -422,20 +486,20 @@ where }; // seal the block - let block = Block { + let block = N::Block::new( header, - body: BlockBody { + BlockBody { transactions: info.executed_transactions, ommers: vec![], withdrawals: ctx.withdrawals().cloned(), }, - }; + ); let sealed_block = Arc::new(block.seal_slow()); debug!(target: "payload_builder", id=%ctx.attributes().payload_id(), sealed_block_header = ?sealed_block.header(), "sealed built block"); // create the executed block data - let executed: ExecutedBlockWithTrieUpdates = ExecutedBlockWithTrieUpdates { + let executed: ExecutedBlockWithTrieUpdates = ExecutedBlockWithTrieUpdates { block: ExecutedBlock { recovered_block: Arc::new(RecoveredBlock::new_sealed( sealed_block.as_ref().clone(), @@ -462,13 +526,15 @@ where } /// Builds the payload and returns its [`ExecutionWitness`] based on the state after execution. - pub fn witness( + pub fn witness( self, state: &mut State, - ctx: &OpPayloadBuilderCtx, + ctx: &OpPayloadBuilderCtx, ) -> Result where - EvmConfig: ConfigureEvm

, + EvmConfig: ConfigureEvmFor, + N: OpPayloadPrimitives, + Txs: PayloadTransactions, DB: Database + AsRef

, P: StateProofProvider, { @@ -481,48 +547,48 @@ where } /// A type that returns a the [`PayloadTransactions`] that should be included in the pool. -pub trait OpPayloadTransactions: Clone + Send + Sync + Unpin + 'static { +pub trait OpPayloadTransactions: + Clone + Send + Sync + Unpin + 'static +{ /// Returns an iterator that yields the transaction in the order they should get included in the /// new payload. fn best_transactions< - Pool: TransactionPool>, + Pool: TransactionPool>, >( &self, pool: Pool, attr: BestTransactionsAttributes, - ) -> impl PayloadTransactions; + ) -> impl PayloadTransactions; } -impl OpPayloadTransactions for () { - fn best_transactions< - Pool: TransactionPool>, - >( +impl OpPayloadTransactions for () { + fn best_transactions>>( &self, pool: Pool, attr: BestTransactionsAttributes, - ) -> impl PayloadTransactions { + ) -> impl PayloadTransactions { BestPayloadTransactions::new(pool.best_transactions_with_attributes(attr)) } } /// Holds the state after execution #[derive(Debug)] -pub struct ExecutedPayload { +pub struct ExecutedPayload { /// Tracked execution info - pub info: ExecutionInfo, + pub info: ExecutionInfo, /// Withdrawal hash. pub withdrawals_root: Option, } /// This acts as the container for executed transactions and its byproducts (receipts, gas used) #[derive(Default, Debug)] -pub struct ExecutionInfo { +pub struct ExecutionInfo { /// All executed transactions (unrecovered). - pub executed_transactions: Vec, + pub executed_transactions: Vec, /// The recovered senders for the executed transactions. pub executed_senders: Vec

, /// The transaction receipts - pub receipts: Vec, + pub receipts: Vec, /// All gas used so far pub cumulative_gas_used: u64, /// Estimated DA size @@ -531,7 +597,7 @@ pub struct ExecutionInfo { pub total_fees: U256, } -impl ExecutionInfo { +impl ExecutionInfo { /// Create a new instance with allocated slots. pub fn with_capacity(capacity: usize) -> Self { Self { @@ -552,7 +618,7 @@ impl ExecutionInfo { /// maximum allowed DA limit per block. pub fn is_tx_over_limits( &self, - tx: &OpTransactionSigned, + tx: &N::SignedTx, block_gas_limit: u64, tx_data_limit: Option, block_data_limit: Option, @@ -573,7 +639,7 @@ impl ExecutionInfo { /// Container type that holds all necessities to build a new payload. #[derive(Debug)] -pub struct OpPayloadBuilderCtx { +pub struct OpPayloadBuilderCtx { /// The type that knows how to perform system calls and configure the evm. pub evm_config: EvmConfig, /// The DA config for the payload builder @@ -581,23 +647,25 @@ pub struct OpPayloadBuilderCtx { /// The chainspec pub chain_spec: Arc, /// How to build the payload. - pub config: PayloadConfig, + pub config: PayloadConfig>, /// Evm Settings pub evm_env: EvmEnv, /// Marker to check whether the job has been cancelled. pub cancel: Cancelled, /// The currently best payload. - pub best_payload: Option, + pub best_payload: Option>, + /// Receipt builder. + pub receipt_builder: Arc>, } -impl OpPayloadBuilderCtx { +impl OpPayloadBuilderCtx { /// Returns the parent block the payload will be build on. pub fn parent(&self) -> &SealedHeader { &self.config.parent_header } /// Returns the builder attributes. - pub const fn attributes(&self) -> &OpPayloadBuilderAttributes { + pub const fn attributes(&self) -> &OpPayloadBuilderAttributes { &self.config.attributes } @@ -730,9 +798,10 @@ impl OpPayloadBuilderCtx { } } -impl OpPayloadBuilderCtx +impl OpPayloadBuilderCtx where - EvmConfig: ConfigureEvm
, + EvmConfig: ConfigureEvmFor, + N: OpPayloadPrimitives, { /// apply eip-4788 pre block contract call pub fn apply_pre_beacon_root_contract_call( @@ -761,11 +830,48 @@ where Ok(()) } + /// Constructs a receipt for the given transaction. + fn build_receipt( + &self, + info: &ExecutionInfo, + result: ExecutionResult, + deposit_nonce: Option, + 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, db: &mut State, - ) -> Result + ) -> Result, PayloadBuilderError> where DB: Database, { @@ -793,11 +899,11 @@ 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 = (self.is_regolith_active() && sequencer_tx.is_deposit()) + let depositor_nonce = (self.is_regolith_active() && sequencer_tx.is_deposit()) .then(|| { evm.db_mut() .load_cache_account(sequencer_tx.signer()) - .map(|acc| acc.account_info().unwrap_or_default()) + .map(|acc| acc.account_info().unwrap_or_default().nonce) }) .transpose() .map_err(|_| { @@ -828,28 +934,13 @@ where // add gas used by the transaction to cumulative gas used, before creating the receipt info.cumulative_gas_used += gas_used; - let receipt = alloy_consensus::Receipt { - status: Eip658Value::Eip658(result.is_success()), - cumulative_gas_used: info.cumulative_gas_used, - logs: result.into_logs().into_iter().collect(), - }; - // Push transaction changeset and calculate header bloom filter for receipt. - info.receipts.push(match sequencer_tx.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: self.is_canyon_active().then_some(1), - }), - }); + 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()); @@ -864,7 +955,7 @@ where /// Returns `Ok(Some(())` if the job was cancelled. pub fn execute_best_transactions( &self, - info: &mut ExecutionInfo, + info: &mut ExecutionInfo, db: &mut State, mut best_txs: impl PayloadTransactions, ) -> Result, PayloadBuilderError> @@ -888,7 +979,7 @@ where } // A sequencer's block should never contain blob or deposit transactions from the pool. - if tx.is_eip4844() || tx.tx_type() == OpTxType::Deposit { + if tx.is_eip4844() || tx.is_deposit() { best_txs.mark_invalid(tx.signer(), tx.nonce()); continue } @@ -932,24 +1023,8 @@ where info.cumulative_gas_used += gas_used; info.cumulative_da_bytes_used += tx.length() as u64; - let receipt = alloy_consensus::Receipt { - status: Eip658Value::Eip658(result.is_success()), - cumulative_gas_used: info.cumulative_gas_used, - logs: result.into_logs().into_iter().collect(), - }; - // Push transaction changeset and calculate header bloom filter for receipt. - info.receipts.push(match tx.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: None, - deposit_receipt_version: None, - }), - }); + info.receipts.push(self.build_receipt(info, result, None, &tx)); // update add to total fees let miner_fee = tx diff --git a/crates/optimism/payload/src/payload.rs b/crates/optimism/payload/src/payload.rs index 0088e3950490..2c57e95fd911 100644 --- a/crates/optimism/payload/src/payload.rs +++ b/crates/optimism/payload/src/payload.rs @@ -1,5 +1,7 @@ //! Payload related types +use std::fmt::Debug; + use alloy_eips::{ eip1559::BaseFeeParams, eip2718::Decodable2718, eip4895::Withdrawals, eip7685::Requests, }; @@ -14,28 +16,41 @@ use op_alloy_consensus::{encode_holocene_extra_data, EIP1559ParamError}; pub use op_alloy_rpc_types_engine::OpPayloadAttributes; use op_alloy_rpc_types_engine::{OpExecutionPayloadEnvelopeV3, OpExecutionPayloadEnvelopeV4}; use reth_chain_state::ExecutedBlockWithTrieUpdates; -use reth_optimism_primitives::{OpBlock, OpPrimitives, OpTransactionSigned}; +use reth_optimism_primitives::OpPrimitives; use reth_payload_builder::EthPayloadBuilderAttributes; use reth_payload_primitives::{BuiltPayload, PayloadBuilderAttributes}; -use reth_primitives::{transaction::WithEncoded, SealedBlock}; +use reth_primitives::{transaction::WithEncoded, Block, NodePrimitives, SealedBlock}; +use reth_primitives_traits::SignedTransaction; /// Optimism Payload Builder Attributes -#[derive(Debug, Clone, PartialEq, Eq, Default)] -pub struct OpPayloadBuilderAttributes { +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct OpPayloadBuilderAttributes { /// Inner ethereum payload builder attributes pub payload_attributes: EthPayloadBuilderAttributes, /// `NoTxPool` option for the generated payload pub no_tx_pool: bool, /// Decoded transactions and the original EIP-2718 encoded bytes as received in the payload /// attributes. - pub transactions: Vec>, + pub transactions: Vec>, /// The gas limit for the generated payload pub gas_limit: Option, /// EIP-1559 parameters for the generated payload pub eip_1559_params: Option, } -impl OpPayloadBuilderAttributes { +impl Default for OpPayloadBuilderAttributes { + fn default() -> Self { + Self { + payload_attributes: Default::default(), + no_tx_pool: Default::default(), + gas_limit: Default::default(), + eip_1559_params: Default::default(), + transactions: Default::default(), + } + } +} + +impl OpPayloadBuilderAttributes { /// Extracts the `eip1559` parameters for the payload. pub fn get_holocene_extra_data( &self, @@ -47,7 +62,9 @@ impl OpPayloadBuilderAttributes { } } -impl PayloadBuilderAttributes for OpPayloadBuilderAttributes { +impl PayloadBuilderAttributes + for OpPayloadBuilderAttributes +{ type RpcPayloadAttributes = OpPayloadAttributes; type Error = alloy_rlp::Error; @@ -127,24 +144,20 @@ impl PayloadBuilderAttributes for OpPayloadBuilderAttributes { /// Contains the built payload. #[derive(Debug, Clone)] -pub struct OpBuiltPayload { +pub struct OpBuiltPayload { /// Identifier of the payload pub(crate) id: PayloadId, /// Block execution data for the payload, if any. - pub(crate) block: ExecutedBlockWithTrieUpdates, + pub(crate) block: ExecutedBlockWithTrieUpdates, /// The fees of the block pub(crate) fees: U256, } // === impl BuiltPayload === -impl OpBuiltPayload { +impl OpBuiltPayload { /// Initializes the payload with the given initial block. - pub const fn new( - id: PayloadId, - fees: U256, - block: ExecutedBlockWithTrieUpdates, - ) -> Self { + pub const fn new(id: PayloadId, fees: U256, block: ExecutedBlockWithTrieUpdates) -> Self { Self { id, block, fees } } @@ -154,7 +167,7 @@ impl OpBuiltPayload { } /// Returns the built block(sealed) - pub fn block(&self) -> &SealedBlock { + pub fn block(&self) -> &SealedBlock { self.block.sealed_block() } @@ -164,10 +177,10 @@ impl OpBuiltPayload { } } -impl BuiltPayload for OpBuiltPayload { - type Primitives = OpPrimitives; +impl BuiltPayload for OpBuiltPayload { + type Primitives = N; - fn block(&self) -> &SealedBlock { + fn block(&self) -> &SealedBlock { self.block() } @@ -175,27 +188,7 @@ impl BuiltPayload for OpBuiltPayload { self.fees } - fn executed_block(&self) -> Option> { - Some(self.block.clone()) - } - - fn requests(&self) -> Option { - None - } -} - -impl BuiltPayload for &OpBuiltPayload { - type Primitives = OpPrimitives; - - fn block(&self) -> &SealedBlock { - (**self).block() - } - - fn fees(&self) -> U256 { - (**self).fees() - } - - fn executed_block(&self) -> Option> { + fn executed_block(&self) -> Option> { Some(self.block.clone()) } @@ -205,8 +198,12 @@ impl BuiltPayload for &OpBuiltPayload { } // V1 engine_getPayloadV1 response -impl From for ExecutionPayloadV1 { - fn from(value: OpBuiltPayload) -> Self { +impl From> for ExecutionPayloadV1 +where + T: SignedTransaction, + N: NodePrimitives>, +{ + fn from(value: OpBuiltPayload) -> Self { Self::from_block_unchecked( value.block().hash(), &value.block.into_sealed_block().into_block(), @@ -215,8 +212,12 @@ impl From for ExecutionPayloadV1 { } // V2 engine_getPayloadV2 response -impl From for ExecutionPayloadEnvelopeV2 { - fn from(value: OpBuiltPayload) -> Self { +impl From> for ExecutionPayloadEnvelopeV2 +where + T: SignedTransaction, + N: NodePrimitives>, +{ + fn from(value: OpBuiltPayload) -> Self { let OpBuiltPayload { block, fees, .. } = value; let block = block.into_sealed_block(); @@ -230,8 +231,12 @@ impl From for ExecutionPayloadEnvelopeV2 { } } -impl From for OpExecutionPayloadEnvelopeV3 { - fn from(value: OpBuiltPayload) -> Self { +impl From> for OpExecutionPayloadEnvelopeV3 +where + T: SignedTransaction, + N: NodePrimitives>, +{ + fn from(value: OpBuiltPayload) -> Self { let OpBuiltPayload { block, fees, .. } = value; let parent_beacon_block_root = @@ -258,8 +263,13 @@ impl From for OpExecutionPayloadEnvelopeV3 { } } } -impl From for OpExecutionPayloadEnvelopeV4 { - fn from(value: OpBuiltPayload) -> Self { + +impl From> for OpExecutionPayloadEnvelopeV4 +where + T: SignedTransaction, + N: NodePrimitives>, +{ + fn from(value: OpBuiltPayload) -> Self { let OpBuiltPayload { block, fees, .. } = value; let parent_beacon_block_root = @@ -347,6 +357,7 @@ mod tests { use crate::OpPayloadAttributes; use alloy_primitives::{address, b256, bytes, FixedBytes}; use alloy_rpc_types_engine::PayloadAttributes; + use reth_optimism_primitives::OpTransactionSigned; use reth_payload_primitives::EngineApiMessageVersion; use std::str::FromStr; @@ -383,17 +394,18 @@ mod tests { #[test] fn test_get_extra_data_post_holocene() { - let attributes = OpPayloadBuilderAttributes { - eip_1559_params: Some(B64::from_str("0x0000000800000008").unwrap()), - ..Default::default() - }; + let attributes: OpPayloadBuilderAttributes = + OpPayloadBuilderAttributes { + eip_1559_params: Some(B64::from_str("0x0000000800000008").unwrap()), + ..Default::default() + }; let extra_data = attributes.get_holocene_extra_data(BaseFeeParams::new(80, 60)); assert_eq!(extra_data.unwrap(), Bytes::copy_from_slice(&[0, 0, 0, 0, 8, 0, 0, 0, 8])); } #[test] fn test_get_extra_data_post_holocene_default() { - let attributes = + let attributes: OpPayloadBuilderAttributes = OpPayloadBuilderAttributes { eip_1559_params: Some(B64::ZERO), ..Default::default() }; let extra_data = attributes.get_holocene_extra_data(BaseFeeParams::new(80, 60)); assert_eq!(extra_data.unwrap(), Bytes::copy_from_slice(&[0, 0, 0, 0, 80, 0, 0, 0, 60])); diff --git a/crates/optimism/rpc/src/witness.rs b/crates/optimism/rpc/src/witness.rs index 387e6597edc2..63ccbc709097 100644 --- a/crates/optimism/rpc/src/witness.rs +++ b/crates/optimism/rpc/src/witness.rs @@ -1,17 +1,17 @@ //! Support for optimism specific witness RPCs. -use alloy_consensus::Header; use alloy_primitives::B256; use alloy_rpc_types_debug::ExecutionWitness; use jsonrpsee_core::{async_trait, RpcResult}; use op_alloy_rpc_types_engine::OpPayloadAttributes; use reth_chainspec::ChainSpecProvider; -use reth_evm::ConfigureEvm; +use reth_evm::ConfigureEvmFor; use reth_optimism_chainspec::OpChainSpec; -use reth_optimism_payload_builder::OpPayloadBuilder; -use reth_optimism_primitives::OpTransactionSigned; +use reth_optimism_payload_builder::{builder::OpPayloadPrimitives, OpPayloadBuilder}; use reth_primitives::SealedHeader; -use reth_provider::{BlockReaderIdExt, ProviderError, ProviderResult, StateProviderFactory}; +use reth_provider::{ + BlockReaderIdExt, NodePrimitivesProvider, ProviderError, ProviderResult, StateProviderFactory, +}; pub use reth_rpc_api::DebugExecutionWitnessApiServer; use reth_rpc_server_types::{result::internal_rpc_err, ToRpcResult}; use reth_tasks::TaskSpawner; @@ -19,18 +19,17 @@ use std::{fmt::Debug, sync::Arc}; use tokio::sync::{oneshot, Semaphore}; /// An extension to the `debug_` namespace of the RPC API. -pub struct OpDebugWitnessApi { +pub struct OpDebugWitnessApi { inner: Arc>, } -impl OpDebugWitnessApi { +impl OpDebugWitnessApi { /// Creates a new instance of the `OpDebugWitnessApi`. pub fn new( provider: Provider, - evm_config: EvmConfig, task_spawner: Box, + builder: OpPayloadBuilder, ) -> Self { - let builder = OpPayloadBuilder::new(evm_config); let semaphore = Arc::new(Semaphore::new(3)); let inner = OpDebugWitnessApiInner { provider, builder, task_spawner, semaphore }; Self { inner: Arc::new(inner) } @@ -39,7 +38,7 @@ impl OpDebugWitnessApi { impl OpDebugWitnessApi where - Provider: BlockReaderIdExt
, + Provider: NodePrimitivesProvider + BlockReaderIdExt
, { /// Fetches the parent header by hash. fn parent_header(&self, parent_block_hash: B256) -> ProviderResult { @@ -55,11 +54,12 @@ impl DebugExecutionWitnessApiServer for OpDebugWitnessApi where Provider: BlockReaderIdExt
+ + NodePrimitivesProvider + StateProviderFactory + ChainSpecProvider + Clone + 'static, - EvmConfig: ConfigureEvm
+ 'static, + EvmConfig: ConfigureEvmFor + 'static, { async fn execute_payload( &self, @@ -84,20 +84,26 @@ where } } -impl Clone for OpDebugWitnessApi { +impl Clone for OpDebugWitnessApi +where + Provider: NodePrimitivesProvider, +{ fn clone(&self) -> Self { Self { inner: Arc::clone(&self.inner) } } } -impl Debug for OpDebugWitnessApi { +impl Debug for OpDebugWitnessApi +where + Provider: NodePrimitivesProvider, +{ fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("OpDebugWitnessApi").finish_non_exhaustive() } } -struct OpDebugWitnessApiInner { +struct OpDebugWitnessApiInner { provider: Provider, - builder: OpPayloadBuilder, + builder: OpPayloadBuilder, task_spawner: Box, semaphore: Arc, } diff --git a/crates/payload/primitives/Cargo.toml b/crates/payload/primitives/Cargo.toml index 7bab266e33e0..9f6de787de87 100644 --- a/crates/payload/primitives/Cargo.toml +++ b/crates/payload/primitives/Cargo.toml @@ -25,6 +25,7 @@ alloy-rpc-types-engine = { workspace = true, features = ["serde"] } op-alloy-rpc-types-engine = { workspace = true, optional = true } # misc +auto_impl.workspace = true serde.workspace = true thiserror.workspace = true tokio = { workspace = true, default-features = false, features = ["sync"] } diff --git a/crates/payload/primitives/src/traits.rs b/crates/payload/primitives/src/traits.rs index 18100c796233..6ae91452fa24 100644 --- a/crates/payload/primitives/src/traits.rs +++ b/crates/payload/primitives/src/traits.rs @@ -11,6 +11,7 @@ use reth_primitives::{NodePrimitives, SealedBlock}; /// Represents a built payload type that contains a built `SealedBlock` and can be converted into /// engine API execution payloads. +#[auto_impl::auto_impl(&, Arc)] pub trait BuiltPayload: Send + Sync + fmt::Debug { /// The node's primitive types type Primitives: NodePrimitives; From 09625b7813ec3116de098b041670a045b6cf4734 Mon Sep 17 00:00:00 2001 From: Arsenii Kulikov Date: Sun, 2 Feb 2025 03:03:40 +0400 Subject: [PATCH 2/5] fix doc --- crates/optimism/consensus/src/proof.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/optimism/consensus/src/proof.rs b/crates/optimism/consensus/src/proof.rs index d6ee80679cc2..f1c7e41f9240 100644 --- a/crates/optimism/consensus/src/proof.rs +++ b/crates/optimism/consensus/src/proof.rs @@ -40,7 +40,7 @@ pub(crate) fn calculate_receipt_root_optimism( ordered_trie_root_with_encoder(receipts, |r, buf| r.encode_2718(buf)) } -/// Calculates the receipt root for a header for the reference type of [`OpReceipt`]. +/// Calculates the receipt root for a header for the reference type of an OP receipt. /// /// NOTE: Prefer calculate receipt root optimism if you have log blooms memoized. pub fn calculate_receipt_root_no_memo_optimism( From 9bad1da4d829f740de1b9c7aae9291c81447cab4 Mon Sep 17 00:00:00 2001 From: Arsenii Kulikov Date: Sun, 2 Feb 2025 03:54:14 +0400 Subject: [PATCH 3/5] OpPayloadTypes --- crates/optimism/node/src/engine.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/crates/optimism/node/src/engine.rs b/crates/optimism/node/src/engine.rs index b85624d1f4bb..5bb8aca45dbb 100644 --- a/crates/optimism/node/src/engine.rs +++ b/crates/optimism/node/src/engine.rs @@ -18,7 +18,7 @@ use reth_node_api::{ use reth_optimism_chainspec::OpChainSpec; use reth_optimism_forks::{OpHardfork, OpHardforks}; use reth_optimism_payload_builder::{OpBuiltPayload, OpPayloadBuilderAttributes}; -use reth_optimism_primitives::{OpBlock, OpTransactionSigned}; +use reth_optimism_primitives::{OpBlock, OpPrimitives}; use reth_payload_validator::ExecutionPayloadValidator; use reth_primitives::SealedBlock; use std::sync::Arc; @@ -61,12 +61,12 @@ where /// A default payload type for [`OpEngineTypes`] #[derive(Debug, Default, Clone, serde::Deserialize, serde::Serialize)] #[non_exhaustive] -pub struct OpPayloadTypes; +pub struct OpPayloadTypes(core::marker::PhantomData); -impl PayloadTypes for OpPayloadTypes { - type BuiltPayload = OpBuiltPayload; +impl PayloadTypes for OpPayloadTypes { + type BuiltPayload = OpBuiltPayload; type PayloadAttributes = OpPayloadAttributes; - type PayloadBuilderAttributes = OpPayloadBuilderAttributes; + type PayloadBuilderAttributes = OpPayloadBuilderAttributes; } /// Validator for Optimism engine API. From 9b912cce942260a928fa93e01ddf95684d0e1071 Mon Sep 17 00:00:00 2001 From: Arsenii Kulikov Date: Sun, 2 Feb 2025 20:39:39 +0400 Subject: [PATCH 4/5] move trait --- crates/optimism/payload/src/builder.rs | 33 +++----------------------- crates/optimism/payload/src/lib.rs | 2 ++ crates/optimism/payload/src/traits.rs | 30 +++++++++++++++++++++++ crates/optimism/rpc/src/witness.rs | 2 +- 4 files changed, 36 insertions(+), 31 deletions(-) create mode 100644 crates/optimism/payload/src/traits.rs diff --git a/crates/optimism/payload/src/builder.rs b/crates/optimism/payload/src/builder.rs index f9f1f3e912df..eaf4248e4dab 100644 --- a/crates/optimism/payload/src/builder.rs +++ b/crates/optimism/payload/src/builder.rs @@ -4,6 +4,7 @@ use crate::{ config::{OpBuilderConfig, OpDAConfig}, error::OpPayloadBuilderError, payload::{OpBuiltPayload, OpPayloadBuilderAttributes}, + OpPayloadPrimitives, }; use alloy_consensus::{Eip658Value, Header, Transaction, Typed2718, EMPTY_OMMER_ROOT_HASH}; use alloy_eips::{eip4895::Withdrawals, merge::BEACON_NONCE}; @@ -25,16 +26,14 @@ use reth_optimism_chainspec::OpChainSpec; use reth_optimism_consensus::calculate_receipt_root_no_memo_optimism; use reth_optimism_evm::{OpReceiptBuilder, ReceiptBuilderCtx}; use reth_optimism_forks::OpHardforks; -use reth_optimism_primitives::{ - transaction::signed::OpTransaction, DepositReceipt, OpTransactionSigned, -}; +use reth_optimism_primitives::{transaction::signed::OpTransaction, OpTransactionSigned}; use reth_payload_builder_primitives::PayloadBuilderError; use reth_payload_primitives::PayloadBuilderAttributes; use reth_payload_util::{NoopPayloadTransactions, PayloadTransactions}; use reth_primitives::{ transaction::SignedTransactionIntoRecoveredExt, BlockBody, NodePrimitives, SealedHeader, }; -use reth_primitives_traits::{block::Block as _, proofs, RecoveredBlock, SignedTransaction}; +use reth_primitives_traits::{block::Block as _, proofs, RecoveredBlock}; use reth_provider::{ HashedPostStateProvider, ProviderError, StateProofProvider, StateProviderFactory, StateRootProvider, @@ -51,32 +50,6 @@ use revm::{ use std::{fmt::Display, sync::Arc}; use tracing::{debug, trace, warn}; -/// Helper trait to encapsulate common bounds on [`NodePrimitives`] for OP payload builder. -pub trait OpPayloadPrimitives: - NodePrimitives< - Receipt: DepositReceipt, - SignedTx = Self::_TX, - BlockHeader = Header, - BlockBody = BlockBody, -> -{ - /// Helper AT to bound [`NodePrimitives::Block`] type without causing bound cycle. - type _TX: SignedTransaction + OpTransaction; -} - -impl OpPayloadPrimitives for T -where - Tx: SignedTransaction + OpTransaction, - T: NodePrimitives< - SignedTx = Tx, - Receipt: DepositReceipt, - BlockHeader = Header, - BlockBody = BlockBody, - >, -{ - type _TX = Tx; -} - /// Optimism's payload builder #[derive(Debug, Clone)] pub struct OpPayloadBuilder { diff --git a/crates/optimism/payload/src/lib.rs b/crates/optimism/payload/src/lib.rs index 53fad1118fd0..2bb66f0cf038 100644 --- a/crates/optimism/payload/src/lib.rs +++ b/crates/optimism/payload/src/lib.rs @@ -16,5 +16,7 @@ pub use builder::OpPayloadBuilder; pub mod error; pub mod payload; pub use payload::{OpBuiltPayload, OpPayloadAttributes, OpPayloadBuilderAttributes}; +mod traits; +pub use traits::*; pub mod config; diff --git a/crates/optimism/payload/src/traits.rs b/crates/optimism/payload/src/traits.rs new file mode 100644 index 000000000000..325e91218390 --- /dev/null +++ b/crates/optimism/payload/src/traits.rs @@ -0,0 +1,30 @@ +use alloy_consensus::{BlockBody, Header}; +use reth_optimism_primitives::{transaction::signed::OpTransaction, DepositReceipt}; +use reth_primitives::NodePrimitives; +use reth_primitives_traits::SignedTransaction; + +/// Helper trait to encapsulate common bounds on [`NodePrimitives`] for OP payload builder. +pub trait OpPayloadPrimitives: + NodePrimitives< + Receipt: DepositReceipt, + SignedTx = Self::_TX, + BlockHeader = Header, + BlockBody = BlockBody, +> +{ + /// Helper AT to bound [`NodePrimitives::Block`] type without causing bound cycle. + type _TX: SignedTransaction + OpTransaction; +} + +impl OpPayloadPrimitives for T +where + Tx: SignedTransaction + OpTransaction, + T: NodePrimitives< + SignedTx = Tx, + Receipt: DepositReceipt, + BlockHeader = Header, + BlockBody = BlockBody, + >, +{ + type _TX = Tx; +} diff --git a/crates/optimism/rpc/src/witness.rs b/crates/optimism/rpc/src/witness.rs index 63ccbc709097..2acd4cd4748c 100644 --- a/crates/optimism/rpc/src/witness.rs +++ b/crates/optimism/rpc/src/witness.rs @@ -7,7 +7,7 @@ use op_alloy_rpc_types_engine::OpPayloadAttributes; use reth_chainspec::ChainSpecProvider; use reth_evm::ConfigureEvmFor; use reth_optimism_chainspec::OpChainSpec; -use reth_optimism_payload_builder::{builder::OpPayloadPrimitives, OpPayloadBuilder}; +use reth_optimism_payload_builder::{OpPayloadBuilder, OpPayloadPrimitives}; use reth_primitives::SealedHeader; use reth_provider::{ BlockReaderIdExt, NodePrimitivesProvider, ProviderError, ProviderResult, StateProviderFactory, From 253995f9c32828141fcf40d5ed86c0a6db99d542 Mon Sep 17 00:00:00 2001 From: Arsenii Kulikov Date: Mon, 3 Feb 2025 18:44:08 +0400 Subject: [PATCH 5/5] re-add SealedBlock --- crates/optimism/payload/src/builder.rs | 3 +- crates/optimism/payload/src/payload.rs | 38 ++++++++++++++------------ 2 files changed, 23 insertions(+), 18 deletions(-) diff --git a/crates/optimism/payload/src/builder.rs b/crates/optimism/payload/src/builder.rs index eaf4248e4dab..458eeae2a6ad 100644 --- a/crates/optimism/payload/src/builder.rs +++ b/crates/optimism/payload/src/builder.rs @@ -486,7 +486,8 @@ where let no_tx_pool = ctx.attributes().no_tx_pool; - let payload = OpBuiltPayload::new(ctx.payload_id(), info.total_fees, executed); + let payload = + OpBuiltPayload::new(ctx.payload_id(), sealed_block, info.total_fees, Some(executed)); if no_tx_pool { // if `no_tx_pool` is set only transactions from the payload attributes will be included diff --git a/crates/optimism/payload/src/payload.rs b/crates/optimism/payload/src/payload.rs index 2c57e95fd911..da2f64c73fa8 100644 --- a/crates/optimism/payload/src/payload.rs +++ b/crates/optimism/payload/src/payload.rs @@ -1,6 +1,6 @@ //! Payload related types -use std::fmt::Debug; +use std::{fmt::Debug, sync::Arc}; use alloy_eips::{ eip1559::BaseFeeParams, eip2718::Decodable2718, eip4895::Withdrawals, eip7685::Requests, @@ -147,8 +147,10 @@ impl PayloadBuilderAttributes pub struct OpBuiltPayload { /// Identifier of the payload pub(crate) id: PayloadId, + /// Sealed block + pub(crate) block: Arc>, /// Block execution data for the payload, if any. - pub(crate) block: ExecutedBlockWithTrieUpdates, + pub(crate) executed_block: Option>, /// The fees of the block pub(crate) fees: U256, } @@ -157,8 +159,13 @@ pub struct OpBuiltPayload { impl OpBuiltPayload { /// Initializes the payload with the given initial block. - pub const fn new(id: PayloadId, fees: U256, block: ExecutedBlockWithTrieUpdates) -> Self { - Self { id, block, fees } + pub const fn new( + id: PayloadId, + block: Arc>, + fees: U256, + executed_block: Option>, + ) -> Self { + Self { id, block, fees, executed_block } } /// Returns the identifier of the payload. @@ -168,7 +175,7 @@ impl OpBuiltPayload { /// Returns the built block(sealed) pub fn block(&self) -> &SealedBlock { - self.block.sealed_block() + &self.block } /// Fees of the block @@ -189,7 +196,7 @@ impl BuiltPayload for OpBuiltPayload { } fn executed_block(&self) -> Option> { - Some(self.block.clone()) + self.executed_block.clone() } fn requests(&self) -> Option { @@ -206,7 +213,7 @@ where fn from(value: OpBuiltPayload) -> Self { Self::from_block_unchecked( value.block().hash(), - &value.block.into_sealed_block().into_block(), + &Arc::unwrap_or_clone(value.block).into_block(), ) } } @@ -220,12 +227,11 @@ where fn from(value: OpBuiltPayload) -> Self { let OpBuiltPayload { block, fees, .. } = value; - let block = block.into_sealed_block(); Self { block_value: fees, execution_payload: ExecutionPayloadFieldV2::from_block_unchecked( block.hash(), - &block.into_block(), + &Arc::unwrap_or_clone(block).into_block(), ), } } @@ -239,13 +245,12 @@ where fn from(value: OpBuiltPayload) -> Self { let OpBuiltPayload { block, fees, .. } = value; - let parent_beacon_block_root = - block.sealed_block().parent_beacon_block_root.unwrap_or_default(); + let parent_beacon_block_root = block.parent_beacon_block_root.unwrap_or_default(); Self { execution_payload: ExecutionPayloadV3::from_block_unchecked( - block.sealed_block().hash(), - &block.into_sealed_block().into_block(), + block.hash(), + &Arc::unwrap_or_clone(block).into_block(), ), block_value: fees, // From the engine API spec: @@ -272,13 +277,12 @@ where fn from(value: OpBuiltPayload) -> Self { let OpBuiltPayload { block, fees, .. } = value; - let parent_beacon_block_root = - block.sealed_block().parent_beacon_block_root.unwrap_or_default(); + let parent_beacon_block_root = block.parent_beacon_block_root.unwrap_or_default(); Self { execution_payload: ExecutionPayloadV3::from_block_unchecked( - block.sealed_block().hash(), - &block.into_sealed_block().into_block(), + block.hash(), + &Arc::unwrap_or_clone(block).into_block(), ), block_value: fees, // From the engine API spec: