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

@ -18,7 +18,10 @@ use reth_evm::{
ConfigureEvm,
};
use reth_payload_validator::ExecutionPayloadValidator;
use reth_primitives::{proofs, Block, BlockBody, BlockExt, Receipt, Receipts};
use reth_primitives::{
proofs, transaction::SignedTransactionIntoRecoveredExt, Block, BlockBody, BlockExt, Receipt,
Receipts,
};
use reth_provider::{BlockReader, ExecutionOutcome, ProviderError, StateProviderFactory};
use reth_revm::{
database::StateProviderDatabase,

View File

@ -13,7 +13,7 @@ use reth_evm::execute::BasicBlockExecutorProvider;
use reth_evm_ethereum::execute::EthExecutionStrategyFactory;
use reth_network::{NetworkHandle, PeersInfo};
use reth_node_api::{
AddOnsContext, ConfigureEvm, EngineValidator, FullNodeComponents, NodeTypesWithDB,
AddOnsContext, ConfigureEvm, EngineValidator, FullNodeComponents, NodeTypesWithDB, TxTy,
};
use reth_node_builder::{
components::{
@ -30,7 +30,7 @@ use reth_provider::{CanonStateSubscriptions, EthStorage};
use reth_rpc::EthApi;
use reth_tracing::tracing::{debug, info};
use reth_transaction_pool::{
blobstore::DiskFileBlobStore, EthTransactionPool, TransactionPool,
blobstore::DiskFileBlobStore, EthTransactionPool, PoolTransaction, TransactionPool,
TransactionValidationTaskExecutor,
};
use reth_trie_db::MerklePatriciaTrie;
@ -243,7 +243,9 @@ impl EthereumPayloadBuilder {
Types: NodeTypesWithEngine<ChainSpec = ChainSpec, Primitives = EthPrimitives>,
Node: FullNodeTypes<Types = Types>,
Evm: ConfigureEvm<Header = Header>,
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
+ Unpin
+ 'static,
Types::Engine: PayloadTypes<
BuiltPayload = EthBuiltPayload,
PayloadAttributes = EthPayloadAttributes,
@ -280,7 +282,9 @@ impl<Types, Node, Pool> PayloadServiceBuilder<Node, Pool> for EthereumPayloadBui
where
Types: NodeTypesWithEngine<ChainSpec = ChainSpec, Primitives = EthPrimitives>,
Node: FullNodeTypes<Types = Types>,
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
+ Unpin
+ 'static,
Types::Engine: PayloadTypes<
BuiltPayload = EthBuiltPayload,
PayloadAttributes = EthPayloadAttributes,
@ -305,7 +309,9 @@ pub struct EthereumNetworkBuilder {
impl<Node, Pool> NetworkBuilder<Node, Pool> for EthereumNetworkBuilder
where
Node: FullNodeTypes<Types: NodeTypes<ChainSpec = ChainSpec, Primitives = EthPrimitives>>,
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
+ Unpin
+ 'static,
{
async fn build_network(
self,

View File

@ -31,12 +31,13 @@ use reth_payload_primitives::PayloadBuilderAttributes;
use reth_primitives::{
proofs::{self},
Block, BlockBody, BlockExt, EthereumHardforks, InvalidTransactionError, Receipt,
TransactionSigned,
};
use reth_provider::{ChainSpecProvider, StateProviderFactory};
use reth_revm::database::StateProviderDatabase;
use reth_transaction_pool::{
error::InvalidPoolTransactionError, noop::NoopTransactionPool, BestTransactions,
BestTransactionsAttributes, TransactionPool, ValidPoolTransaction,
BestTransactionsAttributes, PoolTransaction, TransactionPool, ValidPoolTransaction,
};
use reth_trie::HashedPostState;
use revm::{
@ -93,7 +94,7 @@ impl<EvmConfig, Pool, Client> PayloadBuilder<Pool, Client> for EthereumPayloadBu
where
EvmConfig: ConfigureEvm<Header = Header>,
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = ChainSpec>,
Pool: TransactionPool,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>,
{
type Attributes = EthPayloadBuilderAttributes;
type BuiltPayload = EthBuiltPayload;
@ -157,7 +158,7 @@ pub fn default_ethereum_payload<EvmConfig, Pool, Client, F>(
where
EvmConfig: ConfigureEvm<Header = Header>,
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = ChainSpec>,
Pool: TransactionPool,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>,
F: FnOnce(BestTransactionsAttributes) -> BestTransactionsIter<Pool>,
{
let BuildArguments { client, pool, mut cached_reads, config, cancel, best_payload } = args;
@ -244,7 +245,7 @@ where
}
// convert tx to a signed transaction
let tx = pool_tx.to_recovered_transaction();
let tx = pool_tx.to_consensus();
// There's only limited amount of blob space available per block, so we need to check if
// the EIP-4844 can still fit in the block

View File

@ -30,7 +30,8 @@ use reth_execution_types::Chain;
use reth_exex::{ExExContext, ExExEvent, ExExNotification, ExExNotifications, Wal};
use reth_network::{config::SecretKey, NetworkConfigBuilder, NetworkManager};
use reth_node_api::{
FullNodeTypes, FullNodeTypesAdapter, NodeTypes, NodeTypesWithDBAdapter, NodeTypesWithEngine,
FullNodeTypes, FullNodeTypesAdapter, NodePrimitives, NodeTypes, NodeTypesWithDBAdapter,
NodeTypesWithEngine,
};
use reth_node_builder::{
components::{
@ -45,7 +46,7 @@ use reth_node_ethereum::{
EthEngineTypes, EthEvmConfig,
};
use reth_payload_builder::noop::NoopPayloadBuilderService;
use reth_primitives::{BlockExt, EthPrimitives, Head, SealedBlockWithSenders};
use reth_primitives::{BlockExt, EthPrimitives, Head, SealedBlockWithSenders, TransactionSigned};
use reth_provider::{
providers::{BlockchainProvider, StaticFileProvider},
BlockReader, EthStorage, ProviderFactory,
@ -64,7 +65,7 @@ pub struct TestPoolBuilder;
impl<Node> PoolBuilder<Node> for TestPoolBuilder
where
Node: FullNodeTypes,
Node: FullNodeTypes<Types: NodeTypes<Primitives: NodePrimitives<SignedTx = TransactionSigned>>>,
{
type Pool = TestPool;

View File

@ -17,6 +17,7 @@ use reth_network_api::{
NetworkEvent, NetworkEventListenerProvider, NetworkInfo, Peers,
};
use reth_network_peers::PeerId;
use reth_primitives::TransactionSigned;
use reth_provider::{test_utils::NoopProvider, ChainSpecProvider};
use reth_storage_api::{BlockReader, BlockReaderIdExt, HeaderProvider, StateProviderFactory};
use reth_tasks::TokioTaskExecutor;
@ -24,7 +25,7 @@ use reth_tokio_util::EventStream;
use reth_transaction_pool::{
blobstore::InMemoryBlobStore,
test_utils::{TestPool, TestPoolBuilder},
EthTransactionPool, TransactionPool, TransactionValidationTaskExecutor,
EthTransactionPool, PoolTransaction, TransactionPool, TransactionValidationTaskExecutor,
};
use secp256k1::SecretKey;
use std::{
@ -202,7 +203,9 @@ where
+ Clone
+ Unpin
+ 'static,
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>
+ Unpin
+ 'static,
{
/// Spawns the testnet to a separate task
pub fn spawn(self) -> TestnetHandle<C, Pool> {
@ -267,7 +270,9 @@ where
> + HeaderProvider
+ Unpin
+ 'static,
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>
+ Unpin
+ 'static,
{
type Output = ();
@ -468,7 +473,9 @@ where
> + HeaderProvider
+ Unpin
+ 'static,
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>
+ Unpin
+ 'static,
{
type Output = ();

View File

@ -702,9 +702,8 @@ where
BroadcastedTransaction: SignedTransaction,
PooledTransaction: SignedTransaction,
>,
<<Pool as TransactionPool>::Transaction as PoolTransaction>::Consensus:
Into<N::BroadcastedTransaction>,
<<Pool as TransactionPool>::Transaction as PoolTransaction>::Pooled: Into<N::PooledTransaction>,
Pool::Transaction:
PoolTransaction<Consensus = N::BroadcastedTransaction, Pooled: Into<N::PooledTransaction>>,
{
/// Invoked when transactions in the local mempool are considered __pending__.
///
@ -1011,9 +1010,8 @@ where
impl<Pool> TransactionsManager<Pool>
where
Pool: TransactionPool + 'static,
<<Pool as TransactionPool>::Transaction as PoolTransaction>::Consensus: Into<TransactionSigned>,
<<Pool as TransactionPool>::Transaction as PoolTransaction>::Pooled:
Into<PooledTransactionsElement>,
Pool::Transaction:
PoolTransaction<Consensus = TransactionSigned, Pooled: Into<PooledTransactionsElement>>,
{
/// Handles dedicated transaction events related to the `eth` protocol.
fn on_network_tx_event(&mut self, event: NetworkTransactionEvent) {
@ -1313,9 +1311,8 @@ where
impl<Pool> Future for TransactionsManager<Pool>
where
Pool: TransactionPool + Unpin + 'static,
<<Pool as TransactionPool>::Transaction as PoolTransaction>::Consensus: Into<TransactionSigned>,
<<Pool as TransactionPool>::Transaction as PoolTransaction>::Pooled:
Into<PooledTransactionsElement>,
Pool::Transaction:
PoolTransaction<Consensus = TransactionSigned, Pooled: Into<PooledTransactionsElement>>,
{
type Output = ();
@ -1503,11 +1500,11 @@ impl<T: SignedTransaction> PropagateTransaction<T> {
/// Create a new instance from a pooled transaction
fn new<P>(tx: Arc<ValidPoolTransaction<P>>) -> Self
where
P: PoolTransaction<Consensus: Into<T>>,
P: PoolTransaction<Consensus = T>,
{
let size = tx.encoded_length();
let transaction = tx.transaction.clone_into_consensus().into();
let transaction = Arc::new(transaction);
let transaction = tx.transaction.clone_into_consensus();
let transaction = Arc::new(transaction.into_signed());
Self { size, transaction }
}

View File

@ -8,11 +8,11 @@ use reth_consensus::FullConsensus;
use reth_evm::execute::BlockExecutorProvider;
use reth_network_api::FullNetwork;
use reth_node_core::node_config::NodeConfig;
use reth_node_types::{NodeTypes, NodeTypesWithDB, NodeTypesWithEngine};
use reth_node_types::{NodeTypes, NodeTypesWithDB, NodeTypesWithEngine, TxTy};
use reth_payload_builder_primitives::PayloadBuilder;
use reth_provider::FullProvider;
use reth_tasks::TaskExecutor;
use reth_transaction_pool::TransactionPool;
use reth_transaction_pool::{PoolTransaction, TransactionPool};
use std::{future::Future, marker::PhantomData};
/// A helper trait that is downstream of the [`NodeTypesWithEngine`] trait and adds stateful
@ -47,7 +47,7 @@ where
/// Encapsulates all types and components of the node.
pub trait FullNodeComponents: FullNodeTypes + Clone + 'static {
/// The transaction pool of the node.
type Pool: TransactionPool + Unpin;
type Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Self::Types>>> + Unpin;
/// The node's EVM configuration, defining settings for the Ethereum Virtual Machine.
type Evm: ConfigureEvm<Header = Header>;

View File

@ -37,7 +37,7 @@ use reth_provider::{
BlockReader, ChainSpecProvider, FullProvider,
};
use reth_tasks::TaskExecutor;
use reth_transaction_pool::{PoolConfig, TransactionPool};
use reth_transaction_pool::{PoolConfig, PoolTransaction, TransactionPool};
use revm_primitives::EnvKzgSettings;
use secp256k1::SecretKey;
use std::sync::Arc;
@ -650,7 +650,10 @@ impl<Node: FullNodeTypes> BuilderContext<Node> {
/// connected to that network.
pub fn start_network<Pool>(&self, builder: NetworkBuilder<(), ()>, pool: Pool) -> NetworkHandle
where
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<
Transaction: PoolTransaction<Consensus = reth_primitives::TransactionSigned>,
> + Unpin
+ 'static,
Node::Provider: BlockReader<
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
@ -673,7 +676,10 @@ impl<Node: FullNodeTypes> BuilderContext<Node> {
tx_config: TransactionsManagerConfig,
) -> NetworkHandle
where
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<
Transaction: PoolTransaction<Consensus = reth_primitives::TransactionSigned>,
> + Unpin
+ 'static,
Node::Provider: BlockReader<
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,

View File

@ -10,9 +10,9 @@ use crate::{
use alloy_consensus::Header;
use reth_consensus::FullConsensus;
use reth_evm::execute::BlockExecutorProvider;
use reth_node_api::{NodeTypes, NodeTypesWithEngine};
use reth_node_api::{NodeTypes, NodeTypesWithEngine, TxTy};
use reth_payload_builder::PayloadBuilderHandle;
use reth_transaction_pool::TransactionPool;
use reth_transaction_pool::{PoolTransaction, TransactionPool};
use std::{future::Future, marker::PhantomData};
/// A generic, general purpose and customizable [`NodeComponentsBuilder`] implementation.
@ -375,7 +375,9 @@ where
Node: FullNodeTypes,
F: FnOnce(&BuilderContext<Node>) -> Fut + Send,
Fut: Future<Output = eyre::Result<Components<Node, Pool, EVM, Executor, Cons>>> + Send,
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
+ Unpin
+ 'static,
EVM: ConfigureEvm<Header = Header>,
Executor: BlockExecutorProvider<Primitives = <Node::Types as NodeTypes>::Primitives>,
Cons: FullConsensus<<Node::Types as NodeTypes>::Primitives> + Clone + Unpin + 'static,

View File

@ -27,9 +27,9 @@ use reth_consensus::FullConsensus;
use reth_evm::execute::BlockExecutorProvider;
use reth_network::NetworkHandle;
use reth_network_api::FullNetwork;
use reth_node_api::{NodeTypes, NodeTypesWithEngine};
use reth_node_api::{NodeTypes, NodeTypesWithEngine, TxTy};
use reth_payload_builder::PayloadBuilderHandle;
use reth_transaction_pool::TransactionPool;
use reth_transaction_pool::{PoolTransaction, TransactionPool};
/// An abstraction over the components of a node, consisting of:
/// - evm and executor
@ -38,7 +38,7 @@ use reth_transaction_pool::TransactionPool;
/// - payload builder.
pub trait NodeComponents<T: FullNodeTypes>: Clone + Unpin + Send + Sync + 'static {
/// The transaction pool of the node.
type Pool: TransactionPool + Unpin;
type Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<T::Types>>> + Unpin;
/// The node's EVM configuration, defining settings for the Ethereum Virtual Machine.
type Evm: ConfigureEvm<Header = Header>;
@ -97,7 +97,9 @@ impl<Node, Pool, EVM, Executor, Cons> NodeComponents<Node>
for Components<Node, Pool, EVM, Executor, Cons>
where
Node: FullNodeTypes,
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
+ Unpin
+ 'static,
EVM: ConfigureEvm<Header = Header>,
Executor: BlockExecutorProvider<Primitives = <Node::Types as NodeTypes>::Primitives>,
Cons: FullConsensus<<Node::Types as NodeTypes>::Primitives> + Clone + Unpin + 'static,

View File

@ -1,7 +1,8 @@
//! Pool component for the node builder.
use alloy_primitives::Address;
use reth_transaction_pool::{PoolConfig, SubPoolLimit, TransactionPool};
use reth_node_api::TxTy;
use reth_transaction_pool::{PoolConfig, PoolTransaction, SubPoolLimit, TransactionPool};
use std::{collections::HashSet, future::Future};
use crate::{BuilderContext, FullNodeTypes};
@ -9,7 +10,9 @@ use crate::{BuilderContext, FullNodeTypes};
/// A type that knows how to build the transaction pool.
pub trait PoolBuilder<Node: FullNodeTypes>: Send {
/// The transaction pool to build.
type Pool: TransactionPool + Unpin + 'static;
type Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
+ Unpin
+ 'static;
/// Creates the transaction pool.
fn build_pool(
@ -21,7 +24,9 @@ pub trait PoolBuilder<Node: FullNodeTypes>: Send {
impl<Node, F, Fut, Pool> PoolBuilder<Node> for F
where
Node: FullNodeTypes,
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
+ Unpin
+ 'static,
F: FnOnce(&BuilderContext<Node>) -> Fut + Send,
Fut: Future<Output = eyre::Result<Pool>> + Send,
{

View File

@ -33,6 +33,7 @@ use reth_rpc_builder::{
use reth_rpc_engine_api::{capabilities::EngineCapabilities, EngineApi};
use reth_tasks::TaskExecutor;
use reth_tracing::tracing::{debug, info};
use reth_transaction_pool::TransactionPool;
use crate::EthApiBuilderCtx;
@ -405,6 +406,7 @@ where
N: FullNodeComponents<
Types: ProviderNodeTypes<Primitives = EthPrimitives>,
PayloadBuilder: PayloadBuilder<PayloadType = <N::Types as NodeTypesWithEngine>::Engine>,
Pool: TransactionPool<Transaction = <EthApi::Pool as TransactionPool>::Transaction>,
>,
EthApi: EthApiTypes + FullEthApiServer + AddDevSigners + Unpin + 'static,
EV: EngineValidatorBuilder<N>,
@ -527,6 +529,7 @@ where
N: FullNodeComponents<
Types: ProviderNodeTypes<Primitives = EthPrimitives>,
PayloadBuilder: PayloadBuilder<PayloadType = <N::Types as NodeTypesWithEngine>::Engine>,
Pool: TransactionPool<Transaction = <EthApi::Pool as TransactionPool>::Transaction>,
>,
EthApi: EthApiTypes + FullEthApiServer + AddDevSigners + Unpin + 'static,
EV: EngineValidatorBuilder<N>,

View File

@ -13,7 +13,7 @@ use reth_db::transaction::{DbTx, DbTxMut};
use reth_evm::{execute::BasicBlockExecutorProvider, ConfigureEvm};
use reth_network::{NetworkConfig, NetworkHandle, NetworkManager, PeersInfo};
use reth_node_api::{
AddOnsContext, EngineValidator, FullNodeComponents, NodeAddOns, PayloadBuilder,
AddOnsContext, EngineValidator, FullNodeComponents, NodeAddOns, PayloadBuilder, TxTy,
};
use reth_node_builder::{
components::{
@ -42,7 +42,7 @@ use reth_provider::{
use reth_rpc_server_types::RethRpcModule;
use reth_tracing::tracing::{debug, info};
use reth_transaction_pool::{
blobstore::DiskFileBlobStore, CoinbaseTipOrdering, TransactionPool,
blobstore::DiskFileBlobStore, CoinbaseTipOrdering, PoolTransaction, TransactionPool,
TransactionValidationTaskExecutor,
};
use reth_trie_db::MerklePatriciaTrie;
@ -465,7 +465,9 @@ where
Primitives = OpPrimitives,
>,
>,
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
+ Unpin
+ 'static,
Evm: ConfigureEvm<Header = Header>,
{
let payload_builder = reth_optimism_payload_builder::OpPayloadBuilder::new(evm_config)
@ -505,7 +507,9 @@ where
Primitives = OpPrimitives,
>,
>,
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
+ Unpin
+ 'static,
Txs: OpPayloadTransactions,
{
async fn spawn_payload_service(
@ -577,7 +581,9 @@ impl OpNetworkBuilder {
impl<Node, Pool> NetworkBuilder<Node, Pool> for OpNetworkBuilder
where
Node: FullNodeTypes<Types: NodeTypes<ChainSpec = OpChainSpec, Primitives = OpPrimitives>>,
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TxTy<Node::Types>>>
+ Unpin
+ 'static,
{
async fn build_network(
self,

View File

@ -70,7 +70,7 @@ impl<Client, Tx> OpTransactionValidator<Client, Tx> {
impl<Client, Tx> OpTransactionValidator<Client, Tx>
where
Client: StateProviderFactory + BlockReaderIdExt<Block = reth_primitives::Block>,
Tx: EthPoolTransaction,
Tx: EthPoolTransaction<Consensus = TransactionSigned>,
{
/// Create a new [`OpTransactionValidator`].
pub fn new(inner: EthTransactionValidator<Client, Tx>) -> Self {
@ -142,7 +142,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());
let tx: TransactionSigned = valid_tx.transaction().clone_into_consensus().into();
let tx = valid_tx.transaction().clone_into_consensus();
tx.encode_2718(&mut encoded);
let cost_addition = match l1_block_info.l1_tx_data_fee(
@ -196,7 +196,7 @@ where
impl<Client, Tx> TransactionValidator for OpTransactionValidator<Client, Tx>
where
Client: StateProviderFactory + BlockReaderIdExt<Block = reth_primitives::Block>,
Tx: EthPoolTransaction,
Tx: EthPoolTransaction<Consensus = TransactionSigned>,
{
type Transaction = Tx;

View File

@ -30,7 +30,7 @@ use reth_payload_util::{PayloadTransactions, PayloadTransactionsChain, PayloadTr
use reth_primitives::{RecoveredTx, SealedBlock, Transaction, TransactionSigned};
use reth_provider::providers::BlockchainProvider2;
use reth_tasks::TaskManager;
use reth_transaction_pool::pool::BestPayloadTransactions;
use reth_transaction_pool::{pool::BestPayloadTransactions, PoolTransaction};
use std::sync::Arc;
use tokio::sync::Mutex;
@ -44,9 +44,11 @@ impl OpPayloadTransactions for CustomTxPriority {
&self,
pool: Pool,
attr: reth_transaction_pool::BestTransactionsAttributes,
) -> impl PayloadTransactions
) -> impl PayloadTransactions<Transaction = TransactionSigned>
where
Pool: reth_transaction_pool::TransactionPool,
Pool: reth_transaction_pool::TransactionPool<
Transaction: PoolTransaction<Consensus = TransactionSigned>,
>,
{
// Block composition:
// 1. Best transactions from the pool (up to 250k gas)

View File

@ -19,12 +19,13 @@ use reth_payload_builder_primitives::PayloadBuilderError;
use reth_payload_primitives::PayloadBuilderAttributes;
use reth_payload_util::PayloadTransactions;
use reth_primitives::{
proofs, Block, BlockBody, BlockExt, Receipt, SealedHeader, TransactionSigned, TxType,
proofs, transaction::SignedTransactionIntoRecoveredExt, Block, BlockBody, BlockExt, Receipt,
SealedHeader, TransactionSigned, TxType,
};
use reth_provider::{ProviderError, StateProofProvider, StateProviderFactory, StateRootProvider};
use reth_revm::database::StateProviderDatabase;
use reth_transaction_pool::{
noop::NoopTransactionPool, BestTransactionsAttributes, TransactionPool,
noop::NoopTransactionPool, BestTransactionsAttributes, PoolTransaction, TransactionPool,
};
use reth_trie::HashedPostState;
use revm::{
@ -112,7 +113,7 @@ where
) -> Result<BuildOutcome<OpBuiltPayload>, PayloadBuilderError>
where
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = OpChainSpec>,
Pool: TransactionPool,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>,
{
let (initialized_cfg, initialized_block_env) = self
.cfg_and_block_env(&args.config.attributes, &args.config.parent_header)
@ -213,7 +214,7 @@ where
impl<Pool, Client, EvmConfig, Txs> PayloadBuilder<Pool, Client> for OpPayloadBuilder<EvmConfig, Txs>
where
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = OpChainSpec>,
Pool: TransactionPool,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>,
EvmConfig: ConfigureEvm<Header = Header>,
Txs: OpPayloadTransactions,
{
@ -281,7 +282,7 @@ pub struct OpBuilder<Pool, Txs> {
impl<Pool, Txs> OpBuilder<Pool, Txs>
where
Pool: TransactionPool,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>,
Txs: OpPayloadTransactions,
{
/// Executes the payload and returns the outcome.
@ -479,19 +480,23 @@ where
pub trait OpPayloadTransactions: Clone + Send + Sync + Unpin + 'static {
/// Returns an iterator that yields the transaction in the order they should get included in the
/// new payload.
fn best_transactions<Pool: TransactionPool>(
fn best_transactions<
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>,
>(
&self,
pool: Pool,
attr: BestTransactionsAttributes,
) -> impl PayloadTransactions;
) -> impl PayloadTransactions<Transaction = TransactionSigned>;
}
impl OpPayloadTransactions for () {
fn best_transactions<Pool: TransactionPool>(
fn best_transactions<
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>,
>(
&self,
pool: Pool,
attr: BestTransactionsAttributes,
) -> impl PayloadTransactions {
) -> impl PayloadTransactions<Transaction = TransactionSigned> {
BestPayloadTransactions::new(pool.best_transactions_with_attributes(attr))
}
}
@ -830,11 +835,10 @@ where
&self,
info: &mut ExecutionInfo,
db: &mut State<DB>,
mut best_txs: impl PayloadTransactions,
mut best_txs: impl PayloadTransactions<Transaction = TransactionSigned>,
) -> Result<Option<()>, PayloadBuilderError>
where
DB: Database<Error = ProviderError>,
Pool: TransactionPool,
{
let block_gas_limit = self.block_gas_limit();
let base_fee = self.base_fee();

View File

@ -9,7 +9,7 @@ use reth_evm::ConfigureEvm;
use reth_optimism_consensus::calculate_receipt_root_no_memo_optimism;
use reth_primitives::{Receipt, SealedBlockWithSenders};
use reth_provider::{
BlockReader, BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, ExecutionOutcome,
BlockReader, BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, ExecutionOutcome, ProviderTx,
ReceiptProvider, StateProviderFactory,
};
use reth_rpc_eth_api::{
@ -17,7 +17,7 @@ use reth_rpc_eth_api::{
FromEthApiError, RpcNodeCore,
};
use reth_rpc_eth_types::{EthApiError, PendingBlock};
use reth_transaction_pool::TransactionPool;
use reth_transaction_pool::{PoolTransaction, TransactionPool};
use revm::primitives::BlockEnv;
impl<N> LoadPendingBlock for OpEthApi<N>
@ -25,13 +25,14 @@ where
Self: SpawnBlocking,
N: RpcNodeCore<
Provider: BlockReaderIdExt<
Transaction = reth_primitives::TransactionSigned,
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
Header = reth_primitives::Header,
> + EvmEnvProvider
+ ChainSpecProvider<ChainSpec: EthChainSpec + EthereumHardforks>
+ StateProviderFactory,
Pool: TransactionPool,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = ProviderTx<N::Provider>>>,
Evm: ConfigureEvm<Header = Header>,
>,
{

View File

@ -7,12 +7,15 @@ use reth_primitives::RecoveredTx;
/// Can include transactions from the pool and other sources (alternative pools,
/// sequencer-originated transactions, etc.).
pub trait PayloadTransactions {
/// The transaction type this iterator yields.
type Transaction;
/// Returns the next transaction to include in the block.
fn next(
&mut self,
// In the future, `ctx` can include access to state for block building purposes.
ctx: (),
) -> Option<RecoveredTx>;
) -> Option<RecoveredTx<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.

View File

@ -26,8 +26,10 @@ impl<T> PayloadTransactionsFixed<T> {
}
}
impl PayloadTransactions for PayloadTransactionsFixed<RecoveredTx> {
fn next(&mut self, _ctx: ()) -> Option<RecoveredTx> {
impl<T: Clone> PayloadTransactions for PayloadTransactionsFixed<RecoveredTx<T>> {
type Transaction = T;
fn next(&mut self, _ctx: ()) -> Option<RecoveredTx<T>> {
(self.index < self.transactions.len()).then(|| {
let tx = self.transactions[self.index].clone();
self.index += 1;
@ -87,20 +89,22 @@ impl<B: PayloadTransactions, A: PayloadTransactions> PayloadTransactionsChain<B,
}
}
impl<B, A> PayloadTransactions for PayloadTransactionsChain<B, A>
impl<A, B> PayloadTransactions for PayloadTransactionsChain<A, B>
where
B: PayloadTransactions,
A: PayloadTransactions,
A: PayloadTransactions<Transaction: Transaction>,
B: PayloadTransactions<Transaction = A::Transaction>,
{
fn next(&mut self, ctx: ()) -> Option<RecoveredTx> {
type Transaction = A::Transaction;
fn next(&mut self, ctx: ()) -> Option<RecoveredTx<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.transaction.gas_limit() <= before_max_gas {
self.before_gas += tx.transaction.gas_limit();
if self.before_gas + tx.as_signed().gas_limit() <= before_max_gas {
self.before_gas += tx.as_signed().gas_limit();
return Some(tx);
}
self.before.mark_invalid(tx.signer(), tx.transaction.nonce());
self.after.mark_invalid(tx.signer(), tx.transaction.nonce());
self.before.mark_invalid(tx.signer(), tx.as_signed().nonce());
self.after.mark_invalid(tx.signer(), tx.as_signed().nonce());
} else {
return Some(tx);
}
@ -108,11 +112,11 @@ where
while let Some(tx) = self.after.next(ctx) {
if let Some(after_max_gas) = self.after_max_gas {
if self.after_gas + tx.transaction.gas_limit() <= after_max_gas {
self.after_gas += tx.transaction.gas_limit();
if self.after_gas + tx.as_signed().gas_limit() <= after_max_gas {
self.after_gas += tx.as_signed().gas_limit();
return Some(tx);
}
self.after.mark_invalid(tx.signer(), tx.transaction.nonce());
self.after.mark_invalid(tx.signer(), tx.as_signed().nonce());
} else {
return Some(tx);
}

View File

@ -1130,23 +1130,6 @@ impl TransactionSigned {
Some(RecoveredTx { signed_transaction: self, signer })
}
/// Tries to recover signer and return [`RecoveredTx`] by cloning the type.
pub fn try_ecrecovered(&self) -> Option<RecoveredTx> {
let signer = self.recover_signer()?;
Some(RecoveredTx { signed_transaction: self.clone(), signer })
}
/// Tries to recover signer and return [`RecoveredTx`].
///
/// Returns `Err(Self)` if the transaction's signature is invalid, see also
/// [`Self::recover_signer`].
pub fn try_into_ecrecovered(self) -> Result<RecoveredTx, Self> {
match self.recover_signer() {
None => Err(self),
Some(signer) => Ok(RecoveredTx { signed_transaction: self, signer }),
}
}
/// Tries to recover signer and return [`RecoveredTx`]. _without ensuring that
/// the signature has a low `s` value_ (EIP-2).
///
@ -1707,6 +1690,23 @@ impl<T: Encodable2718> Encodable2718 for RecoveredTx<T> {
/// Extension trait for [`SignedTransaction`] to convert it into [`RecoveredTx`].
pub trait SignedTransactionIntoRecoveredExt: SignedTransaction {
/// Tries to recover signer and return [`RecoveredTx`] by cloning the type.
fn try_ecrecovered(&self) -> Option<RecoveredTx<Self>> {
let signer = self.recover_signer()?;
Some(RecoveredTx { signed_transaction: self.clone(), signer })
}
/// Tries to recover signer and return [`RecoveredTx`].
///
/// Returns `Err(Self)` if the transaction's signature is invalid, see also
/// [`SignedTransaction::recover_signer`].
fn try_into_ecrecovered(self) -> Result<RecoveredTx<Self>, Self> {
match self.recover_signer() {
None => Err(self),
Some(signer) => Ok(RecoveredTx { signed_transaction: self, signer }),
}
}
/// Consumes the type, recover signer and return [`RecoveredTx`] _without
/// ensuring that the signature has a low `s` value_ (EIP-2).
///

View File

@ -26,7 +26,7 @@
//! RethRpcModule, RpcModuleBuilder, RpcServerConfig, ServerBuilder, TransportRpcModuleConfig,
//! };
//! use reth_tasks::TokioTaskExecutor;
//! use reth_transaction_pool::TransactionPool;
//! use reth_transaction_pool::{PoolTransaction, TransactionPool};
//!
//! pub async fn launch<Provider, Pool, Network, Events, EvmConfig, BlockExecutor, Consensus>(
//! provider: Provider,
@ -44,7 +44,9 @@
//! Header = reth_primitives::Header,
//! > + AccountReader
//! + ChangeSetReader,
//! Pool: TransactionPool + Unpin + 'static,
//! Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>
//! + Unpin
//! + 'static,
//! Network: NetworkInfo + Peers + Clone + 'static,
//! Events:
//! CanonStateSubscriptions<Primitives = reth_primitives::EthPrimitives> + Clone + 'static,
@ -95,7 +97,7 @@
//! };
//! use reth_rpc_layer::JwtSecret;
//! use reth_tasks::TokioTaskExecutor;
//! use reth_transaction_pool::TransactionPool;
//! use reth_transaction_pool::{PoolTransaction, TransactionPool};
//! use tokio::try_join;
//!
//! pub async fn launch<
@ -125,7 +127,9 @@
//! Header = reth_primitives::Header,
//! > + AccountReader
//! + ChangeSetReader,
//! Pool: TransactionPool + Unpin + 'static,
//! Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>
//! + Unpin
//! + 'static,
//! Network: NetworkInfo + Peers + Clone + 'static,
//! Events:
//! CanonStateSubscriptions<Primitives = reth_primitives::EthPrimitives> + Clone + 'static,
@ -277,7 +281,7 @@ where
Header = reth_primitives::Header,
> + AccountReader
+ ChangeSetReader,
Pool: TransactionPool + 'static,
Pool: TransactionPool<Transaction = <EthApi::Pool as TransactionPool>::Transaction> + 'static,
Network: NetworkInfo + Peers + Clone + 'static,
Tasks: TaskSpawner + Clone + 'static,
Events: CanonStateSubscriptions<Primitives = EthPrimitives> + Clone + 'static,
@ -674,6 +678,7 @@ where
Receipt = <EthApi::Provider as ReceiptProvider>::Receipt,
Header = <EthApi::Provider as HeaderProvider>::Header,
>,
Pool: TransactionPool<Transaction = <EthApi::Pool as TransactionPool>::Transaction>,
{
let Self {
provider,
@ -793,6 +798,7 @@ where
Receipt = <EthApi::Provider as ReceiptProvider>::Receipt,
Header = <EthApi::Provider as HeaderProvider>::Header,
>,
Pool: TransactionPool<Transaction = <EthApi::Pool as TransactionPool>::Transaction>,
{
let mut modules = TransportRpcModules::default();
@ -1328,7 +1334,7 @@ where
Header = <EthApi::Provider as HeaderProvider>::Header,
> + AccountReader
+ ChangeSetReader,
Pool: TransactionPool + 'static,
Pool: TransactionPool<Transaction = <EthApi::Pool as TransactionPool>::Transaction> + 'static,
Network: NetworkInfo + Peers + Clone + 'static,
Tasks: TaskSpawner + Clone + 'static,
Events: CanonStateSubscriptions<Primitives = EthPrimitives> + Clone + 'static,

View File

@ -22,7 +22,7 @@ use reth_primitives::{
};
use reth_provider::{
BlockReader, BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, ProviderError,
ProviderReceipt, ReceiptProvider, StateProviderFactory,
ProviderReceipt, ProviderTx, ReceiptProvider, StateProviderFactory,
};
use reth_revm::{
database::StateProviderDatabase,
@ -33,7 +33,8 @@ use reth_revm::{
};
use reth_rpc_eth_types::{EthApiError, PendingBlock, PendingBlockEnv, PendingBlockEnvOrigin};
use reth_transaction_pool::{
error::InvalidPoolTransactionError, BestTransactionsAttributes, TransactionPool,
error::InvalidPoolTransactionError, BestTransactionsAttributes, PoolTransaction,
TransactionPool,
};
use reth_trie::HashedPostState;
use revm::{db::states::bundle_state::BundleRetention, DatabaseCommit, State};
@ -48,13 +49,14 @@ pub trait LoadPendingBlock:
EthApiTypes
+ RpcNodeCore<
Provider: BlockReaderIdExt<
Transaction = reth_primitives::TransactionSigned,
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
Header = reth_primitives::Header,
> + EvmEnvProvider
+ ChainSpecProvider<ChainSpec: EthChainSpec + EthereumHardforks>
+ StateProviderFactory,
Pool: TransactionPool,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = ProviderTx<Self::Provider>>>,
Evm: ConfigureEvm<Header = Header>,
>
{
@ -319,7 +321,7 @@ pub trait LoadPendingBlock:
}
// convert tx to a signed transaction
let tx = pool_tx.to_recovered_transaction();
let tx = pool_tx.to_consensus();
// There's only limited amount of blob space available per block, so we need to check if
// the EIP-4844 can still fit in the block

View File

@ -240,7 +240,7 @@ pub trait EthTransactions: LoadTransaction<Provider: BlockReaderIdExt> {
RpcNodeCore::pool(self).get_transaction_by_sender_and_nonce(sender, nonce)
{
let transaction = tx.transaction.clone_into_consensus();
return Ok(Some(from_recovered(transaction.into(), self.tx_resp_builder())?));
return Ok(Some(from_recovered(transaction, self.tx_resp_builder())?));
}
}
@ -385,7 +385,7 @@ pub trait EthTransactions: LoadTransaction<Provider: BlockReaderIdExt> {
let pool_transaction =
<<Self as RpcNodeCore>::Pool as TransactionPool>::Transaction::try_from_consensus(
transaction.into(),
transaction,
)
.map_err(|_| EthApiError::TransactionConversionError)?;

View File

@ -8,8 +8,9 @@ use std::{
use alloy_network::Network;
use alloy_rpc_types_eth::Block;
use reth_primitives::TransactionSigned;
use reth_provider::{ReceiptProvider, TransactionsProvider};
use reth_provider::{ProviderTx, ReceiptProvider, TransactionsProvider};
use reth_rpc_types_compat::TransactionCompat;
use reth_transaction_pool::{PoolTransaction, TransactionPool};
use crate::{AsEthApiError, FromEthApiError, FromEvmError, RpcNodeCore};
@ -50,6 +51,9 @@ where
Self: RpcNodeCore<
Provider: TransactionsProvider<Transaction = TransactionSigned>
+ ReceiptProvider<Receipt = reth_primitives::Receipt>,
Pool: TransactionPool<
Transaction: PoolTransaction<Consensus = ProviderTx<Self::Provider>>,
>,
> + EthApiTypes<
TransactionCompat: TransactionCompat<
<Self::Provider as TransactionsProvider>::Transaction,
@ -64,6 +68,9 @@ impl<T> FullEthApiTypes for T where
T: RpcNodeCore<
Provider: TransactionsProvider<Transaction = TransactionSigned>
+ ReceiptProvider<Receipt = reth_primitives::Receipt>,
Pool: TransactionPool<
Transaction: PoolTransaction<Consensus = ProviderTx<Self::Provider>>,
>,
> + EthApiTypes<
TransactionCompat: TransactionCompat<
<Self::Provider as TransactionsProvider>::Transaction,

View File

@ -9,7 +9,7 @@ use alloy_rpc_types_eth::{
use async_trait::async_trait;
use jsonrpsee::{core::RpcResult, server::IdProvider};
use reth_chainspec::ChainInfo;
use reth_primitives::{Receipt, RecoveredTx, SealedBlockWithSenders};
use reth_primitives::{Receipt, SealedBlockWithSenders};
use reth_provider::{BlockIdReader, BlockReader, ProviderError};
use reth_rpc_eth_api::{
EthApiTypes, EthFilterApiServer, FullEthApiTypes, RpcTransaction, TransactionCompat,
@ -145,7 +145,7 @@ where
impl<Provider, Pool, Eth> EthFilter<Provider, Pool, Eth>
where
Provider: BlockReader + BlockIdReader + 'static,
Pool: TransactionPool<Transaction: 'static> + 'static,
Pool: TransactionPool<Transaction = <Eth::Pool as TransactionPool>::Transaction> + 'static,
Eth: FullEthApiTypes,
{
/// Returns all the filter changes for the given id, if any
@ -245,7 +245,7 @@ impl<Provider, Pool, Eth> EthFilterApiServer<RpcTransaction<Eth::NetworkTypes>>
for EthFilter<Provider, Pool, Eth>
where
Provider: BlockReader + BlockIdReader + 'static,
Pool: TransactionPool + 'static,
Pool: TransactionPool<Transaction = <Eth::Pool as TransactionPool>::Transaction> + 'static,
Eth: FullEthApiTypes + 'static,
{
/// Handler for `eth_newFilter`
@ -611,7 +611,7 @@ struct FullTransactionsReceiver<T: PoolTransaction, TxCompat> {
impl<T, TxCompat> FullTransactionsReceiver<T, TxCompat>
where
T: PoolTransaction + 'static,
TxCompat: TransactionCompat,
TxCompat: TransactionCompat<T::Consensus>,
{
/// Creates a new `FullTransactionsReceiver` encapsulating the provided transaction stream.
fn new(stream: NewSubpoolTransactionStream<T>, tx_resp_builder: TxCompat) -> Self {
@ -619,15 +619,12 @@ where
}
/// Returns all new pending transactions received since the last poll.
async fn drain(&self) -> FilterChanges<TxCompat::Transaction>
where
T: PoolTransaction<Consensus: Into<RecoveredTx>>,
{
async fn drain(&self) -> FilterChanges<TxCompat::Transaction> {
let mut pending_txs = Vec::new();
let mut prepared_stream = self.txs_stream.lock().await;
while let Ok(tx) = prepared_stream.try_recv() {
match from_recovered(tx.transaction.to_recovered_transaction(), &self.tx_resp_builder) {
match from_recovered(tx.transaction.to_consensus(), &self.tx_resp_builder) {
Ok(tx) => pending_txs.push(tx),
Err(err) => {
error!(target: "rpc",
@ -651,8 +648,8 @@ trait FullTransactionsFilter<T>: fmt::Debug + Send + Sync + Unpin + 'static {
impl<T, TxCompat> FullTransactionsFilter<TxCompat::Transaction>
for FullTransactionsReceiver<T, TxCompat>
where
T: PoolTransaction<Consensus: Into<RecoveredTx>> + 'static,
TxCompat: TransactionCompat + 'static,
T: PoolTransaction + 'static,
TxCompat: TransactionCompat<T::Consensus> + 'static,
{
async fn drain(&self) -> FilterChanges<TxCompat::Transaction> {
Self::drain(self).await

View File

@ -3,13 +3,15 @@
use alloy_consensus::Header;
use reth_chainspec::{EthChainSpec, EthereumHardforks};
use reth_evm::ConfigureEvm;
use reth_provider::{BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, StateProviderFactory};
use reth_provider::{
BlockReaderIdExt, ChainSpecProvider, EvmEnvProvider, ProviderTx, StateProviderFactory,
};
use reth_rpc_eth_api::{
helpers::{LoadPendingBlock, SpawnBlocking},
RpcNodeCore,
};
use reth_rpc_eth_types::PendingBlock;
use reth_transaction_pool::TransactionPool;
use reth_transaction_pool::{PoolTransaction, TransactionPool};
use crate::EthApi;
@ -19,13 +21,16 @@ where
Self: SpawnBlocking
+ RpcNodeCore<
Provider: BlockReaderIdExt<
Transaction = reth_primitives::TransactionSigned,
Block = reth_primitives::Block,
Receipt = reth_primitives::Receipt,
Header = reth_primitives::Header,
> + EvmEnvProvider
+ ChainSpecProvider<ChainSpec: EthChainSpec + EthereumHardforks>
+ StateProviderFactory,
Pool: TransactionPool,
Pool: TransactionPool<
Transaction: PoolTransaction<Consensus = ProviderTx<Self::Provider>>,
>,
Evm: ConfigureEvm<Header = Header>,
>,
{

View File

@ -23,7 +23,7 @@ use reth_rpc_eth_types::logs_utils;
use reth_rpc_server_types::result::{internal_rpc_err, invalid_params_rpc_err};
use reth_rpc_types_compat::transaction::from_recovered;
use reth_tasks::{TaskSpawner, TokioTaskExecutor};
use reth_transaction_pool::{NewTransactionEvent, TransactionPool};
use reth_transaction_pool::{NewTransactionEvent, PoolConsensusTx, TransactionPool};
use serde::Serialize;
use tokio_stream::{
wrappers::{BroadcastStream, ReceiverStream},
@ -95,7 +95,7 @@ where
> + Clone
+ 'static,
Network: NetworkInfo + Clone + 'static,
Eth: TransactionCompat + 'static,
Eth: TransactionCompat<PoolConsensusTx<Pool>> + 'static,
{
/// Handler for `eth_subscribe`
async fn subscribe(
@ -135,7 +135,7 @@ where
> + Clone
+ 'static,
Network: NetworkInfo + Clone + 'static,
Eth: TransactionCompat,
Eth: TransactionCompat<PoolConsensusTx<Pool>>,
{
match kind {
SubscriptionKind::NewHeads => {
@ -165,7 +165,7 @@ where
// full transaction objects requested
let stream = pubsub.full_pending_transaction_stream().filter_map(|tx| {
let tx_value = match from_recovered(
tx.transaction.to_recovered_transaction(),
tx.transaction.to_consensus(),
&tx_resp_builder,
) {
Ok(tx) => {

View File

@ -8,10 +8,11 @@ use alloy_rpc_types_txpool::{
};
use async_trait::async_trait;
use jsonrpsee::core::RpcResult;
use reth_primitives::RecoveredTx;
use reth_rpc_api::TxPoolApiServer;
use reth_rpc_types_compat::{transaction::from_recovered, TransactionCompat};
use reth_transaction_pool::{AllPoolTransactions, PoolTransaction, TransactionPool};
use reth_transaction_pool::{
AllPoolTransactions, PoolConsensusTx, PoolTransaction, TransactionPool,
};
use tracing::trace;
/// `txpool` API implementation.
@ -33,8 +34,8 @@ impl<Pool, Eth> TxPoolApi<Pool, Eth> {
impl<Pool, Eth> TxPoolApi<Pool, Eth>
where
Pool: TransactionPool + 'static,
Eth: TransactionCompat,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus: Transaction>> + 'static,
Eth: TransactionCompat<PoolConsensusTx<Pool>>,
{
fn content(&self) -> Result<TxpoolContent<Eth::Transaction>, Eth::Error> {
#[inline]
@ -44,12 +45,12 @@ where
resp_builder: &RpcTxB,
) -> Result<(), RpcTxB::Error>
where
Tx: PoolTransaction<Consensus: Into<RecoveredTx>>,
RpcTxB: TransactionCompat,
Tx: PoolTransaction,
RpcTxB: TransactionCompat<Tx::Consensus>,
{
content.entry(tx.sender()).or_default().insert(
tx.nonce().to_string(),
from_recovered(tx.clone_into_consensus().into(), resp_builder)?,
from_recovered(tx.clone_into_consensus(), resp_builder)?,
);
Ok(())
@ -72,8 +73,8 @@ where
#[async_trait]
impl<Pool, Eth> TxPoolApiServer<Eth::Transaction> for TxPoolApi<Pool, Eth>
where
Pool: TransactionPool + 'static,
Eth: TransactionCompat + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus: Transaction>> + 'static,
Eth: TransactionCompat<PoolConsensusTx<Pool>> + 'static,
{
/// Returns the number of transactions currently pending for inclusion in the next block(s), as
/// well as the ones that are being scheduled for future execution only.
@ -96,19 +97,19 @@ where
trace!(target: "rpc::eth", "Serving txpool_inspect");
#[inline]
fn insert<T: PoolTransaction<Consensus: Into<RecoveredTx>>>(
fn insert<T: PoolTransaction<Consensus: Transaction>>(
tx: &T,
inspect: &mut BTreeMap<Address, BTreeMap<String, TxpoolInspectSummary>>,
) {
let entry = inspect.entry(tx.sender()).or_default();
let tx: RecoveredTx = tx.clone_into_consensus().into();
let tx = tx.clone_into_consensus();
entry.insert(
tx.nonce().to_string(),
TxpoolInspectSummary {
to: tx.to(),
value: tx.value(),
gas: tx.gas_limit() as u128,
gas_price: tx.transaction.max_fee_per_gas(),
gas_price: tx.max_fee_per_gas(),
},
);
}

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 {

View File

@ -38,14 +38,14 @@ use reth::{
},
network::NetworkHandle,
payload::ExecutionPayloadValidator,
primitives::{Block, EthPrimitives, SealedBlockFor},
primitives::{Block, EthPrimitives, SealedBlockFor, TransactionSigned},
providers::{CanonStateSubscriptions, EthStorage, StateProviderFactory},
rpc::{
eth::EthApi,
types::engine::{ExecutionPayload, ExecutionPayloadSidecar, PayloadError},
},
tasks::TaskManager,
transaction_pool::TransactionPool,
transaction_pool::{PoolTransaction, TransactionPool},
};
use reth_basic_payload_builder::{
BasicPayloadJobGenerator, BasicPayloadJobGeneratorConfig, BuildArguments, BuildOutcome,
@ -340,7 +340,9 @@ where
Primitives = EthPrimitives,
>,
>,
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>
+ Unpin
+ 'static,
{
async fn spawn_payload_service(
self,
@ -380,7 +382,7 @@ pub struct CustomPayloadBuilder;
impl<Pool, Client> PayloadBuilder<Pool, Client> for CustomPayloadBuilder
where
Client: StateProviderFactory + ChainSpecProvider<ChainSpec = ChainSpec>,
Pool: TransactionPool,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>,
{
type Attributes = CustomPayloadBuilderAttributes;
type BuiltPayload = EthBuiltPayload;

View File

@ -20,7 +20,7 @@ use reth::{
},
rpc::types::engine::PayloadAttributes,
tasks::TaskManager,
transaction_pool::TransactionPool,
transaction_pool::{PoolTransaction, TransactionPool},
};
use reth_chainspec::{Chain, ChainSpec};
use reth_evm_ethereum::EthEvmConfig;
@ -183,7 +183,9 @@ impl<Types, Node, Pool> PayloadServiceBuilder<Node, Pool> for MyPayloadBuilder
where
Types: NodeTypesWithEngine<ChainSpec = ChainSpec, Primitives = EthPrimitives>,
Node: FullNodeTypes<Types = Types>,
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>
+ Unpin
+ 'static,
Types::Engine: PayloadTypes<
BuiltPayload = EthBuiltPayload,
PayloadAttributes = PayloadAttributes,

View File

@ -54,8 +54,7 @@ fn main() {
if let Some(recipient) = tx.to() {
if args.is_match(&recipient) {
// convert the pool transaction
let call_request =
transaction_to_call_request(tx.to_recovered_transaction());
let call_request = transaction_to_call_request(tx.to_consensus());
let result = eth_api
.spawn_with_call_at(

View File

@ -17,14 +17,14 @@ use reth::{
cli::{config::PayloadBuilderConfig, Cli},
payload::PayloadBuilderHandle,
providers::CanonStateSubscriptions,
transaction_pool::TransactionPool,
transaction_pool::{PoolTransaction, TransactionPool},
};
use reth_basic_payload_builder::BasicPayloadJobGeneratorConfig;
use reth_chainspec::ChainSpec;
use reth_node_api::NodeTypesWithEngine;
use reth_node_ethereum::{node::EthereumAddOns, EthEngineTypes, EthEvmConfig, EthereumNode};
use reth_payload_builder::PayloadBuilderService;
use reth_primitives::EthPrimitives;
use reth_primitives::{EthPrimitives, TransactionSigned};
pub mod generator;
pub mod job;
@ -42,7 +42,9 @@ where
Primitives = EthPrimitives,
>,
>,
Pool: TransactionPool + Unpin + 'static,
Pool: TransactionPool<Transaction: PoolTransaction<Consensus = TransactionSigned>>
+ Unpin
+ 'static,
{
async fn spawn_payload_service(
self,

View File

@ -44,8 +44,7 @@ fn main() {
if let Some(recipient) = tx.to() {
if args.is_match(&recipient) {
// trace the transaction with `trace_call`
let callrequest =
transaction_to_call_request(tx.to_recovered_transaction());
let callrequest = transaction_to_call_request(tx.to_consensus());
let tracerequest = TraceCallRequest::new(callrequest)
.with_trace_type(TraceType::Trace);
if let Ok(trace_result) = traceapi.trace_call(tracerequest).await {