mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
Relax Trait Bounds on TransactionPool::Transaction and EthPoolTransaction (#11079)
Co-authored-by: Matthias Seitz <matthias.seitz@outlook.de>
This commit is contained in:
@ -1033,7 +1033,7 @@ where
|
||||
has_bad_transactions = true;
|
||||
} else {
|
||||
// this is a new transaction that should be imported into the pool
|
||||
let pool_transaction = Pool::Transaction::from_pooled(tx);
|
||||
let pool_transaction = Pool::Transaction::from_pooled(tx.into());
|
||||
new_txs.push(pool_transaction);
|
||||
|
||||
entry.insert(HashSet::from([peer_id]));
|
||||
@ -1396,11 +1396,14 @@ impl PropagateTransaction {
|
||||
}
|
||||
|
||||
/// Create a new instance from a pooled transaction
|
||||
fn new<T: PoolTransaction<Consensus = TransactionSignedEcRecovered>>(
|
||||
tx: Arc<ValidPoolTransaction<T>>,
|
||||
) -> Self {
|
||||
fn new<T>(tx: Arc<ValidPoolTransaction<T>>) -> Self
|
||||
where
|
||||
T: PoolTransaction<Consensus: Into<TransactionSignedEcRecovered>>,
|
||||
{
|
||||
let size = tx.encoded_length();
|
||||
let transaction = Arc::new(tx.transaction.clone().into_consensus().into_signed());
|
||||
let recovered: TransactionSignedEcRecovered =
|
||||
tx.transaction.clone().into_consensus().into();
|
||||
let transaction = Arc::new(recovered.into_signed());
|
||||
Self { size, transaction }
|
||||
}
|
||||
}
|
||||
|
||||
@ -140,7 +140,7 @@ where
|
||||
let l1_block_info = self.block_info.l1_block_info.read().clone();
|
||||
|
||||
let mut encoded = Vec::with_capacity(valid_tx.transaction().encoded_length());
|
||||
valid_tx.transaction().clone().into_consensus().encode_2718(&mut encoded);
|
||||
valid_tx.transaction().clone().into_consensus().into().encode_2718(&mut encoded);
|
||||
|
||||
let cost_addition = match l1_block_info.l1_tx_data_fee(
|
||||
&self.chain_spec(),
|
||||
|
||||
@ -34,7 +34,8 @@ where
|
||||
/// Returns the hash of the transaction.
|
||||
async fn send_raw_transaction(&self, tx: Bytes) -> Result<B256, Self::Error> {
|
||||
let recovered = recover_raw_transaction(tx.clone())?;
|
||||
let pool_transaction = <Self::Pool as TransactionPool>::Transaction::from_pooled(recovered);
|
||||
let pool_transaction =
|
||||
<Self::Pool as TransactionPool>::Transaction::from_pooled(recovered.into());
|
||||
|
||||
// On optimism, transactions are forwarded directly to the sequencer to be included in
|
||||
// blocks that it builds.
|
||||
|
||||
@ -231,7 +231,7 @@ pub trait EthTransactions: LoadTransaction {
|
||||
LoadState::pool(self).get_transaction_by_sender_and_nonce(sender, nonce)
|
||||
{
|
||||
let transaction = tx.transaction.clone().into_consensus();
|
||||
return Ok(Some(from_recovered::<Self::TransactionCompat>(transaction)));
|
||||
return Ok(Some(from_recovered::<Self::TransactionCompat>(transaction.into())));
|
||||
}
|
||||
}
|
||||
|
||||
@ -324,7 +324,7 @@ pub trait EthTransactions: LoadTransaction {
|
||||
async move {
|
||||
let recovered = recover_raw_transaction(tx.clone())?;
|
||||
let pool_transaction =
|
||||
<Self::Pool as TransactionPool>::Transaction::from_pooled(recovered);
|
||||
<Self::Pool as TransactionPool>::Transaction::from_pooled(recovered.into());
|
||||
|
||||
// submit the transaction to the pool with a `Local` origin
|
||||
let hash = self
|
||||
@ -376,7 +376,7 @@ pub trait EthTransactions: LoadTransaction {
|
||||
let recovered =
|
||||
signed_tx.into_ecrecovered().ok_or(EthApiError::InvalidTransactionSignature)?;
|
||||
|
||||
let pool_transaction = <<Self as LoadTransaction>::Pool as TransactionPool>::Transaction::try_from_consensus(recovered).map_err(|_| EthApiError::TransactionConversionError)?;
|
||||
let pool_transaction = <<Self as LoadTransaction>::Pool as TransactionPool>::Transaction::try_from_consensus(recovered.into()).map_err(|_| EthApiError::TransactionConversionError)?;
|
||||
|
||||
// submit the transaction to the pool with a `Local` origin
|
||||
let hash = LoadTransaction::pool(self)
|
||||
@ -518,7 +518,7 @@ pub trait LoadTransaction: SpawnBlocking + FullEthApiTypes {
|
||||
if let Some(tx) =
|
||||
self.pool().get(&hash).map(|tx| tx.transaction.clone().into_consensus())
|
||||
{
|
||||
resp = Some(TransactionSource::Pool(tx));
|
||||
resp = Some(TransactionSource::Pool(tx.into()));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -611,7 +611,7 @@ where
|
||||
/// Returns all new pending transactions received since the last poll.
|
||||
async fn drain(&self) -> FilterChanges<TxCompat::Transaction>
|
||||
where
|
||||
T: PoolTransaction<Consensus = TransactionSignedEcRecovered>,
|
||||
T: PoolTransaction<Consensus: Into<TransactionSignedEcRecovered>>,
|
||||
{
|
||||
let mut pending_txs = Vec::new();
|
||||
let mut prepared_stream = self.txs_stream.lock().await;
|
||||
@ -633,7 +633,7 @@ trait FullTransactionsFilter<T>: fmt::Debug + Send + Sync + Unpin + 'static {
|
||||
impl<T, TxCompat> FullTransactionsFilter<TxCompat::Transaction>
|
||||
for FullTransactionsReceiver<T, TxCompat>
|
||||
where
|
||||
T: PoolTransaction<Consensus = TransactionSignedEcRecovered> + 'static,
|
||||
T: PoolTransaction<Consensus: Into<TransactionSignedEcRecovered>> + 'static,
|
||||
TxCompat: TransactionCompat + 'static,
|
||||
{
|
||||
async fn drain(&self) -> FilterChanges<TxCompat::Transaction> {
|
||||
|
||||
@ -41,12 +41,12 @@ where
|
||||
tx: &Tx,
|
||||
content: &mut BTreeMap<Address, BTreeMap<String, RpcTxB::Transaction>>,
|
||||
) where
|
||||
Tx: PoolTransaction<Consensus = TransactionSignedEcRecovered>,
|
||||
Tx: PoolTransaction<Consensus: Into<TransactionSignedEcRecovered>>,
|
||||
RpcTxB: TransactionCompat,
|
||||
{
|
||||
content.entry(tx.sender()).or_default().insert(
|
||||
tx.nonce().to_string(),
|
||||
from_recovered::<RpcTxB>(tx.clone().into_consensus()),
|
||||
from_recovered::<RpcTxB>(tx.clone().into_consensus().into()),
|
||||
);
|
||||
}
|
||||
|
||||
@ -91,12 +91,12 @@ where
|
||||
trace!(target: "rpc::eth", "Serving txpool_inspect");
|
||||
|
||||
#[inline]
|
||||
fn insert<T: PoolTransaction<Consensus = TransactionSignedEcRecovered>>(
|
||||
fn insert<T: PoolTransaction<Consensus: Into<TransactionSignedEcRecovered>>>(
|
||||
tx: &T,
|
||||
inspect: &mut BTreeMap<Address, BTreeMap<String, TxpoolInspectSummary>>,
|
||||
) {
|
||||
let entry = inspect.entry(tx.sender()).or_default();
|
||||
let tx = tx.clone().into_consensus();
|
||||
let tx: TransactionSignedEcRecovered = tx.clone().into_consensus().into();
|
||||
entry.insert(
|
||||
tx.nonce().to_string(),
|
||||
TxpoolInspectSummary {
|
||||
|
||||
@ -324,6 +324,7 @@ where
|
||||
impl<V, T, S> TransactionPool for Pool<V, T, S>
|
||||
where
|
||||
V: TransactionValidator,
|
||||
<V as TransactionValidator>::Transaction: EthPoolTransaction,
|
||||
T: TransactionOrdering<Transaction = <V as TransactionValidator>::Transaction>,
|
||||
S: BlobStore,
|
||||
{
|
||||
@ -546,6 +547,7 @@ where
|
||||
impl<V, T, S> TransactionPoolExt for Pool<V, T, S>
|
||||
where
|
||||
V: TransactionValidator,
|
||||
<V as TransactionValidator>::Transaction: EthPoolTransaction,
|
||||
T: TransactionOrdering<Transaction = <V as TransactionValidator>::Transaction>,
|
||||
S: BlobStore,
|
||||
{
|
||||
|
||||
@ -18,6 +18,7 @@ use reth_execution_types::ChangedAccount;
|
||||
use reth_fs_util::FsPathError;
|
||||
use reth_primitives::{
|
||||
BlockNumberOrTag, PooledTransactionsElementEcRecovered, SealedHeader, TransactionSigned,
|
||||
TransactionSignedEcRecovered,
|
||||
};
|
||||
use reth_storage_api::{errors::provider::ProviderError, BlockReaderIdExt, StateProviderFactory};
|
||||
use reth_tasks::TaskSpawner;
|
||||
@ -334,11 +335,10 @@ pub async fn maintain_transaction_pool<Client, P, St, Tasks>(
|
||||
.ok()
|
||||
})
|
||||
.map(|tx| {
|
||||
<<P as TransactionPool>::Transaction as PoolTransaction>::from_pooled(tx)
|
||||
<P as TransactionPool>::Transaction::from_pooled(tx.into())
|
||||
})
|
||||
} else {
|
||||
|
||||
<P::Transaction as PoolTransaction>::try_from_consensus(tx).ok()
|
||||
<P as TransactionPool>::Transaction::try_from_consensus(tx.into()).ok()
|
||||
}
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
@ -583,7 +583,7 @@ where
|
||||
.filter_map(|tx| tx.try_ecrecovered())
|
||||
.filter_map(|tx| {
|
||||
// Filter out errors
|
||||
<P::Transaction as PoolTransaction>::try_from_consensus(tx).ok()
|
||||
<P::Transaction as PoolTransaction>::try_from_consensus(tx.into()).ok()
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
@ -606,7 +606,11 @@ where
|
||||
|
||||
let local_transactions = local_transactions
|
||||
.into_iter()
|
||||
.map(|tx| tx.to_recovered_transaction().into_signed())
|
||||
.map(|tx| {
|
||||
let recovered: TransactionSignedEcRecovered =
|
||||
tx.transaction.clone().into_consensus().into();
|
||||
recovered.into_signed()
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
let num_txs = local_transactions.len();
|
||||
|
||||
@ -1,6 +1,5 @@
|
||||
use crate::traits::PoolTransaction;
|
||||
use alloy_primitives::U256;
|
||||
use reth_primitives::{PooledTransactionsElementEcRecovered, TransactionSignedEcRecovered};
|
||||
use std::{fmt, marker::PhantomData};
|
||||
|
||||
/// Priority of the transaction that can be missing.
|
||||
@ -32,10 +31,7 @@ pub trait TransactionOrdering: Send + Sync + 'static {
|
||||
type PriorityValue: Ord + Clone + Default + fmt::Debug + Send + Sync;
|
||||
|
||||
/// The transaction type to determine the priority of.
|
||||
type Transaction: PoolTransaction<
|
||||
Pooled = PooledTransactionsElementEcRecovered,
|
||||
Consensus = TransactionSignedEcRecovered,
|
||||
>;
|
||||
type Transaction: PoolTransaction;
|
||||
|
||||
/// Returns the priority score for the given transaction.
|
||||
fn priority(
|
||||
@ -55,10 +51,7 @@ pub struct CoinbaseTipOrdering<T>(PhantomData<T>);
|
||||
|
||||
impl<T> TransactionOrdering for CoinbaseTipOrdering<T>
|
||||
where
|
||||
T: PoolTransaction<
|
||||
Pooled = PooledTransactionsElementEcRecovered,
|
||||
Consensus = TransactionSignedEcRecovered,
|
||||
> + 'static,
|
||||
T: PoolTransaction + 'static,
|
||||
{
|
||||
type PriorityValue = U256;
|
||||
type Transaction = T;
|
||||
|
||||
@ -88,6 +88,7 @@ use reth_execution_types::ChangedAccount;
|
||||
|
||||
use reth_primitives::{
|
||||
BlobTransaction, BlobTransactionSidecar, PooledTransactionsElement, TransactionSigned,
|
||||
TransactionSignedEcRecovered,
|
||||
};
|
||||
use std::{
|
||||
collections::{HashMap, HashSet},
|
||||
@ -318,13 +319,19 @@ where
|
||||
&self,
|
||||
tx_hashes: Vec<TxHash>,
|
||||
limit: GetPooledTransactionLimit,
|
||||
) -> Vec<PooledTransactionsElement> {
|
||||
) -> Vec<PooledTransactionsElement>
|
||||
where
|
||||
<V as TransactionValidator>::Transaction:
|
||||
PoolTransaction<Consensus: Into<TransactionSignedEcRecovered>>,
|
||||
{
|
||||
let transactions = self.get_all(tx_hashes);
|
||||
let mut elements = Vec::with_capacity(transactions.len());
|
||||
let mut size = 0;
|
||||
for transaction in transactions {
|
||||
let encoded_len = transaction.encoded_length();
|
||||
let tx = transaction.to_recovered_transaction().into_signed();
|
||||
let recovered: TransactionSignedEcRecovered =
|
||||
transaction.transaction.clone().into_consensus().into();
|
||||
let tx = recovered.into_signed();
|
||||
let pooled = if tx.is_eip4844() {
|
||||
// for EIP-4844 transactions, we need to fetch the blob sidecar from the blob store
|
||||
if let Some(blob) = self.get_blob_transaction(tx) {
|
||||
@ -360,9 +367,15 @@ where
|
||||
pub(crate) fn get_pooled_transaction_element(
|
||||
&self,
|
||||
tx_hash: TxHash,
|
||||
) -> Option<PooledTransactionsElement> {
|
||||
) -> Option<PooledTransactionsElement>
|
||||
where
|
||||
<V as TransactionValidator>::Transaction:
|
||||
PoolTransaction<Consensus: Into<TransactionSignedEcRecovered>>,
|
||||
{
|
||||
self.get(&tx_hash).and_then(|transaction| {
|
||||
let tx = transaction.to_recovered_transaction().into_signed();
|
||||
let recovered: TransactionSignedEcRecovered =
|
||||
transaction.transaction.clone().into_consensus().into();
|
||||
let tx = recovered.into_signed();
|
||||
if tx.is_eip4844() {
|
||||
self.get_blob_transaction(tx).map(PooledTransactionsElement::BlobTransaction)
|
||||
} else {
|
||||
|
||||
@ -44,10 +44,7 @@ pub type PeerId = alloy_primitives::B512;
|
||||
#[auto_impl::auto_impl(&, Arc)]
|
||||
pub trait TransactionPool: Send + Sync + Clone {
|
||||
/// The transaction type of the pool
|
||||
type Transaction: PoolTransaction<
|
||||
Pooled = PooledTransactionsElementEcRecovered,
|
||||
Consensus = TransactionSignedEcRecovered,
|
||||
>;
|
||||
type Transaction: EthPoolTransaction;
|
||||
|
||||
/// Returns stats about the pool and all sub-pools.
|
||||
fn pool_size(&self) -> PoolSize;
|
||||
@ -496,12 +493,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 = T::Consensus> + '_ {
|
||||
self.pending.iter().map(|tx| tx.transaction.clone().into_consensus())
|
||||
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 = T::Consensus> + '_ {
|
||||
self.queued.iter().map(|tx| tx.transaction.clone().into_consensus())
|
||||
self.queued.iter().map(|tx| tx.transaction.clone().into())
|
||||
}
|
||||
}
|
||||
|
||||
@ -813,19 +810,25 @@ pub trait PoolTransaction: fmt::Debug + Send + Sync + Clone {
|
||||
type TryFromConsensusError;
|
||||
|
||||
/// Associated type representing the raw consensus variant of the transaction.
|
||||
type Consensus: From<Self> + TryInto<Self>;
|
||||
type Consensus: From<Self> + TryInto<Self, Error = Self::TryFromConsensusError>;
|
||||
|
||||
/// Associated type representing the recovered pooled variant of the transaction.
|
||||
type Pooled: 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: Self::Consensus) -> Result<Self, Self::TryFromConsensusError> {
|
||||
tx.try_into()
|
||||
}
|
||||
|
||||
/// Define a method to convert from the `Self` type to `Consensus`
|
||||
fn into_consensus(self) -> Self::Consensus;
|
||||
fn into_consensus(self) -> Self::Consensus {
|
||||
self.into()
|
||||
}
|
||||
|
||||
/// Define a method to convert from the `Pooled` type to `Self`
|
||||
fn from_pooled(pooled: Self::Pooled) -> Self;
|
||||
fn from_pooled(pooled: Self::Pooled) -> Self {
|
||||
pooled.into()
|
||||
}
|
||||
|
||||
/// Hash of the transaction.
|
||||
fn hash(&self) -> &TxHash;
|
||||
@ -921,12 +924,11 @@ pub trait PoolTransaction: fmt::Debug + Send + Sync + Clone {
|
||||
fn chain_id(&self) -> Option<u64>;
|
||||
}
|
||||
|
||||
/// An extension trait that provides additional interfaces for the
|
||||
/// [`EthTransactionValidator`](crate::EthTransactionValidator).
|
||||
/// Super trait for transactions that can be converted to and from Eth transactions
|
||||
pub trait EthPoolTransaction:
|
||||
PoolTransaction<
|
||||
Pooled = PooledTransactionsElementEcRecovered,
|
||||
Consensus = TransactionSignedEcRecovered,
|
||||
Consensus: From<TransactionSignedEcRecovered> + Into<TransactionSignedEcRecovered>,
|
||||
Pooled: From<PooledTransactionsElementEcRecovered> + Into<PooledTransactionsElementEcRecovered>,
|
||||
>
|
||||
{
|
||||
/// Extracts the blob sidecar from the transaction.
|
||||
@ -1069,18 +1071,6 @@ impl PoolTransaction for EthPooledTransaction {
|
||||
|
||||
type Pooled = PooledTransactionsElementEcRecovered;
|
||||
|
||||
fn try_from_consensus(tx: Self::Consensus) -> Result<Self, Self::TryFromConsensusError> {
|
||||
tx.try_into()
|
||||
}
|
||||
|
||||
fn into_consensus(self) -> Self::Consensus {
|
||||
self.into()
|
||||
}
|
||||
|
||||
fn from_pooled(pooled: Self::Pooled) -> Self {
|
||||
pooled.into()
|
||||
}
|
||||
|
||||
/// Returns hash of the transaction.
|
||||
fn hash(&self) -> &TxHash {
|
||||
self.transaction.hash_ref()
|
||||
|
||||
@ -7,10 +7,7 @@ use crate::{
|
||||
};
|
||||
use alloy_primitives::{Address, TxHash, B256, U256};
|
||||
use futures_util::future::Either;
|
||||
use reth_primitives::{
|
||||
BlobTransactionSidecar, PooledTransactionsElementEcRecovered, SealedBlock,
|
||||
TransactionSignedEcRecovered,
|
||||
};
|
||||
use reth_primitives::{BlobTransactionSidecar, SealedBlock, TransactionSignedEcRecovered};
|
||||
use std::{fmt, future::Future, time::Instant};
|
||||
|
||||
mod constants;
|
||||
@ -154,10 +151,7 @@ impl<T: PoolTransaction> ValidTransaction<T> {
|
||||
/// Provides support for validating transaction at any given state of the chain
|
||||
pub trait TransactionValidator: Send + Sync {
|
||||
/// The transaction type to validate.
|
||||
type Transaction: PoolTransaction<
|
||||
Pooled = PooledTransactionsElementEcRecovered,
|
||||
Consensus = TransactionSignedEcRecovered,
|
||||
>;
|
||||
type Transaction: PoolTransaction;
|
||||
|
||||
/// Validates the transaction and returns a [`TransactionValidationOutcome`] describing the
|
||||
/// validity of the given transaction.
|
||||
@ -380,12 +374,12 @@ impl<T: PoolTransaction> ValidPoolTransaction<T> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: PoolTransaction<Consensus = TransactionSignedEcRecovered>> ValidPoolTransaction<T> {
|
||||
impl<T: PoolTransaction<Consensus: Into<TransactionSignedEcRecovered>>> ValidPoolTransaction<T> {
|
||||
/// Converts to this type into a [`TransactionSignedEcRecovered`].
|
||||
///
|
||||
/// Note: this takes `&self` since indented usage is via `Arc<Self>`.
|
||||
pub fn to_recovered_transaction(&self) -> TransactionSignedEcRecovered {
|
||||
self.transaction.clone().into_consensus()
|
||||
self.transaction.clone().into_consensus().into()
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user