tx-pool: Make txpool independent of primitive tx types (#9916)

This commit is contained in:
Thomas Coratger
2024-08-01 15:39:58 +02:00
committed by GitHub
parent 1ba8eb8361
commit f25367cffd
13 changed files with 72 additions and 131 deletions

View File

@ -84,7 +84,6 @@ impl<Executor, Client, Pool, Engine> Future for MiningTask<Client, Pool, Executo
where
Client: StateProviderFactory + CanonChainTracker + Clone + Unpin + 'static,
Pool: TransactionPool + Unpin + 'static,
<Pool as TransactionPool>::Transaction: IntoRecoveredTransaction,
Engine: EngineTypes,
Executor: BlockExecutorProvider,
{

View File

@ -14,9 +14,7 @@ use reth_network_p2p::{
};
use reth_network_peers::PeerId;
use reth_network_types::ReputationChangeKind;
use reth_primitives::{
FromRecoveredPooledTransaction, PooledTransactionsElement, TransactionSigned, TxHash, B256,
};
use reth_primitives::{PooledTransactionsElement, TransactionSigned, TxHash, B256};
use reth_tokio_util::EventStream;
use reth_transaction_pool::{
error::{PoolError, PoolResult},
@ -1004,7 +1002,7 @@ where
Entry::Vacant(entry) => {
if !self.bad_imports.contains(tx.hash()) {
// this is a new transaction that should be imported into the pool
let pool_transaction = <Pool::Transaction as FromRecoveredPooledTransaction>::from_recovered_pooled_transaction(tx);
let pool_transaction: Pool::Transaction = tx.into();
new_txs.push(pool_transaction);
entry.insert(HashSet::from([peer_id]));
@ -1375,7 +1373,7 @@ impl PropagateTransaction {
/// Create a new instance from a pooled transaction
fn new<T: PoolTransaction>(tx: Arc<ValidPoolTransaction<T>>) -> Self {
let size = tx.encoded_length();
let transaction = Arc::new(tx.transaction.to_recovered_transaction().into_signed());
let transaction = Arc::new(tx.transaction.clone().into().into_signed());
Self { size, transaction }
}
}

View File

@ -113,7 +113,7 @@ where
let l1_block_info = self.block_info.l1_block_info.read().clone();
let mut encoded = Vec::new();
valid_tx.transaction().to_recovered_transaction().encode_enveloped(&mut encoded);
valid_tx.transaction().clone().into().encode_enveloped(&mut encoded);
let cost_addition = match l1_block_info.l1_tx_data_fee(
&self.chain_spec(),

View File

@ -59,8 +59,8 @@ pub use reth_primitives_traits::{
pub use static_file::StaticFileSegment;
pub use transaction::{
BlobTransaction, BlobTransactionSidecar, FromRecoveredPooledTransaction,
PooledTransactionsElement, PooledTransactionsElementEcRecovered,
BlobTransaction, BlobTransactionSidecar, PooledTransactionsElement,
PooledTransactionsElementEcRecovered,
};
#[cfg(feature = "c-kzg")]
@ -70,9 +70,9 @@ pub use transaction::{
util::secp256k1::{public_key_to_address, recover_signer_unchecked, sign_message},
AccessList, AccessListItem, IntoRecoveredTransaction, InvalidTransactionError, Signature,
Transaction, TransactionMeta, TransactionSigned, TransactionSignedEcRecovered,
TransactionSignedNoHash, TryFromRecoveredTransaction, TxEip1559, TxEip2930, TxEip4844,
TxEip7702, TxHashOrNumber, TxLegacy, TxType, EIP1559_TX_TYPE_ID, EIP2930_TX_TYPE_ID,
EIP4844_TX_TYPE_ID, EIP7702_TX_TYPE_ID, LEGACY_TX_TYPE_ID,
TransactionSignedNoHash, TxEip1559, TxEip2930, TxEip4844, TxEip7702, TxHashOrNumber, TxLegacy,
TxType, EIP1559_TX_TYPE_ID, EIP2930_TX_TYPE_ID, EIP4844_TX_TYPE_ID, EIP7702_TX_TYPE_ID,
LEGACY_TX_TYPE_ID,
};
// Re-exports

View File

@ -1580,48 +1580,7 @@ impl Decodable for TransactionSignedEcRecovered {
}
}
/// A transaction type that can be created from a [`TransactionSignedEcRecovered`] transaction.
///
/// This is a conversion trait that'll ensure transactions received via P2P can be converted to the
/// transaction type that the transaction pool uses.
pub trait TryFromRecoveredTransaction {
/// The error type returned by the transaction.
type Error;
/// Converts to this type from the given [`TransactionSignedEcRecovered`].
fn try_from_recovered_transaction(
tx: TransactionSignedEcRecovered,
) -> Result<Self, Self::Error>
where
Self: Sized;
}
// Noop conversion
impl TryFromRecoveredTransaction for TransactionSignedEcRecovered {
type Error = TryFromRecoveredTransactionError;
#[inline]
fn try_from_recovered_transaction(
tx: TransactionSignedEcRecovered,
) -> Result<Self, Self::Error> {
if tx.is_eip4844() {
Err(TryFromRecoveredTransactionError::BlobSidecarMissing)
} else {
Ok(tx)
}
}
}
/// A transaction type that can be created from a [`PooledTransactionsElementEcRecovered`]
/// transaction.
///
/// This is a conversion trait that'll ensure transactions received via P2P can be converted to the
/// transaction type that the transaction pool uses.
pub trait FromRecoveredPooledTransaction {
/// Converts to this type from the given [`PooledTransactionsElementEcRecovered`].
fn from_recovered_pooled_transaction(tx: PooledTransactionsElementEcRecovered) -> Self;
}
/// The inverse of [`TryFromRecoveredTransaction`] that ensure the transaction can be sent over the
/// Ensures the transaction can be sent over the
/// network
pub trait IntoRecoveredTransaction {
/// Converts to this type into a [`TransactionSignedEcRecovered`].

View File

@ -6,8 +6,8 @@ use std::{fmt, ops::Deref, sync::Arc};
use alloy_dyn_abi::TypedData;
use futures::Future;
use reth_primitives::{
Address, BlockId, Bytes, FromRecoveredPooledTransaction, IntoRecoveredTransaction, Receipt,
SealedBlockWithSenders, TransactionMeta, TransactionSigned, TxHash, TxKind, B256, U256,
Address, BlockId, Bytes, Receipt, SealedBlockWithSenders, TransactionMeta, TransactionSigned,
TxHash, TxKind, B256, U256,
};
use reth_provider::{BlockReaderIdExt, ReceiptProvider, TransactionsProvider};
use reth_rpc_eth_types::{
@ -250,10 +250,7 @@ pub trait EthTransactions: LoadTransaction {
) -> impl Future<Output = Result<B256, Self::Error>> + Send {
async move {
let recovered = recover_raw_transaction(tx.clone())?;
let pool_transaction =
<Self::Pool as TransactionPool>::Transaction::from_recovered_pooled_transaction(
recovered,
);
let pool_transaction: <Self::Pool as TransactionPool>::Transaction = recovered.into();
// On optimism, transactions are forwarded directly to the sequencer to be included in
// blocks that it builds.
@ -474,7 +471,7 @@ pub trait EthTransactions: LoadTransaction {
signed_tx.into_ecrecovered().ok_or(EthApiError::InvalidTransactionSignature)?;
let pool_transaction = match recovered.try_into() {
Ok(converted) => <<Self as LoadTransaction>::Pool as TransactionPool>::Transaction::from_recovered_pooled_transaction(converted),
Ok(converted) => converted,
Err(_) => return Err(EthApiError::TransactionConversionError.into()),
};
@ -610,9 +607,7 @@ pub trait LoadTransaction: SpawnBlocking {
if resp.is_none() {
// tx not found on disk, check pool
if let Some(tx) =
self.pool().get(&hash).map(|tx| tx.transaction.to_recovered_transaction())
{
if let Some(tx) = self.pool().get(&hash).map(|tx| tx.transaction.clone().into()) {
resp = Some(TransactionSource::Pool(tx));
}
}

View File

@ -1,6 +1,6 @@
use async_trait::async_trait;
use jsonrpsee::core::RpcResult as Result;
use reth_primitives::Address;
use reth_primitives::{Address, TransactionSignedEcRecovered};
use reth_rpc_api::TxPoolApiServer;
use reth_rpc_types::{
txpool::{TxpoolContent, TxpoolContentFrom, TxpoolInspect, TxpoolInspectSummary, TxpoolStatus},
@ -38,7 +38,7 @@ where
) {
content.entry(tx.sender()).or_default().insert(
tx.nonce().to_string(),
reth_rpc_types_compat::transaction::from_recovered(tx.to_recovered_transaction()),
reth_rpc_types_compat::transaction::from_recovered(tx.clone().into()),
);
}
@ -87,7 +87,7 @@ where
inspect: &mut BTreeMap<Address, BTreeMap<String, TxpoolInspectSummary>>,
) {
let entry = inspect.entry(tx.sender()).or_default();
let tx = tx.to_recovered_transaction();
let tx: TransactionSignedEcRecovered = tx.clone().into();
entry.insert(
tx.nonce().to_string(),
TxpoolInspectSummary {

View File

@ -14,9 +14,8 @@ use futures_util::{
use reth_execution_types::ExecutionOutcome;
use reth_fs_util::FsPathError;
use reth_primitives::{
Address, BlockHash, BlockNumber, BlockNumberOrTag, FromRecoveredPooledTransaction,
IntoRecoveredTransaction, PooledTransactionsElementEcRecovered, TransactionSigned,
TryFromRecoveredTransaction,
Address, BlockHash, BlockNumber, BlockNumberOrTag, IntoRecoveredTransaction,
PooledTransactionsElementEcRecovered, TransactionSigned,
};
use reth_provider::{
BlockReaderIdExt, CanonStateNotification, ChainSpecProvider, ProviderError,
@ -332,13 +331,9 @@ pub async fn maintain_transaction_pool<Client, P, St, Tasks>(
)
.ok()
})
.map(
<P as TransactionPool>::Transaction::from_recovered_pooled_transaction,
)
.map(Into::into)
} else {
<P as TransactionPool>::Transaction::try_from_recovered_transaction(
tx,
).ok()
tx.try_into().ok()
}
})
.collect::<Vec<_>>();
@ -592,7 +587,7 @@ where
.filter_map(|tx| tx.try_ecrecovered())
.filter_map(|tx| {
// Filter out errors
<P as TransactionPool>::Transaction::try_from_recovered_transaction(tx).ok()
tx.try_into().ok()
})
.collect::<Vec<_>>();
@ -705,9 +700,7 @@ mod tests {
let tx_bytes = hex!("02f87201830655c2808505ef61f08482565f94388c818ca8b9251b393131c08a736a67ccb192978801049e39c4b5b1f580c001a01764ace353514e8abdfb92446de356b260e3c1225b73fc4c8876a6258d12a129a04f02294aa61ca7676061cd99f29275491218b4754b46a0248e5e42bc5091f507");
let tx = PooledTransactionsElement::decode_enveloped(&mut &tx_bytes[..]).unwrap();
let provider = MockEthProvider::default();
let transaction = EthPooledTransaction::from_recovered_pooled_transaction(
tx.try_into_ecrecovered().unwrap(),
);
let transaction: EthPooledTransaction = tx.try_into_ecrecovered().unwrap().into();
let tx_to_cmp = transaction.clone();
let sender = hex!("1f9090aaE28b8a3dCeaDf281B0F12828e676c326").into();
provider.add_account(sender, ExtendedAccount::new(42, U256::MAX));

View File

@ -3,8 +3,7 @@ use rand::Rng;
use reth_chainspec::MAINNET;
use reth_primitives::{
constants::MIN_PROTOCOL_BASE_FEE, sign_message, AccessList, Address, Bytes, Transaction,
TransactionSigned, TryFromRecoveredTransaction, TxEip1559, TxEip4844, TxKind, TxLegacy, B256,
U256,
TransactionSigned, TxEip1559, TxEip4844, TxKind, TxLegacy, B256, U256,
};
/// A generator for transactions for testing purposes.
@ -99,10 +98,7 @@ impl<R: Rng> TransactionGenerator<R> {
/// Generates and returns a pooled EIP-1559 transaction with a random signer.
pub fn gen_eip1559_pooled(&mut self) -> EthPooledTransaction {
EthPooledTransaction::try_from_recovered_transaction(
self.gen_eip1559().into_ecrecovered().unwrap(),
)
.unwrap()
self.gen_eip1559().into_ecrecovered().unwrap().try_into().unwrap()
}
/// Generates and returns a pooled EIP-4844 transaction with a random signer.

View File

@ -16,10 +16,10 @@ use reth_primitives::{
constants::{eip4844::DATA_GAS_PER_BLOB, MIN_PROTOCOL_BASE_FEE},
transaction::TryFromRecoveredTransactionError,
AccessList, Address, BlobTransactionSidecar, BlobTransactionValidationError, Bytes, ChainId,
FromRecoveredPooledTransaction, IntoRecoveredTransaction, PooledTransactionsElementEcRecovered,
Signature, Transaction, TransactionSigned, TransactionSignedEcRecovered,
TryFromRecoveredTransaction, TxEip1559, TxEip2930, TxEip4844, TxHash, TxKind, TxLegacy, TxType,
B256, EIP1559_TX_TYPE_ID, EIP2930_TX_TYPE_ID, EIP4844_TX_TYPE_ID, LEGACY_TX_TYPE_ID, U256,
PooledTransactionsElementEcRecovered, Signature, Transaction, TransactionSigned,
TransactionSignedEcRecovered, TxEip1559, TxEip2930, TxEip4844, TxHash, TxKind, TxLegacy,
TxType, B256, EIP1559_TX_TYPE_ID, EIP2930_TX_TYPE_ID, EIP4844_TX_TYPE_ID, LEGACY_TX_TYPE_ID,
U256,
};
use std::{ops::Range, sync::Arc, time::Instant, vec::IntoIter};
@ -558,6 +558,10 @@ impl MockTransaction {
}
impl PoolTransaction for MockTransaction {
type Consensus = TransactionSignedEcRecovered;
type Pooled = PooledTransactionsElementEcRecovered;
fn hash(&self) -> &TxHash {
match self {
Self::Legacy { hash, .. } |
@ -755,12 +759,10 @@ impl EthPoolTransaction for MockTransaction {
}
}
impl TryFromRecoveredTransaction for MockTransaction {
impl TryFrom<TransactionSignedEcRecovered> for MockTransaction {
type Error = TryFromRecoveredTransactionError;
fn try_from_recovered_transaction(
tx: TransactionSignedEcRecovered,
) -> Result<Self, Self::Error> {
fn try_from(tx: TransactionSignedEcRecovered) -> Result<Self, Self::Error> {
let sender = tx.signer();
let transaction = tx.into_signed();
let hash = transaction.hash();
@ -869,26 +871,23 @@ impl TryFromRecoveredTransaction for MockTransaction {
}
}
impl FromRecoveredPooledTransaction for MockTransaction {
fn from_recovered_pooled_transaction(tx: PooledTransactionsElementEcRecovered) -> Self {
TryFromRecoveredTransaction::try_from_recovered_transaction(
tx.into_ecrecovered_transaction(),
impl From<PooledTransactionsElementEcRecovered> for MockTransaction {
fn from(tx: PooledTransactionsElementEcRecovered) -> Self {
tx.into_ecrecovered_transaction().try_into().expect(
"Failed to convert from PooledTransactionsElementEcRecovered to MockTransaction",
)
.expect("Failed to convert from PooledTransactionsElementEcRecovered to MockTransaction")
}
}
impl IntoRecoveredTransaction for MockTransaction {
fn to_recovered_transaction(&self) -> TransactionSignedEcRecovered {
let tx = self.clone().into();
impl From<MockTransaction> for TransactionSignedEcRecovered {
fn from(tx: MockTransaction) -> Self {
let signed_tx = TransactionSigned {
hash: *self.hash(),
hash: *tx.hash(),
signature: Signature::default(),
transaction: tx,
transaction: tx.clone().into(),
};
TransactionSignedEcRecovered::from_signed_transaction(signed_tx, self.sender())
Self::from_signed_transaction(signed_tx, tx.sender())
}
}

View File

@ -11,10 +11,9 @@ use futures_util::{ready, Stream};
use reth_eth_wire_types::HandleMempoolData;
use reth_primitives::{
kzg::KzgSettings, transaction::TryFromRecoveredTransactionError, AccessList, Address,
BlobTransactionSidecar, BlobTransactionValidationError, FromRecoveredPooledTransaction,
IntoRecoveredTransaction, PooledTransactionsElement, PooledTransactionsElementEcRecovered,
SealedBlock, Transaction, TransactionSignedEcRecovered, TryFromRecoveredTransaction, TxHash,
TxKind, B256, EIP1559_TX_TYPE_ID, EIP4844_TX_TYPE_ID, EIP7702_TX_TYPE_ID, U256,
BlobTransactionSidecar, BlobTransactionValidationError, PooledTransactionsElement,
PooledTransactionsElementEcRecovered, SealedBlock, Transaction, TransactionSignedEcRecovered,
TxHash, TxKind, B256, EIP1559_TX_TYPE_ID, EIP4844_TX_TYPE_ID, EIP7702_TX_TYPE_ID, U256,
};
#[cfg(feature = "serde")]
use serde::{Deserialize, Serialize};
@ -460,12 +459,12 @@ pub struct AllPoolTransactions<T: PoolTransaction> {
impl<T: PoolTransaction> AllPoolTransactions<T> {
/// Returns an iterator over all pending [`TransactionSignedEcRecovered`] transactions.
pub fn pending_recovered(&self) -> impl Iterator<Item = TransactionSignedEcRecovered> + '_ {
self.pending.iter().map(|tx| tx.transaction.to_recovered_transaction())
self.pending.iter().map(|tx| tx.transaction.clone().into())
}
/// Returns an iterator over all queued [`TransactionSignedEcRecovered`] transactions.
pub fn queued_recovered(&self) -> impl Iterator<Item = TransactionSignedEcRecovered> + '_ {
self.queued.iter().map(|tx| tx.transaction.to_recovered_transaction())
self.queued.iter().map(|tx| tx.transaction.clone().into())
}
}
@ -758,10 +757,17 @@ pub trait PoolTransaction:
fmt::Debug
+ Send
+ Sync
+ FromRecoveredPooledTransaction
+ TryFromRecoveredTransaction
+ IntoRecoveredTransaction
+ Clone
+ From<PooledTransactionsElementEcRecovered>
+ TryFrom<TransactionSignedEcRecovered>
+ Into<TransactionSignedEcRecovered>
{
/// Associated type representing the raw consensus variant of the transaction.
type Consensus: From<Self> + TryInto<Self>;
/// Associated type representing the recovered pooled variant of the transaction.
type Pooled: Into<Self>;
/// Hash of the transaction.
fn hash(&self) -> &TxHash;
@ -993,6 +999,10 @@ impl From<PooledTransactionsElementEcRecovered> for EthPooledTransaction {
}
impl PoolTransaction for EthPooledTransaction {
type Consensus = TransactionSignedEcRecovered;
type Pooled = PooledTransactionsElementEcRecovered;
/// Returns hash of the transaction.
fn hash(&self) -> &TxHash {
self.transaction.hash_ref()
@ -1142,12 +1152,10 @@ impl EthPoolTransaction for EthPooledTransaction {
}
}
impl TryFromRecoveredTransaction for EthPooledTransaction {
impl TryFrom<TransactionSignedEcRecovered> for EthPooledTransaction {
type Error = TryFromRecoveredTransactionError;
fn try_from_recovered_transaction(
tx: TransactionSignedEcRecovered,
) -> Result<Self, Self::Error> {
fn try_from(tx: TransactionSignedEcRecovered) -> Result<Self, Self::Error> {
// ensure we can handle the transaction type and its format
match tx.tx_type() as u8 {
0..=EIP1559_TX_TYPE_ID | EIP7702_TX_TYPE_ID => {
@ -1171,15 +1179,9 @@ impl TryFromRecoveredTransaction for EthPooledTransaction {
}
}
impl FromRecoveredPooledTransaction for EthPooledTransaction {
fn from_recovered_pooled_transaction(tx: PooledTransactionsElementEcRecovered) -> Self {
Self::from(tx)
}
}
impl IntoRecoveredTransaction for EthPooledTransaction {
fn to_recovered_transaction(&self) -> TransactionSignedEcRecovered {
self.transaction.clone()
impl From<EthPooledTransaction> for TransactionSignedEcRecovered {
fn from(tx: EthPooledTransaction) -> Self {
tx.transaction
}
}

View File

@ -798,7 +798,7 @@ mod tests {
EthPooledTransaction, Pool, TransactionPool,
};
use reth_chainspec::MAINNET;
use reth_primitives::{hex, FromRecoveredPooledTransaction, PooledTransactionsElement, U256};
use reth_primitives::{hex, PooledTransactionsElement, U256};
use reth_provider::test_utils::{ExtendedAccount, MockEthProvider};
fn get_transaction() -> EthPooledTransaction {
@ -807,7 +807,7 @@ mod tests {
let data = hex::decode(raw).unwrap();
let tx = PooledTransactionsElement::decode_enveloped(&mut data.as_ref()).unwrap();
EthPooledTransaction::from_recovered_pooled_transaction(tx.try_into_ecrecovered().unwrap())
tx.try_into_ecrecovered().unwrap().into()
}
// <https://github.com/paradigmxyz/reth/issues/5178>

View File

@ -341,7 +341,7 @@ impl<T: PoolTransaction> ValidPoolTransaction<T> {
impl<T: PoolTransaction> IntoRecoveredTransaction for ValidPoolTransaction<T> {
fn to_recovered_transaction(&self) -> TransactionSignedEcRecovered {
self.transaction.to_recovered_transaction()
self.transaction.clone().into()
}
}