chore: deprecate recoveredtx alias (#13887)

This commit is contained in:
Matthias Seitz
2025-01-20 19:07:14 +01:00
committed by GitHub
parent 1e71002056
commit 64214fd186
22 changed files with 130 additions and 125 deletions

View File

@ -20,7 +20,7 @@ use reth_execution_types::{Chain, ExecutionOutcome};
use reth_primitives::{
proofs::{calculate_receipt_root, calculate_transaction_root, calculate_withdrawals_root},
transaction::SignedTransactionIntoRecoveredExt,
BlockBody, EthPrimitives, NodePrimitives, Receipt, Receipts, RecoveredBlock, RecoveredTx,
BlockBody, EthPrimitives, NodePrimitives, Receipt, Receipts, Recovered, RecoveredBlock,
SealedBlock, SealedHeader, Transaction, TransactionSigned,
};
use reth_primitives_traits::Account;
@ -97,7 +97,7 @@ impl<N: NodePrimitives> TestBlockBuilder<N> {
) -> RecoveredBlock<reth_primitives::Block> {
let mut rng = thread_rng();
let mock_tx = |nonce: u64| -> RecoveredTx<_> {
let mock_tx = |nonce: u64| -> Recovered<_> {
let tx = Transaction::Eip1559(TxEip1559 {
chain_id: self.chain_spec.chain.id(),
nonce,
@ -115,7 +115,7 @@ impl<N: NodePrimitives> TestBlockBuilder<N> {
let num_txs = rng.gen_range(0..5);
let signer_balance_decrease = Self::single_tx_cost() * U256::from(num_txs);
let transactions: Vec<RecoveredTx<_>> = (0..num_txs)
let transactions: Vec<Recovered<_>> = (0..num_txs)
.map(|_| {
let tx = mock_tx(self.signer_build_account_info.nonce);
self.signer_build_account_info.nonce += 1;

View File

@ -8,7 +8,7 @@ use alloy_primitives::{Address, BlockHash, BlockNumber, TxHash};
use core::{fmt, ops::RangeInclusive};
use reth_execution_errors::{BlockExecutionError, InternalBlockExecutionError};
use reth_primitives::{
transaction::SignedTransactionIntoRecoveredExt, RecoveredBlock, RecoveredTx, SealedHeader,
transaction::SignedTransactionIntoRecoveredExt, Recovered, RecoveredBlock, SealedHeader,
};
use reth_primitives_traits::{Block, BlockBody, NodePrimitives, SignedTransaction};
use reth_trie::updates::TrieUpdates;
@ -442,13 +442,13 @@ impl<B: Block<Body: BlockBody<Transaction: SignedTransaction>>> ChainBlocks<'_,
self.blocks.values().flat_map(|block| block.transactions_with_sender())
}
/// Returns an iterator over all [`RecoveredTx`] in the blocks
/// Returns an iterator over all [`Recovered`] in the blocks
///
/// Note: This clones the transactions since it is assumed this is part of a shared [Chain].
#[inline]
pub fn transactions_ecrecovered(
&self,
) -> impl Iterator<Item = RecoveredTx<<B::Body as BlockBody>::Transaction>> + '_ {
) -> impl Iterator<Item = Recovered<<B::Body as BlockBody>::Transaction>> + '_ {
self.transactions_with_sender().map(|(signer, tx)| tx.clone().with_signer(*signer))
}

View File

@ -11,7 +11,7 @@ use reth_node_api::{Block, BlockBody};
use reth_optimism_evm::RethL1BlockInfo;
use reth_optimism_primitives::{OpBlock, OpTransactionSigned};
use reth_primitives::{
transaction::TransactionConversionError, GotExpected, InvalidTransactionError, RecoveredTx,
transaction::TransactionConversionError, GotExpected, InvalidTransactionError, Recovered,
SealedBlock,
};
use reth_primitives_traits::SignedTransaction;
@ -50,7 +50,7 @@ pub struct OpPooledTransaction {
impl OpPooledTransaction {
/// Create new instance of [Self].
pub fn new(transaction: RecoveredTx<OpTransactionSigned>, encoded_length: usize) -> Self {
pub fn new(transaction: Recovered<OpTransactionSigned>, encoded_length: usize) -> Self {
Self {
inner: EthPooledTransaction::new(transaction, encoded_length),
estimated_tx_compressed_size: Default::default(),
@ -77,8 +77,8 @@ fn tx_estimated_size_fjord(input: &[u8]) -> u64 {
fastlz_size.saturating_mul(836_500).saturating_sub(42_585_600).max(100_000_000)
}
impl From<RecoveredTx<op_alloy_consensus::OpPooledTransaction>> for OpPooledTransaction {
fn from(tx: RecoveredTx<op_alloy_consensus::OpPooledTransaction>) -> Self {
impl From<Recovered<op_alloy_consensus::OpPooledTransaction>> for OpPooledTransaction {
fn from(tx: Recovered<op_alloy_consensus::OpPooledTransaction>) -> Self {
let encoded_len = tx.encode_2718_len();
let tx = tx.map_transaction(|tx| tx.into());
Self {
@ -88,37 +88,37 @@ impl From<RecoveredTx<op_alloy_consensus::OpPooledTransaction>> for OpPooledTran
}
}
impl TryFrom<RecoveredTx<OpTransactionSigned>> for OpPooledTransaction {
impl TryFrom<Recovered<OpTransactionSigned>> for OpPooledTransaction {
type Error = TransactionConversionError;
fn try_from(value: RecoveredTx<OpTransactionSigned>) -> Result<Self, Self::Error> {
fn try_from(value: Recovered<OpTransactionSigned>) -> Result<Self, Self::Error> {
let (tx, signer) = value.into_parts();
let pooled: RecoveredTx<op_alloy_consensus::OpPooledTransaction> =
RecoveredTx::new_unchecked(tx.try_into()?, signer);
let pooled: Recovered<op_alloy_consensus::OpPooledTransaction> =
Recovered::new_unchecked(tx.try_into()?, signer);
Ok(pooled.into())
}
}
impl From<OpPooledTransaction> for RecoveredTx<OpTransactionSigned> {
impl From<OpPooledTransaction> for Recovered<OpTransactionSigned> {
fn from(value: OpPooledTransaction) -> Self {
value.inner.transaction
}
}
impl PoolTransaction for OpPooledTransaction {
type TryFromConsensusError = <Self as TryFrom<RecoveredTx<Self::Consensus>>>::Error;
type TryFromConsensusError = <Self as TryFrom<Recovered<Self::Consensus>>>::Error;
type Consensus = OpTransactionSigned;
type Pooled = op_alloy_consensus::OpPooledTransaction;
fn clone_into_consensus(&self) -> RecoveredTx<Self::Consensus> {
fn clone_into_consensus(&self) -> Recovered<Self::Consensus> {
self.inner.transaction().clone()
}
fn try_consensus_into_pooled(
tx: RecoveredTx<Self::Consensus>,
) -> Result<RecoveredTx<Self::Pooled>, Self::TryFromConsensusError> {
tx: Recovered<Self::Consensus>,
) -> Result<Recovered<Self::Pooled>, Self::TryFromConsensusError> {
let (tx, signer) = tx.into_parts();
Ok(RecoveredTx::new_unchecked(tx.try_into()?, signer))
Ok(Recovered::new_unchecked(tx.try_into()?, signer))
}
fn hash(&self) -> &TxHash {
@ -210,12 +210,12 @@ impl EthPoolTransaction for OpPooledTransaction {
fn try_into_pooled_eip4844(
self,
_sidecar: Arc<BlobTransactionSidecar>,
) -> Option<RecoveredTx<Self::Pooled>> {
) -> Option<Recovered<Self::Pooled>> {
None
}
fn try_from_eip4844(
_tx: RecoveredTx<Self::Consensus>,
_tx: Recovered<Self::Consensus>,
_sidecar: BlobTransactionSidecar,
) -> Option<Self> {
None
@ -463,7 +463,7 @@ mod tests {
use op_alloy_consensus::{OpTypedTransaction, TxDeposit};
use reth_chainspec::MAINNET;
use reth_optimism_primitives::OpTransactionSigned;
use reth_primitives::RecoveredTx;
use reth_primitives::Recovered;
use reth_provider::test_utils::MockEthProvider;
use reth_transaction_pool::{
blobstore::InMemoryBlobStore, validate::EthTransactionValidatorBuilder, TransactionOrigin,
@ -492,7 +492,7 @@ mod tests {
});
let signature = Signature::test_signature();
let signed_tx = OpTransactionSigned::new_unhashed(deposit_tx, signature);
let signed_recovered = RecoveredTx::new_unchecked(signed_tx, signer);
let signed_recovered = Recovered::new_unchecked(signed_tx, signer);
let len = signed_recovered.encode_2718_len();
let pooled_tx = OpPooledTransaction::new(signed_recovered, len);
let outcome = validator.validate_one(origin, pooled_tx);

View File

@ -28,7 +28,7 @@ use reth_optimism_node::{
use reth_optimism_payload_builder::builder::OpPayloadTransactions;
use reth_optimism_primitives::{OpPrimitives, OpTransactionSigned};
use reth_payload_util::{PayloadTransactions, PayloadTransactionsChain, PayloadTransactionsFixed};
use reth_primitives::RecoveredTx;
use reth_primitives::Recovered;
use reth_provider::providers::BlockchainProvider;
use reth_tasks::TaskManager;
use reth_transaction_pool::{pool::BestPayloadTransactions, PoolTransaction};
@ -67,7 +67,7 @@ impl OpPayloadTransactions for CustomTxPriority {
..Default::default()
};
let signature = sender.sign_transaction_sync(&mut end_of_block_tx).unwrap();
let end_of_block_tx = RecoveredTx::new_unchecked(
let end_of_block_tx = Recovered::new_unchecked(
OpTransactionSigned::new_unhashed(
OpTypedTransaction::Eip1559(end_of_block_tx),
signature,

View File

@ -4,7 +4,7 @@
#[derive(Debug, thiserror::Error)]
pub enum OpPayloadBuilderError {
/// Thrown when a transaction fails to convert to a
/// [`reth_primitives::RecoveredTx`].
/// [`reth_primitives::Recovered`].
#[error("failed to convert deposit transaction to RecoveredTx")]
TransactionEcRecoverFailed,
/// Thrown when the L1 block info could not be parsed from the calldata of the

View File

@ -7,7 +7,7 @@ use op_alloy_consensus::{OpTxEnvelope, OpTypedTransaction};
use op_alloy_rpc_types::{OpTransactionRequest, Transaction};
use reth_node_api::FullNodeComponents;
use reth_optimism_primitives::{OpReceipt, OpTransactionSigned};
use reth_primitives::RecoveredTx;
use reth_primitives::Recovered;
use reth_primitives_traits::transaction::signed::SignedTransaction;
use reth_provider::{
BlockReader, BlockReaderIdExt, ProviderTx, ReceiptProvider, TransactionsProvider,
@ -84,7 +84,7 @@ where
fn fill(
&self,
tx: RecoveredTx<OpTransactionSigned>,
tx: Recovered<OpTransactionSigned>,
tx_info: TransactionInfo,
) -> Result<Self::Transaction, Self::Error> {
let from = tx.signer();

View File

@ -1,5 +1,5 @@
use alloy_primitives::Address;
use reth_primitives::RecoveredTx;
use reth_primitives::Recovered;
/// Iterator that returns transactions for the block building process in the order they should be
/// included in the block.
@ -15,7 +15,7 @@ pub trait PayloadTransactions {
&mut self,
// In the future, `ctx` can include access to state for block building purposes.
ctx: (),
) -> Option<RecoveredTx<Self::Transaction>>;
) -> Option<Recovered<Self::Transaction>>;
/// Exclude descendants of the transaction with given sender and nonce from the iterator,
/// because this transaction won't be included in the block.
@ -35,7 +35,7 @@ impl<T> Default for NoopPayloadTransactions<T> {
impl<T> PayloadTransactions for NoopPayloadTransactions<T> {
type Transaction = T;
fn next(&mut self, _ctx: ()) -> Option<RecoveredTx<Self::Transaction>> {
fn next(&mut self, _ctx: ()) -> Option<Recovered<Self::Transaction>> {
None
}

View File

@ -1,7 +1,7 @@
use crate::PayloadTransactions;
use alloy_consensus::Transaction;
use alloy_primitives::Address;
use reth_primitives::RecoveredTx;
use reth_primitives::Recovered;
/// An implementation of [`crate::traits::PayloadTransactions`] that yields
/// a pre-defined set of transactions.
@ -26,10 +26,10 @@ impl<T> PayloadTransactionsFixed<T> {
}
}
impl<T: Clone> PayloadTransactions for PayloadTransactionsFixed<RecoveredTx<T>> {
impl<T: Clone> PayloadTransactions for PayloadTransactionsFixed<Recovered<T>> {
type Transaction = T;
fn next(&mut self, _ctx: ()) -> Option<RecoveredTx<T>> {
fn next(&mut self, _ctx: ()) -> Option<Recovered<T>> {
(self.index < self.transactions.len()).then(|| {
let tx = self.transactions[self.index].clone();
self.index += 1;
@ -96,7 +96,7 @@ where
{
type Transaction = A::Transaction;
fn next(&mut self, ctx: ()) -> Option<RecoveredTx<Self::Transaction>> {
fn next(&mut self, ctx: ()) -> Option<Recovered<Self::Transaction>> {
while let Some(tx) = self.before.next(ctx) {
if let Some(before_max_gas) = self.before_max_gas {
if self.before_gas + tx.tx().gas_limit() <= before_max_gas {

View File

@ -40,9 +40,14 @@ pub use reth_primitives_traits::{
pub use static_file::StaticFileSegment;
pub use alloy_consensus::{
transaction::{PooledTransaction, Recovered as RecoveredTx, TransactionMeta},
transaction::{PooledTransaction, Recovered, TransactionMeta},
ReceiptWithBloom,
};
/// Recovered transaction
#[deprecated(note = "use `Recovered` instead")]
pub type RecoveredTx<T> = Recovered<T>;
pub use transaction::{
util::secp256k1::{public_key_to_address, recover_signer_unchecked, sign_message},
InvalidTransactionError, Transaction, TransactionSigned, TxType,

View File

@ -1,6 +1,6 @@
//! Transaction types.
use crate::RecoveredTx;
use crate::Recovered;
pub use alloy_consensus::transaction::PooledTransaction;
use once_cell as _;
#[allow(deprecated)]
@ -31,4 +31,4 @@ pub use reth_ethereum_primitives::{Transaction, TransactionSigned};
/// Type alias kept for backward compatibility.
#[deprecated(note = "Use `Recovered` instead")]
pub type TransactionSignedEcRecovered<T = TransactionSigned> = RecoveredTx<T>;
pub type TransactionSignedEcRecovered<T = TransactionSigned> = Recovered<T>;

View File

@ -1,9 +1,9 @@
//! Defines the types for blob transactions, legacy, and other EIP-2718 transactions included in a
//! response to `GetPooledTransactions`.
use crate::RecoveredTx;
use crate::Recovered;
use alloy_consensus::transaction::PooledTransaction;
/// A signed pooled transaction with recovered signer.
#[deprecated(note = "use `Recovered` instead")]
pub type PooledTransactionsElementEcRecovered<T = PooledTransaction> = RecoveredTx<T>;
pub type PooledTransactionsElementEcRecovered<T = PooledTransaction> = Recovered<T>;

View File

@ -4,7 +4,7 @@
use alloy_primitives::B256;
use alloy_rpc_types_eth::TransactionInfo;
use reth_primitives::{RecoveredTx, TransactionSigned};
use reth_primitives::{Recovered, TransactionSigned};
use reth_primitives_traits::SignedTransaction;
use reth_rpc_types_compat::TransactionCompat;
@ -12,13 +12,13 @@ use reth_rpc_types_compat::TransactionCompat;
#[derive(Debug, Clone, Eq, PartialEq)]
pub enum TransactionSource<T = TransactionSigned> {
/// Transaction exists in the pool (Pending)
Pool(RecoveredTx<T>),
Pool(Recovered<T>),
/// Transaction already included in a block
///
/// This can be a historical block or a pending block (received from the CL)
Block {
/// Transaction fetched via provider
transaction: RecoveredTx<T>,
transaction: Recovered<T>,
/// Index of the transaction in the block
index: u64,
/// Hash of the block.
@ -34,7 +34,7 @@ pub enum TransactionSource<T = TransactionSigned> {
impl<T: SignedTransaction> TransactionSource<T> {
/// Consumes the type and returns the wrapped transaction.
pub fn into_recovered(self) -> RecoveredTx<T> {
pub fn into_recovered(self) -> Recovered<T> {
self.into()
}
@ -60,7 +60,7 @@ impl<T: SignedTransaction> TransactionSource<T> {
}
/// Returns the transaction and block related info, if not pending
pub fn split(self) -> (RecoveredTx<T>, TransactionInfo) {
pub fn split(self) -> (Recovered<T>, TransactionInfo) {
match self {
Self::Pool(tx) => {
let hash = tx.trie_hash();
@ -83,7 +83,7 @@ impl<T: SignedTransaction> TransactionSource<T> {
}
}
impl<T> From<TransactionSource<T>> for RecoveredTx<T> {
impl<T> From<TransactionSource<T>> for Recovered<T> {
fn from(value: TransactionSource<T>) -> Self {
match value {
TransactionSource::Pool(tx) => tx,

View File

@ -1,7 +1,7 @@
//! Commonly used code snippets
use super::{EthApiError, EthResult};
use reth_primitives::{transaction::SignedTransactionIntoRecoveredExt, RecoveredTx};
use reth_primitives::{transaction::SignedTransactionIntoRecoveredExt, Recovered};
use reth_primitives_traits::SignedTransaction;
use std::future::Future;
@ -11,7 +11,7 @@ use std::future::Future;
/// malformed.
///
/// See [`alloy_eips::eip2718::Decodable2718::decode_2718`]
pub fn recover_raw_transaction<T: SignedTransaction>(mut data: &[u8]) -> EthResult<RecoveredTx<T>> {
pub fn recover_raw_transaction<T: SignedTransaction>(mut data: &[u8]) -> EthResult<Recovered<T>> {
if data.is_empty() {
return Err(EthApiError::EmptyRawTransactionData)
}

View File

@ -4,7 +4,7 @@ use core::error;
use std::fmt;
use alloy_rpc_types_eth::{request::TransactionRequest, TransactionInfo};
use reth_primitives::{RecoveredTx, TransactionSigned};
use reth_primitives::{Recovered, TransactionSigned};
use serde::{Deserialize, Serialize};
/// Builds RPC transaction w.r.t. network.
@ -26,7 +26,7 @@ pub trait TransactionCompat<T = TransactionSigned>:
/// Wrapper for `fill()` with default `TransactionInfo`
/// Create a new rpc transaction result for a _pending_ signed transaction, setting block
/// environment related fields to `None`.
fn fill_pending(&self, tx: RecoveredTx<T>) -> Result<Self::Transaction, Self::Error> {
fn fill_pending(&self, tx: Recovered<T>) -> Result<Self::Transaction, Self::Error> {
self.fill(tx, TransactionInfo::default())
}
@ -37,7 +37,7 @@ pub trait TransactionCompat<T = TransactionSigned>:
/// transaction was mined.
fn fill(
&self,
tx: RecoveredTx<T>,
tx: Recovered<T>,
tx_inf: TransactionInfo,
) -> Result<Self::Transaction, Self::Error>;
@ -51,9 +51,9 @@ pub trait TransactionCompat<T = TransactionSigned>:
fn otterscan_api_truncate_input(tx: &mut Self::Transaction);
}
/// Convert [`RecoveredTx`] to [`TransactionRequest`]
/// Convert [`Recovered`] to [`TransactionRequest`]
pub fn transaction_to_call_request<T: alloy_consensus::Transaction>(
tx: RecoveredTx<T>,
tx: Recovered<T>,
) -> TransactionRequest {
let from = tx.signer();
TransactionRequest::from_transaction_with_sender(tx.into_tx(), from)

View File

@ -5,7 +5,7 @@ use alloy_network::{Ethereum, Network};
use alloy_primitives::PrimitiveSignature as Signature;
use alloy_rpc_types::TransactionRequest;
use alloy_rpc_types_eth::{Transaction, TransactionInfo};
use reth_primitives::{RecoveredTx, TransactionSigned};
use reth_primitives::{Recovered, TransactionSigned};
use reth_primitives_traits::SignedTransaction;
use reth_rpc_eth_api::EthApiTypes;
use reth_rpc_eth_types::EthApiError;
@ -40,7 +40,7 @@ where
fn fill(
&self,
tx: RecoveredTx<TransactionSigned>,
tx: Recovered<TransactionSigned>,
tx_info: TransactionInfo,
) -> Result<Self::Transaction, Self::Error> {
let from = tx.signer();

View File

@ -176,7 +176,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_primitives::Recovered;
use reth_primitives_traits::Block;
use reth_storage_api::StateProviderFactory;
use std::{collections::HashSet, sync::Arc};
@ -423,7 +423,7 @@ where
fn get_pooled_transaction_element(
&self,
tx_hash: TxHash,
) -> Option<RecoveredTx<<<V as TransactionValidator>::Transaction as PoolTransaction>::Pooled>>
) -> Option<Recovered<<<V as TransactionValidator>::Transaction as PoolTransaction>::Pooled>>
{
self.pool.get_pooled_transaction_element(tx_hash)
}

View File

@ -22,7 +22,7 @@ use alloy_eips::{
};
use alloy_primitives::{Address, TxHash, B256, U256};
use reth_eth_wire_types::HandleMempoolData;
use reth_primitives::RecoveredTx;
use reth_primitives::Recovered;
use std::{collections::HashSet, marker::PhantomData, sync::Arc};
use tokio::sync::{mpsc, mpsc::Receiver};
@ -143,7 +143,7 @@ impl TransactionPool for NoopTransactionPool {
fn get_pooled_transaction_element(
&self,
_tx_hash: TxHash,
) -> Option<RecoveredTx<<Self::Transaction as PoolTransaction>::Pooled>> {
) -> Option<Recovered<<Self::Transaction as PoolTransaction>::Pooled>> {
None
}

View File

@ -7,7 +7,7 @@ use crate::{
use alloy_primitives::Address;
use core::fmt;
use reth_payload_util::PayloadTransactions;
use reth_primitives::{InvalidTransactionError, RecoveredTx};
use reth_primitives::{InvalidTransactionError, Recovered};
use std::{
collections::{BTreeMap, BTreeSet, HashSet, VecDeque},
sync::Arc,
@ -251,7 +251,7 @@ where
{
type Transaction = T::Consensus;
fn next(&mut self, _ctx: ()) -> Option<RecoveredTx<Self::Transaction>> {
fn next(&mut self, _ctx: ()) -> Option<Recovered<Self::Transaction>> {
loop {
let tx = self.best.next()?;
if self.invalid.contains(&tx.sender()) {

View File

@ -88,7 +88,7 @@ use reth_eth_wire_types::HandleMempoolData;
use reth_execution_types::ChangedAccount;
use alloy_eips::eip4844::BlobTransactionSidecar;
use reth_primitives::RecoveredTx;
use reth_primitives::Recovered;
use rustc_hash::FxHashMap;
use std::{collections::HashSet, fmt, sync::Arc, time::Instant};
use tokio::sync::mpsc;
@ -316,7 +316,7 @@ where
fn to_pooled_transaction(
&self,
transaction: Arc<ValidPoolTransaction<T::Transaction>>,
) -> Option<RecoveredTx<<<V as TransactionValidator>::Transaction as PoolTransaction>::Pooled>>
) -> Option<Recovered<<<V as TransactionValidator>::Transaction as PoolTransaction>::Pooled>>
where
<V as TransactionValidator>::Transaction: EthPoolTransaction,
{
@ -371,7 +371,7 @@ where
pub fn get_pooled_transaction_element(
&self,
tx_hash: TxHash,
) -> Option<RecoveredTx<<<V as TransactionValidator>::Transaction as PoolTransaction>::Pooled>>
) -> Option<Recovered<<<V as TransactionValidator>::Transaction as PoolTransaction>::Pooled>>
where
<V as TransactionValidator>::Transaction: EthPoolTransaction,
{

View File

@ -30,7 +30,7 @@ use rand::{
};
use reth_primitives::{
transaction::{SignedTransactionIntoRecoveredExt, TryFromRecoveredTransactionError},
PooledTransaction, RecoveredTx, Transaction, TransactionSigned, TxType,
PooledTransaction, Recovered, Transaction, TransactionSigned, TxType,
};
use reth_primitives_traits::{InMemorySize, SignedTransaction};
use std::{ops::Range, sync::Arc, time::Instant, vec::IntoIter};
@ -668,22 +668,22 @@ impl PoolTransaction for MockTransaction {
type Pooled = PooledTransaction;
fn try_from_consensus(
tx: RecoveredTx<Self::Consensus>,
tx: Recovered<Self::Consensus>,
) -> Result<Self, Self::TryFromConsensusError> {
tx.try_into()
}
fn into_consensus(self) -> RecoveredTx<Self::Consensus> {
fn into_consensus(self) -> Recovered<Self::Consensus> {
self.into()
}
fn from_pooled(pooled: RecoveredTx<Self::Pooled>) -> Self {
fn from_pooled(pooled: Recovered<Self::Pooled>) -> Self {
pooled.into()
}
fn try_consensus_into_pooled(
tx: RecoveredTx<Self::Consensus>,
) -> Result<RecoveredTx<Self::Pooled>, Self::TryFromConsensusError> {
tx: Recovered<Self::Consensus>,
) -> Result<Recovered<Self::Pooled>, Self::TryFromConsensusError> {
let (tx, signer) = tx.into_parts();
Self::Pooled::try_from(tx)
.map(|tx| tx.with_signer(signer))
@ -869,7 +869,7 @@ impl EthPoolTransaction for MockTransaction {
fn try_into_pooled_eip4844(
self,
sidecar: Arc<BlobTransactionSidecar>,
) -> Option<RecoveredTx<Self::Pooled>> {
) -> Option<Recovered<Self::Pooled>> {
let (tx, signer) = self.into_consensus().into_parts();
tx.try_into_pooled_eip4844(Arc::unwrap_or_clone(sidecar))
.map(|tx| tx.with_signer(signer))
@ -877,7 +877,7 @@ impl EthPoolTransaction for MockTransaction {
}
fn try_from_eip4844(
tx: RecoveredTx<Self::Consensus>,
tx: Recovered<Self::Consensus>,
sidecar: BlobTransactionSidecar,
) -> Option<Self> {
let (tx, signer) = tx.into_parts();
@ -903,10 +903,10 @@ impl EthPoolTransaction for MockTransaction {
}
}
impl TryFrom<RecoveredTx<TransactionSigned>> for MockTransaction {
impl TryFrom<Recovered<TransactionSigned>> for MockTransaction {
type Error = TryFromRecoveredTransactionError;
fn try_from(tx: RecoveredTx<TransactionSigned>) -> Result<Self, Self::Error> {
fn try_from(tx: Recovered<TransactionSigned>) -> Result<Self, Self::Error> {
let sender = tx.signer();
let transaction = tx.into_tx();
let hash = *transaction.tx_hash();
@ -1044,16 +1044,16 @@ impl TryFrom<RecoveredTx<TransactionSigned>> for MockTransaction {
}
}
impl From<RecoveredTx<PooledTransaction>> for MockTransaction {
fn from(tx: RecoveredTx<PooledTransaction>) -> Self {
impl From<Recovered<PooledTransaction>> for MockTransaction {
fn from(tx: Recovered<PooledTransaction>) -> Self {
let (tx, signer) = tx.into_parts();
RecoveredTx::<TransactionSigned>::new_unchecked(tx.into(), signer).try_into().expect(
Recovered::<TransactionSigned>::new_unchecked(tx.into(), signer).try_into().expect(
"Failed to convert from PooledTransactionsElementEcRecovered to MockTransaction",
)
}
}
impl From<MockTransaction> for RecoveredTx<TransactionSigned> {
impl From<MockTransaction> for Recovered<TransactionSigned> {
fn from(tx: MockTransaction) -> Self {
let signed_tx =
TransactionSigned::new(tx.clone().into(), Signature::test_signature(), *tx.hash());
@ -1180,9 +1180,9 @@ impl proptest::arbitrary::Arbitrary for MockTransaction {
arb::<(TransactionSigned, Address)>()
.prop_map(|(signed_transaction, signer)| {
RecoveredTx::new_unchecked(signed_transaction, signer)
Recovered::new_unchecked(signed_transaction, signer)
.try_into()
.expect("Failed to create an Arbitrary MockTransaction via RecoveredTx")
.expect("Failed to create an Arbitrary MockTransaction from a Recovered tx")
})
.boxed()
}

View File

@ -21,7 +21,7 @@ use reth_execution_types::ChangedAccount;
use reth_primitives::{
kzg::KzgSettings,
transaction::{SignedTransactionIntoRecoveredExt, TryFromRecoveredTransactionError},
PooledTransaction, RecoveredTx, SealedBlock, Transaction, TransactionSigned,
PooledTransaction, Recovered, SealedBlock, Transaction, TransactionSigned,
};
use reth_primitives_traits::{Block, SignedTransaction};
#[cfg(feature = "serde")]
@ -257,7 +257,7 @@ pub trait TransactionPool: Send + Sync + Clone {
fn get_pooled_transaction_element(
&self,
tx_hash: TxHash,
) -> Option<RecoveredTx<<Self::Transaction as PoolTransaction>::Pooled>>;
) -> Option<Recovered<<Self::Transaction as PoolTransaction>::Pooled>>;
/// Returns an iterator that yields transactions that are ready for block production.
///
@ -570,18 +570,18 @@ pub struct AllPoolTransactions<T: PoolTransaction> {
// === impl AllPoolTransactions ===
impl<T: PoolTransaction> AllPoolTransactions<T> {
/// Returns an iterator over all pending [`RecoveredTx`] transactions.
pub fn pending_recovered(&self) -> impl Iterator<Item = RecoveredTx<T::Consensus>> + '_ {
/// Returns an iterator over all pending [`Recovered`] transactions.
pub fn pending_recovered(&self) -> impl Iterator<Item = Recovered<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 = RecoveredTx<T::Consensus>> + '_ {
/// Returns an iterator over all queued [`Recovered`] transactions.
pub fn queued_recovered(&self) -> impl Iterator<Item = Recovered<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 = RecoveredTx<T::Consensus>> + '_ {
pub fn all(&self) -> impl Iterator<Item = Recovered<T::Consensus>> + '_ {
self.pending.iter().chain(self.queued.iter()).map(|tx| tx.transaction.clone().into())
}
}
@ -968,9 +968,9 @@ pub trait PoolTransaction:
+ Send
+ Sync
+ Clone
+ TryFrom<RecoveredTx<Self::Consensus>, Error = Self::TryFromConsensusError>
+ Into<RecoveredTx<Self::Consensus>>
+ From<RecoveredTx<Self::Pooled>>
+ TryFrom<Recovered<Self::Consensus>, Error = Self::TryFromConsensusError>
+ Into<Recovered<Self::Consensus>>
+ From<Recovered<Self::Pooled>>
{
/// Associated error type for the `try_from_consensus` method.
type TryFromConsensusError: fmt::Display;
@ -983,7 +983,7 @@ pub trait PoolTransaction:
/// Define a method to convert from the `Consensus` type to `Self`
fn try_from_consensus(
tx: RecoveredTx<Self::Consensus>,
tx: Recovered<Self::Consensus>,
) -> Result<Self, Self::TryFromConsensusError> {
tx.try_into()
}
@ -991,29 +991,29 @@ pub trait PoolTransaction:
/// Clone the transaction into a consensus variant.
///
/// This method is preferred when the [`PoolTransaction`] already wraps the consensus variant.
fn clone_into_consensus(&self) -> RecoveredTx<Self::Consensus> {
fn clone_into_consensus(&self) -> Recovered<Self::Consensus> {
self.clone().into_consensus()
}
/// Define a method to convert from the `Self` type to `Consensus`
fn into_consensus(self) -> RecoveredTx<Self::Consensus> {
fn into_consensus(self) -> Recovered<Self::Consensus> {
self.into()
}
/// Define a method to convert from the `Pooled` type to `Self`
fn from_pooled(pooled: RecoveredTx<Self::Pooled>) -> Self {
fn from_pooled(pooled: Recovered<Self::Pooled>) -> Self {
pooled.into()
}
/// Tries to convert the `Consensus` type into the `Pooled` type.
fn try_into_pooled(self) -> Result<RecoveredTx<Self::Pooled>, Self::TryFromConsensusError> {
fn try_into_pooled(self) -> Result<Recovered<Self::Pooled>, 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<Self::Consensus>,
) -> Result<RecoveredTx<Self::Pooled>, Self::TryFromConsensusError>;
tx: Recovered<Self::Consensus>,
) -> Result<Recovered<Self::Pooled>, Self::TryFromConsensusError>;
/// Converts the `Pooled` type into the `Consensus` type.
fn pooled_into_consensus(tx: Self::Pooled) -> Self::Consensus {
@ -1160,13 +1160,13 @@ pub trait EthPoolTransaction: PoolTransaction {
fn try_into_pooled_eip4844(
self,
sidecar: Arc<BlobTransactionSidecar>,
) -> Option<RecoveredTx<Self::Pooled>>;
) -> Option<Recovered<Self::Pooled>>;
/// 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<Self::Consensus>,
tx: Recovered<Self::Consensus>,
sidecar: BlobTransactionSidecar,
) -> Option<Self>;
@ -1183,12 +1183,12 @@ pub trait EthPoolTransaction: PoolTransaction {
/// The default [`PoolTransaction`] for the [Pool](crate::Pool) for Ethereum.
///
/// This type is essentially a wrapper around [`RecoveredTx`] with additional
/// This type is essentially a wrapper around [`Recovered`] with additional
/// fields derived from the transaction that are frequently used by the pools for ordering.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct EthPooledTransaction<T = TransactionSigned> {
/// `EcRecovered` transaction, the consensus format.
pub transaction: RecoveredTx<T>,
pub transaction: Recovered<T>,
/// For EIP-1559 transactions: `max_fee_per_gas * gas_limit + tx_value`.
/// For legacy transactions: `gas_price * gas_limit + tx_value`.
@ -1209,7 +1209,7 @@ impl<T: SignedTransaction> EthPooledTransaction<T> {
///
/// Caution: In case of blob transactions, this does marks the blob sidecar as
/// [`EthBlobTransactionSidecar::Missing`]
pub fn new(transaction: RecoveredTx<T>, encoded_length: usize) -> Self {
pub fn new(transaction: Recovered<T>, encoded_length: usize) -> Self {
let mut blob_sidecar = EthBlobTransactionSidecar::None;
let gas_cost = U256::from(transaction.max_fee_per_gas())
@ -1234,14 +1234,14 @@ impl<T: SignedTransaction> EthPooledTransaction<T> {
}
/// Return the reference to the underlying transaction.
pub const fn transaction(&self) -> &RecoveredTx<T> {
pub const fn transaction(&self) -> &Recovered<T> {
&self.transaction
}
}
/// Conversion from the network transaction type to the pool transaction type.
impl From<RecoveredTx<PooledTransaction>> for EthPooledTransaction {
fn from(tx: RecoveredTx<PooledTransaction>) -> Self {
impl From<Recovered<PooledTransaction>> for EthPooledTransaction {
fn from(tx: Recovered<PooledTransaction>) -> Self {
let encoded_length = tx.encode_2718_len();
let (tx, signer) = tx.into_parts();
match tx {
@ -1251,14 +1251,14 @@ impl From<RecoveredTx<PooledTransaction>> for EthPooledTransaction {
let (tx, blob) = tx.into_parts();
let tx = Signed::new_unchecked(tx, sig, hash);
let tx = TransactionSigned::from(tx);
let tx = RecoveredTx::new_unchecked(tx, signer);
let tx = Recovered::new_unchecked(tx, signer);
let mut pooled = Self::new(tx, encoded_length);
pooled.blob_sidecar = EthBlobTransactionSidecar::Present(blob);
pooled
}
tx => {
// no blob sidecar
let tx = RecoveredTx::new_unchecked(tx.into(), signer);
let tx = Recovered::new_unchecked(tx.into(), signer);
Self::new(tx, encoded_length)
}
}
@ -1272,17 +1272,17 @@ impl PoolTransaction for EthPooledTransaction {
type Pooled = PooledTransaction;
fn clone_into_consensus(&self) -> RecoveredTx<Self::Consensus> {
fn clone_into_consensus(&self) -> Recovered<Self::Consensus> {
self.transaction().clone()
}
fn try_consensus_into_pooled(
tx: RecoveredTx<Self::Consensus>,
) -> Result<RecoveredTx<Self::Pooled>, Self::TryFromConsensusError> {
tx: Recovered<Self::Consensus>,
) -> Result<Recovered<Self::Pooled>, Self::TryFromConsensusError> {
let (tx, signer) = tx.into_parts();
let pooled =
tx.try_into().map_err(|_| TryFromRecoveredTransactionError::BlobSidecarMissing)?;
Ok(RecoveredTx::new_unchecked(pooled, signer))
Ok(Recovered::new_unchecked(pooled, signer))
}
/// Returns hash of the transaction.
@ -1413,16 +1413,16 @@ impl EthPoolTransaction for EthPooledTransaction {
fn try_into_pooled_eip4844(
self,
sidecar: Arc<BlobTransactionSidecar>,
) -> Option<RecoveredTx<Self::Pooled>> {
) -> Option<Recovered<Self::Pooled>> {
let (signed_transaction, signer) = self.into_consensus().into_parts();
let pooled_transaction =
signed_transaction.try_into_pooled_eip4844(Arc::unwrap_or_clone(sidecar)).ok()?;
Some(RecoveredTx::new_unchecked(pooled_transaction, signer))
Some(Recovered::new_unchecked(pooled_transaction, signer))
}
fn try_from_eip4844(
tx: RecoveredTx<Self::Consensus>,
tx: Recovered<Self::Consensus>,
sidecar: BlobTransactionSidecar,
) -> Option<Self> {
let (tx, signer) = tx.into_parts();
@ -1451,10 +1451,10 @@ impl EthPoolTransaction for EthPooledTransaction {
}
}
impl TryFrom<RecoveredTx<TransactionSigned>> for EthPooledTransaction {
impl TryFrom<Recovered<TransactionSigned>> for EthPooledTransaction {
type Error = TryFromRecoveredTransactionError;
fn try_from(tx: RecoveredTx<TransactionSigned>) -> Result<Self, Self::Error> {
fn try_from(tx: Recovered<TransactionSigned>) -> Result<Self, Self::Error> {
// ensure we can handle the transaction type and its format
match tx.ty() {
0..=EIP1559_TX_TYPE_ID | EIP7702_TX_TYPE_ID => {
@ -1478,7 +1478,7 @@ impl TryFrom<RecoveredTx<TransactionSigned>> for EthPooledTransaction {
}
}
impl From<EthPooledTransaction> for RecoveredTx<TransactionSigned> {
impl From<EthPooledTransaction> for Recovered<TransactionSigned> {
fn from(tx: EthPooledTransaction) -> Self {
tx.transaction
}
@ -1690,7 +1690,7 @@ mod tests {
});
let signature = Signature::test_signature();
let signed_tx = TransactionSigned::new_unhashed(tx, signature);
let transaction = RecoveredTx::new_unchecked(signed_tx, Default::default());
let transaction = Recovered::new_unchecked(signed_tx, Default::default());
let pooled_tx = EthPooledTransaction::new(transaction.clone(), 200);
// Check that the pooled transaction is created correctly
@ -1711,7 +1711,7 @@ mod tests {
});
let signature = Signature::test_signature();
let signed_tx = TransactionSigned::new_unhashed(tx, signature);
let transaction = RecoveredTx::new_unchecked(signed_tx, Default::default());
let transaction = Recovered::new_unchecked(signed_tx, Default::default());
let pooled_tx = EthPooledTransaction::new(transaction.clone(), 200);
// Check that the pooled transaction is created correctly
@ -1732,7 +1732,7 @@ mod tests {
});
let signature = Signature::test_signature();
let signed_tx = TransactionSigned::new_unhashed(tx, signature);
let transaction = RecoveredTx::new_unchecked(signed_tx, Default::default());
let transaction = Recovered::new_unchecked(signed_tx, Default::default());
let pooled_tx = EthPooledTransaction::new(transaction.clone(), 200);
// Check that the pooled transaction is created correctly
@ -1755,7 +1755,7 @@ mod tests {
});
let signature = Signature::test_signature();
let signed_tx = TransactionSigned::new_unhashed(tx, signature);
let transaction = RecoveredTx::new_unchecked(signed_tx, Default::default());
let transaction = Recovered::new_unchecked(signed_tx, Default::default());
let pooled_tx = EthPooledTransaction::new(transaction.clone(), 300);
// Check that the pooled transaction is created correctly
@ -1778,7 +1778,7 @@ mod tests {
});
let signature = Signature::test_signature();
let signed_tx = TransactionSigned::new_unhashed(tx, signature);
let transaction = RecoveredTx::new_unchecked(signed_tx, Default::default());
let transaction = Recovered::new_unchecked(signed_tx, Default::default());
let pooled_tx = EthPooledTransaction::new(transaction.clone(), 200);
// Check that the pooled transaction is created correctly

View File

@ -9,7 +9,7 @@ use crate::{
use alloy_eips::eip4844::BlobTransactionSidecar;
use alloy_primitives::{Address, TxHash, B256, U256};
use futures_util::future::Either;
use reth_primitives::{RecoveredTx, SealedBlock};
use reth_primitives::{Recovered, SealedBlock};
use std::{fmt, future::Future, time::Instant};
mod constants;
@ -391,7 +391,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) -> RecoveredTx<T::Consensus> {
pub fn to_consensus(&self) -> Recovered<T::Consensus> {
self.transaction.clone_into_consensus()
}