reth_optimism_payload_builder/
builder.rs

1//! Optimism payload builder implementation.
2use crate::{
3    config::OpBuilderConfig, error::OpPayloadBuilderError, payload::OpBuiltPayload, OpAttributes,
4    OpPayloadBuilderAttributes, OpPayloadPrimitives,
5};
6use alloy_consensus::{BlockHeader, Transaction, Typed2718};
7use alloy_evm::Evm as AlloyEvm;
8use alloy_primitives::{B256, U256};
9use alloy_rpc_types_debug::ExecutionWitness;
10use alloy_rpc_types_engine::PayloadId;
11use reth_basic_payload_builder::*;
12use reth_chainspec::{ChainSpecProvider, EthChainSpec};
13use reth_evm::{
14    block::BlockExecutorFor,
15    execute::{
16        BlockBuilder, BlockBuilderOutcome, BlockExecutionError, BlockExecutor, BlockValidationError,
17    },
18    op_revm::{constants::L1_BLOCK_CONTRACT, L1BlockInfo},
19    ConfigureEvm, Database,
20};
21use reth_execution_types::ExecutionOutcome;
22use reth_optimism_forks::OpHardforks;
23use reth_optimism_primitives::{transaction::OpTransaction, L2_TO_L1_MESSAGE_PASSER_ADDRESS};
24use reth_optimism_txpool::{
25    estimated_da_size::DataAvailabilitySized,
26    interop::{is_valid_interop, MaybeInteropTransaction},
27    OpPooledTx,
28};
29use reth_payload_builder_primitives::PayloadBuilderError;
30use reth_payload_primitives::{BuildNextEnv, BuiltPayloadExecutedBlock, PayloadBuilderAttributes};
31use reth_payload_util::{BestPayloadTransactions, NoopPayloadTransactions, PayloadTransactions};
32use reth_primitives_traits::{
33    HeaderTy, NodePrimitives, SealedHeader, SealedHeaderFor, SignedTransaction, TxTy,
34};
35use reth_revm::{
36    cancelled::CancelOnDrop, database::StateProviderDatabase, db::State,
37    witness::ExecutionWitnessRecord,
38};
39use reth_storage_api::{errors::ProviderError, StateProvider, StateProviderFactory};
40use reth_transaction_pool::{BestTransactionsAttributes, PoolTransaction, TransactionPool};
41use revm::context::{Block, BlockEnv};
42use std::{marker::PhantomData, sync::Arc};
43use tracing::{debug, trace, warn};
44
45/// Optimism's payload builder
46#[derive(Debug)]
47pub struct OpPayloadBuilder<
48    Pool,
49    Client,
50    Evm,
51    Txs = (),
52    Attrs = OpPayloadBuilderAttributes<TxTy<<Evm as ConfigureEvm>::Primitives>>,
53> {
54    /// The rollup's compute pending block configuration option.
55    // TODO(clabby): Implement this feature.
56    pub compute_pending_block: bool,
57    /// The type responsible for creating the evm.
58    pub evm_config: Evm,
59    /// Transaction pool.
60    pub pool: Pool,
61    /// Node client.
62    pub client: Client,
63    /// Settings for the builder, e.g. DA settings.
64    pub config: OpBuilderConfig,
65    /// The type responsible for yielding the best transactions for the payload if mempool
66    /// transactions are allowed.
67    pub best_transactions: Txs,
68    /// Marker for the payload attributes type.
69    _pd: PhantomData<Attrs>,
70}
71
72impl<Pool, Client, Evm, Txs, Attrs> Clone for OpPayloadBuilder<Pool, Client, Evm, Txs, Attrs>
73where
74    Pool: Clone,
75    Client: Clone,
76    Evm: ConfigureEvm,
77    Txs: Clone,
78{
79    fn clone(&self) -> Self {
80        Self {
81            evm_config: self.evm_config.clone(),
82            pool: self.pool.clone(),
83            client: self.client.clone(),
84            config: self.config.clone(),
85            best_transactions: self.best_transactions.clone(),
86            compute_pending_block: self.compute_pending_block,
87            _pd: PhantomData,
88        }
89    }
90}
91
92impl<Pool, Client, Evm, Attrs> OpPayloadBuilder<Pool, Client, Evm, (), Attrs> {
93    /// `OpPayloadBuilder` constructor.
94    ///
95    /// Configures the builder with the default settings.
96    pub fn new(pool: Pool, client: Client, evm_config: Evm) -> Self {
97        Self::with_builder_config(pool, client, evm_config, Default::default())
98    }
99
100    /// Configures the builder with the given [`OpBuilderConfig`].
101    pub const fn with_builder_config(
102        pool: Pool,
103        client: Client,
104        evm_config: Evm,
105        config: OpBuilderConfig,
106    ) -> Self {
107        Self {
108            pool,
109            client,
110            compute_pending_block: true,
111            evm_config,
112            config,
113            best_transactions: (),
114            _pd: PhantomData,
115        }
116    }
117}
118
119impl<Pool, Client, Evm, Txs, Attrs> OpPayloadBuilder<Pool, Client, Evm, Txs, Attrs> {
120    /// Sets the rollup's compute pending block configuration option.
121    pub const fn set_compute_pending_block(mut self, compute_pending_block: bool) -> Self {
122        self.compute_pending_block = compute_pending_block;
123        self
124    }
125
126    /// Configures the type responsible for yielding the transactions that should be included in the
127    /// payload.
128    pub fn with_transactions<T>(
129        self,
130        best_transactions: T,
131    ) -> OpPayloadBuilder<Pool, Client, Evm, T, Attrs> {
132        let Self { pool, client, compute_pending_block, evm_config, config, .. } = self;
133        OpPayloadBuilder {
134            pool,
135            client,
136            compute_pending_block,
137            evm_config,
138            best_transactions,
139            config,
140            _pd: PhantomData,
141        }
142    }
143
144    /// Enables the rollup's compute pending block configuration option.
145    pub const fn compute_pending_block(self) -> Self {
146        self.set_compute_pending_block(true)
147    }
148
149    /// Returns the rollup's compute pending block configuration option.
150    pub const fn is_compute_pending_block(&self) -> bool {
151        self.compute_pending_block
152    }
153}
154
155impl<Pool, Client, Evm, N, T, Attrs> OpPayloadBuilder<Pool, Client, Evm, T, Attrs>
156where
157    Pool: TransactionPool<Transaction: OpPooledTx<Consensus = N::SignedTx>>,
158    Client: StateProviderFactory + ChainSpecProvider<ChainSpec: OpHardforks>,
159    N: OpPayloadPrimitives,
160    Evm: ConfigureEvm<
161        Primitives = N,
162        NextBlockEnvCtx: BuildNextEnv<Attrs, N::BlockHeader, Client::ChainSpec>,
163    >,
164    Attrs: OpAttributes<Transaction = TxTy<Evm::Primitives>>,
165{
166    /// Constructs an Optimism payload from the transactions sent via the
167    /// Payload attributes by the sequencer. If the `no_tx_pool` argument is passed in
168    /// the payload attributes, the transaction pool will be ignored and the only transactions
169    /// included in the payload will be those sent through the attributes.
170    ///
171    /// Given build arguments including an Optimism client, transaction pool,
172    /// and configuration, this function creates a transaction payload. Returns
173    /// a result indicating success with the payload or an error in case of failure.
174    fn build_payload<'a, Txs>(
175        &self,
176        args: BuildArguments<Attrs, OpBuiltPayload<N>>,
177        best: impl FnOnce(BestTransactionsAttributes) -> Txs + Send + Sync + 'a,
178    ) -> Result<BuildOutcome<OpBuiltPayload<N>>, PayloadBuilderError>
179    where
180        Txs:
181            PayloadTransactions<Transaction: PoolTransaction<Consensus = N::SignedTx> + OpPooledTx>,
182    {
183        let BuildArguments { mut cached_reads, config, cancel, best_payload } = args;
184
185        let ctx = OpPayloadBuilderCtx {
186            evm_config: self.evm_config.clone(),
187            builder_config: self.config.clone(),
188            chain_spec: self.client.chain_spec(),
189            config,
190            cancel,
191            best_payload,
192        };
193
194        let builder = OpBuilder::new(best);
195
196        let state_provider = self.client.state_by_block_hash(ctx.parent().hash())?;
197        let state = StateProviderDatabase::new(&state_provider);
198
199        if ctx.attributes().no_tx_pool() {
200            builder.build(state, &state_provider, ctx)
201        } else {
202            // sequencer mode we can reuse cachedreads from previous runs
203            builder.build(cached_reads.as_db_mut(state), &state_provider, ctx)
204        }
205        .map(|out| out.with_cached_reads(cached_reads))
206    }
207
208    /// Computes the witness for the payload.
209    pub fn payload_witness(
210        &self,
211        parent: SealedHeader<N::BlockHeader>,
212        attributes: Attrs::RpcPayloadAttributes,
213    ) -> Result<ExecutionWitness, PayloadBuilderError>
214    where
215        Attrs: PayloadBuilderAttributes,
216    {
217        let attributes =
218            Attrs::try_new(parent.hash(), attributes, 3).map_err(PayloadBuilderError::other)?;
219
220        let config = PayloadConfig { parent_header: Arc::new(parent), attributes };
221        let ctx = OpPayloadBuilderCtx {
222            evm_config: self.evm_config.clone(),
223            builder_config: self.config.clone(),
224            chain_spec: self.client.chain_spec(),
225            config,
226            cancel: Default::default(),
227            best_payload: Default::default(),
228        };
229
230        let state_provider = self.client.state_by_block_hash(ctx.parent().hash())?;
231
232        let builder = OpBuilder::new(|_| NoopPayloadTransactions::<Pool::Transaction>::default());
233        builder.witness(state_provider, &ctx)
234    }
235}
236
237/// Implementation of the [`PayloadBuilder`] trait for [`OpPayloadBuilder`].
238impl<Pool, Client, Evm, N, Txs, Attrs> PayloadBuilder
239    for OpPayloadBuilder<Pool, Client, Evm, Txs, Attrs>
240where
241    N: OpPayloadPrimitives,
242    Client: StateProviderFactory + ChainSpecProvider<ChainSpec: OpHardforks> + Clone,
243    Pool: TransactionPool<Transaction: OpPooledTx<Consensus = N::SignedTx>>,
244    Evm: ConfigureEvm<
245        Primitives = N,
246        NextBlockEnvCtx: BuildNextEnv<Attrs, N::BlockHeader, Client::ChainSpec>,
247    >,
248    Txs: OpPayloadTransactions<Pool::Transaction>,
249    Attrs: OpAttributes<Transaction = N::SignedTx>,
250{
251    type Attributes = Attrs;
252    type BuiltPayload = OpBuiltPayload<N>;
253
254    fn try_build(
255        &self,
256        args: BuildArguments<Self::Attributes, Self::BuiltPayload>,
257    ) -> Result<BuildOutcome<Self::BuiltPayload>, PayloadBuilderError> {
258        let pool = self.pool.clone();
259        self.build_payload(args, |attrs| self.best_transactions.best_transactions(pool, attrs))
260    }
261
262    fn on_missing_payload(
263        &self,
264        _args: BuildArguments<Self::Attributes, Self::BuiltPayload>,
265    ) -> MissingPayloadBehaviour<Self::BuiltPayload> {
266        // we want to await the job that's already in progress because that should be returned as
267        // is, there's no benefit in racing another job
268        MissingPayloadBehaviour::AwaitInProgress
269    }
270
271    // NOTE: this should only be used for testing purposes because this doesn't have access to L1
272    // system txs, hence on_missing_payload we return [MissingPayloadBehaviour::AwaitInProgress].
273    fn build_empty_payload(
274        &self,
275        config: PayloadConfig<Self::Attributes, N::BlockHeader>,
276    ) -> Result<Self::BuiltPayload, PayloadBuilderError> {
277        let args = BuildArguments {
278            config,
279            cached_reads: Default::default(),
280            cancel: Default::default(),
281            best_payload: None,
282        };
283        self.build_payload(args, |_| NoopPayloadTransactions::<Pool::Transaction>::default())?
284            .into_payload()
285            .ok_or_else(|| PayloadBuilderError::MissingPayload)
286    }
287}
288
289/// The type that builds the payload.
290///
291/// Payload building for optimism is composed of several steps.
292/// The first steps are mandatory and defined by the protocol.
293///
294/// 1. first all System calls are applied.
295/// 2. After canyon the forced deployed `create2deployer` must be loaded
296/// 3. all sequencer transactions are executed (part of the payload attributes)
297///
298/// Depending on whether the node acts as a sequencer and is allowed to include additional
299/// transactions (`no_tx_pool == false`):
300/// 4. include additional transactions
301///
302/// And finally
303/// 5. build the block: compute all roots (txs, state)
304#[derive(derive_more::Debug)]
305pub struct OpBuilder<'a, Txs> {
306    /// Yields the best transaction to include if transactions from the mempool are allowed.
307    #[debug(skip)]
308    best: Box<dyn FnOnce(BestTransactionsAttributes) -> Txs + 'a>,
309}
310
311impl<'a, Txs> OpBuilder<'a, Txs> {
312    /// Creates a new [`OpBuilder`].
313    pub fn new(best: impl FnOnce(BestTransactionsAttributes) -> Txs + Send + Sync + 'a) -> Self {
314        Self { best: Box::new(best) }
315    }
316}
317
318impl<Txs> OpBuilder<'_, Txs> {
319    /// Builds the payload on top of the state.
320    pub fn build<Evm, ChainSpec, N, Attrs>(
321        self,
322        db: impl Database<Error = ProviderError>,
323        state_provider: impl StateProvider,
324        ctx: OpPayloadBuilderCtx<Evm, ChainSpec, Attrs>,
325    ) -> Result<BuildOutcomeKind<OpBuiltPayload<N>>, PayloadBuilderError>
326    where
327        Evm: ConfigureEvm<
328            Primitives = N,
329            NextBlockEnvCtx: BuildNextEnv<Attrs, N::BlockHeader, ChainSpec>,
330        >,
331        ChainSpec: EthChainSpec + OpHardforks,
332        N: OpPayloadPrimitives,
333        Txs:
334            PayloadTransactions<Transaction: PoolTransaction<Consensus = N::SignedTx> + OpPooledTx>,
335        Attrs: OpAttributes<Transaction = N::SignedTx>,
336    {
337        let Self { best } = self;
338        debug!(target: "payload_builder", id=%ctx.payload_id(), parent_header = ?ctx.parent().hash(), parent_number = ctx.parent().number(), "building new payload");
339
340        let mut db = State::builder().with_database(db).with_bundle_update().build();
341
342        // Load the L1 block contract into the database cache. If the L1 block contract is not
343        // pre-loaded the database will panic when trying to fetch the DA footprint gas
344        // scalar.
345        db.load_cache_account(L1_BLOCK_CONTRACT).map_err(BlockExecutionError::other)?;
346
347        let mut builder = ctx.block_builder(&mut db)?;
348
349        // 1. apply pre-execution changes
350        builder.apply_pre_execution_changes().map_err(|err| {
351            warn!(target: "payload_builder", %err, "failed to apply pre-execution changes");
352            PayloadBuilderError::Internal(err.into())
353        })?;
354
355        // 2. execute sequencer transactions
356        let mut info = ctx.execute_sequencer_transactions(&mut builder)?;
357
358        // 3. if mem pool transactions are requested we execute them
359        if !ctx.attributes().no_tx_pool() {
360            let best_txs = best(ctx.best_transaction_attributes(builder.evm_mut().block()));
361            if ctx.execute_best_transactions(&mut info, &mut builder, best_txs)?.is_some() {
362                return Ok(BuildOutcomeKind::Cancelled)
363            }
364
365            // check if the new payload is even more valuable
366            if !ctx.is_better_payload(info.total_fees) {
367                // can skip building the block
368                return Ok(BuildOutcomeKind::Aborted { fees: info.total_fees })
369            }
370        }
371
372        let BlockBuilderOutcome { execution_result, hashed_state, trie_updates, block } =
373            builder.finish(state_provider)?;
374
375        let sealed_block = Arc::new(block.sealed_block().clone());
376        debug!(target: "payload_builder", id=%ctx.attributes().payload_id(), sealed_block_header = ?sealed_block.header(), "sealed built block");
377
378        let execution_outcome = ExecutionOutcome::new(
379            db.take_bundle(),
380            vec![execution_result.receipts],
381            block.number(),
382            Vec::new(),
383        );
384
385        // create the executed block data
386        let executed: BuiltPayloadExecutedBlock<N> = BuiltPayloadExecutedBlock {
387            recovered_block: Arc::new(block),
388            execution_output: Arc::new(execution_outcome),
389            // Keep unsorted; conversion to sorted happens when needed downstream
390            hashed_state: either::Either::Left(Arc::new(hashed_state)),
391            trie_updates: either::Either::Left(Arc::new(trie_updates)),
392        };
393
394        let no_tx_pool = ctx.attributes().no_tx_pool();
395
396        let payload =
397            OpBuiltPayload::new(ctx.payload_id(), sealed_block, info.total_fees, Some(executed));
398
399        if no_tx_pool {
400            // if `no_tx_pool` is set only transactions from the payload attributes will be included
401            // in the payload. In other words, the payload is deterministic and we can
402            // freeze it once we've successfully built it.
403            Ok(BuildOutcomeKind::Freeze(payload))
404        } else {
405            Ok(BuildOutcomeKind::Better { payload })
406        }
407    }
408
409    /// Builds the payload and returns its [`ExecutionWitness`] based on the state after execution.
410    pub fn witness<Evm, ChainSpec, N, Attrs>(
411        self,
412        state_provider: impl StateProvider,
413        ctx: &OpPayloadBuilderCtx<Evm, ChainSpec, Attrs>,
414    ) -> Result<ExecutionWitness, PayloadBuilderError>
415    where
416        Evm: ConfigureEvm<
417            Primitives = N,
418            NextBlockEnvCtx: BuildNextEnv<Attrs, N::BlockHeader, ChainSpec>,
419        >,
420        ChainSpec: EthChainSpec + OpHardforks,
421        N: OpPayloadPrimitives,
422        Txs: PayloadTransactions<Transaction: PoolTransaction<Consensus = N::SignedTx>>,
423        Attrs: OpAttributes<Transaction = N::SignedTx>,
424    {
425        let mut db = State::builder()
426            .with_database(StateProviderDatabase::new(&state_provider))
427            .with_bundle_update()
428            .build();
429        let mut builder = ctx.block_builder(&mut db)?;
430
431        builder.apply_pre_execution_changes()?;
432        ctx.execute_sequencer_transactions(&mut builder)?;
433        builder.into_executor().apply_post_execution_changes()?;
434
435        if ctx.chain_spec.is_isthmus_active_at_timestamp(ctx.attributes().timestamp()) {
436            // force load `L2ToL1MessagePasser.sol` so l2 withdrawals root can be computed even if
437            // no l2 withdrawals in block
438            _ = db.load_cache_account(L2_TO_L1_MESSAGE_PASSER_ADDRESS)?;
439        }
440
441        let ExecutionWitnessRecord { hashed_state, codes, keys, lowest_block_number: _ } =
442            ExecutionWitnessRecord::from_executed_state(&db);
443        let state = state_provider.witness(Default::default(), hashed_state)?;
444        Ok(ExecutionWitness {
445            state: state.into_iter().collect(),
446            codes,
447            keys,
448            ..Default::default()
449        })
450    }
451}
452
453/// A type that returns a the [`PayloadTransactions`] that should be included in the pool.
454pub trait OpPayloadTransactions<Transaction>: Clone + Send + Sync + Unpin + 'static {
455    /// Returns an iterator that yields the transaction in the order they should get included in the
456    /// new payload.
457    fn best_transactions<Pool: TransactionPool<Transaction = Transaction>>(
458        &self,
459        pool: Pool,
460        attr: BestTransactionsAttributes,
461    ) -> impl PayloadTransactions<Transaction = Transaction>;
462}
463
464impl<T: PoolTransaction + MaybeInteropTransaction> OpPayloadTransactions<T> for () {
465    fn best_transactions<Pool: TransactionPool<Transaction = T>>(
466        &self,
467        pool: Pool,
468        attr: BestTransactionsAttributes,
469    ) -> impl PayloadTransactions<Transaction = T> {
470        BestPayloadTransactions::new(pool.best_transactions_with_attributes(attr))
471    }
472}
473
474/// Holds the state after execution
475#[derive(Debug)]
476pub struct ExecutedPayload<N: NodePrimitives> {
477    /// Tracked execution info
478    pub info: ExecutionInfo,
479    /// Withdrawal hash.
480    pub withdrawals_root: Option<B256>,
481    /// The transaction receipts.
482    pub receipts: Vec<N::Receipt>,
483    /// The block env used during execution.
484    pub block_env: BlockEnv,
485}
486
487/// This acts as the container for executed transactions and its byproducts (receipts, gas used)
488#[derive(Default, Debug)]
489pub struct ExecutionInfo {
490    /// All gas used so far
491    pub cumulative_gas_used: u64,
492    /// Estimated DA size
493    pub cumulative_da_bytes_used: u64,
494    /// Tracks fees from executed mempool transactions
495    pub total_fees: U256,
496}
497
498impl ExecutionInfo {
499    /// Create a new instance with allocated slots.
500    pub const fn new() -> Self {
501        Self { cumulative_gas_used: 0, cumulative_da_bytes_used: 0, total_fees: U256::ZERO }
502    }
503
504    /// Returns true if the transaction would exceed the block limits:
505    /// - block gas limit: ensures the transaction still fits into the block.
506    /// - tx DA limit: if configured, ensures the tx does not exceed the maximum allowed DA limit
507    ///   per tx.
508    /// - block DA limit: if configured, ensures the transaction's DA size does not exceed the
509    ///   maximum allowed DA limit per block.
510    pub fn is_tx_over_limits(
511        &self,
512        tx_da_size: u64,
513        block_gas_limit: u64,
514        tx_data_limit: Option<u64>,
515        block_data_limit: Option<u64>,
516        tx_gas_limit: u64,
517        da_footprint_gas_scalar: Option<u16>,
518    ) -> bool {
519        if tx_data_limit.is_some_and(|da_limit| tx_da_size > da_limit) {
520            return true;
521        }
522
523        let total_da_bytes_used = self.cumulative_da_bytes_used.saturating_add(tx_da_size);
524
525        if block_data_limit.is_some_and(|da_limit| total_da_bytes_used > da_limit) {
526            return true;
527        }
528
529        // Post Jovian: the tx DA footprint must be less than the block gas limit
530        if let Some(da_footprint_gas_scalar) = da_footprint_gas_scalar {
531            let tx_da_footprint =
532                total_da_bytes_used.saturating_mul(da_footprint_gas_scalar as u64);
533            if tx_da_footprint > block_gas_limit {
534                return true;
535            }
536        }
537
538        self.cumulative_gas_used + tx_gas_limit > block_gas_limit
539    }
540}
541
542/// Container type that holds all necessities to build a new payload.
543#[derive(derive_more::Debug)]
544pub struct OpPayloadBuilderCtx<
545    Evm: ConfigureEvm,
546    ChainSpec,
547    Attrs = OpPayloadBuilderAttributes<TxTy<<Evm as ConfigureEvm>::Primitives>>,
548> {
549    /// The type that knows how to perform system calls and configure the evm.
550    pub evm_config: Evm,
551    /// Additional config for the builder/sequencer, e.g. DA and gas limit
552    pub builder_config: OpBuilderConfig,
553    /// The chainspec
554    pub chain_spec: Arc<ChainSpec>,
555    /// How to build the payload.
556    pub config: PayloadConfig<Attrs, HeaderTy<Evm::Primitives>>,
557    /// Marker to check whether the job has been cancelled.
558    pub cancel: CancelOnDrop,
559    /// The currently best payload.
560    pub best_payload: Option<OpBuiltPayload<Evm::Primitives>>,
561}
562
563impl<Evm, ChainSpec, Attrs> OpPayloadBuilderCtx<Evm, ChainSpec, Attrs>
564where
565    Evm: ConfigureEvm<
566        Primitives: OpPayloadPrimitives,
567        NextBlockEnvCtx: BuildNextEnv<Attrs, HeaderTy<Evm::Primitives>, ChainSpec>,
568    >,
569    ChainSpec: EthChainSpec + OpHardforks,
570    Attrs: OpAttributes<Transaction = TxTy<Evm::Primitives>>,
571{
572    /// Returns the parent block the payload will be build on.
573    pub fn parent(&self) -> &SealedHeaderFor<Evm::Primitives> {
574        self.config.parent_header.as_ref()
575    }
576
577    /// Returns the builder attributes.
578    pub const fn attributes(&self) -> &Attrs {
579        &self.config.attributes
580    }
581
582    /// Returns the current fee settings for transactions from the mempool
583    pub fn best_transaction_attributes(&self, block_env: impl Block) -> BestTransactionsAttributes {
584        BestTransactionsAttributes::new(
585            block_env.basefee(),
586            block_env.blob_gasprice().map(|p| p as u64),
587        )
588    }
589
590    /// Returns the unique id for this payload job.
591    pub fn payload_id(&self) -> PayloadId {
592        self.attributes().payload_id()
593    }
594
595    /// Returns true if the fees are higher than the previous payload.
596    pub fn is_better_payload(&self, total_fees: U256) -> bool {
597        is_better_payload(self.best_payload.as_ref(), total_fees)
598    }
599
600    /// Prepares a [`BlockBuilder`] for the next block.
601    pub fn block_builder<'a, DB: Database>(
602        &'a self,
603        db: &'a mut State<DB>,
604    ) -> Result<
605        impl BlockBuilder<
606                Primitives = Evm::Primitives,
607                Executor: BlockExecutorFor<'a, Evm::BlockExecutorFactory, DB>,
608            > + 'a,
609        PayloadBuilderError,
610    > {
611        self.evm_config
612            .builder_for_next_block(
613                db,
614                self.parent(),
615                Evm::NextBlockEnvCtx::build_next_env(
616                    self.attributes(),
617                    self.parent(),
618                    self.chain_spec.as_ref(),
619                )
620                .map_err(PayloadBuilderError::other)?,
621            )
622            .map_err(PayloadBuilderError::other)
623    }
624
625    /// Executes all sequencer transactions that are included in the payload attributes.
626    pub fn execute_sequencer_transactions(
627        &self,
628        builder: &mut impl BlockBuilder<Primitives = Evm::Primitives>,
629    ) -> Result<ExecutionInfo, PayloadBuilderError> {
630        let mut info = ExecutionInfo::new();
631
632        for sequencer_tx in self.attributes().sequencer_transactions() {
633            // A sequencer's block should never contain blob transactions.
634            if sequencer_tx.value().is_eip4844() {
635                return Err(PayloadBuilderError::other(
636                    OpPayloadBuilderError::BlobTransactionRejected,
637                ))
638            }
639
640            // Convert the transaction to a [RecoveredTx]. This is
641            // purely for the purposes of utilizing the `evm_config.tx_env`` function.
642            // Deposit transactions do not have signatures, so if the tx is a deposit, this
643            // will just pull in its `from` address.
644            let sequencer_tx = sequencer_tx.value().try_clone_into_recovered().map_err(|_| {
645                PayloadBuilderError::other(OpPayloadBuilderError::TransactionEcRecoverFailed)
646            })?;
647
648            let gas_used = match builder.execute_transaction(sequencer_tx.clone()) {
649                Ok(gas_used) => gas_used,
650                Err(BlockExecutionError::Validation(BlockValidationError::InvalidTx {
651                    error,
652                    ..
653                })) => {
654                    trace!(target: "payload_builder", %error, ?sequencer_tx, "Error in sequencer transaction, skipping.");
655                    continue
656                }
657                Err(err) => {
658                    // this is an error that we should treat as fatal for this attempt
659                    return Err(PayloadBuilderError::EvmExecutionError(Box::new(err)))
660                }
661            };
662
663            // add gas used by the transaction to cumulative gas used, before creating the receipt
664            info.cumulative_gas_used += gas_used;
665        }
666
667        Ok(info)
668    }
669
670    /// Executes the given best transactions and updates the execution info.
671    ///
672    /// Returns `Ok(Some(())` if the job was cancelled.
673    pub fn execute_best_transactions<Builder>(
674        &self,
675        info: &mut ExecutionInfo,
676        builder: &mut Builder,
677        mut best_txs: impl PayloadTransactions<
678            Transaction: PoolTransaction<Consensus = TxTy<Evm::Primitives>> + OpPooledTx,
679        >,
680    ) -> Result<Option<()>, PayloadBuilderError>
681    where
682        Builder: BlockBuilder<Primitives = Evm::Primitives>,
683        <<Builder::Executor as BlockExecutor>::Evm as AlloyEvm>::DB: Database,
684    {
685        let mut block_gas_limit = builder.evm_mut().block().gas_limit();
686        if let Some(gas_limit_config) = self.builder_config.gas_limit_config.gas_limit() {
687            // If a gas limit is configured, use that limit as target if it's smaller, otherwise use
688            // the block's actual gas limit.
689            block_gas_limit = gas_limit_config.min(block_gas_limit);
690        };
691        let block_da_limit = self.builder_config.da_config.max_da_block_size();
692        let tx_da_limit = self.builder_config.da_config.max_da_tx_size();
693        let base_fee = builder.evm_mut().block().basefee();
694
695        while let Some(tx) = best_txs.next(()) {
696            let interop = tx.interop_deadline();
697            let tx_da_size = tx.estimated_da_size();
698            let tx = tx.into_consensus();
699
700            let da_footprint_gas_scalar = self
701                .chain_spec
702                .is_jovian_active_at_timestamp(self.attributes().timestamp())
703                .then_some(
704                    L1BlockInfo::fetch_da_footprint_gas_scalar(builder.evm_mut().db_mut()).expect(
705                        "DA footprint should always be available from the database post jovian",
706                    ),
707                );
708
709            if info.is_tx_over_limits(
710                tx_da_size,
711                block_gas_limit,
712                tx_da_limit,
713                block_da_limit,
714                tx.gas_limit(),
715                da_footprint_gas_scalar,
716            ) {
717                // we can't fit this transaction into the block, so we need to mark it as
718                // invalid which also removes all dependent transaction from
719                // the iterator before we can continue
720                best_txs.mark_invalid(tx.signer(), tx.nonce());
721                continue
722            }
723
724            // A sequencer's block should never contain blob or deposit transactions from the pool.
725            if tx.is_eip4844() || tx.is_deposit() {
726                best_txs.mark_invalid(tx.signer(), tx.nonce());
727                continue
728            }
729
730            // We skip invalid cross chain txs, they would be removed on the next block update in
731            // the maintenance job
732            if let Some(interop) = interop &&
733                !is_valid_interop(interop, self.config.attributes.timestamp())
734            {
735                best_txs.mark_invalid(tx.signer(), tx.nonce());
736                continue
737            }
738            // check if the job was cancelled, if so we can exit early
739            if self.cancel.is_cancelled() {
740                return Ok(Some(()))
741            }
742
743            let gas_used = match builder.execute_transaction(tx.clone()) {
744                Ok(gas_used) => gas_used,
745                Err(BlockExecutionError::Validation(BlockValidationError::InvalidTx {
746                    error,
747                    ..
748                })) => {
749                    if error.is_nonce_too_low() {
750                        // if the nonce is too low, we can skip this transaction
751                        trace!(target: "payload_builder", %error, ?tx, "skipping nonce too low transaction");
752                    } else {
753                        // if the transaction is invalid, we can skip it and all of its
754                        // descendants
755                        trace!(target: "payload_builder", %error, ?tx, "skipping invalid transaction and its descendants");
756                        best_txs.mark_invalid(tx.signer(), tx.nonce());
757                    }
758                    continue
759                }
760                Err(err) => {
761                    // this is an error that we should treat as fatal for this attempt
762                    return Err(PayloadBuilderError::EvmExecutionError(Box::new(err)))
763                }
764            };
765
766            // add gas used by the transaction to cumulative gas used, before creating the
767            // receipt
768            info.cumulative_gas_used += gas_used;
769            info.cumulative_da_bytes_used += tx_da_size;
770
771            // update and add to total fees
772            let miner_fee = tx
773                .effective_tip_per_gas(base_fee)
774                .expect("fee is always valid; execution succeeded");
775            info.total_fees += U256::from(miner_fee) * U256::from(gas_used);
776        }
777
778        Ok(None)
779    }
780}