From 63d5feab33c04cf79973b0870da76036069ed8a3 Mon Sep 17 00:00:00 2001 From: Arsenii Kulikov Date: Fri, 7 Feb 2025 00:31:51 +0400 Subject: [PATCH] refactor: extract OP txpool types to a separate crate (#14280) --- .github/assets/check_wasm.sh | 1 + Cargo.lock | 26 ++ Cargo.toml | 2 + crates/optimism/node/Cargo.toml | 2 + crates/optimism/node/src/lib.rs | 2 +- crates/optimism/node/src/txpool.rs | 533 ---------------------- crates/optimism/txpool/Cargo.toml | 51 +++ crates/optimism/txpool/src/lib.rs | 26 ++ crates/optimism/txpool/src/transaction.rs | 280 ++++++++++++ crates/optimism/txpool/src/validator.rs | 250 ++++++++++ 10 files changed, 639 insertions(+), 534 deletions(-) delete mode 100644 crates/optimism/node/src/txpool.rs create mode 100644 crates/optimism/txpool/Cargo.toml create mode 100644 crates/optimism/txpool/src/lib.rs create mode 100644 crates/optimism/txpool/src/transaction.rs create mode 100644 crates/optimism/txpool/src/validator.rs diff --git a/.github/assets/check_wasm.sh b/.github/assets/check_wasm.sh index 827d30dc7..d24f998fc 100755 --- a/.github/assets/check_wasm.sh +++ b/.github/assets/check_wasm.sh @@ -67,6 +67,7 @@ exclude_crates=( reth-payload-util # reth-transaction-pool reth-trie-parallel # tokio reth-testing-utils + reth-optimism-txpool # reth-transaction-pool ) # Array to hold the results diff --git a/Cargo.lock b/Cargo.lock index b198f6968..2e0ebfeba 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8415,6 +8415,7 @@ dependencies = [ "reth-optimism-payload-builder", "reth-optimism-primitives", "reth-optimism-rpc", + "reth-optimism-txpool", "reth-payload-builder", "reth-payload-util", "reth-payload-validator", @@ -8555,6 +8556,31 @@ dependencies = [ "reth-stages-types", ] +[[package]] +name = "reth-optimism-txpool" +version = "1.1.5" +dependencies = [ + "alloy-consensus", + "alloy-eips", + "alloy-primitives", + "alloy-rpc-types-eth", + "c-kzg", + "derive_more", + "op-alloy-consensus", + "op-alloy-flz", + "parking_lot", + "reth-chainspec", + "reth-optimism-chainspec", + "reth-optimism-evm", + "reth-optimism-forks", + "reth-optimism-primitives", + "reth-primitives-traits", + "reth-provider", + "reth-storage-api", + "reth-transaction-pool", + "revm", +] + [[package]] name = "reth-payload-builder" version = "1.1.5" diff --git a/Cargo.toml b/Cargo.toml index 9be4cfd67..b97838d91 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -76,6 +76,7 @@ members = [ "crates/optimism/primitives/", "crates/optimism/rpc/", "crates/optimism/storage", + "crates/optimism/txpool/", "crates/payload/basic/", "crates/payload/builder/", "crates/payload/builder-primitives/", @@ -379,6 +380,7 @@ reth-optimism-payload-builder = { path = "crates/optimism/payload" } reth-optimism-primitives = { path = "crates/optimism/primitives" } reth-optimism-rpc = { path = "crates/optimism/rpc" } reth-optimism-storage = { path = "crates/optimism/storage" } +reth-optimism-txpool = { path = "crates/optimism/txpool" } reth-payload-builder = { path = "crates/payload/builder" } reth-payload-builder-primitives = { path = "crates/payload/builder-primitives" } reth-payload-primitives = { path = "crates/payload/primitives" } diff --git a/crates/optimism/node/Cargo.toml b/crates/optimism/node/Cargo.toml index 38c957b30..7e1a20721 100644 --- a/crates/optimism/node/Cargo.toml +++ b/crates/optimism/node/Cargo.toml @@ -40,6 +40,7 @@ reth-tasks = { workspace = true, optional = true } reth-optimism-payload-builder.workspace = true reth-optimism-evm.workspace = true reth-optimism-rpc.workspace = true +reth-optimism-txpool.workspace = true reth-optimism-chainspec.workspace = true reth-optimism-consensus.workspace = true reth-optimism-forks.workspace = true @@ -102,6 +103,7 @@ optimism = [ "reth-optimism-node/optimism", "reth-node-core/optimism", "reth-optimism-primitives/optimism", + "reth-optimism-txpool/optimism", ] asm-keccak = [ "reth-primitives/asm-keccak", diff --git a/crates/optimism/node/src/lib.rs b/crates/optimism/node/src/lib.rs index 18c571547..77102dd2c 100644 --- a/crates/optimism/node/src/lib.rs +++ b/crates/optimism/node/src/lib.rs @@ -23,7 +23,7 @@ pub use engine::OpEngineTypes; pub mod node; pub use node::{OpNetworkPrimitives, OpNode}; -pub mod txpool; +pub use reth_optimism_txpool as txpool; /// Helpers for running test node instances. #[cfg(feature = "test-utils")] diff --git a/crates/optimism/node/src/txpool.rs b/crates/optimism/node/src/txpool.rs deleted file mode 100644 index 35bc9635c..000000000 --- a/crates/optimism/node/src/txpool.rs +++ /dev/null @@ -1,533 +0,0 @@ -//! OP transaction pool types -use alloy_consensus::{ - BlobTransactionSidecar, BlobTransactionValidationError, BlockHeader, Transaction, Typed2718, -}; -use alloy_eips::{eip2718::Encodable2718, eip7702::SignedAuthorization}; -use alloy_primitives::{Address, Bytes, TxHash, TxKind, B256, U256}; -use alloy_rpc_types_eth::erc4337::TransactionConditional; -use parking_lot::RwLock; -use reth_node_api::{Block, BlockBody}; -use reth_optimism_evm::RethL1BlockInfo; -use reth_optimism_forks::OpHardforks; -use reth_optimism_primitives::OpTransactionSigned; -use reth_primitives::{GotExpected, InvalidTransactionError, Recovered, SealedBlock}; -use reth_primitives_traits::{InMemorySize, SignedTransaction}; -use reth_provider::{BlockReaderIdExt, ChainSpecProvider, StateProviderFactory}; -use reth_revm::L1BlockInfo; -use reth_transaction_pool::{ - CoinbaseTipOrdering, EthBlobTransactionSidecar, EthPoolTransaction, EthPooledTransaction, - EthTransactionValidator, Pool, PoolTransaction, TransactionOrigin, - TransactionValidationOutcome, TransactionValidationTaskExecutor, TransactionValidator, -}; -use revm::primitives::{AccessList, KzgSettings}; -use std::{ - fmt::Debug, - sync::{ - atomic::{AtomicU64, Ordering}, - Arc, OnceLock, - }, -}; - -/// Type alias for default optimism transaction pool -pub type OpTransactionPool = Pool< - TransactionValidationTaskExecutor>, - CoinbaseTipOrdering, - S, ->; - -/// Pool transaction for OP. -/// -/// This type wraps the actual transaction and caches values that are frequently used by the pool. -/// For payload building this lazily tracks values that are required during payload building: -/// - Estimated compressed size of this transaction -#[derive(Debug, Clone, derive_more::Deref)] -pub struct OpPooledTransaction< - Cons = OpTransactionSigned, - Pooled = op_alloy_consensus::OpPooledTransaction, -> { - #[deref] - inner: EthPooledTransaction, - /// The estimated size of this transaction, lazily computed. - estimated_tx_compressed_size: OnceLock, - /// The pooled transaction type. - _pd: core::marker::PhantomData, - - /// Optional conditional attached to this transaction. - conditional: Option>, -} - -impl OpPooledTransaction { - /// Create new instance of [Self]. - pub fn new(transaction: Recovered, encoded_length: usize) -> Self { - Self { - inner: EthPooledTransaction::new(transaction, encoded_length), - estimated_tx_compressed_size: Default::default(), - conditional: None, - _pd: core::marker::PhantomData, - } - } - - /// Returns the estimated compressed size of a transaction in bytes scaled by 1e6. - /// This value is computed based on the following formula: - /// `max(minTransactionSize, intercept + fastlzCoef*fastlzSize)` - pub fn estimated_compressed_size(&self) -> u64 { - *self.estimated_tx_compressed_size.get_or_init(|| { - op_alloy_flz::tx_estimated_size_fjord(&self.inner.transaction().encoded_2718()) - }) - } - - /// Conditional setter. - pub fn with_conditional(mut self, conditional: TransactionConditional) -> Self { - self.conditional = Some(Box::new(conditional)); - self - } - - /// Conditional getter. - pub fn conditional(&self) -> Option<&TransactionConditional> { - self.conditional.as_deref() - } -} - -impl PoolTransaction for OpPooledTransaction -where - Cons: SignedTransaction + From, - Pooled: SignedTransaction + TryFrom, -{ - type TryFromConsensusError = >::Error; - type Consensus = Cons; - type Pooled = Pooled; - - fn clone_into_consensus(&self) -> Recovered { - self.inner.transaction().clone() - } - - fn into_consensus(self) -> Recovered { - self.inner.transaction - } - - fn from_pooled(tx: Recovered) -> Self { - let encoded_len = tx.encode_2718_len(); - let tx = tx.map_transaction(|tx| tx.into()); - Self::new(tx, encoded_len) - } - - fn hash(&self) -> &TxHash { - self.inner.transaction.tx_hash() - } - - fn sender(&self) -> Address { - self.inner.transaction.signer() - } - - fn sender_ref(&self) -> &Address { - self.inner.transaction.signer_ref() - } - - fn cost(&self) -> &U256 { - &self.inner.cost - } - - fn encoded_length(&self) -> usize { - self.inner.encoded_length - } -} - -impl Typed2718 for OpPooledTransaction { - fn ty(&self) -> u8 { - self.inner.ty() - } -} - -impl InMemorySize for OpPooledTransaction { - fn size(&self) -> usize { - self.inner.size() - } -} - -impl alloy_consensus::Transaction for OpPooledTransaction -where - Cons: alloy_consensus::Transaction, - Pooled: Debug + Send + Sync + 'static, -{ - fn chain_id(&self) -> Option { - self.inner.chain_id() - } - - fn nonce(&self) -> u64 { - self.inner.nonce() - } - - fn gas_limit(&self) -> u64 { - self.inner.gas_limit() - } - - fn gas_price(&self) -> Option { - self.inner.gas_price() - } - - fn max_fee_per_gas(&self) -> u128 { - self.inner.max_fee_per_gas() - } - - fn max_priority_fee_per_gas(&self) -> Option { - self.inner.max_priority_fee_per_gas() - } - - fn max_fee_per_blob_gas(&self) -> Option { - self.inner.max_fee_per_blob_gas() - } - - fn priority_fee_or_price(&self) -> u128 { - self.inner.priority_fee_or_price() - } - - fn effective_gas_price(&self, base_fee: Option) -> u128 { - self.inner.effective_gas_price(base_fee) - } - - fn is_dynamic_fee(&self) -> bool { - self.inner.is_dynamic_fee() - } - - fn kind(&self) -> TxKind { - self.inner.kind() - } - - fn is_create(&self) -> bool { - self.inner.is_create() - } - - fn value(&self) -> U256 { - self.inner.value() - } - - fn input(&self) -> &Bytes { - self.inner.input() - } - - fn access_list(&self) -> Option<&AccessList> { - self.inner.access_list() - } - - fn blob_versioned_hashes(&self) -> Option<&[B256]> { - self.inner.blob_versioned_hashes() - } - - fn authorization_list(&self) -> Option<&[SignedAuthorization]> { - self.inner.authorization_list() - } -} - -impl EthPoolTransaction for OpPooledTransaction -where - Cons: SignedTransaction + From, - Pooled: SignedTransaction + TryFrom, - >::Error: core::error::Error, -{ - fn take_blob(&mut self) -> EthBlobTransactionSidecar { - EthBlobTransactionSidecar::None - } - - fn try_into_pooled_eip4844( - self, - _sidecar: Arc, - ) -> Option> { - None - } - - fn try_from_eip4844( - _tx: Recovered, - _sidecar: BlobTransactionSidecar, - ) -> Option { - None - } - - fn validate_blob( - &self, - _sidecar: &BlobTransactionSidecar, - _settings: &KzgSettings, - ) -> Result<(), BlobTransactionValidationError> { - Err(BlobTransactionValidationError::NotBlobTransaction(self.ty())) - } -} - -/// Validator for Optimism transactions. -#[derive(Debug, Clone)] -pub struct OpTransactionValidator { - /// The type that performs the actual validation. - inner: EthTransactionValidator, - /// Additional block info required for validation. - block_info: Arc, - /// If true, ensure that the transaction's sender has enough balance to cover the L1 gas fee - /// derived from the tracked L1 block info that is extracted from the first transaction in the - /// L2 block. - require_l1_data_gas_fee: bool, -} - -impl OpTransactionValidator { - /// Returns the configured chain spec - pub fn chain_spec(&self) -> Arc - where - Client: ChainSpecProvider, - { - self.inner.chain_spec() - } - - /// Returns the configured client - pub fn client(&self) -> &Client { - self.inner.client() - } - - /// Returns the current block timestamp. - fn block_timestamp(&self) -> u64 { - self.block_info.timestamp.load(Ordering::Relaxed) - } - - /// Returns the current block number. - fn block_number(&self) -> u64 { - self.block_info.number.load(Ordering::Relaxed) - } - - /// Whether to ensure that the transaction's sender has enough balance to also cover the L1 gas - /// fee. - pub fn require_l1_data_gas_fee(self, require_l1_data_gas_fee: bool) -> Self { - Self { require_l1_data_gas_fee, ..self } - } - - /// Returns whether this validator also requires the transaction's sender to have enough balance - /// to cover the L1 gas fee. - pub const fn requires_l1_data_gas_fee(&self) -> bool { - self.require_l1_data_gas_fee - } -} - -impl OpTransactionValidator -where - Client: ChainSpecProvider + StateProviderFactory + BlockReaderIdExt, - Tx: EthPoolTransaction, -{ - /// Create a new [`OpTransactionValidator`]. - pub fn new(inner: EthTransactionValidator) -> Self { - let this = Self::with_block_info(inner, OpL1BlockInfo::default()); - if let Ok(Some(block)) = - this.inner.client().block_by_number_or_tag(alloy_eips::BlockNumberOrTag::Latest) - { - // genesis block has no txs, so we can't extract L1 info, we set the block info to empty - // so that we will accept txs into the pool before the first block - if block.header().number() == 0 { - this.block_info.timestamp.store(block.header().timestamp(), Ordering::Relaxed); - this.block_info.number.store(block.header().number(), Ordering::Relaxed); - } else { - this.update_l1_block_info(block.header(), block.body().transactions().first()); - } - } - - this - } - - /// Create a new [`OpTransactionValidator`] with the given [`OpL1BlockInfo`]. - pub fn with_block_info( - inner: EthTransactionValidator, - block_info: OpL1BlockInfo, - ) -> Self { - Self { inner, block_info: Arc::new(block_info), require_l1_data_gas_fee: true } - } - - /// Update the L1 block info for the given header and system transaction, if any. - /// - /// Note: this supports optional system transaction, in case this is used in a dev setuo - pub fn update_l1_block_info(&self, header: &H, tx: Option<&T>) - where - H: BlockHeader, - T: Transaction, - { - self.block_info.timestamp.store(header.timestamp(), Ordering::Relaxed); - self.block_info.number.store(header.number(), Ordering::Relaxed); - - if let Some(Ok(cost_addition)) = tx.map(reth_optimism_evm::extract_l1_info_from_tx) { - *self.block_info.l1_block_info.write() = cost_addition; - } - } - - /// Validates a single transaction. - /// - /// See also [`TransactionValidator::validate_transaction`] - /// - /// This behaves the same as [`EthTransactionValidator::validate_one`], but in addition, ensures - /// that the account has enough balance to cover the L1 gas cost. - pub fn validate_one( - &self, - origin: TransactionOrigin, - transaction: Tx, - ) -> TransactionValidationOutcome { - if transaction.is_eip4844() { - return TransactionValidationOutcome::Invalid( - transaction, - InvalidTransactionError::TxTypeNotSupported.into(), - ) - } - - let outcome = self.inner.validate_one(origin, transaction); - - if !self.requires_l1_data_gas_fee() { - // no need to check L1 gas fee - return outcome - } - - // ensure that the account has enough balance to cover the L1 gas cost - if let TransactionValidationOutcome::Valid { - balance, - state_nonce, - transaction: valid_tx, - propagate, - } = outcome - { - let mut l1_block_info = self.block_info.l1_block_info.read().clone(); - - let mut encoded = Vec::with_capacity(valid_tx.transaction().encoded_length()); - let tx = valid_tx.transaction().clone_into_consensus(); - tx.encode_2718(&mut encoded); - - let cost_addition = match l1_block_info.l1_tx_data_fee( - self.chain_spec(), - self.block_timestamp(), - self.block_number(), - &encoded, - false, - ) { - Ok(cost) => cost, - Err(err) => { - return TransactionValidationOutcome::Error(*valid_tx.hash(), Box::new(err)) - } - }; - let cost = valid_tx.transaction().cost().saturating_add(cost_addition); - - // Checks for max cost - if cost > balance { - return TransactionValidationOutcome::Invalid( - valid_tx.into_transaction(), - InvalidTransactionError::InsufficientFunds( - GotExpected { got: balance, expected: cost }.into(), - ) - .into(), - ) - } - - return TransactionValidationOutcome::Valid { - balance, - state_nonce, - transaction: valid_tx, - propagate, - } - } - - outcome - } - - /// Validates all given transactions. - /// - /// Returns all outcomes for the given transactions in the same order. - /// - /// See also [`Self::validate_one`] - pub fn validate_all( - &self, - transactions: Vec<(TransactionOrigin, Tx)>, - ) -> Vec> { - transactions.into_iter().map(|(origin, tx)| self.validate_one(origin, tx)).collect() - } -} - -impl TransactionValidator for OpTransactionValidator -where - Client: ChainSpecProvider + StateProviderFactory + BlockReaderIdExt, - Tx: EthPoolTransaction, -{ - type Transaction = Tx; - - async fn validate_transaction( - &self, - origin: TransactionOrigin, - transaction: Self::Transaction, - ) -> TransactionValidationOutcome { - self.validate_one(origin, transaction) - } - - async fn validate_transactions( - &self, - transactions: Vec<(TransactionOrigin, Self::Transaction)>, - ) -> Vec> { - self.validate_all(transactions) - } - - fn on_new_head_block(&self, new_tip_block: &SealedBlock) - where - B: Block, - { - self.inner.on_new_head_block(new_tip_block); - self.update_l1_block_info( - new_tip_block.header(), - new_tip_block.body().transactions().first(), - ); - } -} - -/// Tracks additional infos for the current block. -#[derive(Debug, Default)] -pub struct OpL1BlockInfo { - /// The current L1 block info. - l1_block_info: RwLock, - /// Current block timestamp. - timestamp: AtomicU64, - /// Current block number. - number: AtomicU64, -} - -#[cfg(test)] -mod tests { - use crate::txpool::{OpPooledTransaction, OpTransactionValidator}; - use alloy_eips::eip2718::Encodable2718; - use alloy_primitives::{PrimitiveSignature as Signature, TxKind, U256}; - use op_alloy_consensus::{OpTypedTransaction, TxDeposit}; - use reth_optimism_chainspec::OP_MAINNET; - use reth_optimism_primitives::OpTransactionSigned; - use reth_primitives::Recovered; - use reth_provider::test_utils::MockEthProvider; - use reth_transaction_pool::{ - blobstore::InMemoryBlobStore, validate::EthTransactionValidatorBuilder, TransactionOrigin, - TransactionValidationOutcome, - }; - #[test] - fn validate_optimism_transaction() { - let client = MockEthProvider::default().with_chain_spec(OP_MAINNET.clone()); - let validator = EthTransactionValidatorBuilder::new(client) - .no_shanghai() - .no_cancun() - .build(InMemoryBlobStore::default()); - let validator = OpTransactionValidator::new(validator); - - let origin = TransactionOrigin::External; - let signer = Default::default(); - let deposit_tx = OpTypedTransaction::Deposit(TxDeposit { - source_hash: Default::default(), - from: signer, - to: TxKind::Create, - mint: None, - value: U256::ZERO, - gas_limit: 0, - is_system_transaction: false, - input: Default::default(), - }); - let signature = Signature::test_signature(); - let signed_tx = OpTransactionSigned::new_unhashed(deposit_tx, signature); - let signed_recovered = Recovered::new_unchecked(signed_tx, signer); - let len = signed_recovered.encode_2718_len(); - let pooled_tx: OpPooledTransaction = OpPooledTransaction::new(signed_recovered, len); - let outcome = validator.validate_one(origin, pooled_tx); - - let err = match outcome { - TransactionValidationOutcome::Invalid(_, err) => err, - _ => panic!("Expected invalid transaction"), - }; - assert_eq!(err.to_string(), "transaction type not supported"); - } -} diff --git a/crates/optimism/txpool/Cargo.toml b/crates/optimism/txpool/Cargo.toml new file mode 100644 index 000000000..09569854f --- /dev/null +++ b/crates/optimism/txpool/Cargo.toml @@ -0,0 +1,51 @@ +[package] +name = "reth-optimism-txpool" +version.workspace = true +edition.workspace = true +rust-version.workspace = true +license.workspace = true +homepage.workspace = true +repository.workspace = true +description = "OP-Reth Transaction Pool" + +[lints] +workspace = true + +[dependencies] +# ethereum +alloy-consensus.workspace = true +alloy-eips.workspace = true +alloy-primitives.workspace = true +alloy-rpc-types-eth.workspace = true + +# reth +reth-chainspec.workspace = true +reth-primitives-traits.workspace = true +reth-storage-api.workspace = true +reth-transaction-pool.workspace = true + +# revm +revm.workspace = true + +# optimism +op-alloy-consensus.workspace = true +op-alloy-flz.workspace = true +reth-optimism-evm.workspace = true +reth-optimism-forks.workspace = true +reth-optimism-primitives = { workspace = true, features = ["reth-codec"] } + +# misc +c-kzg.workspace = true +derive_more.workspace = true +parking_lot.workspace = true + +[dev-dependencies] +reth-optimism-chainspec.workspace = true +reth-provider = { workspace = true, features = ["test-utils"] } + +[features] +optimism = [ + "reth-optimism-evm/optimism", + "reth-optimism-primitives/optimism", + "revm/optimism", +] diff --git a/crates/optimism/txpool/src/lib.rs b/crates/optimism/txpool/src/lib.rs new file mode 100644 index 000000000..f9c1165f3 --- /dev/null +++ b/crates/optimism/txpool/src/lib.rs @@ -0,0 +1,26 @@ +//! OP-Reth Transaction pool. + +#![doc( + html_logo_url = "https://raw.githubusercontent.com/paradigmxyz/reth/main/assets/reth-docs.png", + html_favicon_url = "https://avatars0.githubusercontent.com/u/97369466?s=256", + issue_tracker_base_url = "https://github.com/paradigmxyz/reth/issues/" +)] +#![cfg_attr(all(not(test), feature = "optimism"), warn(unused_crate_dependencies))] +#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))] +// The `optimism` feature must be enabled to use this crate. +#![cfg(feature = "optimism")] + +mod validator; +pub use validator::{OpL1BlockInfo, OpTransactionValidator}; + +mod transaction; +pub use transaction::OpPooledTransaction; + +use reth_transaction_pool::{CoinbaseTipOrdering, Pool, TransactionValidationTaskExecutor}; + +/// Type alias for default optimism transaction pool +pub type OpTransactionPool = Pool< + TransactionValidationTaskExecutor>, + CoinbaseTipOrdering, + S, +>; diff --git a/crates/optimism/txpool/src/transaction.rs b/crates/optimism/txpool/src/transaction.rs new file mode 100644 index 000000000..3cc83ed70 --- /dev/null +++ b/crates/optimism/txpool/src/transaction.rs @@ -0,0 +1,280 @@ +use alloy_consensus::{ + transaction::Recovered, BlobTransactionSidecar, BlobTransactionValidationError, Typed2718, +}; +use alloy_eips::{eip2930::AccessList, eip7702::SignedAuthorization}; +use alloy_primitives::{Address, Bytes, TxHash, TxKind, B256, U256}; +use alloy_rpc_types_eth::erc4337::TransactionConditional; +use c_kzg::KzgSettings; +use core::fmt::Debug; +use reth_optimism_primitives::OpTransactionSigned; +use reth_primitives_traits::{InMemorySize, SignedTransaction}; +use reth_transaction_pool::{ + EthBlobTransactionSidecar, EthPoolTransaction, EthPooledTransaction, PoolTransaction, +}; +use std::sync::{Arc, OnceLock}; + +/// Pool transaction for OP. +/// +/// This type wraps the actual transaction and caches values that are frequently used by the pool. +/// For payload building this lazily tracks values that are required during payload building: +/// - Estimated compressed size of this transaction +#[derive(Debug, Clone, derive_more::Deref)] +pub struct OpPooledTransaction< + Cons = OpTransactionSigned, + Pooled = op_alloy_consensus::OpPooledTransaction, +> { + #[deref] + inner: EthPooledTransaction, + /// The estimated size of this transaction, lazily computed. + estimated_tx_compressed_size: OnceLock, + /// The pooled transaction type. + _pd: core::marker::PhantomData, + + /// Optional conditional attached to this transaction. + conditional: Option>, +} + +impl OpPooledTransaction { + /// Create new instance of [Self]. + pub fn new(transaction: Recovered, encoded_length: usize) -> Self { + Self { + inner: EthPooledTransaction::new(transaction, encoded_length), + estimated_tx_compressed_size: Default::default(), + conditional: None, + _pd: core::marker::PhantomData, + } + } + + /// Returns the estimated compressed size of a transaction in bytes scaled by 1e6. + /// This value is computed based on the following formula: + /// `max(minTransactionSize, intercept + fastlzCoef*fastlzSize)` + pub fn estimated_compressed_size(&self) -> u64 { + *self.estimated_tx_compressed_size.get_or_init(|| { + op_alloy_flz::tx_estimated_size_fjord(&self.inner.transaction().encoded_2718()) + }) + } + + /// Conditional setter. + pub fn with_conditional(mut self, conditional: TransactionConditional) -> Self { + self.conditional = Some(Box::new(conditional)); + self + } + + /// Conditional getter. + pub fn conditional(&self) -> Option<&TransactionConditional> { + self.conditional.as_deref() + } +} + +impl PoolTransaction for OpPooledTransaction +where + Cons: SignedTransaction + From, + Pooled: SignedTransaction + TryFrom, +{ + type TryFromConsensusError = >::Error; + type Consensus = Cons; + type Pooled = Pooled; + + fn clone_into_consensus(&self) -> Recovered { + self.inner.transaction().clone() + } + + fn into_consensus(self) -> Recovered { + self.inner.transaction + } + + fn from_pooled(tx: Recovered) -> Self { + let encoded_len = tx.encode_2718_len(); + let tx = tx.map_transaction(|tx| tx.into()); + Self::new(tx, encoded_len) + } + + fn hash(&self) -> &TxHash { + self.inner.transaction.tx_hash() + } + + fn sender(&self) -> Address { + self.inner.transaction.signer() + } + + fn sender_ref(&self) -> &Address { + self.inner.transaction.signer_ref() + } + + fn cost(&self) -> &U256 { + &self.inner.cost + } + + fn encoded_length(&self) -> usize { + self.inner.encoded_length + } +} + +impl Typed2718 for OpPooledTransaction { + fn ty(&self) -> u8 { + self.inner.ty() + } +} + +impl InMemorySize for OpPooledTransaction { + fn size(&self) -> usize { + self.inner.size() + } +} + +impl alloy_consensus::Transaction for OpPooledTransaction +where + Cons: alloy_consensus::Transaction, + Pooled: Debug + Send + Sync + 'static, +{ + fn chain_id(&self) -> Option { + self.inner.chain_id() + } + + fn nonce(&self) -> u64 { + self.inner.nonce() + } + + fn gas_limit(&self) -> u64 { + self.inner.gas_limit() + } + + fn gas_price(&self) -> Option { + self.inner.gas_price() + } + + fn max_fee_per_gas(&self) -> u128 { + self.inner.max_fee_per_gas() + } + + fn max_priority_fee_per_gas(&self) -> Option { + self.inner.max_priority_fee_per_gas() + } + + fn max_fee_per_blob_gas(&self) -> Option { + self.inner.max_fee_per_blob_gas() + } + + fn priority_fee_or_price(&self) -> u128 { + self.inner.priority_fee_or_price() + } + + fn effective_gas_price(&self, base_fee: Option) -> u128 { + self.inner.effective_gas_price(base_fee) + } + + fn is_dynamic_fee(&self) -> bool { + self.inner.is_dynamic_fee() + } + + fn kind(&self) -> TxKind { + self.inner.kind() + } + + fn is_create(&self) -> bool { + self.inner.is_create() + } + + fn value(&self) -> U256 { + self.inner.value() + } + + fn input(&self) -> &Bytes { + self.inner.input() + } + + fn access_list(&self) -> Option<&AccessList> { + self.inner.access_list() + } + + fn blob_versioned_hashes(&self) -> Option<&[B256]> { + self.inner.blob_versioned_hashes() + } + + fn authorization_list(&self) -> Option<&[SignedAuthorization]> { + self.inner.authorization_list() + } +} + +impl EthPoolTransaction for OpPooledTransaction +where + Cons: SignedTransaction + From, + Pooled: SignedTransaction + TryFrom, + >::Error: core::error::Error, +{ + fn take_blob(&mut self) -> EthBlobTransactionSidecar { + EthBlobTransactionSidecar::None + } + + fn try_into_pooled_eip4844( + self, + _sidecar: Arc, + ) -> Option> { + None + } + + fn try_from_eip4844( + _tx: Recovered, + _sidecar: BlobTransactionSidecar, + ) -> Option { + None + } + + fn validate_blob( + &self, + _sidecar: &BlobTransactionSidecar, + _settings: &KzgSettings, + ) -> Result<(), BlobTransactionValidationError> { + Err(BlobTransactionValidationError::NotBlobTransaction(self.ty())) + } +} + +#[cfg(test)] +mod tests { + use crate::{OpPooledTransaction, OpTransactionValidator}; + use alloy_consensus::transaction::Recovered; + use alloy_eips::eip2718::Encodable2718; + use alloy_primitives::{PrimitiveSignature as Signature, TxKind, U256}; + use op_alloy_consensus::{OpTypedTransaction, TxDeposit}; + use reth_optimism_chainspec::OP_MAINNET; + use reth_optimism_primitives::OpTransactionSigned; + use reth_provider::test_utils::MockEthProvider; + use reth_transaction_pool::{ + blobstore::InMemoryBlobStore, validate::EthTransactionValidatorBuilder, TransactionOrigin, + TransactionValidationOutcome, + }; + #[test] + fn validate_optimism_transaction() { + let client = MockEthProvider::default().with_chain_spec(OP_MAINNET.clone()); + let validator = EthTransactionValidatorBuilder::new(client) + .no_shanghai() + .no_cancun() + .build(InMemoryBlobStore::default()); + let validator = OpTransactionValidator::new(validator); + + let origin = TransactionOrigin::External; + let signer = Default::default(); + let deposit_tx = OpTypedTransaction::Deposit(TxDeposit { + source_hash: Default::default(), + from: signer, + to: TxKind::Create, + mint: None, + value: U256::ZERO, + gas_limit: 0, + is_system_transaction: false, + input: Default::default(), + }); + let signature = Signature::test_signature(); + let signed_tx = OpTransactionSigned::new_unhashed(deposit_tx, signature); + let signed_recovered = Recovered::new_unchecked(signed_tx, signer); + let len = signed_recovered.encode_2718_len(); + let pooled_tx: OpPooledTransaction = OpPooledTransaction::new(signed_recovered, len); + let outcome = validator.validate_one(origin, pooled_tx); + + let err = match outcome { + TransactionValidationOutcome::Invalid(_, err) => err, + _ => panic!("Expected invalid transaction"), + }; + assert_eq!(err.to_string(), "transaction type not supported"); + } +} diff --git a/crates/optimism/txpool/src/validator.rs b/crates/optimism/txpool/src/validator.rs new file mode 100644 index 000000000..cd41475dc --- /dev/null +++ b/crates/optimism/txpool/src/validator.rs @@ -0,0 +1,250 @@ +use alloy_consensus::{BlockHeader, Transaction}; +use alloy_eips::Encodable2718; +use parking_lot::RwLock; +use reth_chainspec::ChainSpecProvider; +use reth_optimism_evm::RethL1BlockInfo; +use reth_optimism_forks::OpHardforks; +use reth_primitives_traits::{ + transaction::error::InvalidTransactionError, Block, BlockBody, GotExpected, SealedBlock, +}; +use reth_storage_api::{BlockReaderIdExt, StateProviderFactory}; +use reth_transaction_pool::{ + EthPoolTransaction, EthTransactionValidator, TransactionOrigin, TransactionValidationOutcome, + TransactionValidator, +}; +use revm::L1BlockInfo; +use std::sync::{ + atomic::{AtomicU64, Ordering}, + Arc, +}; + +/// Tracks additional infos for the current block. +#[derive(Debug, Default)] +pub struct OpL1BlockInfo { + /// The current L1 block info. + l1_block_info: RwLock, + /// Current block timestamp. + timestamp: AtomicU64, + /// Current block number. + number: AtomicU64, +} + +/// Validator for Optimism transactions. +#[derive(Debug, Clone)] +pub struct OpTransactionValidator { + /// The type that performs the actual validation. + inner: EthTransactionValidator, + /// Additional block info required for validation. + block_info: Arc, + /// If true, ensure that the transaction's sender has enough balance to cover the L1 gas fee + /// derived from the tracked L1 block info that is extracted from the first transaction in the + /// L2 block. + require_l1_data_gas_fee: bool, +} + +impl OpTransactionValidator { + /// Returns the configured chain spec + pub fn chain_spec(&self) -> Arc + where + Client: ChainSpecProvider, + { + self.inner.chain_spec() + } + + /// Returns the configured client + pub fn client(&self) -> &Client { + self.inner.client() + } + + /// Returns the current block timestamp. + fn block_timestamp(&self) -> u64 { + self.block_info.timestamp.load(Ordering::Relaxed) + } + + /// Returns the current block number. + fn block_number(&self) -> u64 { + self.block_info.number.load(Ordering::Relaxed) + } + + /// Whether to ensure that the transaction's sender has enough balance to also cover the L1 gas + /// fee. + pub fn require_l1_data_gas_fee(self, require_l1_data_gas_fee: bool) -> Self { + Self { require_l1_data_gas_fee, ..self } + } + + /// Returns whether this validator also requires the transaction's sender to have enough balance + /// to cover the L1 gas fee. + pub const fn requires_l1_data_gas_fee(&self) -> bool { + self.require_l1_data_gas_fee + } +} + +impl OpTransactionValidator +where + Client: ChainSpecProvider + StateProviderFactory + BlockReaderIdExt, + Tx: EthPoolTransaction, +{ + /// Create a new [`OpTransactionValidator`]. + pub fn new(inner: EthTransactionValidator) -> Self { + let this = Self::with_block_info(inner, OpL1BlockInfo::default()); + if let Ok(Some(block)) = + this.inner.client().block_by_number_or_tag(alloy_eips::BlockNumberOrTag::Latest) + { + // genesis block has no txs, so we can't extract L1 info, we set the block info to empty + // so that we will accept txs into the pool before the first block + if block.header().number() == 0 { + this.block_info.timestamp.store(block.header().timestamp(), Ordering::Relaxed); + this.block_info.number.store(block.header().number(), Ordering::Relaxed); + } else { + this.update_l1_block_info(block.header(), block.body().transactions().first()); + } + } + + this + } + + /// Create a new [`OpTransactionValidator`] with the given [`OpL1BlockInfo`]. + pub fn with_block_info( + inner: EthTransactionValidator, + block_info: OpL1BlockInfo, + ) -> Self { + Self { inner, block_info: Arc::new(block_info), require_l1_data_gas_fee: true } + } + + /// Update the L1 block info for the given header and system transaction, if any. + /// + /// Note: this supports optional system transaction, in case this is used in a dev setuo + pub fn update_l1_block_info(&self, header: &H, tx: Option<&T>) + where + H: BlockHeader, + T: Transaction, + { + self.block_info.timestamp.store(header.timestamp(), Ordering::Relaxed); + self.block_info.number.store(header.number(), Ordering::Relaxed); + + if let Some(Ok(cost_addition)) = tx.map(reth_optimism_evm::extract_l1_info_from_tx) { + *self.block_info.l1_block_info.write() = cost_addition; + } + } + + /// Validates a single transaction. + /// + /// See also [`TransactionValidator::validate_transaction`] + /// + /// This behaves the same as [`EthTransactionValidator::validate_one`], but in addition, ensures + /// that the account has enough balance to cover the L1 gas cost. + pub fn validate_one( + &self, + origin: TransactionOrigin, + transaction: Tx, + ) -> TransactionValidationOutcome { + if transaction.is_eip4844() { + return TransactionValidationOutcome::Invalid( + transaction, + InvalidTransactionError::TxTypeNotSupported.into(), + ) + } + + let outcome = self.inner.validate_one(origin, transaction); + + if !self.requires_l1_data_gas_fee() { + // no need to check L1 gas fee + return outcome + } + + // ensure that the account has enough balance to cover the L1 gas cost + if let TransactionValidationOutcome::Valid { + balance, + state_nonce, + transaction: valid_tx, + propagate, + } = outcome + { + let mut l1_block_info = self.block_info.l1_block_info.read().clone(); + + let mut encoded = Vec::with_capacity(valid_tx.transaction().encoded_length()); + let tx = valid_tx.transaction().clone_into_consensus(); + tx.encode_2718(&mut encoded); + + let cost_addition = match l1_block_info.l1_tx_data_fee( + self.chain_spec(), + self.block_timestamp(), + self.block_number(), + &encoded, + false, + ) { + Ok(cost) => cost, + Err(err) => { + return TransactionValidationOutcome::Error(*valid_tx.hash(), Box::new(err)) + } + }; + let cost = valid_tx.transaction().cost().saturating_add(cost_addition); + + // Checks for max cost + if cost > balance { + return TransactionValidationOutcome::Invalid( + valid_tx.into_transaction(), + InvalidTransactionError::InsufficientFunds( + GotExpected { got: balance, expected: cost }.into(), + ) + .into(), + ) + } + + return TransactionValidationOutcome::Valid { + balance, + state_nonce, + transaction: valid_tx, + propagate, + } + } + + outcome + } + + /// Validates all given transactions. + /// + /// Returns all outcomes for the given transactions in the same order. + /// + /// See also [`Self::validate_one`] + pub fn validate_all( + &self, + transactions: Vec<(TransactionOrigin, Tx)>, + ) -> Vec> { + transactions.into_iter().map(|(origin, tx)| self.validate_one(origin, tx)).collect() + } +} + +impl TransactionValidator for OpTransactionValidator +where + Client: ChainSpecProvider + StateProviderFactory + BlockReaderIdExt, + Tx: EthPoolTransaction, +{ + type Transaction = Tx; + + async fn validate_transaction( + &self, + origin: TransactionOrigin, + transaction: Self::Transaction, + ) -> TransactionValidationOutcome { + self.validate_one(origin, transaction) + } + + async fn validate_transactions( + &self, + transactions: Vec<(TransactionOrigin, Self::Transaction)>, + ) -> Vec> { + self.validate_all(transactions) + } + + fn on_new_head_block(&self, new_tip_block: &SealedBlock) + where + B: Block, + { + self.inner.on_new_head_block(new_tip_block); + self.update_l1_block_info( + new_tip_block.header(), + new_tip_block.body().transactions().first(), + ); + } +}