test: TransactionGenerator and p2p testing support (#5023)

This commit is contained in:
Matthias Seitz
2023-10-14 23:13:36 +02:00
committed by GitHub
parent 3bf2c8865a
commit d58e4a46f8
11 changed files with 650 additions and 68 deletions

View File

@ -1,16 +1,24 @@
//! A network implementation for testing purposes. //! A network implementation for testing purposes.
use crate::{ use crate::{
builder::ETH_REQUEST_CHANNEL_CAPACITY, error::NetworkError, eth_requests::EthRequestHandler, builder::ETH_REQUEST_CHANNEL_CAPACITY,
transactions::TransactionsManager, NetworkConfig, NetworkConfigBuilder, NetworkEvent, error::NetworkError,
NetworkHandle, NetworkManager, eth_requests::EthRequestHandler,
transactions::{TransactionsHandle, TransactionsManager},
NetworkConfig, NetworkConfigBuilder, NetworkEvent, NetworkHandle, NetworkManager,
}; };
use futures::{FutureExt, StreamExt}; use futures::{FutureExt, StreamExt};
use pin_project::pin_project; use pin_project::pin_project;
use reth_eth_wire::{capability::Capability, DisconnectReason, HelloBuilder}; use reth_eth_wire::{capability::Capability, DisconnectReason, HelloBuilder};
use reth_primitives::PeerId; use reth_network_api::{NetworkInfo, Peers};
use reth_provider::{test_utils::NoopProvider, BlockReader, HeaderProvider}; use reth_primitives::{PeerId, MAINNET};
use reth_transaction_pool::test_utils::TestPool; use reth_provider::{test_utils::NoopProvider, BlockReader, HeaderProvider, StateProviderFactory};
use reth_tasks::TokioTaskExecutor;
use reth_transaction_pool::{
blobstore::InMemoryBlobStore,
test_utils::{testing_pool, TestPool},
EthTransactionPool, TransactionPool, TransactionValidationTaskExecutor,
};
use secp256k1::SecretKey; use secp256k1::SecretKey;
use std::{ use std::{
fmt, fmt,
@ -29,15 +37,14 @@ use tokio::{
use tokio_stream::wrappers::UnboundedReceiverStream; use tokio_stream::wrappers::UnboundedReceiverStream;
/// A test network consisting of multiple peers. /// A test network consisting of multiple peers.
#[derive(Default)] pub struct Testnet<C, Pool> {
pub struct Testnet<C> {
/// All running peers in the network. /// All running peers in the network.
peers: Vec<Peer<C>>, peers: Vec<Peer<C, Pool>>,
} }
// === impl Testnet === // === impl Testnet ===
impl<C> Testnet<C> impl<C> Testnet<C, TestPool>
where where
C: BlockReader + HeaderProvider + Clone, C: BlockReader + HeaderProvider + Clone,
{ {
@ -56,26 +63,6 @@ where
Ok(this) Ok(this)
} }
/// Return a mutable slice of all peers.
pub fn peers_mut(&mut self) -> &mut [Peer<C>] {
&mut self.peers
}
/// Return a slice of all peers.
pub fn peers(&self) -> &[Peer<C>] {
&self.peers
}
/// Return a mutable iterator over all peers.
pub fn peers_iter_mut(&mut self) -> impl Iterator<Item = &mut Peer<C>> + '_ {
self.peers.iter_mut()
}
/// Return an iterator over all peers.
pub fn peers_iter(&self) -> impl Iterator<Item = &Peer<C>> + '_ {
self.peers.iter()
}
/// Extend the list of peers with new peers that are configured with each of the given /// Extend the list of peers with new peers that are configured with each of the given
/// [`PeerConfig`]s. /// [`PeerConfig`]s.
pub async fn extend_peer_with_config( pub async fn extend_peer_with_config(
@ -89,6 +76,32 @@ where
} }
Ok(()) Ok(())
} }
}
impl<C, Pool> Testnet<C, Pool>
where
C: BlockReader + HeaderProvider + Clone,
Pool: TransactionPool,
{
/// Return a mutable slice of all peers.
pub fn peers_mut(&mut self) -> &mut [Peer<C, Pool>] {
&mut self.peers
}
/// Return a slice of all peers.
pub fn peers(&self) -> &[Peer<C, Pool>] {
&self.peers
}
/// Return a mutable iterator over all peers.
pub fn peers_iter_mut(&mut self) -> impl Iterator<Item = &mut Peer<C, Pool>> + '_ {
self.peers.iter_mut()
}
/// Return an iterator over all peers.
pub fn peers_iter(&self) -> impl Iterator<Item = &Peer<C, Pool>> + '_ {
self.peers.iter()
}
/// Add a peer to the [`Testnet`] with the given [`PeerConfig`]. /// Add a peer to the [`Testnet`] with the given [`PeerConfig`].
pub async fn add_peer_with_config( pub async fn add_peer_with_config(
@ -98,8 +111,14 @@ where
let PeerConfig { config, client, secret_key } = config; let PeerConfig { config, client, secret_key } = config;
let network = NetworkManager::new(config).await?; let network = NetworkManager::new(config).await?;
let peer = let peer = Peer {
Peer { network, client, secret_key, request_handler: None, transactions_manager: None }; network,
client,
secret_key,
request_handler: None,
transactions_manager: None,
pool: None,
};
self.peers.push(peer); self.peers.push(peer);
Ok(()) Ok(())
} }
@ -109,10 +128,19 @@ where
self.peers.iter().map(|p| p.handle()) self.peers.iter().map(|p| p.handle())
} }
/// Maps the pool of each peer with the given closure
pub fn map_pool<F, P>(self, f: F) -> Testnet<C, P>
where
F: Fn(Peer<C, Pool>) -> Peer<C, P>,
P: TransactionPool,
{
Testnet { peers: self.peers.into_iter().map(f).collect() }
}
/// Apply a closure on each peer /// Apply a closure on each peer
pub fn for_each<F>(&self, f: F) pub fn for_each<F>(&self, f: F)
where where
F: Fn(&Peer<C>), F: Fn(&Peer<C, Pool>),
{ {
self.peers.iter().for_each(f) self.peers.iter().for_each(f)
} }
@ -120,19 +148,45 @@ where
/// Apply a closure on each peer /// Apply a closure on each peer
pub fn for_each_mut<F>(&mut self, f: F) pub fn for_each_mut<F>(&mut self, f: F)
where where
F: FnMut(&mut Peer<C>), F: FnMut(&mut Peer<C, Pool>),
{ {
self.peers.iter_mut().for_each(f) self.peers.iter_mut().for_each(f)
} }
} }
impl<C> Testnet<C> impl<C, Pool> Testnet<C, Pool>
where where
C: BlockReader + HeaderProvider + Unpin + 'static, C: StateProviderFactory + BlockReader + HeaderProvider + Clone + 'static,
Pool: TransactionPool,
{
/// Installs an eth pool on each peer
pub fn with_eth_pool(self) -> Testnet<C, EthTransactionPool<C, InMemoryBlobStore>> {
self.map_pool(|peer| {
let blob_store = InMemoryBlobStore::default();
let pool = TransactionValidationTaskExecutor::eth(
peer.client.clone(),
MAINNET.clone(),
blob_store.clone(),
TokioTaskExecutor::default(),
);
peer.map_transactions_manager(EthTransactionPool::eth_pool(
pool,
blob_store,
Default::default(),
))
})
}
}
impl<C, Pool> Testnet<C, Pool>
where
C: BlockReader + HeaderProvider + Clone + Unpin + 'static,
Pool: TransactionPool + Unpin + 'static,
{ {
/// Spawns the testnet to a separate task /// Spawns the testnet to a separate task
pub fn spawn(self) -> TestnetHandle<C> { pub fn spawn(self) -> TestnetHandle<C, Pool> {
let (tx, rx) = oneshot::channel::<oneshot::Sender<Self>>(); let (tx, rx) = oneshot::channel::<oneshot::Sender<Self>>();
let peers = self.peers.iter().map(|peer| peer.peer_handle()).collect::<Vec<_>>();
let mut net = self; let mut net = self;
let handle = tokio::task::spawn(async move { let handle = tokio::task::spawn(async move {
let mut tx = None; let mut tx = None;
@ -147,11 +201,11 @@ where
} }
}); });
TestnetHandle { _handle: handle, terminate: tx } TestnetHandle { _handle: handle, peers, terminate: tx }
} }
} }
impl Testnet<NoopProvider> { impl Testnet<NoopProvider, TestPool> {
/// Same as [`Self::try_create`] but panics on error /// Same as [`Self::try_create`] but panics on error
pub async fn create(num_peers: usize) -> Self { pub async fn create(num_peers: usize) -> Self {
Self::try_create(num_peers).await.unwrap() Self::try_create(num_peers).await.unwrap()
@ -171,15 +225,22 @@ impl Testnet<NoopProvider> {
} }
} }
impl<C> fmt::Debug for Testnet<C> { impl<C, Pool> Default for Testnet<C, Pool> {
fn default() -> Self {
Self { peers: Vec::new() }
}
}
impl<C, Pool> fmt::Debug for Testnet<C, Pool> {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
f.debug_struct("Testnet {{}}").finish_non_exhaustive() f.debug_struct("Testnet {{}}").finish_non_exhaustive()
} }
} }
impl<C> Future for Testnet<C> impl<C, Pool> Future for Testnet<C, Pool>
where where
C: BlockReader + HeaderProvider + Unpin, C: BlockReader + HeaderProvider + Unpin,
Pool: TransactionPool + Unpin + 'static,
{ {
type Output = (); type Output = ();
@ -194,47 +255,81 @@ where
/// A handle to a [`Testnet`] that can be shared. /// A handle to a [`Testnet`] that can be shared.
#[derive(Debug)] #[derive(Debug)]
pub struct TestnetHandle<C> { pub struct TestnetHandle<C, Pool> {
_handle: JoinHandle<()>, _handle: JoinHandle<()>,
terminate: oneshot::Sender<oneshot::Sender<Testnet<C>>>, peers: Vec<PeerHandle<Pool>>,
terminate: oneshot::Sender<oneshot::Sender<Testnet<C, Pool>>>,
} }
// === impl TestnetHandle === // === impl TestnetHandle ===
impl<C> TestnetHandle<C> { impl<C, Pool> TestnetHandle<C, Pool> {
/// Terminates the task and returns the [`Testnet`] back. /// Terminates the task and returns the [`Testnet`] back.
pub async fn terminate(self) -> Testnet<C> { pub async fn terminate(self) -> Testnet<C, Pool> {
let (tx, rx) = oneshot::channel(); let (tx, rx) = oneshot::channel();
self.terminate.send(tx).unwrap(); self.terminate.send(tx).unwrap();
rx.await.unwrap() rx.await.unwrap()
} }
/// Returns the [`PeerHandle`]s of this [`Testnet`].
pub fn peers(&self) -> &[PeerHandle<Pool>] {
&self.peers
}
/// Connects all peers with each other
pub async fn connect_peers(&self) {
if self.peers.len() < 2 {
return
}
for (idx, handle) in self.peers.iter().enumerate().take(self.peers.len() - 1) {
let mut events = NetworkEventStream::new(handle.event_listener());
for idx in (idx + 1)..self.peers.len() {
let neighbour = &self.peers[idx];
handle.network.add_peer(*neighbour.peer_id(), neighbour.local_addr());
let connected = events.next_session_established().await.unwrap();
assert_eq!(connected, *neighbour.peer_id());
}
}
}
} }
/// A peer in the [`Testnet`]. /// A peer in the [`Testnet`].
#[pin_project] #[pin_project]
#[derive(Debug)] #[derive(Debug)]
pub struct Peer<C> { pub struct Peer<C, Pool = TestPool> {
#[pin] #[pin]
network: NetworkManager<C>, network: NetworkManager<C>,
#[pin] #[pin]
request_handler: Option<EthRequestHandler<C>>, request_handler: Option<EthRequestHandler<C>>,
#[pin] #[pin]
transactions_manager: Option<TransactionsManager<TestPool>>, transactions_manager: Option<TransactionsManager<Pool>>,
pool: Option<Pool>,
client: C, client: C,
secret_key: SecretKey, secret_key: SecretKey,
} }
// === impl Peer === // === impl Peer ===
impl<C> Peer<C> impl<C, Pool> Peer<C, Pool>
where where
C: BlockReader + HeaderProvider + Clone, C: BlockReader + HeaderProvider + Clone,
Pool: TransactionPool,
{ {
/// Returns the number of connected peers. /// Returns the number of connected peers.
pub fn num_peers(&self) -> usize { pub fn num_peers(&self) -> usize {
self.network.num_connected_peers() self.network.num_connected_peers()
} }
/// Returns a handle to the peer's network.
pub fn peer_handle(&self) -> PeerHandle<Pool> {
PeerHandle {
network: self.network.handle().clone(),
pool: self.pool.clone(),
transactions: self.transactions_manager.as_ref().map(|mgr| mgr.handle()),
}
}
/// The address that listens for incoming connections. /// The address that listens for incoming connections.
pub fn local_addr(&self) -> SocketAddr { pub fn local_addr(&self) -> SocketAddr {
self.network.local_addr() self.network.local_addr()
@ -245,6 +340,11 @@ where
self.network.handle().clone() self.network.handle().clone()
} }
/// Returns the [`TestPool`] of this peer.
pub fn pool(&self) -> Option<&Pool> {
self.pool.as_ref()
}
/// Set a new request handler that's connected to the peer's network /// Set a new request handler that's connected to the peer's network
pub fn install_request_handler(&mut self) { pub fn install_request_handler(&mut self) {
let (tx, rx) = channel(ETH_REQUEST_CHANNEL_CAPACITY); let (tx, rx) = channel(ETH_REQUEST_CHANNEL_CAPACITY);
@ -255,17 +355,49 @@ where
} }
/// Set a new transactions manager that's connected to the peer's network /// Set a new transactions manager that's connected to the peer's network
pub fn install_transactions_manager(&mut self, pool: TestPool) { pub fn install_transactions_manager(&mut self, pool: Pool) {
let (tx, rx) = unbounded_channel(); let (tx, rx) = unbounded_channel();
self.network.set_transactions(tx); self.network.set_transactions(tx);
let transactions_manager = TransactionsManager::new(self.handle(), pool, rx); let transactions_manager = TransactionsManager::new(self.handle(), pool.clone(), rx);
self.transactions_manager = Some(transactions_manager); self.transactions_manager = Some(transactions_manager);
self.pool = Some(pool);
}
/// Set a new transactions manager that's connected to the peer's network
pub fn map_transactions_manager<P>(self, pool: P) -> Peer<C, P>
where
P: TransactionPool,
{
let Self { mut network, request_handler, client, secret_key, .. } = self;
let (tx, rx) = unbounded_channel();
network.set_transactions(tx);
let transactions_manager =
TransactionsManager::new(network.handle().clone(), pool.clone(), rx);
Peer {
network,
request_handler,
transactions_manager: Some(transactions_manager),
pool: Some(pool),
client,
secret_key,
}
} }
} }
impl<C> Future for Peer<C> impl<C> Peer<C>
where
C: BlockReader + HeaderProvider + Clone,
{
/// Installs a new [testing_pool]
pub fn install_test_pool(&mut self) {
self.install_transactions_manager(testing_pool())
}
}
impl<C, Pool> Future for Peer<C, Pool>
where where
C: BlockReader + HeaderProvider + Unpin, C: BlockReader + HeaderProvider + Unpin,
Pool: TransactionPool + Unpin + 'static,
{ {
type Output = (); type Output = ();
@ -292,6 +424,47 @@ pub struct PeerConfig<C = NoopProvider> {
secret_key: SecretKey, secret_key: SecretKey,
} }
/// A handle to a peer in the [`Testnet`].
#[derive(Debug)]
pub struct PeerHandle<Pool> {
network: NetworkHandle,
transactions: Option<TransactionsHandle>,
pool: Option<Pool>,
}
// === impl PeerHandle ===
impl<Pool> PeerHandle<Pool> {
/// Returns the [`PeerId`] used in the network.
pub fn peer_id(&self) -> &PeerId {
self.network.peer_id()
}
pub fn local_addr(&self) -> SocketAddr {
self.network.local_addr()
}
/// Creates a new [`NetworkEvent`] listener channel.
pub fn event_listener(&self) -> UnboundedReceiverStream<NetworkEvent> {
self.network.event_listener()
}
/// Returns the [`TransactionsHandle`] of this peer.
pub fn transactions(&self) -> Option<&TransactionsHandle> {
self.transactions.as_ref()
}
/// Returns the [`TestPool`] of this peer.
pub fn pool(&self) -> Option<&Pool> {
self.pool.as_ref()
}
/// Returns the [`NetworkHandle`] of this peer.
pub fn network(&self) -> &NetworkHandle {
&self.network
}
}
// === impl PeerConfig === // === impl PeerConfig ===
impl<C> PeerConfig<C> impl<C> PeerConfig<C>
@ -302,8 +475,14 @@ where
pub async fn launch(self) -> Result<Peer<C>, NetworkError> { pub async fn launch(self) -> Result<Peer<C>, NetworkError> {
let PeerConfig { config, client, secret_key } = self; let PeerConfig { config, client, secret_key } = self;
let network = NetworkManager::new(config).await?; let network = NetworkManager::new(config).await?;
let peer = let peer = Peer {
Peer { network, client, secret_key, request_handler: None, transactions_manager: None }; network,
client,
secret_key,
request_handler: None,
transactions_manager: None,
pool: None,
};
Ok(peer) Ok(peer)
} }

View File

@ -59,7 +59,7 @@ const GET_POOLED_TRANSACTION_SOFT_LIMIT_SIZE: GetPooledTransactionLimit =
pub type PoolImportFuture = Pin<Box<dyn Future<Output = PoolResult<TxHash>> + Send + 'static>>; pub type PoolImportFuture = Pin<Box<dyn Future<Output = PoolResult<TxHash>> + Send + 'static>>;
/// Api to interact with [`TransactionsManager`] task. /// Api to interact with [`TransactionsManager`] task.
#[derive(Debug)] #[derive(Debug, Clone)]
pub struct TransactionsHandle { pub struct TransactionsHandle {
/// Command channel to the [`TransactionsManager`] /// Command channel to the [`TransactionsManager`]
manager_tx: mpsc::UnboundedSender<TransactionsCommand>, manager_tx: mpsc::UnboundedSender<TransactionsCommand>,
@ -212,13 +212,18 @@ impl<Pool: TransactionPool> TransactionsManager<Pool> {
impl<Pool> TransactionsManager<Pool> impl<Pool> TransactionsManager<Pool>
where where
Pool: TransactionPool + 'static, Pool: TransactionPool,
{ {
/// Returns a new handle that can send commands to this type. /// Returns a new handle that can send commands to this type.
pub fn handle(&self) -> TransactionsHandle { pub fn handle(&self) -> TransactionsHandle {
TransactionsHandle { manager_tx: self.command_tx.clone() } TransactionsHandle { manager_tx: self.command_tx.clone() }
} }
}
impl<Pool> TransactionsManager<Pool>
where
Pool: TransactionPool + 'static,
{
#[inline] #[inline]
fn update_import_metrics(&self) { fn update_import_metrics(&self) {
self.metrics.pending_pool_imports.set(self.pool_imports.len() as f64); self.metrics.pending_pool_imports.set(self.pool_imports.len() as f64);

View File

@ -1,5 +1,3 @@
use std::sync::Arc;
use reth_eth_wire::{GetPooledTransactions, PooledTransactions}; use reth_eth_wire::{GetPooledTransactions, PooledTransactions};
use reth_interfaces::sync::{NetworkSyncUpdater, SyncState}; use reth_interfaces::sync::{NetworkSyncUpdater, SyncState};
use reth_network::{ use reth_network::{
@ -11,7 +9,7 @@ use reth_primitives::{Signature, TransactionSigned, B256};
use reth_provider::test_utils::MockEthProvider; use reth_provider::test_utils::MockEthProvider;
use reth_transaction_pool::{ use reth_transaction_pool::{
test_utils::{testing_pool, MockTransaction}, test_utils::{testing_pool, MockTransaction},
TransactionOrigin, TransactionPool, TransactionPool,
}; };
use tokio::sync::oneshot; use tokio::sync::oneshot;
@ -39,15 +37,14 @@ async fn test_large_tx_req() {
let txs_hashes: Vec<B256> = txs.iter().map(|tx| tx.get_hash()).collect(); let txs_hashes: Vec<B256> = txs.iter().map(|tx| tx.get_hash()).collect();
// setup testnet // setup testnet
let mock_provider = Arc::new(MockEthProvider::default()); let mut net = Testnet::create_with(2, MockEthProvider::default()).await;
let mut net = Testnet::create_with(2, mock_provider.clone()).await;
// install request handlers // install request handlers
net.for_each_mut(|peer| peer.install_request_handler()); net.for_each_mut(|peer| peer.install_request_handler());
// insert generated txs into responding peer's pool // insert generated txs into responding peer's pool
let pool1 = testing_pool(); let pool1 = testing_pool();
pool1.add_transactions(TransactionOrigin::Private, txs).await.unwrap(); pool1.add_external_transactions(txs).await.unwrap();
// install transactions managers // install transactions managers
net.peers_mut()[0].install_transactions_manager(testing_pool()); net.peers_mut()[0].install_transactions_manager(testing_pool());

View File

@ -22,6 +22,21 @@ use secp256k1::SecretKey;
use std::{collections::HashSet, net::SocketAddr, time::Duration}; use std::{collections::HashSet, net::SocketAddr, time::Duration};
use tokio::task; use tokio::task;
#[tokio::test(flavor = "multi_thread")]
async fn test_connect_all() {
reth_tracing::init_test_tracing();
let num_peers = 3;
let net = Testnet::create(num_peers).await;
let handle = net.spawn();
handle.connect_peers().await;
handle.peers().iter().for_each(|peer| {
assert_eq!(peer.network().num_connected_peers(), num_peers - 1);
});
}
#[tokio::test(flavor = "multi_thread")] #[tokio::test(flavor = "multi_thread")]
async fn test_establish_connections() { async fn test_establish_connections() {
reth_tracing::init_test_tracing(); reth_tracing::init_test_tracing();

View File

@ -5,5 +5,6 @@ mod geth;
mod requests; mod requests;
mod session; mod session;
mod startup; mod startup;
mod txgossip;
fn main() {} fn main() {}

View File

@ -0,0 +1,45 @@
//! Testing gossiping of transactions.
use rand::thread_rng;
use reth_network::test_utils::Testnet;
use reth_primitives::U256;
use reth_provider::test_utils::{ExtendedAccount, MockEthProvider};
use reth_transaction_pool::{test_utils::TransactionGenerator, PoolTransaction, TransactionPool};
#[tokio::test(flavor = "multi_thread")]
async fn test_tx_gossip() {
reth_tracing::init_test_tracing();
let provider = MockEthProvider::default();
let net = Testnet::create_with(2, provider.clone()).await;
// install request handlers
let net = net.with_eth_pool();
let handle = net.spawn();
// connect all the peers
handle.connect_peers().await;
let peer0 = &handle.peers()[0];
let peer1 = &handle.peers()[1];
let peer0_pool = peer0.pool().unwrap();
let mut peer0_tx_listener = peer0.pool().unwrap().pending_transactions_listener();
let mut peer1_tx_listener = peer1.pool().unwrap().pending_transactions_listener();
let mut gen = TransactionGenerator::new(thread_rng());
let tx = gen.gen_eip1559_pooled();
// ensure the sender has balance
let sender = tx.sender();
provider.add_account(sender, ExtendedAccount::new(0, U256::from(100_000_000)));
// insert pending tx in peer0's pool
let hash = peer0_pool.add_external_transaction(tx).await.unwrap();
let inserted = peer0_tx_listener.recv().await.unwrap();
assert_eq!(inserted, hash);
// ensure tx is gossiped to peer1
let received = peer1_tx_listener.recv().await.unwrap();
assert_eq!(received, hash);
}

View File

@ -473,6 +473,30 @@ impl Transaction {
} }
} }
impl From<TxLegacy> for Transaction {
fn from(tx: TxLegacy) -> Self {
Transaction::Legacy(tx)
}
}
impl From<TxEip2930> for Transaction {
fn from(tx: TxEip2930) -> Self {
Transaction::Eip2930(tx)
}
}
impl From<TxEip1559> for Transaction {
fn from(tx: TxEip1559) -> Self {
Transaction::Eip1559(tx)
}
}
impl From<TxEip4844> for Transaction {
fn from(tx: TxEip4844) -> Self {
Transaction::Eip4844(tx)
}
}
impl Compact for Transaction { impl Compact for Transaction {
fn to_compact<B>(self, buf: &mut B) -> usize fn to_compact<B>(self, buf: &mut B) -> usize
where where

View File

@ -192,6 +192,13 @@ mod traits;
/// Common test helpers for mocking a pool /// Common test helpers for mocking a pool
pub mod test_utils; pub mod test_utils;
/// Type alias for default ethereum transaction pool
pub type EthTransactionPool<Client, S> = Pool<
TransactionValidationTaskExecutor<EthTransactionValidator<Client, EthPooledTransaction>>,
CoinbaseTipOrdering<EthPooledTransaction>,
S,
>;
/// A shareable, generic, customizable `TransactionPool` implementation. /// A shareable, generic, customizable `TransactionPool` implementation.
#[derive(Debug)] #[derive(Debug)]
pub struct Pool<V, T: TransactionOrdering, S> { pub struct Pool<V, T: TransactionOrdering, S> {
@ -262,12 +269,7 @@ where
} }
} }
impl<Client, S> impl<Client, S> EthTransactionPool<Client, S>
Pool<
TransactionValidationTaskExecutor<EthTransactionValidator<Client, EthPooledTransaction>>,
CoinbaseTipOrdering<EthPooledTransaction>,
S,
>
where where
Client: StateProviderFactory + Clone + 'static, Client: StateProviderFactory + Clone + 'static,
S: BlobStore, S: BlobStore,

View File

@ -94,7 +94,6 @@ use std::{
}; };
use tokio::sync::mpsc; use tokio::sync::mpsc;
use tracing::{debug, trace, warn}; use tracing::{debug, trace, warn};
mod events; mod events;
pub use events::{FullTransactionEvent, TransactionEvent}; pub use events::{FullTransactionEvent, TransactionEvent};

View File

@ -0,0 +1,313 @@
#![allow(missing_docs)]
use crate::{
test_utils::{MockTransactionFactory, MockValidTx},
EthPooledTransaction,
};
use rand::Rng;
use reth_primitives::{
constants::MIN_PROTOCOL_BASE_FEE, sign_message, AccessList, Address, Bytes,
FromRecoveredTransaction, Transaction, TransactionKind, TransactionSigned, TxEip1559, TxLegacy,
TxValue, B256, MAINNET,
};
/// A generator for transactions for testing purposes
pub struct TransactionGenerator<R> {
rng: R,
signer_keys: Vec<B256>,
base_fee: u128,
gas_limit: u64,
}
impl<R: Rng> TransactionGenerator<R> {
/// Initializes the generator with 10 random signers
pub fn new(rng: R) -> Self {
Self::with_num_signers(rng, 10)
}
/// Generates random random signers
pub fn with_num_signers(mut rng: R, num_signers: usize) -> Self {
let mut signer_keys = Vec::with_capacity(num_signers);
for _ in 0..num_signers {
signer_keys.push(B256::random());
}
Self { rng, signer_keys, base_fee: MIN_PROTOCOL_BASE_FEE as u128, gas_limit: 300_000 }
}
/// Adds a new signer to the set
pub fn push_signer(&mut self, signer: B256) -> &mut Self {
self.signer_keys.push(signer);
self
}
/// Sets the default gas limit for all generated transactions
pub fn set_gas_limit(&mut self, gas_limit: u64) -> &mut Self {
self.gas_limit = gas_limit;
self
}
/// Sets the default gas limit for all generated transactions
pub fn with_gas_limit(mut self, gas_limit: u64) -> Self {
self.gas_limit = gas_limit;
self
}
/// Sets the base fee for the generated transactions
pub fn set_base_fee(&mut self, base_fee: u64) -> &mut Self {
self.base_fee = base_fee as u128;
self
}
/// Sets the base fee for the generated transactions
pub fn with_base_fee(mut self, base_fee: u64) -> Self {
self.base_fee = base_fee as u128;
self
}
/// Adds the given signers to the set.
pub fn extend_signers(&mut self, signers: impl IntoIterator<Item = B256>) -> &mut Self {
self.signer_keys.extend(signers);
self
}
/// Returns a random signer from the set
fn rng_signer(&mut self) -> B256 {
let idx = self.rng.gen_range(0..self.signer_keys.len());
self.signer_keys[idx]
}
/// Creates a new transaction with a random signer
pub fn transaction(&mut self) -> TransactionBuilder {
TransactionBuilder::default()
.signer(self.rng_signer())
.max_fee_per_gas(self.base_fee)
.max_priority_fee_per_gas(self.base_fee)
.gas_limit(self.gas_limit)
}
/// Creates a new transaction with a random signer
pub fn gen_eip1559(&mut self) -> TransactionSigned {
self.transaction().into_eip1559()
}
pub fn gen_eip1559_pooled(&mut self) -> EthPooledTransaction {
let tx = self.gen_eip1559().into_ecrecovered().unwrap();
EthPooledTransaction::from_recovered_transaction(tx)
}
}
/// A Builder type to configure and create a transaction.
pub struct TransactionBuilder {
signer: B256,
chain_id: u64,
nonce: u64,
gas_limit: u64,
max_fee_per_gas: u128,
max_priority_fee_per_gas: u128,
to: TransactionKind,
value: TxValue,
access_list: AccessList,
input: Bytes,
}
impl TransactionBuilder {
pub fn into_legacy(self) -> TransactionSigned {
let Self {
signer,
chain_id,
nonce,
gas_limit,
max_fee_per_gas,
max_priority_fee_per_gas,
to,
value,
access_list,
input,
} = self;
let tx: Transaction = TxLegacy {
chain_id: Some(chain_id),
nonce,
gas_limit,
gas_price: max_fee_per_gas,
to,
value,
input,
}
.into();
TransactionBuilder::signed(tx, signer)
}
pub fn into_eip1559(self) -> TransactionSigned {
let Self {
signer,
chain_id,
nonce,
gas_limit,
max_fee_per_gas,
max_priority_fee_per_gas,
to,
value,
access_list,
input,
} = self;
let tx: Transaction = TxEip1559 {
chain_id,
nonce,
gas_limit,
max_fee_per_gas,
max_priority_fee_per_gas,
to,
value,
access_list,
input,
}
.into();
TransactionBuilder::signed(tx, signer)
}
fn signed(transaction: Transaction, signer: B256) -> TransactionSigned {
let signature = sign_message(signer, transaction.signature_hash()).unwrap();
TransactionSigned::from_transaction_and_signature(transaction, signature)
}
pub fn signer(mut self, signer: B256) -> Self {
self.signer = signer;
self
}
pub fn gas_limit(mut self, gas_limit: u64) -> Self {
self.gas_limit = gas_limit;
self
}
pub fn nonce(mut self, nonce: u64) -> Self {
self.nonce = nonce;
self
}
pub fn inc_nonce(mut self) -> Self {
self.nonce += 1;
self
}
pub fn decr_nonce(mut self) -> Self {
self.nonce = self.nonce.saturating_sub(1);
self
}
pub fn max_fee_per_gas(mut self, max_fee_per_gas: u128) -> Self {
self.max_fee_per_gas = max_fee_per_gas;
self
}
pub fn max_priority_fee_per_gas(mut self, max_priority_fee_per_gas: u128) -> Self {
self.max_priority_fee_per_gas = max_priority_fee_per_gas;
self
}
pub fn to(mut self, to: Address) -> Self {
self.to = TransactionKind::Call(to);
self
}
pub fn value(mut self, value: u128) -> Self {
self.value = value.into();
self
}
pub fn access_list(mut self, access_list: AccessList) -> Self {
self.access_list = access_list;
self
}
pub fn input(mut self, input: impl Into<Bytes>) -> Self {
self.input = input.into();
self
}
pub fn chain_id(mut self, chain_id: u64) -> Self {
self.chain_id = chain_id;
self
}
pub fn set_chain_id(&mut self, chain_id: u64) -> &mut Self {
self.chain_id = chain_id;
self
}
pub fn set_nonce(&mut self, nonce: u64) -> &mut Self {
self.nonce = nonce;
self
}
pub fn set_gas_limit(&mut self, gas_limit: u64) -> &mut Self {
self.gas_limit = gas_limit;
self
}
pub fn set_max_fee_per_gas(&mut self, max_fee_per_gas: u128) -> &mut Self {
self.max_fee_per_gas = max_fee_per_gas;
self
}
pub fn set_max_priority_fee_per_gas(&mut self, max_priority_fee_per_gas: u128) -> &mut Self {
self.max_priority_fee_per_gas = max_priority_fee_per_gas;
self
}
pub fn set_to(&mut self, to: Address) -> &mut Self {
self.to = TransactionKind::Call(to);
self
}
pub fn set_value(&mut self, value: u128) -> &mut Self {
self.value = value.into();
self
}
pub fn set_access_list(&mut self, access_list: AccessList) -> &mut Self {
self.access_list = access_list;
self
}
pub fn set_signer(&mut self, signer: B256) -> &mut Self {
self.signer = signer;
self
}
pub fn set_input(&mut self, input: impl Into<Bytes>) -> &mut Self {
self.input = input.into();
self
}
}
impl Default for TransactionBuilder {
fn default() -> Self {
Self {
signer: B256::random(),
chain_id: MAINNET.chain.id(),
nonce: 0,
gas_limit: 0,
max_fee_per_gas: 0,
max_priority_fee_per_gas: 0,
to: Default::default(),
value: Default::default(),
access_list: Default::default(),
input: Default::default(),
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use rand::thread_rng;
#[test]
fn test_generate_transaction() {
let rng = thread_rng();
let mut gen = TransactionGenerator::new(rng);
let _tx = gen.transaction().into_legacy();
let _tx = gen.transaction().into_eip1559();
}
}

View File

@ -1,6 +1,7 @@
//! Internal helpers for testing. //! Internal helpers for testing.
#![allow(missing_docs, unused, missing_debug_implementations, unreachable_pub)] #![allow(missing_docs, unused, missing_debug_implementations, unreachable_pub)]
mod gen;
mod mock; mod mock;
mod pool; mod pool;
@ -9,6 +10,7 @@ use crate::{
TransactionOrigin, TransactionValidationOutcome, TransactionValidator, TransactionOrigin, TransactionValidationOutcome, TransactionValidator,
}; };
use async_trait::async_trait; use async_trait::async_trait;
pub use gen::*;
pub use mock::*; pub use mock::*;
use std::{marker::PhantomData, sync::Arc}; use std::{marker::PhantomData, sync::Arc};