From 8226fa0cacc42c5ca55f7638b3fed57a16cac008 Mon Sep 17 00:00:00 2001 From: Arsenii Kulikov Date: Thu, 5 Dec 2024 22:50:43 +0400 Subject: [PATCH] feat: use network tx for `Pool::Pooled` (#13159) Co-authored-by: Matthias Seitz --- crates/ethereum/node/src/node.rs | 10 ++- crates/net/network/src/test_utils/testnet.rs | 26 ++++-- crates/net/network/src/transactions/mod.rs | 21 ++--- crates/node/builder/src/builder/mod.rs | 10 ++- crates/optimism/node/src/node.rs | 10 ++- crates/optimism/rpc/src/eth/transaction.rs | 3 +- .../rpc-eth-api/src/helpers/transaction.rs | 2 +- crates/rpc/rpc-eth-types/src/utils.rs | 12 +-- crates/rpc/rpc/src/eth/bundle.rs | 2 +- crates/rpc/rpc/src/eth/sim_bundle.rs | 5 +- crates/rpc/rpc/src/trace.rs | 4 +- crates/transaction-pool/src/lib.rs | 15 +--- crates/transaction-pool/src/maintain.rs | 11 +-- crates/transaction-pool/src/noop.rs | 14 +--- crates/transaction-pool/src/pool/mod.rs | 20 +---- .../transaction-pool/src/test_utils/mock.rs | 41 +++++++--- crates/transaction-pool/src/traits.rs | 80 ++++++++++--------- 17 files changed, 148 insertions(+), 138 deletions(-) diff --git a/crates/ethereum/node/src/node.rs b/crates/ethereum/node/src/node.rs index dd4f1e5802c8..b2fc7e677aca 100644 --- a/crates/ethereum/node/src/node.rs +++ b/crates/ethereum/node/src/node.rs @@ -25,7 +25,7 @@ use reth_node_builder::{ BuilderContext, Node, NodeAdapter, NodeComponentsBuilder, PayloadBuilderConfig, PayloadTypes, }; use reth_payload_builder::{PayloadBuilderHandle, PayloadBuilderService}; -use reth_primitives::EthPrimitives; +use reth_primitives::{EthPrimitives, PooledTransactionsElement}; use reth_provider::{CanonStateSubscriptions, EthStorage}; use reth_rpc::EthApi; use reth_tracing::tracing::{debug, info}; @@ -309,8 +309,12 @@ pub struct EthereumNetworkBuilder { impl NetworkBuilder for EthereumNetworkBuilder where Node: FullNodeTypes>, - Pool: TransactionPool>> - + Unpin + Pool: TransactionPool< + Transaction: PoolTransaction< + Consensus = TxTy, + Pooled = PooledTransactionsElement, + >, + > + Unpin + 'static, { async fn build_network( diff --git a/crates/net/network/src/test_utils/testnet.rs b/crates/net/network/src/test_utils/testnet.rs index 249cd1a6beb0..3a50d890e70a 100644 --- a/crates/net/network/src/test_utils/testnet.rs +++ b/crates/net/network/src/test_utils/testnet.rs @@ -20,7 +20,7 @@ use reth_network_api::{ NetworkEvent, NetworkEventListenerProvider, NetworkInfo, Peers, }; use reth_network_peers::PeerId; -use reth_primitives::TransactionSigned; +use reth_primitives::{PooledTransactionsElement, TransactionSigned}; use reth_provider::{test_utils::NoopProvider, ChainSpecProvider}; use reth_storage_api::{BlockReader, BlockReaderIdExt, HeaderProvider, StateProviderFactory}; use reth_tasks::TokioTaskExecutor; @@ -206,8 +206,12 @@ where + Clone + Unpin + 'static, - Pool: TransactionPool> - + Unpin + Pool: TransactionPool< + Transaction: PoolTransaction< + Consensus = TransactionSigned, + Pooled = PooledTransactionsElement, + >, + > + Unpin + 'static, { /// Spawns the testnet to a separate task @@ -273,8 +277,12 @@ where > + HeaderProvider + Unpin + 'static, - Pool: TransactionPool> - + Unpin + Pool: TransactionPool< + Transaction: PoolTransaction< + Consensus = TransactionSigned, + Pooled = PooledTransactionsElement, + >, + > + Unpin + 'static, { type Output = (); @@ -476,8 +484,12 @@ where > + HeaderProvider + Unpin + 'static, - Pool: TransactionPool> - + Unpin + Pool: TransactionPool< + Transaction: PoolTransaction< + Consensus = TransactionSigned, + Pooled = PooledTransactionsElement, + >, + > + Unpin + 'static, { type Output = (); diff --git a/crates/net/network/src/transactions/mod.rs b/crates/net/network/src/transactions/mod.rs index 3864c0fcc0b0..fefff2bac358 100644 --- a/crates/net/network/src/transactions/mod.rs +++ b/crates/net/network/src/transactions/mod.rs @@ -49,9 +49,7 @@ use reth_network_p2p::{ }; use reth_network_peers::PeerId; use reth_network_types::ReputationChangeKind; -use reth_primitives::{ - transaction::SignedTransactionIntoRecoveredExt, RecoveredTx, TransactionSigned, -}; +use reth_primitives::{transaction::SignedTransactionIntoRecoveredExt, TransactionSigned}; use reth_primitives_traits::{SignedTransaction, TxType}; use reth_tokio_util::EventStream; use reth_transaction_pool::{ @@ -703,10 +701,8 @@ where BroadcastedTransaction: SignedTransaction, PooledTransaction: SignedTransaction, >, - Pool::Transaction: PoolTransaction< - Consensus = N::BroadcastedTransaction, - Pooled: Into + From>, - >, + Pool::Transaction: + PoolTransaction, { /// Invoked when transactions in the local mempool are considered __pending__. /// @@ -991,13 +987,12 @@ where let _ = response.send(Ok(PooledTransactions::default())); return } - let transactions = self.pool.get_pooled_transactions_as::( + let transactions = self.pool.get_pooled_transaction_elements( request.0, GetPooledTransactionLimit::ResponseSizeSoftLimit( self.transaction_fetcher.info.soft_limit_byte_size_pooled_transactions_response, ), ); - trace!(target: "net::tx::propagation", sent_txs=?transactions.iter().map(|tx| tx.tx_hash()), "Sending requested transactions to peer"); // we sent a response at which point we assume that the peer is aware of the @@ -1247,7 +1242,7 @@ where } else { // this is a new transaction that should be imported into the pool - let pool_transaction = Pool::Transaction::from_pooled(tx.into()); + let pool_transaction = Pool::Transaction::from_pooled(tx); new_txs.push(pool_transaction); entry.insert(HashSet::from([peer_id])); @@ -1338,10 +1333,8 @@ where BroadcastedTransaction: SignedTransaction, PooledTransaction: SignedTransaction, >, - Pool::Transaction: PoolTransaction< - Consensus = N::BroadcastedTransaction, - Pooled: Into + From>, - >, + Pool::Transaction: + PoolTransaction, { type Output = (); diff --git a/crates/node/builder/src/builder/mod.rs b/crates/node/builder/src/builder/mod.rs index 3cab01aa71be..e2b18f666c76 100644 --- a/crates/node/builder/src/builder/mod.rs +++ b/crates/node/builder/src/builder/mod.rs @@ -651,7 +651,10 @@ impl BuilderContext { pub fn start_network(&self, builder: NetworkBuilder<(), ()>, pool: Pool) -> NetworkHandle where Pool: TransactionPool< - Transaction: PoolTransaction, + Transaction: PoolTransaction< + Consensus = reth_primitives::TransactionSigned, + Pooled = reth_primitives::PooledTransactionsElement, + >, > + Unpin + 'static, Node::Provider: BlockReader< @@ -677,7 +680,10 @@ impl BuilderContext { ) -> NetworkHandle where Pool: TransactionPool< - Transaction: PoolTransaction, + Transaction: PoolTransaction< + Consensus = reth_primitives::TransactionSigned, + Pooled = reth_primitives::PooledTransactionsElement, + >, > + Unpin + 'static, Node::Provider: BlockReader< diff --git a/crates/optimism/node/src/node.rs b/crates/optimism/node/src/node.rs index bd045320f527..b7dcf2741c6d 100644 --- a/crates/optimism/node/src/node.rs +++ b/crates/optimism/node/src/node.rs @@ -35,7 +35,7 @@ use reth_optimism_rpc::{ OpEthApi, SequencerClient, }; use reth_payload_builder::{PayloadBuilderHandle, PayloadBuilderService}; -use reth_primitives::{BlockBody, TransactionSigned}; +use reth_primitives::{BlockBody, PooledTransactionsElement, TransactionSigned}; use reth_provider::{ providers::ChainStorage, BlockBodyReader, BlockBodyWriter, CanonStateSubscriptions, ChainSpecProvider, DBProvider, EthStorage, ProviderResult, ReadBodyInput, @@ -633,8 +633,12 @@ impl OpNetworkBuilder { impl NetworkBuilder for OpNetworkBuilder where Node: FullNodeTypes>, - Pool: TransactionPool>> - + Unpin + Pool: TransactionPool< + Transaction: PoolTransaction< + Consensus = TxTy, + Pooled = PooledTransactionsElement, + >, + > + Unpin + 'static, { async fn build_network( diff --git a/crates/optimism/rpc/src/eth/transaction.rs b/crates/optimism/rpc/src/eth/transaction.rs index b5d4ce2bc555..cfc81ab644eb 100644 --- a/crates/optimism/rpc/src/eth/transaction.rs +++ b/crates/optimism/rpc/src/eth/transaction.rs @@ -31,8 +31,7 @@ where /// Returns the hash of the transaction. async fn send_raw_transaction(&self, tx: Bytes) -> Result { let recovered = recover_raw_transaction(tx.clone())?; - let pool_transaction = - ::Transaction::from_pooled(recovered.into()); + let pool_transaction = ::Transaction::from_pooled(recovered); // On optimism, transactions are forwarded directly to the sequencer to be included in // blocks that it builds. diff --git a/crates/rpc/rpc-eth-api/src/helpers/transaction.rs b/crates/rpc/rpc-eth-api/src/helpers/transaction.rs index f73d761600e8..5c6478540f75 100644 --- a/crates/rpc/rpc-eth-api/src/helpers/transaction.rs +++ b/crates/rpc/rpc-eth-api/src/helpers/transaction.rs @@ -344,7 +344,7 @@ pub trait EthTransactions: LoadTransaction { async move { let recovered = recover_raw_transaction(tx)?; let pool_transaction = - ::Transaction::from_pooled(recovered.into()); + ::Transaction::from_pooled(recovered); // submit the transaction to the pool with a `Local` origin let hash = self diff --git a/crates/rpc/rpc-eth-types/src/utils.rs b/crates/rpc/rpc-eth-types/src/utils.rs index 596acc74ce1e..64f159ea0e8b 100644 --- a/crates/rpc/rpc-eth-types/src/utils.rs +++ b/crates/rpc/rpc-eth-types/src/utils.rs @@ -1,21 +1,21 @@ //! Commonly used code snippets -use alloy_eips::eip2718::Decodable2718; use alloy_primitives::Bytes; -use reth_primitives::{PooledTransactionsElement, PooledTransactionsElementEcRecovered}; +use reth_primitives::{transaction::SignedTransactionIntoRecoveredExt, RecoveredTx}; +use reth_primitives_traits::SignedTransaction; use std::future::Future; use super::{EthApiError, EthResult}; -/// Recovers a [`PooledTransactionsElementEcRecovered`] from an enveloped encoded byte stream. +/// Recovers a [`SignedTransaction`] from an enveloped encoded byte stream. /// -/// See [`Decodable2718::decode_2718`] -pub fn recover_raw_transaction(data: Bytes) -> EthResult { +/// See [`alloy_eips::eip2718::Decodable2718::decode_2718`] +pub fn recover_raw_transaction(data: Bytes) -> EthResult> { if data.is_empty() { return Err(EthApiError::EmptyRawTransactionData) } - let transaction = PooledTransactionsElement::decode_2718(&mut data.as_ref()) + let transaction = T::decode_2718(&mut data.as_ref()) .map_err(|_| EthApiError::FailedToDecodeSignedTransaction)?; transaction.try_into_ecrecovered().or(Err(EthApiError::InvalidTransactionSignature)) diff --git a/crates/rpc/rpc/src/eth/bundle.rs b/crates/rpc/rpc/src/eth/bundle.rs index 2924e6ea25fc..d5117650a134 100644 --- a/crates/rpc/rpc/src/eth/bundle.rs +++ b/crates/rpc/rpc/src/eth/bundle.rs @@ -79,7 +79,7 @@ where let transactions = txs .into_iter() - .map(recover_raw_transaction) + .map(recover_raw_transaction::) .collect::, _>>()? .into_iter() .map(|tx| tx.to_components()) diff --git a/crates/rpc/rpc/src/eth/sim_bundle.rs b/crates/rpc/rpc/src/eth/sim_bundle.rs index 87778ec6e650..0702d7df0e42 100644 --- a/crates/rpc/rpc/src/eth/sim_bundle.rs +++ b/crates/rpc/rpc/src/eth/sim_bundle.rs @@ -10,7 +10,7 @@ use alloy_rpc_types_mev::{ use jsonrpsee::core::RpcResult; use reth_chainspec::EthChainSpec; use reth_evm::{ConfigureEvm, ConfigureEvmEnv}; -use reth_primitives::TransactionSigned; +use reth_primitives::{PooledTransactionsElement, TransactionSigned}; use reth_provider::{ChainSpecProvider, HeaderProvider}; use reth_revm::database::StateProviderDatabase; use reth_rpc_api::MevSimApiServer; @@ -171,7 +171,8 @@ where match &body[idx] { BundleItem::Tx { tx, can_revert } => { let recovered_tx = - recover_raw_transaction(tx.clone()).map_err(EthApiError::from)?; + recover_raw_transaction::(tx.clone()) + .map_err(EthApiError::from)?; let (tx, signer) = recovered_tx.to_components(); let tx = tx.into_transaction(); diff --git a/crates/rpc/rpc/src/trace.rs b/crates/rpc/rpc/src/trace.rs index f81eefdc5ff1..009203f757dd 100644 --- a/crates/rpc/rpc/src/trace.rs +++ b/crates/rpc/rpc/src/trace.rs @@ -19,6 +19,7 @@ use reth_consensus_common::calc::{ base_block_reward, base_block_reward_pre_merge, block_reward, ommer_reward, }; use reth_evm::ConfigureEvmEnv; +use reth_primitives::PooledTransactionsElement; use reth_provider::{BlockReader, ChainSpecProvider, EvmEnvProvider, StateProviderFactory}; use reth_revm::database::StateProviderDatabase; use reth_rpc_api::TraceApiServer; @@ -115,7 +116,8 @@ where trace_types: HashSet, block_id: Option, ) -> Result { - let tx = recover_raw_transaction(tx)?.into_ecrecovered_transaction(); + let tx = recover_raw_transaction::(tx)? + .into_ecrecovered_transaction(); let (cfg, block, at) = self.eth_api().evm_env_at(block_id.unwrap_or_default()).await?; diff --git a/crates/transaction-pool/src/lib.rs b/crates/transaction-pool/src/lib.rs index 1c383e8edf01..0e069291e735 100644 --- a/crates/transaction-pool/src/lib.rs +++ b/crates/transaction-pool/src/lib.rs @@ -156,6 +156,7 @@ use alloy_primitives::{Address, TxHash, B256, U256}; use aquamarine as _; use reth_eth_wire_types::HandleMempoolData; use reth_execution_types::ChangedAccount; +use reth_primitives::RecoveredTx; use reth_storage_api::StateProviderFactory; use std::{collections::HashSet, sync::Arc}; use tokio::sync::mpsc::Receiver; @@ -419,21 +420,11 @@ where self.pool.get_pooled_transaction_elements(tx_hashes, limit) } - fn get_pooled_transactions_as

( - &self, - tx_hashes: Vec, - limit: GetPooledTransactionLimit, - ) -> Vec

- where - ::Pooled: Into

, - { - self.pool.get_pooled_transactions_as(tx_hashes, limit) - } - fn get_pooled_transaction_element( &self, tx_hash: TxHash, - ) -> Option<<::Transaction as PoolTransaction>::Pooled> { + ) -> Option::Transaction as PoolTransaction>::Pooled>> + { self.pool.get_pooled_transaction_element(tx_hash) } diff --git a/crates/transaction-pool/src/maintain.rs b/crates/transaction-pool/src/maintain.rs index fa7b75e34ad1..96971b487f0d 100644 --- a/crates/transaction-pool/src/maintain.rs +++ b/crates/transaction-pool/src/maintain.rs @@ -4,7 +4,7 @@ use crate::{ blobstore::{BlobStoreCanonTracker, BlobStoreUpdates}, error::PoolError, metrics::MaintainPoolMetrics, - traits::{CanonicalStateUpdate, TransactionPool, TransactionPoolExt}, + traits::{CanonicalStateUpdate, EthPoolTransaction, TransactionPool, TransactionPoolExt}, BlockInfo, PoolTransaction, PoolUpdateKind, }; use alloy_consensus::BlockHeader; @@ -20,8 +20,7 @@ use reth_chainspec::{ChainSpecProvider, EthChainSpec}; use reth_execution_types::ChangedAccount; use reth_fs_util::FsPathError; use reth_primitives::{ - transaction::SignedTransactionIntoRecoveredExt, PooledTransactionsElementEcRecovered, - SealedHeader, TransactionSigned, + transaction::SignedTransactionIntoRecoveredExt, SealedHeader, TransactionSigned, }; use reth_primitives_traits::SignedTransaction; use reth_storage_api::{errors::provider::ProviderError, BlockReaderIdExt, StateProviderFactory}; @@ -335,13 +334,9 @@ pub async fn maintain_transaction_pool( .flatten() .map(Arc::unwrap_or_clone) .and_then(|sidecar| { - PooledTransactionsElementEcRecovered::try_from_blob_transaction( +

::Transaction::try_from_eip4844( tx, sidecar, ) - .ok() - }) - .map(|tx| { -

::Transaction::from_pooled(tx.into()) }) } else {

::Transaction::try_from_consensus(tx).ok() diff --git a/crates/transaction-pool/src/noop.rs b/crates/transaction-pool/src/noop.rs index 3a068d3a5936..8d880994aa9e 100644 --- a/crates/transaction-pool/src/noop.rs +++ b/crates/transaction-pool/src/noop.rs @@ -22,6 +22,7 @@ use alloy_eips::{ }; use alloy_primitives::{Address, TxHash, B256, U256}; use reth_eth_wire_types::HandleMempoolData; +use reth_primitives::RecoveredTx; use std::{collections::HashSet, marker::PhantomData, sync::Arc}; use tokio::sync::{mpsc, mpsc::Receiver}; @@ -139,21 +140,10 @@ impl TransactionPool for NoopTransactionPool { vec![] } - fn get_pooled_transactions_as( - &self, - _tx_hashes: Vec, - _limit: GetPooledTransactionLimit, - ) -> Vec - where - ::Pooled: Into, - { - vec![] - } - fn get_pooled_transaction_element( &self, _tx_hash: TxHash, - ) -> Option<::Pooled> { + ) -> Option::Pooled>> { None } diff --git a/crates/transaction-pool/src/pool/mod.rs b/crates/transaction-pool/src/pool/mod.rs index fce1b9acef80..044b192fe590 100644 --- a/crates/transaction-pool/src/pool/mod.rs +++ b/crates/transaction-pool/src/pool/mod.rs @@ -88,6 +88,7 @@ use reth_eth_wire_types::HandleMempoolData; use reth_execution_types::ChangedAccount; use alloy_eips::eip4844::BlobTransactionSidecar; +use reth_primitives::RecoveredTx; use std::{ collections::{HashMap, HashSet}, fmt, @@ -312,7 +313,7 @@ where fn to_pooled_transaction( &self, transaction: Arc>, - ) -> Option<<::Transaction as PoolTransaction>::Pooled> + ) -> Option::Transaction as PoolTransaction>::Pooled>> where ::Transaction: EthPoolTransaction, { @@ -342,19 +343,6 @@ where ) -> Vec<<::Transaction as PoolTransaction>::Pooled> where ::Transaction: EthPoolTransaction, - { - self.get_pooled_transactions_as(tx_hashes, limit) - } - - /// Returns pooled transactions for the given transaction hashes as the requested type. - pub fn get_pooled_transactions_as

( - &self, - tx_hashes: Vec, - limit: GetPooledTransactionLimit, - ) -> Vec

- where - ::Transaction: EthPoolTransaction, - <::Transaction as PoolTransaction>::Pooled: Into

, { let transactions = self.get_all(tx_hashes); let mut elements = Vec::with_capacity(transactions.len()); @@ -366,7 +354,7 @@ where }; size += encoded_len; - elements.push(pooled.into()); + elements.push(pooled.into_signed()); if limit.exceeds(size) { break @@ -380,7 +368,7 @@ where pub fn get_pooled_transaction_element( &self, tx_hash: TxHash, - ) -> Option<<::Transaction as PoolTransaction>::Pooled> + ) -> Option::Transaction as PoolTransaction>::Pooled>> where ::Transaction: EthPoolTransaction, { diff --git a/crates/transaction-pool/src/test_utils/mock.rs b/crates/transaction-pool/src/test_utils/mock.rs index 0e8b26faf83b..d174c7b16047 100644 --- a/crates/transaction-pool/src/test_utils/mock.rs +++ b/crates/transaction-pool/src/test_utils/mock.rs @@ -25,8 +25,9 @@ use rand::{ prelude::Distribution, }; use reth_primitives::{ - transaction::TryFromRecoveredTransactionError, PooledTransactionsElementEcRecovered, - RecoveredTx, Transaction, TransactionSigned, TxType, + transaction::{SignedTransactionIntoRecoveredExt, TryFromRecoveredTransactionError}, + PooledTransactionsElement, PooledTransactionsElementEcRecovered, RecoveredTx, Transaction, + TransactionSigned, TxType, }; use reth_primitives_traits::InMemorySize; use std::{ops::Range, sync::Arc, time::Instant, vec::IntoIter}; @@ -594,7 +595,7 @@ impl PoolTransaction for MockTransaction { type Consensus = TransactionSigned; - type Pooled = PooledTransactionsElementEcRecovered; + type Pooled = PooledTransactionsElement; fn try_from_consensus( tx: RecoveredTx, @@ -606,14 +607,17 @@ impl PoolTransaction for MockTransaction { self.into() } - fn from_pooled(pooled: Self::Pooled) -> Self { + fn from_pooled(pooled: RecoveredTx) -> Self { pooled.into() } fn try_consensus_into_pooled( tx: RecoveredTx, - ) -> Result { - Self::Pooled::try_from(tx).map_err(|_| TryFromRecoveredTransactionError::BlobSidecarMissing) + ) -> Result, Self::TryFromConsensusError> { + let (tx, signer) = tx.to_components(); + Self::Pooled::try_from(tx) + .map(|tx| tx.with_signer(signer)) + .map_err(|_| TryFromRecoveredTransactionError::BlobSidecarMissing) } fn hash(&self) -> &TxHash { @@ -786,12 +790,25 @@ impl EthPoolTransaction for MockTransaction { } } - fn try_into_pooled_eip4844(self, sidecar: Arc) -> Option { - Self::Pooled::try_from_blob_transaction( - self.into_consensus(), - Arc::unwrap_or_clone(sidecar), - ) - .ok() + fn try_into_pooled_eip4844( + self, + sidecar: Arc, + ) -> Option> { + let (tx, signer) = self.into_consensus().to_components(); + Self::Pooled::try_from_blob_transaction(tx, Arc::unwrap_or_clone(sidecar)) + .map(|tx| tx.with_signer(signer)) + .ok() + } + + fn try_from_eip4844( + tx: RecoveredTx, + sidecar: BlobTransactionSidecar, + ) -> Option { + let (tx, signer) = tx.to_components(); + Self::Pooled::try_from_blob_transaction(tx, sidecar) + .map(|tx| tx.with_signer(signer)) + .ok() + .map(Self::from_pooled) } fn validate_blob( diff --git a/crates/transaction-pool/src/traits.rs b/crates/transaction-pool/src/traits.rs index 15f824e7d436..aa238ded2f0d 100644 --- a/crates/transaction-pool/src/traits.rs +++ b/crates/transaction-pool/src/traits.rs @@ -19,8 +19,10 @@ use futures_util::{ready, Stream}; use reth_eth_wire_types::HandleMempoolData; use reth_execution_types::ChangedAccount; use reth_primitives::{ - kzg::KzgSettings, transaction::TryFromRecoveredTransactionError, PooledTransactionsElement, - PooledTransactionsElementEcRecovered, RecoveredTx, SealedBlock, Transaction, TransactionSigned, + kzg::KzgSettings, + transaction::{SignedTransactionIntoRecoveredExt, TryFromRecoveredTransactionError}, + PooledTransactionsElement, PooledTransactionsElementEcRecovered, RecoveredTx, SealedBlock, + Transaction, TransactionSigned, }; use reth_primitives_traits::SignedTransaction; #[cfg(feature = "serde")] @@ -236,15 +238,6 @@ pub trait TransactionPool: Send + Sync + Clone { limit: GetPooledTransactionLimit, ) -> Vec<::Pooled>; - /// Returns the pooled transaction variant for the given transaction hash as the requested type. - fn get_pooled_transactions_as( - &self, - tx_hashes: Vec, - limit: GetPooledTransactionLimit, - ) -> Vec - where - ::Pooled: Into; - /// Returns the pooled transaction variant for the given transaction hash. /// /// This adheres to the expected behavior of @@ -260,15 +253,7 @@ pub trait TransactionPool: Send + Sync + Clone { fn get_pooled_transaction_element( &self, tx_hash: TxHash, - ) -> Option<::Pooled>; - - /// Returns the pooled transaction variant for the given transaction hash as the requested type. - fn get_pooled_transaction_as(&self, tx_hash: TxHash) -> Option - where - ::Pooled: Into, - { - self.get_pooled_transaction_element(tx_hash).map(Into::into) - } + ) -> Option::Pooled>>; /// Returns an iterator that yields transactions that are ready for block production. /// @@ -973,6 +958,7 @@ pub trait PoolTransaction: + Clone + TryFrom, Error = Self::TryFromConsensusError> + Into> + + From> { /// Associated error type for the `try_from_consensus` method. type TryFromConsensusError: fmt::Display; @@ -981,7 +967,7 @@ pub trait PoolTransaction: type Consensus; /// Associated type representing the recovered pooled variant of the transaction. - type Pooled: Encodable2718 + Into; + type Pooled: SignedTransaction; /// Define a method to convert from the `Consensus` type to `Self` fn try_from_consensus( @@ -1003,19 +989,19 @@ pub trait PoolTransaction: } /// Define a method to convert from the `Pooled` type to `Self` - fn from_pooled(pooled: Self::Pooled) -> Self { + fn from_pooled(pooled: RecoveredTx) -> Self { pooled.into() } /// Tries to convert the `Consensus` type into the `Pooled` type. - fn try_into_pooled(self) -> Result { + fn try_into_pooled(self) -> Result, Self::TryFromConsensusError> { Self::try_consensus_into_pooled(self.into_consensus()) } /// Tries to convert the `Consensus` type into the `Pooled` type. fn try_consensus_into_pooled( tx: RecoveredTx, - ) -> Result; + ) -> Result, Self::TryFromConsensusError>; /// Hash of the transaction. fn hash(&self) -> &TxHash; @@ -1144,13 +1130,7 @@ pub trait PoolTransaction: /// /// This extends the [`PoolTransaction`] trait with additional methods that are specific to the /// Ethereum pool. -pub trait EthPoolTransaction: - PoolTransaction< - Pooled: From - + Into - + Into, -> -{ +pub trait EthPoolTransaction: PoolTransaction { /// Extracts the blob sidecar from the transaction. fn take_blob(&mut self) -> EthBlobTransactionSidecar; @@ -1162,7 +1142,18 @@ pub trait EthPoolTransaction: /// /// This returns an option, but callers should ensure that the transaction is an EIP-4844 /// transaction: [`PoolTransaction::is_eip4844`]. - fn try_into_pooled_eip4844(self, sidecar: Arc) -> Option; + fn try_into_pooled_eip4844( + self, + sidecar: Arc, + ) -> Option>; + + /// Tries to convert the `Consensus` type with a blob sidecar into the `Pooled` type. + /// + /// Returns `None` if passed transaction is not a blob transaction. + fn try_from_eip4844( + tx: RecoveredTx, + sidecar: BlobTransactionSidecar, + ) -> Option; /// Validates the blob sidecar of the transaction with the given settings. fn validate_blob( @@ -1258,7 +1249,7 @@ impl PoolTransaction for EthPooledTransaction { type Consensus = TransactionSigned; - type Pooled = PooledTransactionsElementEcRecovered; + type Pooled = PooledTransactionsElement; fn clone_into_consensus(&self) -> RecoveredTx { self.transaction().clone() @@ -1266,8 +1257,11 @@ impl PoolTransaction for EthPooledTransaction { fn try_consensus_into_pooled( tx: RecoveredTx, - ) -> Result { - Self::Pooled::try_from(tx).map_err(|_| TryFromRecoveredTransactionError::BlobSidecarMissing) + ) -> Result, Self::TryFromConsensusError> { + let (tx, signer) = tx.to_components(); + let pooled = PooledTransactionsElement::try_from_broadcast(tx) + .map_err(|_| TryFromRecoveredTransactionError::BlobSidecarMissing)?; + Ok(RecoveredTx::from_signed_transaction(pooled, signer)) } /// Returns hash of the transaction. @@ -1395,7 +1389,10 @@ impl EthPoolTransaction for EthPooledTransaction { } } - fn try_into_pooled_eip4844(self, sidecar: Arc) -> Option { + fn try_into_pooled_eip4844( + self, + sidecar: Arc, + ) -> Option> { PooledTransactionsElementEcRecovered::try_from_blob_transaction( self.into_consensus(), Arc::unwrap_or_clone(sidecar), @@ -1403,6 +1400,17 @@ impl EthPoolTransaction for EthPooledTransaction { .ok() } + fn try_from_eip4844( + tx: RecoveredTx, + sidecar: BlobTransactionSidecar, + ) -> Option { + let (tx, signer) = tx.to_components(); + PooledTransactionsElement::try_from_blob_transaction(tx, sidecar) + .ok() + .map(|tx| tx.with_signer(signer)) + .map(Self::from_pooled) + } + fn validate_blob( &self, sidecar: &BlobTransactionSidecar,