feat: use primitive transaction as PoolTransaction::Consensus (#13086)

This commit is contained in:
Arsenii Kulikov
2024-12-03 12:46:37 +04:00
committed by GitHub
parent ae8912fa73
commit 5724114947
38 changed files with 283 additions and 202 deletions

View File

@ -10,6 +10,7 @@ use crate::{
use alloy_consensus::BlockHeader;
use alloy_eips::BlockNumberOrTag;
use alloy_primitives::{Address, BlockHash, BlockNumber};
use alloy_rlp::Encodable;
use futures_util::{
future::{BoxFuture, Fuse, FusedFuture},
FutureExt, Stream, StreamExt,
@ -19,7 +20,8 @@ use reth_chainspec::{ChainSpecProvider, EthChainSpec};
use reth_execution_types::ChangedAccount;
use reth_fs_util::FsPathError;
use reth_primitives::{
PooledTransactionsElementEcRecovered, RecoveredTx, SealedHeader, TransactionSigned,
transaction::SignedTransactionIntoRecoveredExt, PooledTransactionsElementEcRecovered,
SealedHeader, TransactionSigned,
};
use reth_primitives_traits::SignedTransaction;
use reth_storage_api::{errors::provider::ProviderError, BlockReaderIdExt, StateProviderFactory};
@ -78,7 +80,7 @@ pub fn maintain_transaction_pool_future<Client, P, St, Tasks>(
) -> BoxFuture<'static, ()>
where
Client: StateProviderFactory + BlockReaderIdExt + ChainSpecProvider + Clone + Send + 'static,
P: TransactionPoolExt + 'static,
P: TransactionPoolExt<Transaction: PoolTransaction<Consensus = TransactionSigned>> + 'static,
St: Stream<Item = CanonStateNotification> + Send + Unpin + 'static,
Tasks: TaskSpawner + 'static,
{
@ -99,7 +101,7 @@ pub async fn maintain_transaction_pool<Client, P, St, Tasks>(
config: MaintainPoolConfig,
) where
Client: StateProviderFactory + BlockReaderIdExt + ChainSpecProvider + Clone + Send + 'static,
P: TransactionPoolExt + 'static,
P: TransactionPoolExt<Transaction: PoolTransaction<Consensus = TransactionSigned>> + 'static,
St: Stream<Item = CanonStateNotification> + Send + Unpin + 'static,
Tasks: TaskSpawner + 'static,
{
@ -342,7 +344,7 @@ pub async fn maintain_transaction_pool<Client, P, St, Tasks>(
<P as TransactionPool>::Transaction::from_pooled(tx.into())
})
} else {
<P as TransactionPool>::Transaction::try_from_consensus(tx.into()).ok()
<P as TransactionPool>::Transaction::try_from_consensus(tx).ok()
}
})
.collect::<Vec<_>>();
@ -559,7 +561,7 @@ async fn load_and_reinsert_transactions<P>(
file_path: &Path,
) -> Result<(), TransactionsBackupError>
where
P: TransactionPool,
P: TransactionPool<Transaction: PoolTransaction<Consensus: SignedTransaction>>,
{
if !file_path.exists() {
return Ok(())
@ -572,14 +574,15 @@ where
return Ok(())
}
let txs_signed: Vec<TransactionSigned> = alloy_rlp::Decodable::decode(&mut data.as_slice())?;
let txs_signed: Vec<<P::Transaction as PoolTransaction>::Consensus> =
alloy_rlp::Decodable::decode(&mut data.as_slice())?;
let pool_transactions = txs_signed
.into_iter()
.filter_map(|tx| tx.try_ecrecovered())
.filter_map(|tx| {
// Filter out errors
<P::Transaction as PoolTransaction>::try_from_consensus(tx.into()).ok()
<P::Transaction as PoolTransaction>::try_from_consensus(tx).ok()
})
.collect();
@ -592,7 +595,7 @@ where
fn save_local_txs_backup<P>(pool: P, file_path: &Path)
where
P: TransactionPool,
P: TransactionPool<Transaction: PoolTransaction<Consensus: Encodable>>,
{
let local_transactions = pool.get_local_transactions();
if local_transactions.is_empty() {
@ -602,10 +605,7 @@ where
let local_transactions = local_transactions
.into_iter()
.map(|tx| {
let recovered: RecoveredTx = tx.transaction.clone_into_consensus().into();
recovered.into_signed()
})
.map(|tx| tx.transaction.clone_into_consensus().into_signed())
.collect::<Vec<_>>();
let num_txs = local_transactions.len();
@ -645,7 +645,7 @@ pub async fn backup_local_transactions_task<P>(
pool: P,
config: LocalTransactionBackupConfig,
) where
P: TransactionPool + Clone,
P: TransactionPool<Transaction: PoolTransaction<Consensus: SignedTransaction>> + Clone,
{
let Some(transactions_path) = config.transactions_path else {
// nothing to do

View File

@ -226,7 +226,7 @@ impl<T: TransactionOrdering> Iterator for BestTransactions<T> {
#[derive(Debug)]
pub struct BestPayloadTransactions<T, I>
where
T: PoolTransaction<Consensus: Into<RecoveredTx>>,
T: PoolTransaction,
I: Iterator<Item = Arc<ValidPoolTransaction<T>>>,
{
invalid: HashSet<Address>,
@ -235,7 +235,7 @@ where
impl<T, I> BestPayloadTransactions<T, I>
where
T: PoolTransaction<Consensus: Into<RecoveredTx>>,
T: PoolTransaction,
I: Iterator<Item = Arc<ValidPoolTransaction<T>>>,
{
/// Create a new `BestPayloadTransactions` with the given iterator.
@ -246,16 +246,18 @@ where
impl<T, I> PayloadTransactions for BestPayloadTransactions<T, I>
where
T: PoolTransaction<Consensus: Into<RecoveredTx>>,
T: PoolTransaction,
I: Iterator<Item = Arc<ValidPoolTransaction<T>>>,
{
fn next(&mut self, _ctx: ()) -> Option<RecoveredTx> {
type Transaction = T::Consensus;
fn next(&mut self, _ctx: ()) -> Option<RecoveredTx<Self::Transaction>> {
loop {
let tx = self.best.next()?;
if self.invalid.contains(&tx.sender()) {
continue
}
return Some(tx.to_recovered_transaction())
return Some(tx.to_consensus())
}
}

View File

@ -592,15 +592,17 @@ impl MockTransaction {
impl PoolTransaction for MockTransaction {
type TryFromConsensusError = TryFromRecoveredTransactionError;
type Consensus = RecoveredTx;
type Consensus = TransactionSigned;
type Pooled = PooledTransactionsElementEcRecovered;
fn try_from_consensus(tx: Self::Consensus) -> Result<Self, Self::TryFromConsensusError> {
fn try_from_consensus(
tx: RecoveredTx<Self::Consensus>,
) -> Result<Self, Self::TryFromConsensusError> {
tx.try_into()
}
fn into_consensus(self) -> Self::Consensus {
fn into_consensus(self) -> RecoveredTx<Self::Consensus> {
self.into()
}
@ -609,7 +611,7 @@ impl PoolTransaction for MockTransaction {
}
fn try_consensus_into_pooled(
tx: Self::Consensus,
tx: RecoveredTx<Self::Consensus>,
) -> Result<Self::Pooled, Self::TryFromConsensusError> {
Self::Pooled::try_from(tx).map_err(|_| TryFromRecoveredTransactionError::BlobSidecarMissing)
}

View File

@ -38,6 +38,9 @@ use tokio::sync::mpsc::Receiver;
/// The `PeerId` type.
pub type PeerId = alloy_primitives::B512;
/// Helper type alias to access [`PoolTransaction::Consensus`] for a given [`TransactionPool`].
pub type PoolConsensusTx<P> = <<P as TransactionPool>::Transaction as PoolTransaction>::Consensus;
/// General purpose abstraction of a transaction-pool.
///
/// This is intended to be used by API-consumers such as RPC that need inject new incoming,
@ -577,17 +580,17 @@ pub struct AllPoolTransactions<T: PoolTransaction> {
impl<T: PoolTransaction> AllPoolTransactions<T> {
/// Returns an iterator over all pending [`RecoveredTx`] transactions.
pub fn pending_recovered(&self) -> impl Iterator<Item = T::Consensus> + '_ {
pub fn pending_recovered(&self) -> impl Iterator<Item = RecoveredTx<T::Consensus>> + '_ {
self.pending.iter().map(|tx| tx.transaction.clone().into())
}
/// Returns an iterator over all queued [`RecoveredTx`] transactions.
pub fn queued_recovered(&self) -> impl Iterator<Item = T::Consensus> + '_ {
pub fn queued_recovered(&self) -> impl Iterator<Item = RecoveredTx<T::Consensus>> + '_ {
self.queued.iter().map(|tx| tx.transaction.clone().into())
}
/// Returns an iterator over all transactions, both pending and queued.
pub fn all(&self) -> impl Iterator<Item = T::Consensus> + '_ {
pub fn all(&self) -> impl Iterator<Item = RecoveredTx<T::Consensus>> + '_ {
self.pending.iter().chain(self.queued.iter()).map(|tx| tx.transaction.clone().into())
}
}
@ -963,30 +966,39 @@ impl BestTransactionsAttributes {
/// This distinction is necessary for the EIP-4844 blob transactions, which require an additional
/// sidecar when they are gossiped around the network. It is expected that the `Consensus` format is
/// a subset of the `Pooled` format.
pub trait PoolTransaction: fmt::Debug + Send + Sync + Clone {
pub trait PoolTransaction:
fmt::Debug
+ Send
+ Sync
+ Clone
+ TryFrom<RecoveredTx<Self::Consensus>, Error = Self::TryFromConsensusError>
+ Into<RecoveredTx<Self::Consensus>>
{
/// Associated error type for the `try_from_consensus` method.
type TryFromConsensusError: fmt::Display;
/// Associated type representing the raw consensus variant of the transaction.
type Consensus: From<Self> + TryInto<Self, Error = Self::TryFromConsensusError>;
type Consensus;
/// Associated type representing the recovered pooled variant of the transaction.
type Pooled: Encodable2718 + Into<Self>;
/// Define a method to convert from the `Consensus` type to `Self`
fn try_from_consensus(tx: Self::Consensus) -> Result<Self, Self::TryFromConsensusError> {
fn try_from_consensus(
tx: RecoveredTx<Self::Consensus>,
) -> Result<Self, Self::TryFromConsensusError> {
tx.try_into()
}
/// Clone the transaction into a consensus variant.
///
/// This method is preferred when the [`PoolTransaction`] already wraps the consensus variant.
fn clone_into_consensus(&self) -> Self::Consensus {
fn clone_into_consensus(&self) -> RecoveredTx<Self::Consensus> {
self.clone().into_consensus()
}
/// Define a method to convert from the `Self` type to `Consensus`
fn into_consensus(self) -> Self::Consensus {
fn into_consensus(self) -> RecoveredTx<Self::Consensus> {
self.into()
}
@ -1002,7 +1014,7 @@ pub trait PoolTransaction: fmt::Debug + Send + Sync + Clone {
/// Tries to convert the `Consensus` type into the `Pooled` type.
fn try_consensus_into_pooled(
tx: Self::Consensus,
tx: RecoveredTx<Self::Consensus>,
) -> Result<Self::Pooled, Self::TryFromConsensusError>;
/// Hash of the transaction.
@ -1131,7 +1143,6 @@ pub trait PoolTransaction: fmt::Debug + Send + Sync + Clone {
/// Ethereum pool.
pub trait EthPoolTransaction:
PoolTransaction<
Consensus: From<RecoveredTx> + Into<RecoveredTx> + Into<TransactionSigned>,
Pooled: From<PooledTransactionsElementEcRecovered>
+ Into<PooledTransactionsElementEcRecovered>
+ Into<PooledTransactionsElement>,
@ -1242,16 +1253,16 @@ impl From<PooledTransactionsElementEcRecovered> for EthPooledTransaction {
impl PoolTransaction for EthPooledTransaction {
type TryFromConsensusError = TryFromRecoveredTransactionError;
type Consensus = RecoveredTx;
type Consensus = TransactionSigned;
type Pooled = PooledTransactionsElementEcRecovered;
fn clone_into_consensus(&self) -> Self::Consensus {
fn clone_into_consensus(&self) -> RecoveredTx<Self::Consensus> {
self.transaction().clone()
}
fn try_consensus_into_pooled(
tx: Self::Consensus,
tx: RecoveredTx<Self::Consensus>,
) -> Result<Self::Pooled, Self::TryFromConsensusError> {
Self::Pooled::try_from(tx).map_err(|_| TryFromRecoveredTransactionError::BlobSidecarMissing)
}

View File

@ -378,7 +378,7 @@ impl<T: PoolTransaction> ValidPoolTransaction<T> {
/// Converts to this type into the consensus transaction of the pooled transaction.
///
/// Note: this takes `&self` since indented usage is via `Arc<Self>`.
pub fn to_consensus(&self) -> T::Consensus {
pub fn to_consensus(&self) -> RecoveredTx<T::Consensus> {
self.transaction.clone_into_consensus()
}
@ -435,15 +435,6 @@ impl<T: PoolTransaction> ValidPoolTransaction<T> {
}
}
impl<T: PoolTransaction<Consensus: Into<RecoveredTx>>> ValidPoolTransaction<T> {
/// Converts to this type into a [`RecoveredTx`].
///
/// Note: this takes `&self` since indented usage is via `Arc<Self>`.
pub fn to_recovered_transaction(&self) -> RecoveredTx {
self.to_consensus().into()
}
}
#[cfg(test)]
impl<T: PoolTransaction> Clone for ValidPoolTransaction<T> {
fn clone(&self) -> Self {