From dfe83f770a4cff2a67c17babbe6a6f39d4eb68d8 Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Thu, 31 Oct 2024 16:44:19 +0100 Subject: [PATCH 1/8] feat: restructure op builder --- crates/optimism/payload/src/builder.rs | 662 ++++++++++++++++++++++++- crates/transaction-pool/src/traits.rs | 5 + 2 files changed, 661 insertions(+), 6 deletions(-) diff --git a/crates/optimism/payload/src/builder.rs b/crates/optimism/payload/src/builder.rs index 3095ce3512eb..5a00c7f700aa 100644 --- a/crates/optimism/payload/src/builder.rs +++ b/crates/optimism/payload/src/builder.rs @@ -1,9 +1,11 @@ //! 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, B256, B64, U256}; +use alloy_rpc_types_engine::PayloadId; use reth_basic_payload_builder::*; use reth_chain_state::ExecutedBlock; use reth_chainspec::ChainSpecProvider; @@ -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, ProviderResult, StateProviderBox, StateProviderFactory}; use reth_revm::database::StateProviderDatabase; use reth_transaction_pool::{ - noop::NoopTransactionPool, BestTransactionsAttributes, TransactionPool, + noop::NoopTransactionPool, BestTransactions, BestTransactionsAttributes, BestTransactionsFor, + TransactionPool, ValidPoolTransaction, }; 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}; @@ -36,6 +39,9 @@ use crate::{ payload::{OpPayloadBuilderAttributes, OptimismBuiltPayload}, }; use op_alloy_consensus::DepositTransaction; +use op_alloy_rpc_types_engine::OpPayloadAttributes; +use reth_payload_builder::database::{CachedReads, CachedReadsDBRef}; +use revm::db::WrapDatabaseRef; /// Optimism's payload builder #[derive(Debug, Clone, PartialEq, Eq)] @@ -106,6 +112,9 @@ where let (cfg_env, block_env) = self .cfg_and_block_env(&args.config, &args.config.parent_header) .map_err(PayloadBuilderError::other)?; + + // Builder::new().build(Context::new()) + optimism_payload(&self.evm_config, args, cfg_env, block_env, self.compute_pending_block) } @@ -143,6 +152,647 @@ where } } +/// The type that builds the payload. +/// +/// 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) +pub struct Builder { + /// The transaction pool + pool: Pool, + /// Yields the best transaction to include if transactions from the mempool are allowed. + best: Best, +} + +impl Builder +where + Pool: TransactionPool, + Best: FnOnce(Pool, BestTransactionsAttributes) -> BestTransactionsFor, +{ + /// Builds the payload on top of the state. + pub fn build( + self, + mut db: State, + ctx: OpPayloadBuilderCtx, + ) -> Result, PayloadBuilderError> + where + EvmConfig: ConfigureEvm
, + DB: Database + DatabaseCommit, + { + 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"); + + // 1. apply eip-4788 pre block contract call + ctx.apply_pre_beacon_root_contract_call(&mut db)?; + + // 2. ensure create2deployer is force deployed + ctx.ensure_create2_deployer(&mut db)?; + + // 3. execute sequencer transactions + let mut info = ctx.execute_sequencer_transactions(&mut db)?; + + // 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 + // TODO need workaround for cached reads here + // return Ok(BuildOutcome::Aborted { fees: info.total_fees, cached_reads }) + } + } + + 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 + // TODO need a work around to access the underlying state provider + // let hashed_state = HashedPostState::from_bundle_state(&execution_outcome.state().state); + // let (state_root, trie_output) = { + // let state_provider = db.database.0.inner.borrow_mut(); + // state_provider.db.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(&info.executed_transactions); + + // 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 extradata = ctx.extra_data()?; + + 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, + }; + + // seal the block + let block = Block { + header, + body: BlockBody { + transactions: info.executed_transactions, + 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(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 = OptimismBuiltPayload::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(BuildOutcome::Freeze(payload)) + } else { + Ok(BuildOutcome::Better { payload, cached_reads }) + } + + todo!() + } +} + +/// 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 + } + + /// Returns the builder attributes. + pub 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_canyon_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) + .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 + DatabaseCommit, + { + 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 + DatabaseCommit, + 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 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 + best_txs.mark_invalid(&pool_tx); + continue + } + + // A sequencer's block should never contain blob or deposit transactions from the pool. + if pool_tx.is_eip4844() || pool_tx.tx_type() == TxType::Deposit as u8 { + best_txs.mark_invalid(&pool_tx); + continue + } + + // check if the job was cancelled, if so we can exit early + if self.cancel.is_cancelled() { + return Ok(Some(BuildOutcome::Cancelled)) + } + + // convert tx to a signed transaction + let tx = pool_tx.to_recovered_transaction(); + let env = EnvWithHandlerCfg::new_with_cfg_env( + 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 = 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) => { + if matches!(err, InvalidTransaction::NonceTooLow { .. }) { + // if the nonce is too low, we can skip this transaction + trace!(target: "payload_builder", %err, ?tx, "skipping nonce too low transaction"); + } else { + // if the transaction is invalid, we can skip it and all of its + // descendants + trace!(target: "payload_builder", %err, ?tx, "skipping invalid transaction and its descendants"); + best_txs.mark_invalid(&pool_tx); + } + + continue + } + err => { + // this is an error that we should treat as fatal for this attempt + return Err(PayloadBuilderError::EvmExecutionError(err)) + } + } + } + }; + // drop evm so db is released. + 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: 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: None, + deposit_receipt_version: None, + })); + + // update add to total fees + let miner_fee = tx + .effective_tip_per_gas(Some(base_fee)) + .expect("fee is always valid; execution succeeded"); + info.total_fees += U256::from(miner_fee) * U256::from(gas_used); + + // append sender and transaction to the respective lists + info.executed_senders.push(tx.signer()); + info.executed_transactions.push(tx.into_signed()); + } + + Ok(None) + } +} + +// /// Creates the context and opens the state. +// fn with_state_at( +// attributes: OpPayloadBuilderAttributes, +// evm_config: C, +// client: C, +// ) -> ProviderResult>> +// where +// C: StateProviderFactory + ChainSpecProvider, +// { +// todo!() +// } +// +// /// Creates the context and opens the state with cached reads. +// /// +// /// This reuses reads captured from a previous run. +// /// Note: this is only relevant if the attributes allows transactions from the mempool, +// because /// if no transactions from the mempool should be included the payload is +// deterministic and we /// can freeze it [`BuildOutcome::Freeze`] preventing another job. +// fn with_cached_state_at( +// attributes: OpPayloadBuilderAttributes, +// evm_config: C, +// client: C, +// cached_reads: &mut CachedReads, +// ) -> ProviderResult< +// OpPayloadBuildingCtx< +// EvmConfig, +// WrapDatabaseRef>>, +// >, +// > +// where +// C: StateProviderFactory + ChainSpecProvider, +// { +// todo!() +// } + /// 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 diff --git a/crates/transaction-pool/src/traits.rs b/crates/transaction-pool/src/traits.rs index 0d8f6dbb54ad..e92574f7f6e5 100644 --- a/crates/transaction-pool/src/traits.rs +++ b/crates/transaction-pool/src/traits.rs @@ -729,6 +729,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_ From 7f02c595cc65aeeaa180c4c0d1c99dba774891fb Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Thu, 31 Oct 2024 17:44:31 +0100 Subject: [PATCH 2/8] require stateroot provider --- crates/optimism/payload/src/builder.rs | 55 ++++++++++++-------------- 1 file changed, 26 insertions(+), 29 deletions(-) diff --git a/crates/optimism/payload/src/builder.rs b/crates/optimism/payload/src/builder.rs index d359fce98218..66481378901f 100644 --- a/crates/optimism/payload/src/builder.rs +++ b/crates/optimism/payload/src/builder.rs @@ -4,7 +4,7 @@ use std::{fmt::Display, sync::Arc}; use alloy_consensus::EMPTY_OMMER_ROOT_HASH; use alloy_eips::merge::BEACON_NONCE; -use alloy_primitives::{Address, Bytes, B256, 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; @@ -20,11 +20,11 @@ use reth_primitives::{ revm_primitives::{BlockEnv, CfgEnvWithHandlerCfg}, Block, BlockBody, Header, Receipt, SealedHeader, TransactionSigned, TxType, }; -use reth_provider::{ProviderError, ProviderResult, StateProviderBox, StateProviderFactory}; +use reth_provider::{ProviderError, StateProviderFactory, StateRootProvider}; use reth_revm::database::StateProviderDatabase; use reth_transaction_pool::{ noop::NoopTransactionPool, BestTransactions, BestTransactionsAttributes, BestTransactionsFor, - TransactionPool, ValidPoolTransaction, + TransactionPool, }; use reth_trie::HashedPostState; use revm::{ @@ -39,9 +39,7 @@ use crate::{ payload::{OpBuiltPayload, OpPayloadBuilderAttributes}, }; use op_alloy_consensus::DepositTransaction; -use op_alloy_rpc_types_engine::OpPayloadAttributes; -use reth_payload_builder::database::{CachedReads, CachedReadsDBRef}; -use revm::db::WrapDatabaseRef; +use reth_payload_builder::database::CachedReads; /// Optimism's payload builder #[derive(Debug, Clone, PartialEq, Eq)] @@ -167,7 +165,10 @@ where /// /// And finally /// 5. build the block: compute all roots (txs, state) +#[derive(Debug)] pub struct Builder { + /// Cached reads from previous run. + cached_reads: CachedReads, /// The transaction pool pool: Pool, /// Yields the best transaction to include if transactions from the mempool are allowed. @@ -184,12 +185,12 @@ where self, mut db: State, ctx: OpPayloadBuilderCtx, - ) -> Result, PayloadBuilderError> + ) -> Result, PayloadBuilderError> where EvmConfig: ConfigureEvm
, - DB: Database + DatabaseCommit, + DB: Database + DatabaseCommit + StateRootProvider, { - let Self { pool, best } = self; + let Self { cached_reads, pool, best } = self; debug!(target: "payload_builder", id=%ctx.payload_id(), parent_header = ?ctx.parent().hash(), parent_number = ctx.parent().number, "building new payload"); // 1. apply eip-4788 pre block contract call @@ -213,8 +214,7 @@ where // check if the new payload is even more valuable if !ctx.is_better_payload(info.total_fees) { // can skip building the block - // TODO need workaround for cached reads here - // return Ok(BuildOutcome::Aborted { fees: info.total_fees, cached_reads }) + return Ok(BuildOutcome::Aborted { fees: info.total_fees, cached_reads }) } } @@ -245,18 +245,17 @@ where execution_outcome.block_logs_bloom(block_number).expect("Number is in range"); // // calculate the state root - // TODO need a work around to access the underlying state provider - // let hashed_state = HashedPostState::from_bundle_state(&execution_outcome.state().state); - // let (state_root, trie_output) = { - // let state_provider = db.database.0.inner.borrow_mut(); - // state_provider.db.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" - // ); - // })? - // }; + let hashed_state = HashedPostState::from_bundle_state(&execution_outcome.state().state); + let (state_root, trie_output) = { + let state_provider = db.database; + state_provider.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" + ); + })? + }; // create the block header let transactions_root = proofs::calculate_transaction_root(&info.executed_transactions); @@ -265,7 +264,7 @@ where // 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 extradata = ctx.extra_data()?; + let extra_data = ctx.extra_data()?; let header = Header { parent_hash: ctx.parent().hash(), @@ -315,7 +314,7 @@ where let no_tx_pool = ctx.attributes().no_tx_pool; - let payload = OptimismBuiltPayload::new( + let payload = OpBuiltPayload::new( ctx.payload_id(), sealed_block, info.total_fees, @@ -332,8 +331,6 @@ where } else { Ok(BuildOutcome::Better { payload, cached_reads }) } - - todo!() } } @@ -381,7 +378,7 @@ pub struct OpPayloadBuilderCtx { /// Marker to check whether the job has been cancelled. pub cancel: Cancelled, /// The currently best payload. - pub best_payload: Option, + pub best_payload: Option, } impl OpPayloadBuilderCtx { @@ -659,7 +656,7 @@ where info: &mut ExecutionInfo, db: &mut State, mut best_txs: BestTransactionsFor, - ) -> Result>, PayloadBuilderError> + ) -> Result>, PayloadBuilderError> where DB: Database + DatabaseCommit, Pool: TransactionPool, From 67c08f637f35637967fd7509ba4af46ad9548da7 Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Thu, 31 Oct 2024 18:14:44 +0100 Subject: [PATCH 3/8] more unblock --- crates/optimism/payload/src/builder.rs | 64 ++++++++++++++++++++++++-- 1 file changed, 60 insertions(+), 4 deletions(-) diff --git a/crates/optimism/payload/src/builder.rs b/crates/optimism/payload/src/builder.rs index 66481378901f..482fd56f4cdc 100644 --- a/crates/optimism/payload/src/builder.rs +++ b/crates/optimism/payload/src/builder.rs @@ -73,9 +73,10 @@ impl OptimismPayloadBuilder { self.compute_pending_block } } + impl OptimismPayloadBuilder where - EvmConfig: ConfigureEvmEnv
, + EvmConfig: ConfigureEvm
, { /// Returns the configured [`CfgEnvWithHandlerCfg`] and [`BlockEnv`] for the targeted payload /// (that has the `parent` as its parent). @@ -91,6 +92,48 @@ where }; self.evm_config.next_cfg_and_block_env(parent, next_attributes) } + + 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 mut builder = Builder { + cached_reads, + pool, + best: best_txs:: + }; + + let state_provider = client.state_by_block_hash(ctx.parent().hash())?; + let state = StateProviderDatabase::new(state_provider); + let mut db = + State::builder().with_database_ref(builder.cached_reads.as_db(state)).with_bundle_update().build(); + + // Builder::new().build(Context::new()) + builder.build2(db, ctx); + todo!() + } +} + +fn best_txs(pool: Pool, attr: BestTransactionsAttributes) -> BestTransactionsFor { + pool.best_transactions_with_attributes(attr) } /// Implementation of the [`PayloadBuilder`] trait for [`OptimismPayloadBuilder`]. @@ -111,6 +154,7 @@ where .cfg_and_block_env(&args.config, &args.config.parent_header) .map_err(PayloadBuilderError::other)?; + // Builder::new().build(Context::new()) optimism_payload(&self.evm_config, args, cfg_env, block_env, self.compute_pending_block) @@ -180,6 +224,18 @@ where Pool: TransactionPool, Best: FnOnce(Pool, BestTransactionsAttributes) -> BestTransactionsFor, { + pub fn build2( + self, + mut db: State, + ctx: OpPayloadBuilderCtx, + ) -> Result, PayloadBuilderError> + where + EvmConfig: ConfigureEvm
, + DB: Database, + { + todo!() + } + /// Builds the payload on top of the state. pub fn build( self, @@ -188,7 +244,7 @@ where ) -> Result, PayloadBuilderError> where EvmConfig: ConfigureEvm
, - DB: Database + DatabaseCommit + StateRootProvider, + DB: Database + StateRootProvider, { let Self { cached_reads, pool, best } = self; debug!(target: "payload_builder", id=%ctx.payload_id(), parent_header = ?ctx.parent().hash(), parent_number = ctx.parent().number, "building new payload"); @@ -555,7 +611,7 @@ where db: &mut State, ) -> Result where - DB: Database + DatabaseCommit, + DB: Database, { let mut info = ExecutionInfo::with_capacity(self.attributes().transactions.len()); @@ -658,7 +714,7 @@ where mut best_txs: BestTransactionsFor, ) -> Result>, PayloadBuilderError> where - DB: Database + DatabaseCommit, + DB: Database, Pool: TransactionPool, { let block_gas_limit = self.block_gas_limit(); From ec974b762a3495a47e17548003781b507e8703ba Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Mon, 4 Nov 2024 13:16:09 +0100 Subject: [PATCH 4/8] make it compile --- crates/optimism/payload/src/builder.rs | 163 +++++++++---------------- crates/payload/basic/src/lib.rs | 31 +++++ 2 files changed, 91 insertions(+), 103 deletions(-) diff --git a/crates/optimism/payload/src/builder.rs b/crates/optimism/payload/src/builder.rs index 6cdf4fe59456..5db52dcbf99f 100644 --- a/crates/optimism/payload/src/builder.rs +++ b/crates/optimism/payload/src/builder.rs @@ -9,7 +9,7 @@ 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; @@ -39,7 +39,6 @@ use crate::{ payload::{OpBuiltPayload, OpPayloadBuilderAttributes}, }; use op_alloy_consensus::DepositTransaction; -use reth_payload_builder::database::CachedReads; /// Optimism's payload builder #[derive(Debug, Clone, PartialEq, Eq)] @@ -92,11 +91,21 @@ where self.evm_config.next_cfg_and_block_env(parent, next_attributes) } - fn build_payload(&self, args: BuildArguments) -> Result, PayloadBuilderError> + /// 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) @@ -114,27 +123,26 @@ where best_payload, }; - let mut builder = Builder { - cached_reads, - pool, - best: best_txs:: - }; + let builder = OpBuilder { pool, best: best_txs:: }; let state_provider = client.state_by_block_hash(ctx.parent().hash())?; let state = StateProviderDatabase::new(state_provider); - let mut db = - State::builder().with_database_ref(builder.cached_reads.as_db(state)).with_bundle_update().build(); - // Builder::new().build(Context::new()) - builder.build2(db, ctx); - todo!() + 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)) } } -fn best_txs(pool: Pool, attr: BestTransactionsAttributes) -> BestTransactionsFor { - pool.best_transactions_with_attributes(attr) -} - /// Implementation of the [`PayloadBuilder`] trait for [`OpPayloadBuilder`]. impl PayloadBuilder for OpPayloadBuilder where @@ -149,14 +157,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)?; - - - // Builder::new().build(Context::new()) - - optimism_payload(&self.evm_config, args, cfg_env, block_env, self.compute_pending_block) + self.build_payload(args) } fn on_missing_payload( @@ -184,12 +185,7 @@ 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) } } @@ -209,43 +205,31 @@ where /// And finally /// 5. build the block: compute all roots (txs, state) #[derive(Debug)] -pub struct Builder { - /// Cached reads from previous run. - cached_reads: CachedReads, +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 Builder +impl OpBuilder where Pool: TransactionPool, Best: FnOnce(Pool, BestTransactionsAttributes) -> BestTransactionsFor, { - pub fn build2( - self, - mut db: State, - ctx: OpPayloadBuilderCtx, - ) -> Result, PayloadBuilderError> - where - EvmConfig: ConfigureEvm
, - DB: Database, - { - todo!() - } - /// Builds the payload on top of the state. - pub fn build( + pub fn build( self, mut db: State, ctx: OpPayloadBuilderCtx, - ) -> Result, PayloadBuilderError> + ) -> Result, PayloadBuilderError> where EvmConfig: ConfigureEvm
, - DB: Database + StateRootProvider, + DB: Database + AsRef

, + P: StateRootProvider, { - let Self { cached_reads, pool, best } = self; + 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"); // 1. apply eip-4788 pre block contract call @@ -269,7 +253,7 @@ where // check if the new payload is even more valuable if !ctx.is_better_payload(info.total_fees) { // can skip building the block - return Ok(BuildOutcome::Aborted { fees: info.total_fees, cached_reads }) + return Ok(BuildOutcomeKind::Aborted { fees: info.total_fees }) } } @@ -302,14 +286,15 @@ where // // calculate the state root let hashed_state = HashedPostState::from_bundle_state(&execution_outcome.state().state); let (state_root, trie_output) = { - let state_provider = db.database; - state_provider.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" - ); - })? + 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" + ); + }, + )? }; // create the block header @@ -382,13 +367,20 @@ where // 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)) + Ok(BuildOutcomeKind::Freeze(payload)) } else { - Ok(BuildOutcome::Better { payload, cached_reads }) + 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 { @@ -443,7 +435,7 @@ impl OpPayloadBuilderCtx { } /// Returns the builder attributes. - pub fn attributes(&self) -> &OpPayloadBuilderAttributes { + pub const fn attributes(&self) -> &OpPayloadBuilderAttributes { &self.config.attributes } @@ -585,7 +577,7 @@ where DB: Database + DatabaseCommit, DB::Error: Display, { - SystemCaller::new(self.evm_config.clone(), &self.chain_spec) + SystemCaller::new(self.evm_config.clone(), self.chain_spec.clone()) .pre_block_beacon_root_contract_call( db, &self.initialized_cfg, @@ -711,7 +703,7 @@ where info: &mut ExecutionInfo, db: &mut State, mut best_txs: BestTransactionsFor, - ) -> Result>, PayloadBuilderError> + ) -> Result>, PayloadBuilderError> where DB: Database, Pool: TransactionPool, @@ -736,7 +728,7 @@ where // check if the job was cancelled, if so we can exit early if self.cancel.is_cancelled() { - return Ok(Some(BuildOutcome::Cancelled)) + return Ok(Some(BuildOutcomeKind::Cancelled)) } // convert tx to a signed transaction @@ -810,41 +802,6 @@ where } } -// /// Creates the context and opens the state. -// fn with_state_at( -// attributes: OpPayloadBuilderAttributes, -// evm_config: C, -// client: C, -// ) -> ProviderResult>> -// where -// C: StateProviderFactory + ChainSpecProvider, -// { -// todo!() -// } -// -// /// Creates the context and opens the state with cached reads. -// /// -// /// This reuses reads captured from a previous run. -// /// Note: this is only relevant if the attributes allows transactions from the mempool, -// because /// if no transactions from the mempool should be included the payload is -// deterministic and we /// can freeze it [`BuildOutcome::Freeze`] preventing another job. -// fn with_cached_state_at( -// attributes: OpPayloadBuilderAttributes, -// evm_config: C, -// client: C, -// cached_reads: &mut CachedReads, -// ) -> ProviderResult< -// OpPayloadBuildingCtx< -// EvmConfig, -// WrapDatabaseRef>>, -// >, -// > -// where -// C: StateProviderFactory + ChainSpecProvider, -// { -// todo!() -// } - /// 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 @@ -854,7 +811,7 @@ where /// 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( +pub fn optimism_payload( evm_config: &EvmConfig, args: BuildArguments, initialized_cfg: CfgEnvWithHandlerCfg, diff --git a/crates/payload/basic/src/lib.rs b/crates/payload/basic/src/lib.rs index 7b1de980ce9c..267535a8894f 100644 --- a/crates/payload/basic/src/lib.rs +++ b/crates/payload/basic/src/lib.rs @@ -792,6 +792,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 { + BuildOutcomeKind::Better { payload } => BuildOutcome::Better { payload, cached_reads }, + BuildOutcomeKind::Aborted { fees } => BuildOutcome::Aborted { fees, cached_reads }, + BuildOutcomeKind::Cancelled => BuildOutcome::Cancelled, + BuildOutcomeKind::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 From bbbe3dc265e704e3b7e9681f34c9879019d8a477 Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Mon, 4 Nov 2024 13:18:26 +0100 Subject: [PATCH 5/8] clippy --- crates/payload/basic/src/lib.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/crates/payload/basic/src/lib.rs b/crates/payload/basic/src/lib.rs index 267535a8894f..27795deea68b 100644 --- a/crates/payload/basic/src/lib.rs +++ b/crates/payload/basic/src/lib.rs @@ -815,10 +815,10 @@ impl BuildOutcomeKind { /// Attaches the [`CachedReads`] to the outcome. pub fn with_cached_reads(self, cached_reads: CachedReads) -> BuildOutcome { match self { - BuildOutcomeKind::Better { payload } => BuildOutcome::Better { payload, cached_reads }, - BuildOutcomeKind::Aborted { fees } => BuildOutcome::Aborted { fees, cached_reads }, - BuildOutcomeKind::Cancelled => BuildOutcome::Cancelled, - BuildOutcomeKind::Freeze(payload) => BuildOutcome::Freeze(payload), + 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), } } } From 8e60c6aa47d3b2039027efc73dea6b1f00bab054 Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Mon, 4 Nov 2024 13:29:51 +0100 Subject: [PATCH 6/8] fix: use holecene fn --- crates/optimism/payload/src/builder.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/optimism/payload/src/builder.rs b/crates/optimism/payload/src/builder.rs index 5db52dcbf99f..685bb5ea2651 100644 --- a/crates/optimism/payload/src/builder.rs +++ b/crates/optimism/payload/src/builder.rs @@ -479,7 +479,7 @@ impl OpPayloadBuilderCtx { /// /// After holocene this extracts the extradata from the paylpad pub fn extra_data(&self) -> Result { - if self.is_canyon_active() { + if self.is_holocene_active() { self.attributes() .get_holocene_extra_data( self.chain_spec.base_fee_params_at_timestamp( From c805e2c83acadbb3baafa29c2ce8bfae3793e0e7 Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Mon, 4 Nov 2024 14:36:45 +0100 Subject: [PATCH 7/8] add todo --- crates/optimism/payload/src/builder.rs | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/crates/optimism/payload/src/builder.rs b/crates/optimism/payload/src/builder.rs index 685bb5ea2651..fa5596231103 100644 --- a/crates/optimism/payload/src/builder.rs +++ b/crates/optimism/payload/src/builder.rs @@ -123,7 +123,12 @@ where best_payload, }; - let builder = OpBuilder { pool, best: best_txs:: }; + 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); From 9cc71f88a530a1249fc12b1f6a3dd66d078f6fba Mon Sep 17 00:00:00 2001 From: Matthias Seitz Date: Tue, 5 Nov 2024 14:49:52 +0100 Subject: [PATCH 8/8] rm unused fn --- crates/optimism/payload/src/builder.rs | 398 ------------------------- 1 file changed, 398 deletions(-) diff --git a/crates/optimism/payload/src/builder.rs b/crates/optimism/payload/src/builder.rs index fa5596231103..35e1d905a463 100644 --- a/crates/optimism/payload/src/builder.rs +++ b/crates/optimism/payload/src/builder.rs @@ -807,404 +807,6 @@ where } } -/// 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. -/// -/// 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 fn optimism_payload( - evm_config: &EvmConfig, - args: BuildArguments, - initialized_cfg: CfgEnvWithHandlerCfg, - initialized_block_env: BlockEnv, - _compute_pending_block: bool, -) -> Result, PayloadBuilderError> -where - EvmConfig: ConfigureEvm

, - Client: StateProviderFactory + ChainSpecProvider, - Pool: TransactionPool, -{ - let BuildArguments { client, pool, mut cached_reads, config, cancel, best_payload } = args; - - 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; - - debug!(target: "payload_builder", id=%attributes.payload_attributes.payload_id(), parent_header = ?parent_header.hash(), parent_number = parent_header.number, "building new payload"); - - 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::(); - - 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)) - } - } - } - }; - - // 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 - cumulative_gas_used += gas_used; - - // 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), - })); - - // append sender and transaction to the respective lists - executed_senders.push(sequencer_tx.signer()); - executed_txs.push(sequencer_tx.into_signed()); - } - - if !attributes.no_tx_pool { - 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 { - // 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 - best_txs.mark_invalid(&pool_tx); - continue - } - - // A sequencer's block should never contain blob or deposit transactions from the pool. - if pool_tx.is_eip4844() || pool_tx.tx_type() == TxType::Deposit as u8 { - best_txs.mark_invalid(&pool_tx); - continue - } - - // check if the job was cancelled, if so we can exit early - if cancel.is_cancelled() { - return Ok(BuildOutcome::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()), - ); - - // Configure the environment for the block. - 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) => { - if matches!(err, InvalidTransaction::NonceTooLow { .. }) { - // if the nonce is too low, we can skip this transaction - trace!(target: "payload_builder", %err, ?tx, "skipping nonce too low transaction"); - } else { - // if the transaction is invalid, we can skip it and all of its - // descendants - trace!(target: "payload_builder", %err, ?tx, "skipping invalid transaction and its descendants"); - best_txs.mark_invalid(&pool_tx); - } - - continue - } - err => { - // this is an error that we should treat as fatal for this attempt - return Err(PayloadBuilderError::EvmExecutionError(err)) - } - } - } - }; - // drop evm so db is released. - 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 - cumulative_gas_used += gas_used; - - // Push transaction changeset and calculate header bloom filter for receipt. - receipts.push(Some(Receipt { - tx_type: tx.tx_type(), - success: result.is_success(), - cumulative_gas_used, - logs: result.into_logs().into_iter().map(Into::into).collect(), - deposit_nonce: None, - deposit_receipt_version: None, - })); - - // update add to total fees - 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); - - // append sender and transaction to the respective lists - executed_senders.push(tx.signer()); - executed_txs.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 }) - } -} - /// Extracts the Holocene 1599 parameters from the encoded form: /// pub fn decode_eip_1559_params(eip_1559_params: B64) -> (u32, u32) {