diff --git a/crates/optimism/payload/src/builder.rs b/crates/optimism/payload/src/builder.rs index 0cf45835a..35e1d905a 100644 --- a/crates/optimism/payload/src/builder.rs +++ b/crates/optimism/payload/src/builder.rs @@ -1,13 +1,15 @@ //! Optimism payload builder implementation. -use std::sync::Arc; + +use std::{fmt::Display, sync::Arc}; use alloy_consensus::EMPTY_OMMER_ROOT_HASH; use alloy_eips::merge::BEACON_NONCE; -use alloy_primitives::{B64, U256}; +use alloy_primitives::{Address, Bytes, B64, U256}; +use alloy_rpc_types_engine::PayloadId; use reth_basic_payload_builder::*; use reth_chain_state::ExecutedBlock; use reth_chainspec::ChainSpecProvider; -use reth_evm::{system_calls::SystemCaller, ConfigureEvm, ConfigureEvmEnv, NextBlockEnvAttributes}; +use reth_evm::{system_calls::SystemCaller, ConfigureEvm, NextBlockEnvAttributes}; use reth_execution_types::ExecutionOutcome; use reth_optimism_chainspec::OpChainSpec; use reth_optimism_consensus::calculate_receipt_root_no_memo_optimism; @@ -16,18 +18,19 @@ use reth_payload_primitives::{PayloadBuilderAttributes, PayloadBuilderError}; use reth_primitives::{ proofs, revm_primitives::{BlockEnv, CfgEnvWithHandlerCfg}, - Block, BlockBody, Header, Receipt, TxType, + Block, BlockBody, Header, Receipt, SealedHeader, TransactionSigned, TxType, }; -use reth_provider::StateProviderFactory; +use reth_provider::{ProviderError, StateProviderFactory, StateRootProvider}; use reth_revm::database::StateProviderDatabase; use reth_transaction_pool::{ - noop::NoopTransactionPool, BestTransactionsAttributes, TransactionPool, + noop::NoopTransactionPool, BestTransactions, BestTransactionsAttributes, BestTransactionsFor, + TransactionPool, }; use reth_trie::HashedPostState; use revm::{ db::{states::bundle_state::BundleRetention, State}, primitives::{EVMError, EnvWithHandlerCfg, InvalidTransaction, ResultAndState}, - DatabaseCommit, + Database, DatabaseCommit, }; use tracing::{debug, trace, warn}; @@ -71,7 +74,7 @@ impl OpPayloadBuilder { } impl OpPayloadBuilder where - EvmConfig: ConfigureEvmEnv
, + EvmConfig: ConfigureEvm
, { /// Returns the configured [`CfgEnvWithHandlerCfg`] and [`BlockEnv`] for the targeted payload /// (that has the `parent` as its parent). @@ -87,6 +90,62 @@ where }; self.evm_config.next_cfg_and_block_env(parent, next_attributes) } + + /// Constructs an Optimism payload from the transactions sent via the + /// Payload attributes by the sequencer. If the `no_tx_pool` argument is passed in + /// the payload attributes, the transaction pool will be ignored and the only transactions + /// included in the payload will be those sent through the attributes. + /// + /// Given build arguments including an Optimism client, transaction pool, + /// and configuration, this function creates a transaction payload. Returns + /// a result indicating success with the payload or an error in case of failure. + fn build_payload( + &self, + args: BuildArguments, + ) -> Result, PayloadBuilderError> + where + Client: StateProviderFactory + ChainSpecProvider, + Pool: TransactionPool, + { + let (initialized_cfg, initialized_block_env) = self + .cfg_and_block_env(&args.config, &args.config.parent_header) + .map_err(PayloadBuilderError::other)?; + + let BuildArguments { client, pool, mut cached_reads, config, cancel, best_payload } = args; + + let ctx = OpPayloadBuilderCtx { + evm_config: self.evm_config.clone(), + chain_spec: client.chain_spec(), + config, + initialized_cfg, + initialized_block_env, + cancel, + best_payload, + }; + + let builder = OpBuilder { + pool, + // TODO(mattsse): make this configurable in the `OpPayloadBuilder` directly via an + // additional generic + best: best_txs::, + }; + + let state_provider = client.state_by_block_hash(ctx.parent().hash())?; + let state = StateProviderDatabase::new(state_provider); + + if ctx.attributes().no_tx_pool { + let db = State::builder().with_database(state).with_bundle_update().build(); + builder.build(db, ctx) + } else { + // sequencer mode we can reuse cachedreads from previous runs + let db = State::builder() + .with_database(cached_reads.as_db_mut(state)) + .with_bundle_update() + .build(); + builder.build(db, ctx) + } + .map(|out| out.with_cached_reads(cached_reads)) + } } /// Implementation of the [`PayloadBuilder`] trait for [`OpPayloadBuilder`]. @@ -103,10 +162,7 @@ where &self, args: BuildArguments, ) -> Result, PayloadBuilderError> { - let (cfg_env, block_env) = self - .cfg_and_block_env(&args.config, &args.config.parent_header) - .map_err(PayloadBuilderError::other)?; - optimism_payload(&self.evm_config, args, cfg_env, block_env, self.compute_pending_block) + self.build_payload(args) } fn on_missing_payload( @@ -134,198 +190,534 @@ where cancel: Default::default(), best_payload: None, }; - let (cfg_env, block_env) = self - .cfg_and_block_env(&args.config, &args.config.parent_header) - .map_err(PayloadBuilderError::other)?; - optimism_payload(&self.evm_config, args, cfg_env, block_env, false)? - .into_payload() - .ok_or_else(|| PayloadBuilderError::MissingPayload) + self.build_payload(args)?.into_payload().ok_or_else(|| PayloadBuilderError::MissingPayload) } } -/// Constructs an Optimism transaction payload from the transactions sent through the -/// Payload attributes by the sequencer. If the `no_tx_pool` argument is passed in -/// the payload attributes, the transaction pool will be ignored and the only transactions -/// included in the payload will be those sent through the attributes. +/// The type that builds the payload. /// -/// Given build arguments including an Optimism client, transaction pool, -/// and configuration, this function creates a transaction payload. Returns -/// a result indicating success with the payload or an error in case of failure. -#[inline] -pub(crate) fn optimism_payload( - evm_config: &EvmConfig, - args: BuildArguments, - initialized_cfg: CfgEnvWithHandlerCfg, - initialized_block_env: BlockEnv, - _compute_pending_block: bool, -) -> Result, PayloadBuilderError> +/// Payload building for optimism is composed of several steps. +/// The first steps are mandatory and defined by the protocol. +/// +/// 1. first all System calls are applied. +/// 2. After canyon the forced deployed `create2deployer` must be loaded +/// 3. all sequencer transactions are executed (part of the payload attributes) +/// +/// Depending on whether the node acts as a sequencer and is allowed to include additional +/// transactions (`no_tx_pool == false`): +/// 4. include additional transactions +/// +/// And finally +/// 5. build the block: compute all roots (txs, state) +#[derive(Debug)] +pub struct OpBuilder { + /// The transaction pool + pool: Pool, + /// Yields the best transaction to include if transactions from the mempool are allowed. + // TODO(mattsse): convert this to a trait + best: Best, +} + +impl OpBuilder where - EvmConfig: ConfigureEvm
, - Client: StateProviderFactory + ChainSpecProvider, Pool: TransactionPool, + Best: FnOnce(Pool, BestTransactionsAttributes) -> BestTransactionsFor, { - let BuildArguments { client, pool, mut cached_reads, config, cancel, best_payload } = args; + /// Builds the payload on top of the state. + pub fn build( + self, + mut db: State, + ctx: OpPayloadBuilderCtx, + ) -> Result, PayloadBuilderError> + where + EvmConfig: ConfigureEvm
, + DB: Database + AsRef

, + P: StateRootProvider, + { + let Self { pool, best } = self; + debug!(target: "payload_builder", id=%ctx.payload_id(), parent_header = ?ctx.parent().hash(), parent_number = ctx.parent().number, "building new payload"); - let chain_spec = client.chain_spec(); - let state_provider = client.state_by_block_hash(config.parent_header.hash())?; - let state = StateProviderDatabase::new(state_provider); - let mut db = - State::builder().with_database(cached_reads.as_db_mut(state)).with_bundle_update().build(); - let PayloadConfig { parent_header, attributes, mut extra_data } = config; + // 1. apply eip-4788 pre block contract call + ctx.apply_pre_beacon_root_contract_call(&mut db)?; - debug!(target: "payload_builder", id=%attributes.payload_attributes.payload_id(), parent_header = ?parent_header.hash(), parent_number = parent_header.number, "building new payload"); + // 2. ensure create2deployer is force deployed + ctx.ensure_create2_deployer(&mut db)?; - let mut cumulative_gas_used = 0; - let block_gas_limit: u64 = attributes.gas_limit.unwrap_or_else(|| { - initialized_block_env.gas_limit.try_into().unwrap_or(chain_spec.max_gas_limit) - }); - let base_fee = initialized_block_env.basefee.to::(); + // 3. execute sequencer transactions + let mut info = ctx.execute_sequencer_transactions(&mut db)?; - let mut executed_txs = Vec::with_capacity(attributes.transactions.len()); - let mut executed_senders = Vec::with_capacity(attributes.transactions.len()); - - let mut best_txs = pool.best_transactions_with_attributes(BestTransactionsAttributes::new( - base_fee, - initialized_block_env.get_blob_gasprice().map(|gasprice| gasprice as u64), - )); - - let mut total_fees = U256::ZERO; - - let block_number = initialized_block_env.number.to::(); - - let is_regolith = - chain_spec.is_regolith_active_at_timestamp(attributes.payload_attributes.timestamp); - - // apply eip-4788 pre block contract call - let mut system_caller = SystemCaller::new(evm_config.clone(), chain_spec.clone()); - - system_caller - .pre_block_beacon_root_contract_call( - &mut db, - &initialized_cfg, - &initialized_block_env, - attributes.payload_attributes.parent_beacon_block_root, - ) - .map_err(|err| { - warn!(target: "payload_builder", - parent_header=%parent_header.hash(), - %err, - "failed to apply beacon root contract call for payload" - ); - PayloadBuilderError::Internal(err.into()) - })?; - - // Ensure that the create2deployer is force-deployed at the canyon transition. Optimism - // blocks will always have at least a single transaction in them (the L1 info transaction), - // so we can safely assume that this will always be triggered upon the transition and that - // the above check for empty blocks will never be hit on OP chains. - reth_optimism_evm::ensure_create2_deployer( - chain_spec.clone(), - attributes.payload_attributes.timestamp, - &mut db, - ) - .map_err(|err| { - warn!(target: "payload_builder", %err, "missing create2 deployer, skipping block."); - PayloadBuilderError::other(OptimismPayloadBuilderError::ForceCreate2DeployerFail) - })?; - - let mut receipts = Vec::with_capacity(attributes.transactions.len()); - for sequencer_tx in &attributes.transactions { - // Check if the job was cancelled, if so we can exit early. - if cancel.is_cancelled() { - return Ok(BuildOutcome::Cancelled) - } - - // A sequencer's block should never contain blob transactions. - if sequencer_tx.value().is_eip4844() { - return Err(PayloadBuilderError::other( - OptimismPayloadBuilderError::BlobTransactionRejected, - )) - } - - // Convert the transaction to a [TransactionSignedEcRecovered]. This is - // purely for the purposes of utilizing the `evm_config.tx_env`` function. - // Deposit transactions do not have signatures, so if the tx is a deposit, this - // will just pull in its `from` address. - let sequencer_tx = sequencer_tx.value().clone().try_into_ecrecovered().map_err(|_| { - PayloadBuilderError::other(OptimismPayloadBuilderError::TransactionEcRecoverFailed) - })?; - - // Cache the depositor account prior to the state transition for the deposit nonce. - // - // Note that this *only* needs to be done post-regolith hardfork, as deposit nonces - // were not introduced in Bedrock. In addition, regular transactions don't have deposit - // nonces, so we don't need to touch the DB for those. - let depositor = (is_regolith && sequencer_tx.is_deposit()) - .then(|| { - db.load_cache_account(sequencer_tx.signer()) - .map(|acc| acc.account_info().unwrap_or_default()) - }) - .transpose() - .map_err(|_| { - PayloadBuilderError::other(OptimismPayloadBuilderError::AccountLoadFailed( - sequencer_tx.signer(), - )) - })?; - - let env = EnvWithHandlerCfg::new_with_cfg_env( - initialized_cfg.clone(), - initialized_block_env.clone(), - evm_config.tx_env(sequencer_tx.as_signed(), sequencer_tx.signer()), - ); - - let mut evm = evm_config.evm_with_env(&mut db, env); - - let ResultAndState { result, state } = match evm.transact() { - Ok(res) => res, - Err(err) => { - match err { - EVMError::Transaction(err) => { - trace!(target: "payload_builder", %err, ?sequencer_tx, "Error in sequencer transaction, skipping."); - continue - } - err => { - // this is an error that we should treat as fatal for this attempt - return Err(PayloadBuilderError::EvmExecutionError(err)) - } - } + // 4. if mem pool transactions are requested we execute them + if !ctx.attributes().no_tx_pool { + let best_txs = best(pool, ctx.best_transaction_attributes()); + if let Some(cancelled) = + ctx.execute_best_transactions::<_, Pool>(&mut info, &mut db, best_txs)? + { + return Ok(cancelled) } + + // check if the new payload is even more valuable + if !ctx.is_better_payload(info.total_fees) { + // can skip building the block + return Ok(BuildOutcomeKind::Aborted { fees: info.total_fees }) + } + } + + let WithdrawalsOutcome { withdrawals_root, withdrawals } = + ctx.commit_withdrawals(&mut db)?; + + // merge all transitions into bundle state, this would apply the withdrawal balance changes + // and 4788 contract call + db.merge_transitions(BundleRetention::Reverts); + + let block_number = ctx.block_number(); + let execution_outcome = ExecutionOutcome::new( + db.take_bundle(), + vec![info.receipts.clone()].into(), + block_number, + Vec::new(), + ); + let receipts_root = execution_outcome + .generic_receipts_root_slow(block_number, |receipts| { + calculate_receipt_root_no_memo_optimism( + receipts, + &ctx.chain_spec, + ctx.attributes().timestamp(), + ) + }) + .expect("Number is in range"); + let logs_bloom = + execution_outcome.block_logs_bloom(block_number).expect("Number is in range"); + + // // calculate the state root + let hashed_state = HashedPostState::from_bundle_state(&execution_outcome.state().state); + let (state_root, trie_output) = { + db.database.as_ref().state_root_with_updates(hashed_state.clone()).inspect_err( + |err| { + warn!(target: "payload_builder", + parent_header=%ctx.parent().hash(), + %err, + "failed to calculate state root for payload" + ); + }, + )? }; - // to release the db reference drop evm. - drop(evm); - // commit changes - db.commit(state); + // create the block header + let transactions_root = proofs::calculate_transaction_root(&info.executed_transactions); - let gas_used = result.gas_used(); + // OP doesn't support blobs/EIP-4844. + // https://specs.optimism.io/protocol/exec-engine.html#ecotone-disable-blob-transactions + // Need [Some] or [None] based on hardfork to match block hash. + let (excess_blob_gas, blob_gas_used) = ctx.blob_fields(); + let extra_data = ctx.extra_data()?; - // add gas used by the transaction to cumulative gas used, before creating the receipt - cumulative_gas_used += gas_used; + let header = Header { + parent_hash: ctx.parent().hash(), + ommers_hash: EMPTY_OMMER_ROOT_HASH, + beneficiary: ctx.initialized_block_env.coinbase, + state_root, + transactions_root, + receipts_root, + withdrawals_root, + logs_bloom, + timestamp: ctx.attributes().payload_attributes.timestamp, + mix_hash: ctx.attributes().payload_attributes.prev_randao, + nonce: BEACON_NONCE.into(), + base_fee_per_gas: Some(ctx.base_fee()), + number: ctx.parent().number + 1, + gas_limit: ctx.block_gas_limit(), + difficulty: U256::ZERO, + gas_used: info.cumulative_gas_used, + extra_data, + parent_beacon_block_root: ctx.attributes().payload_attributes.parent_beacon_block_root, + blob_gas_used, + excess_blob_gas, + requests_hash: None, + }; - // Push transaction changeset and calculate header bloom filter for receipt. - receipts.push(Some(Receipt { - tx_type: sequencer_tx.tx_type(), - success: result.is_success(), - cumulative_gas_used, - logs: result.into_logs().into_iter().map(Into::into).collect(), - 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: chain_spec - .is_canyon_active_at_timestamp(attributes.payload_attributes.timestamp) - .then_some(1), - })); + // seal the block + let block = Block { + header, + body: BlockBody { + transactions: info.executed_transactions, + ommers: vec![], + withdrawals, + }, + }; - // append sender and transaction to the respective lists - executed_senders.push(sequencer_tx.signer()); - executed_txs.push(sequencer_tx.into_signed()); + let sealed_block = block.seal_slow(); + debug!(target: "payload_builder", ?sealed_block, "sealed built block"); + + // create the executed block data + let executed = ExecutedBlock { + block: Arc::new(sealed_block.clone()), + senders: Arc::new(info.executed_senders), + execution_output: Arc::new(execution_outcome), + hashed_state: Arc::new(hashed_state), + trie: Arc::new(trie_output), + }; + + let no_tx_pool = ctx.attributes().no_tx_pool; + + let payload = OpBuiltPayload::new( + ctx.payload_id(), + sealed_block, + info.total_fees, + ctx.chain_spec.clone(), + ctx.config.attributes, + Some(executed), + ); + + if no_tx_pool { + // if `no_tx_pool` is set only transactions from the payload attributes will be included + // in the payload. In other words, the payload is deterministic and we can + // freeze it once we've successfully built it. + Ok(BuildOutcomeKind::Freeze(payload)) + } else { + Ok(BuildOutcomeKind::Better { payload }) + } + } +} + +fn best_txs( + pool: Pool, + attr: BestTransactionsAttributes, +) -> BestTransactionsFor { + pool.best_transactions_with_attributes(attr) +} + +/// This acts as the container for executed transactions and its byproducts (receipts, gas used) +#[derive(Default, Debug)] +pub struct ExecutionInfo { + /// All executed transactions (unrecovered). + pub executed_transactions: Vec, + /// The recovered senders for the executed transactions. + pub executed_senders: Vec

, + /// The transaction receipts + pub receipts: Vec>, + /// All gas used so far + pub cumulative_gas_used: u64, + /// Tracks fees from executed mempool transactions + pub total_fees: U256, +} + +impl ExecutionInfo { + /// Create a new instance with allocated slots. + pub fn with_capacity(capacity: usize) -> Self { + Self { + executed_transactions: Vec::with_capacity(capacity), + executed_senders: Vec::with_capacity(capacity), + receipts: Vec::with_capacity(capacity), + cumulative_gas_used: 0, + total_fees: U256::ZERO, + } + } +} + +/// Container type that holds all necessities to build a new payload. +#[derive(Debug)] +pub struct OpPayloadBuilderCtx { + /// The type that knows how to perform system calls and configure the evm. + pub evm_config: EvmConfig, + /// The chainspec + pub chain_spec: Arc, + /// How to build the payload. + pub config: PayloadConfig, + /// Evm Settings + pub initialized_cfg: CfgEnvWithHandlerCfg, + /// Block config + pub initialized_block_env: BlockEnv, + /// Marker to check whether the job has been cancelled. + pub cancel: Cancelled, + /// The currently best payload. + pub best_payload: Option, +} + +impl OpPayloadBuilderCtx { + /// Returns the parent block the payload will be build on. + pub fn parent(&self) -> &SealedHeader { + &self.config.parent_header } - if !attributes.no_tx_pool { + /// Returns the builder attributes. + pub const fn attributes(&self) -> &OpPayloadBuilderAttributes { + &self.config.attributes + } + + /// Returns the block gas limit to target. + pub fn block_gas_limit(&self) -> u64 { + self.attributes() + .gas_limit + .unwrap_or_else(|| self.initialized_block_env.gas_limit.saturating_to()) + } + + /// Returns the block number for the block. + pub fn block_number(&self) -> u64 { + self.initialized_block_env.number.to() + } + + /// Returns the current base fee + pub fn base_fee(&self) -> u64 { + self.initialized_block_env.basefee.to() + } + + /// Returns the current blob gas price. + pub fn get_blob_gasprice(&self) -> Option { + self.initialized_block_env.get_blob_gasprice().map(|gasprice| gasprice as u64) + } + + /// Returns the blob fields for the header. + /// + /// This will always return `Some(0)` after ecotone. + pub fn blob_fields(&self) -> (Option, Option) { + // OP doesn't support blobs/EIP-4844. + // https://specs.optimism.io/protocol/exec-engine.html#ecotone-disable-blob-transactions + // Need [Some] or [None] based on hardfork to match block hash. + if self.is_ecotone_active() { + (Some(0), Some(0)) + } else { + (None, None) + } + } + + /// Returns the extra data for the block. + /// + /// After holocene this extracts the extradata from the paylpad + pub fn extra_data(&self) -> Result { + if self.is_holocene_active() { + self.attributes() + .get_holocene_extra_data( + self.chain_spec.base_fee_params_at_timestamp( + self.attributes().payload_attributes.timestamp, + ), + ) + .map_err(PayloadBuilderError::other) + } else { + Ok(self.config.extra_data.clone()) + } + } + + /// Returns the current fee settings for transactions from the mempool + pub fn best_transaction_attributes(&self) -> BestTransactionsAttributes { + BestTransactionsAttributes::new(self.base_fee(), self.get_blob_gasprice()) + } + + /// Returns the unique id for this payload job. + pub fn payload_id(&self) -> PayloadId { + self.attributes().payload_id() + } + + /// Returns true if regolith is active for the payload. + pub fn is_regolith_active(&self) -> bool { + self.chain_spec.is_regolith_active_at_timestamp(self.attributes().timestamp()) + } + + /// Returns true if ecotone is active for the payload. + pub fn is_ecotone_active(&self) -> bool { + self.chain_spec.is_ecotone_active_at_timestamp(self.attributes().timestamp()) + } + + /// Returns true if canyon is active for the payload. + pub fn is_canyon_active(&self) -> bool { + self.chain_spec.is_canyon_active_at_timestamp(self.attributes().timestamp()) + } + + /// Returns true if holocene is active for the payload. + pub fn is_holocene_active(&self) -> bool { + self.chain_spec.is_holocene_active_at_timestamp(self.attributes().timestamp()) + } + + /// Returns true if the fees are higher than the previous payload. + pub fn is_better_payload(&self, total_fees: U256) -> bool { + is_better_payload(self.best_payload.as_ref(), total_fees) + } + + /// Commits the withdrawals from the payload attributes to the state. + pub fn commit_withdrawals( + &self, + db: &mut State, + ) -> Result + where + DB: Database, + { + commit_withdrawals( + db, + &self.chain_spec, + self.attributes().payload_attributes.timestamp, + self.attributes().payload_attributes.withdrawals.clone(), + ) + } + + /// Ensure that the create2deployer is force-deployed at the canyon transition. Optimism + /// blocks will always have at least a single transaction in them (the L1 info transaction), + /// so we can safely assume that this will always be triggered upon the transition and that + /// the above check for empty blocks will never be hit on OP chains. + pub fn ensure_create2_deployer(&self, db: &mut State) -> Result<(), PayloadBuilderError> + where + DB: Database, + DB::Error: Display, + { + reth_optimism_evm::ensure_create2_deployer( + self.chain_spec.clone(), + self.attributes().payload_attributes.timestamp, + db, + ) + .map_err(|err| { + warn!(target: "payload_builder", %err, "missing create2 deployer, skipping block."); + PayloadBuilderError::other(OptimismPayloadBuilderError::ForceCreate2DeployerFail) + }) + } +} + +impl OpPayloadBuilderCtx +where + EvmConfig: ConfigureEvm
, +{ + /// apply eip-4788 pre block contract call + pub fn apply_pre_beacon_root_contract_call( + &self, + db: &mut DB, + ) -> Result<(), PayloadBuilderError> + where + DB: Database + DatabaseCommit, + DB::Error: Display, + { + SystemCaller::new(self.evm_config.clone(), self.chain_spec.clone()) + .pre_block_beacon_root_contract_call( + db, + &self.initialized_cfg, + &self.initialized_block_env, + self.attributes().payload_attributes.parent_beacon_block_root, + ) + .map_err(|err| { + warn!(target: "payload_builder", + parent_header=%self.parent().hash(), + %err, + "failed to apply beacon root contract call for payload" + ); + PayloadBuilderError::Internal(err.into()) + })?; + + Ok(()) + } + + /// Executes all sequencer transactions that are included in the payload attributes. + pub fn execute_sequencer_transactions( + &self, + db: &mut State, + ) -> Result + where + DB: Database, + { + let mut info = ExecutionInfo::with_capacity(self.attributes().transactions.len()); + + for sequencer_tx in &self.attributes().transactions { + // A sequencer's block should never contain blob transactions. + if sequencer_tx.value().is_eip4844() { + return Err(PayloadBuilderError::other( + OptimismPayloadBuilderError::BlobTransactionRejected, + )) + } + + // Convert the transaction to a [TransactionSignedEcRecovered]. This is + // purely for the purposes of utilizing the `evm_config.tx_env`` function. + // Deposit transactions do not have signatures, so if the tx is a deposit, this + // will just pull in its `from` address. + let sequencer_tx = + sequencer_tx.value().clone().try_into_ecrecovered().map_err(|_| { + PayloadBuilderError::other( + OptimismPayloadBuilderError::TransactionEcRecoverFailed, + ) + })?; + + // Cache the depositor account prior to the state transition for the deposit nonce. + // + // Note that this *only* needs to be done post-regolith hardfork, as deposit nonces + // were not introduced in Bedrock. In addition, regular transactions don't have deposit + // nonces, so we don't need to touch the DB for those. + let depositor = (self.is_regolith_active() && sequencer_tx.is_deposit()) + .then(|| { + db.load_cache_account(sequencer_tx.signer()) + .map(|acc| acc.account_info().unwrap_or_default()) + }) + .transpose() + .map_err(|_| { + PayloadBuilderError::other(OptimismPayloadBuilderError::AccountLoadFailed( + sequencer_tx.signer(), + )) + })?; + + let env = EnvWithHandlerCfg::new_with_cfg_env( + self.initialized_cfg.clone(), + self.initialized_block_env.clone(), + self.evm_config.tx_env(sequencer_tx.as_signed(), sequencer_tx.signer()), + ); + + let mut evm = self.evm_config.evm_with_env(&mut *db, env); + + let ResultAndState { result, state } = match evm.transact() { + Ok(res) => res, + Err(err) => { + match err { + EVMError::Transaction(err) => { + trace!(target: "payload_builder", %err, ?sequencer_tx, "Error in sequencer transaction, skipping."); + continue + } + err => { + // this is an error that we should treat as fatal for this attempt + return Err(PayloadBuilderError::EvmExecutionError(err)) + } + } + } + }; + + // to release the db reference drop evm. + drop(evm); + // commit changes + db.commit(state); + + let gas_used = result.gas_used(); + + // add gas used by the transaction to cumulative gas used, before creating the receipt + info.cumulative_gas_used += gas_used; + + // Push transaction changeset and calculate header bloom filter for receipt. + info.receipts.push(Some(Receipt { + tx_type: sequencer_tx.tx_type(), + success: result.is_success(), + cumulative_gas_used: info.cumulative_gas_used, + logs: result.into_logs().into_iter().map(Into::into).collect(), + 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), + })); + + // append sender and transaction to the respective lists + info.executed_senders.push(sequencer_tx.signer()); + info.executed_transactions.push(sequencer_tx.into_signed()); + } + + Ok(info) + } + + /// Executes the given best transactions and updates the execution info + pub fn execute_best_transactions( + &self, + info: &mut ExecutionInfo, + db: &mut State, + mut best_txs: BestTransactionsFor, + ) -> Result>, PayloadBuilderError> + where + DB: Database, + Pool: TransactionPool, + { + let block_gas_limit = self.block_gas_limit(); + let base_fee = self.base_fee(); while let Some(pool_tx) = best_txs.next() { // ensure we still have capacity for this transaction - if cumulative_gas_used + pool_tx.gas_limit() > block_gas_limit { + if info.cumulative_gas_used + pool_tx.gas_limit() > block_gas_limit { // we can't fit this transaction into the block, so we need to mark it as // invalid which also removes all dependent transaction from // the iterator before we can continue @@ -340,20 +732,20 @@ where } // check if the job was cancelled, if so we can exit early - if cancel.is_cancelled() { - return Ok(BuildOutcome::Cancelled) + if self.cancel.is_cancelled() { + return Ok(Some(BuildOutcomeKind::Cancelled)) } // convert tx to a signed transaction let tx = pool_tx.to_recovered_transaction(); let env = EnvWithHandlerCfg::new_with_cfg_env( - initialized_cfg.clone(), - initialized_block_env.clone(), - evm_config.tx_env(tx.as_signed(), tx.signer()), + self.initialized_cfg.clone(), + self.initialized_block_env.clone(), + self.evm_config.tx_env(tx.as_signed(), tx.signer()), ); // Configure the environment for the block. - let mut evm = evm_config.evm_with_env(&mut db, env); + let mut evm = self.evm_config.evm_with_env(&mut *db, env); let ResultAndState { result, state } = match evm.transact() { Ok(res) => res, @@ -388,13 +780,13 @@ where // add gas used by the transaction to cumulative gas used, before creating the // receipt - cumulative_gas_used += gas_used; + info.cumulative_gas_used += gas_used; // Push transaction changeset and calculate header bloom filter for receipt. - receipts.push(Some(Receipt { + info.receipts.push(Some(Receipt { tx_type: tx.tx_type(), success: result.is_success(), - cumulative_gas_used, + cumulative_gas_used: info.cumulative_gas_used, logs: result.into_logs().into_iter().map(Into::into).collect(), deposit_nonce: None, deposit_receipt_version: None, @@ -404,140 +796,14 @@ where let miner_fee = tx .effective_tip_per_gas(Some(base_fee)) .expect("fee is always valid; execution succeeded"); - total_fees += U256::from(miner_fee) * U256::from(gas_used); + info.total_fees += U256::from(miner_fee) * U256::from(gas_used); // append sender and transaction to the respective lists - executed_senders.push(tx.signer()); - executed_txs.push(tx.into_signed()); + info.executed_senders.push(tx.signer()); + info.executed_transactions.push(tx.into_signed()); } - } - // check if we have a better block, but only if we included transactions from the pool - if !attributes.no_tx_pool && !is_better_payload(best_payload.as_ref(), total_fees) { - // can skip building the block - return Ok(BuildOutcome::Aborted { fees: total_fees, cached_reads }) - } - - let WithdrawalsOutcome { withdrawals_root, withdrawals } = commit_withdrawals( - &mut db, - &chain_spec, - attributes.payload_attributes.timestamp, - attributes.payload_attributes.withdrawals.clone(), - )?; - - // merge all transitions into bundle state, this would apply the withdrawal balance changes - // and 4788 contract call - db.merge_transitions(BundleRetention::Reverts); - - let execution_outcome = ExecutionOutcome::new( - db.take_bundle(), - vec![receipts.clone()].into(), - block_number, - Vec::new(), - ); - let receipts_root = execution_outcome - .generic_receipts_root_slow(block_number, |receipts| { - calculate_receipt_root_no_memo_optimism(receipts, &chain_spec, attributes.timestamp()) - }) - .expect("Number is in range"); - let logs_bloom = execution_outcome.block_logs_bloom(block_number).expect("Number is in range"); - - // calculate the state root - let hashed_state = HashedPostState::from_bundle_state(&execution_outcome.state().state); - let (state_root, trie_output) = { - db.database.inner().state_root_with_updates(hashed_state.clone()).inspect_err(|err| { - warn!(target: "payload_builder", - parent_header=%parent_header.hash(), - %err, - "failed to calculate state root for payload" - ); - })? - }; - - // create the block header - let transactions_root = proofs::calculate_transaction_root(&executed_txs); - - // OP doesn't support blobs/EIP-4844. - // https://specs.optimism.io/protocol/exec-engine.html#ecotone-disable-blob-transactions - // Need [Some] or [None] based on hardfork to match block hash. - let (excess_blob_gas, blob_gas_used) = - if chain_spec.is_ecotone_active_at_timestamp(attributes.payload_attributes.timestamp) { - (Some(0), Some(0)) - } else { - (None, None) - }; - - let is_holocene = - chain_spec.is_holocene_active_at_timestamp(attributes.payload_attributes.timestamp); - - if is_holocene { - extra_data = attributes - .get_holocene_extra_data( - chain_spec.base_fee_params_at_timestamp(attributes.payload_attributes.timestamp), - ) - .map_err(PayloadBuilderError::other)?; - } - - let header = Header { - parent_hash: parent_header.hash(), - ommers_hash: EMPTY_OMMER_ROOT_HASH, - beneficiary: initialized_block_env.coinbase, - state_root, - transactions_root, - receipts_root, - withdrawals_root, - logs_bloom, - timestamp: attributes.payload_attributes.timestamp, - mix_hash: attributes.payload_attributes.prev_randao, - nonce: BEACON_NONCE.into(), - base_fee_per_gas: Some(base_fee), - number: parent_header.number + 1, - gas_limit: block_gas_limit, - difficulty: U256::ZERO, - gas_used: cumulative_gas_used, - extra_data, - parent_beacon_block_root: attributes.payload_attributes.parent_beacon_block_root, - blob_gas_used, - excess_blob_gas, - requests_hash: None, - }; - - // seal the block - let block = Block { - header, - body: BlockBody { transactions: executed_txs, ommers: vec![], withdrawals }, - }; - - let sealed_block = block.seal_slow(); - debug!(target: "payload_builder", ?sealed_block, "sealed built block"); - - // create the executed block data - let executed = ExecutedBlock { - block: Arc::new(sealed_block.clone()), - senders: Arc::new(executed_senders), - execution_output: Arc::new(execution_outcome), - hashed_state: Arc::new(hashed_state), - trie: Arc::new(trie_output), - }; - - let no_tx_pool = attributes.no_tx_pool; - - let payload = OpBuiltPayload::new( - attributes.payload_attributes.id, - sealed_block, - total_fees, - chain_spec, - attributes, - Some(executed), - ); - - if no_tx_pool { - // if `no_tx_pool` is set only transactions from the payload attributes will be included in - // the payload. In other words, the payload is deterministic and we can freeze it once we've - // successfully built it. - Ok(BuildOutcome::Freeze(payload)) - } else { - Ok(BuildOutcome::Better { payload, cached_reads }) + Ok(None) } } diff --git a/crates/payload/basic/src/lib.rs b/crates/payload/basic/src/lib.rs index d6dcdf114..e57cc668d 100644 --- a/crates/payload/basic/src/lib.rs +++ b/crates/payload/basic/src/lib.rs @@ -790,6 +790,37 @@ impl BuildOutcome { } } +/// The possible outcomes of a payload building attempt without reused [`CachedReads`] +#[derive(Debug)] +pub enum BuildOutcomeKind { + /// Successfully built a better block. + Better { + /// The new payload that was built. + payload: Payload, + }, + /// Aborted payload building because resulted in worse block wrt. fees. + Aborted { + /// The total fees associated with the attempted payload. + fees: U256, + }, + /// Build job was cancelled + Cancelled, + /// The payload is final and no further building should occur + Freeze(Payload), +} + +impl BuildOutcomeKind { + /// Attaches the [`CachedReads`] to the outcome. + pub fn with_cached_reads(self, cached_reads: CachedReads) -> BuildOutcome { + match self { + Self::Better { payload } => BuildOutcome::Better { payload, cached_reads }, + Self::Aborted { fees } => BuildOutcome::Aborted { fees, cached_reads }, + Self::Cancelled => BuildOutcome::Cancelled, + Self::Freeze(payload) => BuildOutcome::Freeze(payload), + } + } +} + /// A collection of arguments used for building payloads. /// /// This struct encapsulates the essential components and configuration required for the payload diff --git a/crates/transaction-pool/src/traits.rs b/crates/transaction-pool/src/traits.rs index 512e3e31f..b62b7ff23 100644 --- a/crates/transaction-pool/src/traits.rs +++ b/crates/transaction-pool/src/traits.rs @@ -739,6 +739,11 @@ impl fmt::Display for CanonicalStateUpdate<'_> { } } +/// Alias to restrict the [`BestTransactions`] items to the pool's transaction type. +pub type BestTransactionsFor = Box< + dyn BestTransactions::Transaction>>>, +>; + /// An `Iterator` that only returns transactions that are ready to be executed. /// /// This makes no assumptions about the order of the transactions, but expects that _all_