diff --git a/CHANGELOG.md b/CHANGELOG.md index 8ee0e4b12dc..35283a98080 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -11,6 +11,8 @@ and this project adheres to [Semantic Versioning](http://semver.org/). - [2182](https://github.com/FuelLabs/fuel-core/pull/2151): Limit number of transactions that can be fetched via TxSource::next - [2189](https://github.com/FuelLabs/fuel-core/pull/2151): Select next DA height to never include more than u16::MAX -1 transactions from L1. - [2162](https://github.com/FuelLabs/fuel-core/pull/2162): Pool structure with dependencies, etc.. for the next transaction pool module. +- [2193](https://github.com/FuelLabs/fuel-core/pull/2193): Insertion in PoolV2 and tests refactoring + ### Changed diff --git a/Cargo.lock b/Cargo.lock index f32bd10821f..72524c2d6c2 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3747,10 +3747,13 @@ dependencies = [ "fuel-core-services", "fuel-core-storage", "fuel-core-types 0.36.0", + "futures", + "mockall", "num-rational", + "parking_lot", "petgraph", + "rayon", "tokio", - "tokio-rayon", "tracing", ] diff --git a/crates/services/txpool_v2/Cargo.toml b/crates/services/txpool_v2/Cargo.toml index 8c984c7e3fd..25fafbeb4e1 100644 --- a/crates/services/txpool_v2/Cargo.toml +++ b/crates/services/txpool_v2/Cargo.toml @@ -18,8 +18,18 @@ derive_more = { workspace = true } fuel-core-services = { workspace = true } fuel-core-storage = { workspace = true, features = ["std"] } fuel-core-types = { workspace = true, features = ["test-helpers"] } +futures = { workspace = true } +mockall = { workspace = true, optional = true } num-rational = { workspace = true } +parking_lot = { workspace = true } petgraph = "0.6.5" +rayon = { workspace = true } tokio = { workspace = true, default-features = false, features = ["sync"] } -tokio-rayon = { workspace = true } tracing = { workspace = true } + +[features] +test-helpers = [ + "dep:mockall", + "fuel-core-types/test-helpers", + "fuel-core-storage/test-helpers", +] diff --git a/crates/services/txpool_v2/src/collision_manager/basic.rs b/crates/services/txpool_v2/src/collision_manager/basic.rs index b6c2caf3ef4..1edfc65410d 100644 --- a/crates/services/txpool_v2/src/collision_manager/basic.rs +++ b/crates/services/txpool_v2/src/collision_manager/basic.rs @@ -31,15 +31,14 @@ use fuel_core_types::{ use num_rational::Ratio; use crate::{ - error::Error, + error::{ + CollisionReason, + Error, + }, storage::StorageData, }; -use super::{ - CollisionManager, - CollisionReason, - Collisions, -}; +use super::CollisionManager; pub trait BasicCollisionManagerStorage { type StorageIndex: Copy + Debug + Hash + PartialEq + Eq; @@ -76,26 +75,47 @@ impl Default for BasicCollisionManager { } impl BasicCollisionManager { - fn gather_colliding_txs( + fn is_better_than_collision( &self, tx: &PoolTransaction, - ) -> Result, Error> { - let mut collisions = Collisions::new(); - if let PoolTransaction::Blob(checked_tx, _) = tx { + collision: S::StorageIndex, + storage: &S, + ) -> bool { + let new_tx_ratio = Ratio::new(tx.tip(), tx.max_gas()); + let colliding_tx = storage + .get(&collision) + .expect("Transaction always should exist in storage"); + let colliding_tx_ratio = Ratio::new( + colliding_tx.dependents_cumulative_tip, + colliding_tx.dependents_cumulative_gas, + ); + new_tx_ratio > colliding_tx_ratio + } +} + +impl CollisionManager for BasicCollisionManager { + type Storage = S; + type StorageIndex = S::StorageIndex; + + fn collect_colliding_transactions( + &self, + transaction: &PoolTransaction, + ) -> Result>, Error> { + let mut collisions = HashMap::new(); + if let PoolTransaction::Blob(checked_tx, _) = transaction { let blob_id = checked_tx.transaction().blob_id(); if let Some(state) = self.blobs_users.get(blob_id) { - collisions.reasons.insert(CollisionReason::Blob(*blob_id)); - collisions.colliding_txs.insert(*state); + collisions.insert(*state, vec![CollisionReason::Blob(*blob_id)]); } } - for input in tx.inputs() { + for input in transaction.inputs() { match input { Input::CoinSigned(CoinSigned { utxo_id, .. }) | Input::CoinPredicate(CoinPredicate { utxo_id, .. }) => { // Check if the utxo is already spent by another transaction in the pool - if let Some(tx_id) = self.coins_spenders.get(utxo_id) { - collisions.reasons.insert(CollisionReason::Coin(*utxo_id)); - collisions.colliding_txs.insert(*tx_id); + if let Some(storage_id) = self.coins_spenders.get(utxo_id) { + let entry = collisions.entry(*storage_id).or_default(); + entry.push(CollisionReason::Utxo(*utxo_id)); } } Input::MessageCoinSigned(MessageCoinSigned { nonce, .. }) @@ -103,9 +123,9 @@ impl BasicCollisionManager { | Input::MessageDataSigned(MessageDataSigned { nonce, .. }) | Input::MessageDataPredicate(MessageDataPredicate { nonce, .. }) => { // Check if the message is already spent by another transaction in the pool - if let Some(tx_id) = self.messages_spenders.get(nonce) { - collisions.reasons.insert(CollisionReason::Message(*nonce)); - collisions.colliding_txs.insert(*tx_id); + if let Some(storage_id) = self.messages_spenders.get(nonce) { + let entry = collisions.entry(*storage_id).or_default(); + entry.push(CollisionReason::Message(*nonce)); } } // No collision for contract inputs @@ -113,67 +133,55 @@ impl BasicCollisionManager { } } - for output in tx.outputs() { + for output in transaction.outputs() { if let Output::ContractCreated { contract_id, .. } = output { // Check if the contract is already created by another transaction in the pool - if let Some(tx_id) = self.contracts_creators.get(contract_id) { - collisions - .reasons - .insert(CollisionReason::ContractCreation(*contract_id)); - collisions.colliding_txs.insert(*tx_id); + if let Some(storage_id) = self.contracts_creators.get(contract_id) { + let entry = collisions.entry(*storage_id).or_default(); + entry.push(CollisionReason::ContractCreation(*contract_id)); } } } Ok(collisions) } - fn is_better_than_collisions( - &self, - tx: &PoolTransaction, - collisions: &Collisions, - storage: &S, - ) -> bool { - let new_tx_ratio = Ratio::new(tx.tip(), tx.max_gas()); - let (total_tip, total_gas) = collisions.colliding_txs.iter().fold( - (0u64, 0u64), - |(total_tip, total_gas), node_id| { - let dependent_tx = storage - .get(node_id) - .expect("Transaction always should exist in storage"); - let total_tip = - total_tip.saturating_add(dependent_tx.dependents_cumulative_tip); - let total_gas = - total_gas.saturating_add(dependent_tx.dependents_cumulative_gas); - (total_tip, total_gas) - }, - ); - - let collision_tx_ratio = Ratio::new(total_tip, total_gas); - - new_tx_ratio > collision_tx_ratio - } -} - -impl CollisionManager for BasicCollisionManager { - type Storage = S; - type StorageIndex = S::StorageIndex; - - fn collect_colliding_transactions( + /// Rules: + // A transaction with dependencies can collide only with one other transaction if it is less worth it + // A transaction without dependencies can collide with multiple transaction if they are less worth it + fn can_store_transaction( &self, transaction: &PoolTransaction, - storage: &S, - ) -> Result, Error> { - let collisions = self.gather_colliding_txs(transaction)?; - if collisions.colliding_txs.is_empty() { - Ok(Collisions::new()) - } else if self.is_better_than_collisions(transaction, &collisions, storage) { - Ok(collisions) + has_dependencies: bool, + colliding_transactions: &HashMap>, + storage: &Self::Storage, + ) -> Result<(), CollisionReason> { + if colliding_transactions.is_empty() { + return Ok(()); + } + if has_dependencies { + if colliding_transactions.len() > 1 { + return Err(CollisionReason::MultipleCollisions); + } + let (collision, reason) = colliding_transactions.iter().next().unwrap(); + if !self.is_better_than_collision(transaction, *collision, storage) { + if let Some(reason) = reason.first() { + return Err(reason.clone()); + } else { + return Err(CollisionReason::Unknown); + } + } } else { - Err(Error::Collided(format!( - "Transaction collides with other transactions: {:?}", - collisions.colliding_txs - ))) + for (collision, reason) in colliding_transactions.iter() { + if !self.is_better_than_collision(transaction, *collision, storage) { + if let Some(reason) = reason.first() { + return Err(reason.clone()); + } else { + return Err(CollisionReason::Unknown); + } + } + } } + Ok(()) } fn on_stored_transaction( diff --git a/crates/services/txpool_v2/src/collision_manager/mod.rs b/crates/services/txpool_v2/src/collision_manager/mod.rs index ddff3f2abf4..5fcc5409d63 100644 --- a/crates/services/txpool_v2/src/collision_manager/mod.rs +++ b/crates/services/txpool_v2/src/collision_manager/mod.rs @@ -1,5 +1,8 @@ use std::{ - collections::HashSet, + collections::{ + HashMap, + HashSet, + }, fmt::Debug, }; @@ -13,61 +16,35 @@ use fuel_core_types::{ services::txpool::PoolTransaction, }; -use crate::error::Error; +use crate::error::{ + CollisionReason, + Error, +}; pub mod basic; -/// The reason why a transaction collides with another. -/// It also contains additional information about the collision. -#[derive(Debug, Copy, Clone, Ord, PartialOrd, Eq, PartialEq, Hash)] -pub enum CollisionReason { - Coin(UtxoId), - Blob(BlobId), - Message(Nonce), - ContractCreation(ContractId), -} - -/// Contains all the information about the collisions of a transaction. -#[derive(Default, Debug)] -pub struct Collisions { - reasons: HashSet, - colliding_txs: HashSet, -} - -impl Collisions { - /// Create a new empty collision information. - pub fn new() -> Self { - Self { - reasons: HashSet::default(), - colliding_txs: HashSet::default(), - } - } - - /// Get the reasons of the collisions. - pub fn reasons(&self) -> &HashSet { - &self.reasons - } - - /// Get the transactions that collide with the transaction. - pub fn colliding_txs(&self) -> &HashSet { - &self.colliding_txs - } -} - pub trait CollisionManager { /// Storage type of the collision manager. type Storage; /// Index that identifies a transaction in the storage. type StorageIndex; - /// Collect all the transactions that collide with the given transaction. - /// It returns an error if the transaction is less worthy than the colliding transactions. - /// It returns the information about the collisions. + /// Collect the transaction that collide with the given transaction. + /// Returns a list of storage indexes of the colliding transactions and the reason of the collision. fn collect_colliding_transactions( &self, transaction: &PoolTransaction, + ) -> Result>, Error>; + + /// Determine if the collisions allow the transaction to be stored. + /// Returns the reason of the collision if the transaction cannot be stored. + fn can_store_transaction( + &self, + transaction: &PoolTransaction, + has_dependencies: bool, + colliding_transactions: &HashMap>, storage: &Self::Storage, - ) -> Result, Error>; + ) -> Result<(), CollisionReason>; /// Inform the collision manager that a transaction was stored. fn on_stored_transaction( diff --git a/crates/services/txpool_v2/src/config.rs b/crates/services/txpool_v2/src/config.rs index f7bf0ee4534..52d14d4234b 100644 --- a/crates/services/txpool_v2/src/config.rs +++ b/crates/services/txpool_v2/src/config.rs @@ -115,6 +115,7 @@ impl BlackList { } } +#[derive(Clone)] pub struct Config { /// Enable UTXO validation (will check if UTXO exists in the database and has correct data). pub utxo_validation: bool, @@ -124,14 +125,34 @@ pub struct Config { pub max_block_gas: u64, /// Maximum transactions per dependencies chain. pub max_txs_chain_count: usize, - /// Maximum transactions in the pool. - pub max_txs: u64, + /// Pool limits + pub pool_limits: PoolLimits, /// Maximum transaction time to live. pub max_txs_ttl: Duration, + /// Heavy async processing configuration. + pub heavy_work: HeavyWorkConfig, /// Blacklist. Transactions with blacklisted inputs will not be accepted. pub black_list: BlackList, } +#[derive(Clone)] +pub struct PoolLimits { + /// Maximum number of transactions in the pool. + pub max_txs: usize, + /// Maximum number of gas in the pool. + pub max_gas: u64, + /// Maximum number of bytes in the pool. + pub max_bytes_size: usize, +} + +#[derive(Clone)] +pub struct HeavyWorkConfig { + /// Maximum of threads for managing verifications/insertions. + pub number_threads_to_verify_transactions: usize, + /// Maximum of tasks in the heavy async processing queue. + pub size_of_verification_queue: usize, +} + #[cfg(test)] impl Default for Config { fn default() -> Self { @@ -140,9 +161,17 @@ impl Default for Config { max_block_gas: 100000000, max_block_size: 1000000000, max_txs_chain_count: 1000, - max_txs: 10000, + pool_limits: PoolLimits { + max_txs: 10000, + max_gas: 100_000_000_000, + max_bytes_size: 10_000_000_000, + }, max_txs_ttl: Duration::from_secs(60 * 10), black_list: BlackList::default(), + heavy_work: HeavyWorkConfig { + number_threads_to_verify_transactions: 4, + size_of_verification_queue: 100, + }, } } } diff --git a/crates/services/txpool_v2/src/error.rs b/crates/services/txpool_v2/src/error.rs index 288eab642d1..36459d18a24 100644 --- a/crates/services/txpool_v2/src/error.rs +++ b/crates/services/txpool_v2/src/error.rs @@ -9,6 +9,8 @@ use fuel_core_types::{ fuel_vm::checked_transaction::CheckError, }; +use crate::ports::WasmValidityError; + #[derive(Debug, derive_more::Display)] pub enum Error { #[display(fmt = "Gas price not found for block height {_0}")] @@ -27,9 +29,10 @@ pub enum Error { fmt = "Transaction is not inserted. Transaction chain dependency is already too big" )] NotInsertedChainDependencyTooBig, - // TODO: Make more specific errors: https://github.com/FuelLabs/fuel-core/issues/2185 #[display(fmt = "Transaction collided: {_0}")] - Collided(String), + Collided(CollisionReason), + #[display(fmt = "Transaction is not inserted. Collision is also a dependency")] + NotInsertedCollisionIsDependency, #[display(fmt = "Utxo not found: {_0}")] UtxoNotFound(UtxoId), #[display(fmt = "The UTXO `{_0}` is blacklisted")] @@ -82,6 +85,36 @@ pub enum Error { NotInsertedLimitHit, #[display(fmt = "Storage error: {_0}")] Storage(String), + #[display(fmt = "Error with Wasm validity: {:?}", _0)] + WasmValidity(WasmValidityError), + #[display(fmt = "Transaction is not inserted. Mint transaction is not allowed")] + MintIsDisallowed, +} + +#[derive(Debug, Clone, derive_more::Display)] +pub enum CollisionReason { + #[display( + fmt = "Transaction with the same UTXO (id: {_0}) already exists and is more worth it" + )] + Utxo(UtxoId), + #[display( + fmt = "Transaction that create the same contract (id: {_0}) already exists and is more worth it" + )] + ContractCreation(ContractId), + #[display( + fmt = "Transaction that use the same blob (id: {_0}) already exists and is more worth it" + )] + Blob(BlobId), + #[display( + fmt = "Transaction that use the same message (id: {_0}) already exists and is more worth it" + )] + Message(Nonce), + #[display(fmt = "This transaction have an unknown collision")] + Unknown, + #[display( + fmt = "This transaction have dependencies and is colliding with multiple transactions" + )] + MultipleCollisions, } impl From for Error { diff --git a/crates/services/txpool_v2/src/heavy_async_processing.rs b/crates/services/txpool_v2/src/heavy_async_processing.rs new file mode 100644 index 00000000000..94fabad1fc7 --- /dev/null +++ b/crates/services/txpool_v2/src/heavy_async_processing.rs @@ -0,0 +1,195 @@ +use std::{ + future::Future, + sync::Arc, +}; +use tokio::sync::Semaphore; + +pub struct HeavyAsyncProcessor { + rayon_thread_pool: rayon::ThreadPool, + semaphore: Arc, +} + +#[derive(Debug, PartialEq, Eq)] +pub struct OutOfCapacity; + +impl HeavyAsyncProcessor { + pub fn new( + number_of_threads: usize, + number_of_pending_tasks: usize, + ) -> anyhow::Result { + let rayon_thread_pool = rayon::ThreadPoolBuilder::new() + .num_threads(number_of_threads) + .build() + .map_err(|e| anyhow::anyhow!("Failed to create a rayon pool: {}", e))?; + let semaphore = Arc::new(Semaphore::new(number_of_pending_tasks)); + Ok(Self { + rayon_thread_pool, + semaphore, + }) + } + + pub fn spawn(&self, future: F) -> Result<(), OutOfCapacity> + where + F: Future + Send + 'static, + { + let permit = self.semaphore.clone().try_acquire_owned(); + + if let Ok(permit) = permit { + self.rayon_thread_pool.spawn_fifo(move || { + let _drop = permit; + futures::executor::block_on(future); + }); + Ok(()) + } else { + Err(OutOfCapacity) + } + } +} + +#[cfg(test)] +#[allow(clippy::bool_assert_comparison)] +mod tests { + use super::*; + use std::{ + thread::sleep, + time::Duration, + }; + use tokio::time::Instant; + + #[tokio::test] + async fn one_spawn_single_tasks_works() { + // Given + let number_of_pending_tasks = 1; + let heavy_task_processor = + HeavyAsyncProcessor::new(1, number_of_pending_tasks).unwrap(); + + // When + let (sender, receiver) = tokio::sync::oneshot::channel(); + let result = heavy_task_processor.spawn(async move { + sender.send(()).unwrap(); + }); + + // Then + assert_eq!(result, Ok(())); + let timeout = tokio::time::timeout(Duration::from_secs(1), receiver).await; + timeout + .expect("Shouldn't timeout") + .expect("Should receive a message"); + } + + #[tokio::test] + async fn second_spawn_fails_when_limit_is_one_and_first_in_progress() { + // Given + let number_of_pending_tasks = 1; + let heavy_task_processor = + HeavyAsyncProcessor::new(1, number_of_pending_tasks).unwrap(); + let first_spawn_result = heavy_task_processor.spawn(async move { + sleep(Duration::from_secs(1)); + }); + assert_eq!(first_spawn_result, Ok(())); + + // When + let second_spawn_result = heavy_task_processor.spawn(async move { + sleep(Duration::from_secs(1)); + }); + + // Then + assert_eq!(second_spawn_result, Err(OutOfCapacity)); + } + + #[tokio::test] + async fn second_spawn_works_when_first_is_finished() { + let number_of_pending_tasks = 1; + let heavy_task_processor = + HeavyAsyncProcessor::new(1, number_of_pending_tasks).unwrap(); + + // Given + let (sender, receiver) = tokio::sync::oneshot::channel(); + let first_spawn = heavy_task_processor.spawn(async move { + sleep(Duration::from_secs(1)); + sender.send(()).unwrap(); + }); + assert_eq!(first_spawn, Ok(())); + receiver.await.unwrap(); + + // When + let second_spawn = heavy_task_processor.spawn(async move { + sleep(Duration::from_secs(1)); + }); + + // Then + assert_eq!(second_spawn, Ok(())); + } + + #[tokio::test] + async fn can_spawn_10_tasks_when_limit_is_10() { + // Given + let number_of_pending_tasks = 10; + let heavy_task_processor = + HeavyAsyncProcessor::new(1, number_of_pending_tasks).unwrap(); + + for _ in 0..number_of_pending_tasks { + // When + let result = heavy_task_processor.spawn(async move { + sleep(Duration::from_secs(1)); + }); + + // Then + assert_eq!(result, Ok(())); + } + } + + #[tokio::test] + async fn executes_10_tasks_for_10_seconds_with_one_thread() { + // Given + let number_of_pending_tasks = 10; + let number_of_threads = 1; + let heavy_task_processor = + HeavyAsyncProcessor::new(number_of_threads, number_of_pending_tasks).unwrap(); + + // When + let (broadcast_sender, mut broadcast_receiver) = + tokio::sync::broadcast::channel(1024); + let instant = Instant::now(); + for _ in 0..number_of_pending_tasks { + let broadcast_sender = broadcast_sender.clone(); + let result = heavy_task_processor.spawn(async move { + sleep(Duration::from_secs(1)); + broadcast_sender.send(()).unwrap(); + }); + assert_eq!(result, Ok(())); + } + drop(broadcast_sender); + + // Then + while broadcast_receiver.recv().await.is_ok() {} + assert!(instant.elapsed() >= Duration::from_secs(10)); + } + + #[tokio::test] + async fn executes_10_tasks_for_2_seconds_with_10_thread() { + // Given + let number_of_pending_tasks = 10; + let number_of_threads = 10; + let heavy_task_processor = + HeavyAsyncProcessor::new(number_of_threads, number_of_pending_tasks).unwrap(); + + // When + let (broadcast_sender, mut broadcast_receiver) = + tokio::sync::broadcast::channel(1024); + let instant = Instant::now(); + for _ in 0..number_of_pending_tasks { + let broadcast_sender = broadcast_sender.clone(); + let result = heavy_task_processor.spawn(async move { + sleep(Duration::from_secs(1)); + broadcast_sender.send(()).unwrap(); + }); + assert_eq!(result, Ok(())); + } + drop(broadcast_sender); + + // Then + while broadcast_receiver.recv().await.is_ok() {} + assert!(instant.elapsed() <= Duration::from_secs(2)); + } +} diff --git a/crates/services/txpool_v2/src/lib.rs b/crates/services/txpool_v2/src/lib.rs index 982cc4e3bc3..b67bd14c1c5 100644 --- a/crates/services/txpool_v2/src/lib.rs +++ b/crates/services/txpool_v2/src/lib.rs @@ -8,12 +8,13 @@ mod collision_manager; mod config; mod error; +mod heavy_async_processing; mod pool; mod ports; mod selection_algorithms; mod service; mod storage; -mod transaction_conversion; +mod verifications; type GasPrice = Word; diff --git a/crates/services/txpool_v2/src/pool.rs b/crates/services/txpool_v2/src/pool.rs index d6cefbc5189..65007f89203 100644 --- a/crates/services/txpool_v2/src/pool.rs +++ b/crates/services/txpool_v2/src/pool.rs @@ -1,18 +1,27 @@ +use std::{ + collections::HashMap, + time::Instant, +}; + use fuel_core_types::{ - fuel_tx::field::BlobId, + fuel_tx::{ + field::BlobId, + Transaction, + TxId, + }, + fuel_vm::checked_transaction::Checked, services::txpool::PoolTransaction, }; +use num_rational::Ratio; use tracing::instrument; -#[cfg(test)] -use fuel_core_types::fuel_tx::TxId; -#[cfg(test)] -use std::collections::HashMap; - use crate::{ collision_manager::CollisionManager, config::Config, - error::Error, + error::{ + CollisionReason, + Error, + }, ports::{ AtomicView, TxPoolPersistentStorage, @@ -21,7 +30,11 @@ use crate::{ Constraints, SelectionAlgorithm, }, - storage::Storage, + storage::{ + RemovedTransactions, + Storage, + }, + verifications::FullyVerifiedTx, }; /// The pool is the main component of the txpool service. It is responsible for storing transactions @@ -37,8 +50,12 @@ pub struct Pool { selection_algorithm: SA, /// The persistent storage of the pool. persistent_storage_provider: PSProvider, - #[cfg(test)] + /// Mapping from tx_id to storage_id. tx_id_to_storage_id: HashMap, + /// Current pool gas stored. + current_gas: u64, + /// Current pool size in bytes. + current_bytes_size: usize, } impl Pool { @@ -56,8 +73,9 @@ impl Pool { selection_algorithm, persistent_storage_provider, config, - #[cfg(test)] tx_id_to_storage_id: HashMap::new(), + current_gas: 0, + current_bytes_size: 0, } } } @@ -75,55 +93,96 @@ where /// Each result is a list of transactions that were removed from the pool /// because of the insertion of the new transaction. #[instrument(skip(self))] - pub fn insert( - &mut self, - transactions: Vec, - ) -> Result, Error>>, Error> { + pub fn insert(&mut self, tx: PoolTransaction) -> Result, Error> { let latest_view = self .persistent_storage_provider .latest_view() .map_err(|e| Error::Database(format!("{:?}", e)))?; - Ok(transactions - .into_iter() - .map(|tx| { - #[cfg(test)] - let tx_id = tx.id(); - self.check_pool_is_not_full()?; - self.config.black_list.check_blacklisting(&tx)?; - Self::check_blob_does_not_exist(&tx, &latest_view)?; - self.storage.validate_inputs( - &tx, - &latest_view, - self.config.utxo_validation, - )?; - let collisions = self - .collision_manager - .collect_colliding_transactions(&tx, &self.storage)?; - let dependencies = self.storage.collect_transaction_dependencies(&tx)?; - let has_dependencies = !dependencies.is_empty(); - let (storage_id, removed_transactions) = self.storage.store_transaction( - tx, - dependencies, - collisions.colliding_txs(), - )?; - #[cfg(test)] - { - self.tx_id_to_storage_id.insert(tx_id, storage_id); - } - // No dependencies directly in the graph and the sorted transactions - if !has_dependencies { - self.selection_algorithm - .new_executable_transactions(vec![storage_id], &self.storage)?; - } - self.update_components_and_caches_on_removal( - removed_transactions.iter(), - )?; - let tx = Storage::get(&self.storage, &storage_id)?; - self.collision_manager - .on_stored_transaction(&tx.transaction, storage_id)?; - Ok(removed_transactions) - }) - .collect()) + let tx_id = tx.id(); + let gas = tx.max_gas(); + let creation_instant = Instant::now(); + let bytes_size = tx.metered_bytes_size(); + self.config.black_list.check_blacklisting(&tx)?; + Self::check_blob_does_not_exist(&tx, &latest_view)?; + self.storage + .validate_inputs(&tx, &latest_view, self.config.utxo_validation)?; + let colliding_transactions = + self.collision_manager.collect_colliding_transactions(&tx)?; + let dependencies = self.storage.collect_transaction_dependencies(&tx)?; + let has_dependencies = !dependencies.is_empty(); + self.collision_manager + .can_store_transaction( + &tx, + has_dependencies, + &colliding_transactions, + &self.storage, + ) + .map_err(Error::Collided)?; + let transactions_to_remove = + self.check_pool_size_available(&tx, &colliding_transactions, &dependencies)?; + let mut removed_transactions = vec![]; + for tx in transactions_to_remove { + let removed = self.storage.remove_transaction_and_dependents_subtree(tx)?; + removed_transactions.extend(removed); + } + for collision in colliding_transactions.keys() { + removed_transactions.extend( + self.storage + .remove_transaction_and_dependents_subtree(*collision)?, + ); + } + let storage_id = + self.storage + .store_transaction(tx, creation_instant, dependencies)?; + self.tx_id_to_storage_id.insert(tx_id, storage_id); + self.current_gas = self.current_gas.saturating_add(gas); + self.current_bytes_size = self.current_bytes_size.saturating_add(bytes_size); + // No dependencies directly in the graph and the sorted transactions + if !has_dependencies { + self.selection_algorithm + .new_executable_transactions(vec![storage_id], &self.storage)?; + } + self.update_components_and_caches_on_removal(removed_transactions.iter())?; + let tx = Storage::get(&self.storage, &storage_id)?; + self.collision_manager + .on_stored_transaction(&tx.transaction, storage_id)?; + self.selection_algorithm.on_stored_transaction( + &tx.transaction, + creation_instant, + storage_id, + ); + Ok(removed_transactions) + } + + /// Check if a transaction can be inserted into the pool. + pub fn can_insert_transaction(&self, tx: &PoolTransaction) -> Result<(), Error> { + let persistent_storage = self + .persistent_storage_provider + .latest_view() + .map_err(|e| Error::Database(format!("{:?}", e)))?; + self.config.black_list.check_blacklisting(tx)?; + Self::check_blob_does_not_exist(tx, &persistent_storage)?; + let colliding_transaction = + self.collision_manager.collect_colliding_transactions(tx)?; + self.storage.validate_inputs( + tx, + &persistent_storage, + self.config.utxo_validation, + )?; + let dependencies = self.storage.collect_transaction_dependencies(tx)?; + let has_dependencies = !dependencies.is_empty(); + self.collision_manager + .can_store_transaction( + tx, + has_dependencies, + &colliding_transaction, + &self.storage, + ) + .map_err(Error::Collided)?; + self.check_pool_size_available(tx, &colliding_transaction, &dependencies)?; + self.storage + .can_store_transaction(tx, &dependencies, &colliding_transaction)?; + Ok(()) } // TODO: Use block space also (https://github.com/FuelLabs/fuel-core/issues/2133) @@ -153,18 +212,86 @@ where Ok(extracted_transactions) } - #[cfg(test)] pub fn find_one(&self, tx_id: &TxId) -> Option<&PoolTransaction> { Storage::get(&self.storage, self.tx_id_to_storage_id.get(tx_id)?) .map(|data| &data.transaction) .ok() } - fn check_pool_is_not_full(&self) -> Result<(), Error> { - if self.storage.count() >= self.config.max_txs { + /// Check if the pool has enough space to store a transaction. + /// It will try to see if we can free some space depending on defined rules + /// If the pool is not full, it will return an empty list + /// If the pool is full, it will return the list of transactions that must be removed from the pool along all of their dependent subtree + /// If the pool is full and we can't make enough space by removing transactions, it will return an error + /// Currently, the rules are: + /// If a transaction is colliding with another verify if deleting the colliding transaction and dependents subtree is enough otherwise refuses the tx + /// If a transaction is dependent and not enough space, don't accept transaction + /// If a transaction is executable, try to free has much space used by less profitable transactions as possible in the pool to include it + fn check_pool_size_available( + &self, + tx: &PoolTransaction, + collided_transactions: &HashMap>, + dependencies: &[S::StorageIndex], + ) -> Result, Error> { + let tx_gas = tx.max_gas(); + let bytes_size = tx.metered_bytes_size(); + let mut removed_transactions = vec![]; + let mut gas_left = self.current_gas.saturating_add(tx_gas); + let mut bytes_left = self.current_bytes_size.saturating_add(bytes_size); + let mut txs_left = self.storage.count().saturating_add(1); + if gas_left <= self.config.pool_limits.max_gas + && bytes_left <= self.config.pool_limits.max_bytes_size + && txs_left <= self.config.pool_limits.max_txs + { + return Ok(vec![]); + } + + // If the transaction has a collision verify that by removing the transaction we can free enough space + // otherwise return an error + for collision in collided_transactions.keys() { + let collision_data = self.storage.get(collision)?; + gas_left = gas_left.saturating_sub(collision_data.dependents_cumulative_gas); + bytes_left = bytes_left + .saturating_sub(collision_data.dependents_cumulative_bytes_size); + txs_left = txs_left.saturating_sub(1); + removed_transactions.push(*collision); + if gas_left <= self.config.pool_limits.max_gas + && bytes_left <= self.config.pool_limits.max_bytes_size + && txs_left <= self.config.pool_limits.max_txs + { + return Ok(removed_transactions); + } + } + + // If the transaction has a dependency and the pool is full, we refuse it + if !dependencies.is_empty() { return Err(Error::NotInsertedLimitHit); } - Ok(()) + + // Here the transaction has no dependencies which means that it's an executable transaction + // and we want to make space for it + let current_ratio = Ratio::new(tx.tip(), tx_gas); + let mut sorted_txs = self.selection_algorithm.get_less_worth_txs(); + while gas_left > self.config.pool_limits.max_gas + || bytes_left > self.config.pool_limits.max_bytes_size + || txs_left > self.config.pool_limits.max_txs + { + let storage_id = sorted_txs.next().ok_or(Error::NotInsertedLimitHit)?; + let storage_data = self.storage.get(&storage_id)?; + let ratio = Ratio::new( + storage_data.dependents_cumulative_tip, + storage_data.dependents_cumulative_gas, + ); + if ratio > current_ratio { + return Err(Error::NotInsertedLimitHit); + } + gas_left = gas_left.saturating_sub(storage_data.dependents_cumulative_gas); + bytes_left = + bytes_left.saturating_sub(storage_data.dependents_cumulative_bytes_size); + txs_left = txs_left.saturating_sub(1); + removed_transactions.push(storage_id); + } + Ok(removed_transactions) } fn check_blob_does_not_exist( @@ -190,10 +317,11 @@ where for tx in removed_transactions { self.collision_manager.on_removed_transaction(tx)?; self.selection_algorithm.on_removed_transaction(tx)?; - #[cfg(test)] - { - self.tx_id_to_storage_id.remove(&tx.id()); - } + self.tx_id_to_storage_id.remove(&tx.id()); + self.current_gas = self.current_gas.saturating_sub(tx.max_gas()); + self.current_bytes_size = self + .current_bytes_size + .saturating_sub(tx.metered_bytes_size()); } Ok(()) } diff --git a/crates/services/txpool_v2/src/ports.rs b/crates/services/txpool_v2/src/ports.rs index dc888c51c8a..d6740a6d470 100644 --- a/crates/services/txpool_v2/src/ports.rs +++ b/crates/services/txpool_v2/src/ports.rs @@ -1,5 +1,8 @@ +use std::sync::Arc; + use fuel_core_storage::Result as StorageResult; use fuel_core_types::{ + blockchain::header::ConsensusParametersVersion, entities::{ coins::coin::CompressedCoin, relayer::message::Message, @@ -7,6 +10,7 @@ use fuel_core_types::{ fuel_tx::{ BlobId, Bytes32, + ConsensusParameters, ContractId, UtxoId, }, @@ -22,6 +26,15 @@ use crate::{ pub use fuel_core_storage::transactional::AtomicView; +/// Trait for getting the latest consensus parameters. +#[cfg_attr(feature = "test-helpers", mockall::automock)] +pub trait ConsensusParametersProvider { + /// Get latest consensus parameters. + fn latest_consensus_parameters( + &self, + ) -> (ConsensusParametersVersion, Arc); +} + pub trait TxPoolPersistentStorage: Send + Sync { /// Get the UTXO by its ID. fn utxo(&self, utxo_id: &UtxoId) -> StorageResult>; diff --git a/crates/services/txpool_v2/src/selection_algorithms/mod.rs b/crates/services/txpool_v2/src/selection_algorithms/mod.rs index bcbe27c9467..c5bb0ef1a97 100644 --- a/crates/services/txpool_v2/src/selection_algorithms/mod.rs +++ b/crates/services/txpool_v2/src/selection_algorithms/mod.rs @@ -1,6 +1,11 @@ +use std::time::Instant; + use fuel_core_types::services::txpool::PoolTransaction; -use crate::error::Error; +use crate::{ + error::Error, + storage::Storage, +}; pub mod ratio_tip_gas; @@ -29,6 +34,17 @@ pub trait SelectionAlgorithm { storage: &Self::Storage, ) -> Result<(), Error>; + /// Get less worth transactions iterator + fn get_less_worth_txs(&self) -> impl Iterator; + + /// Inform the collision manager that a transaction was stored. + fn on_stored_transaction( + &mut self, + transaction: &PoolTransaction, + creation_instant: Instant, + transaction_id: Self::StorageIndex, + ) -> Result<(), Error>; + /// Inform the selection algorithm that a transaction was removed from the pool. fn on_removed_transaction( &mut self, diff --git a/crates/services/txpool_v2/src/selection_algorithms/ratio_tip_gas.rs b/crates/services/txpool_v2/src/selection_algorithms/ratio_tip_gas.rs index 021a4d10ece..96b12c60229 100644 --- a/crates/services/txpool_v2/src/selection_algorithms/ratio_tip_gas.rs +++ b/crates/services/txpool_v2/src/selection_algorithms/ratio_tip_gas.rs @@ -3,7 +3,10 @@ use std::{ Ordering, Reverse, }, - collections::BTreeMap, + collections::{ + BTreeMap, + HashMap, + }, fmt::Debug, time::Instant, }; @@ -69,13 +72,17 @@ impl PartialOrd for Key { /// The selection algorithm that selects transactions based on the tip/gas ratio. pub struct RatioTipGasSelection { - transactions_sorted_tip_gas_ratio: BTreeMap, S::StorageIndex>, + executable_transactions_sorted_tip_gas_ratio: BTreeMap, S::StorageIndex>, + all_transactions_sorted_tip_gas_ratio: BTreeMap, S::StorageIndex>, + tx_id_to_creation_instant: HashMap, } impl RatioTipGasSelection { pub fn new() -> Self { Self { - transactions_sorted_tip_gas_ratio: BTreeMap::new(), + executable_transactions_sorted_tip_gas_ratio: BTreeMap::new(), + all_transactions_sorted_tip_gas_ratio: BTreeMap::new(), + tx_id_to_creation_instant: HashMap::new(), } } } @@ -102,11 +109,13 @@ impl SelectionAlgorithm // Take the first transaction with the highest tip/gas ratio if it fits in the gas limit // then promote all its dependents to the list of transactions to be executed // and repeat the process until the gas limit is reached - while gas_left > 0 && !self.transactions_sorted_tip_gas_ratio.is_empty() { + while gas_left > 0 + && !self.executable_transactions_sorted_tip_gas_ratio.is_empty() + { let mut new_executables = vec![]; let mut best_transaction = None; - let sorted_iter = self.transactions_sorted_tip_gas_ratio.iter(); + let sorted_iter = self.executable_transactions_sorted_tip_gas_ratio.iter(); for (key, storage_id) in sorted_iter { let enough_gas = { let stored_transaction = storage.get(storage_id)?; @@ -125,7 +134,8 @@ impl SelectionAlgorithm self.new_executable_transactions(new_executables, storage)?; // Remove the best transaction from the sorted list if let Some((key, best_transaction)) = best_transaction { - self.transactions_sorted_tip_gas_ratio.remove(&key); + self.executable_transactions_sorted_tip_gas_ratio + .remove(&key); best_transactions.push(best_transaction); } else { // If no transaction fits in the gas limit, @@ -149,26 +159,64 @@ impl SelectionAlgorithm ); let key = Key { ratio: tip_gas_ratio, - creation_instant: Instant::now(), + creation_instant: stored_transaction.creation_instant, tx_id: stored_transaction.transaction.id(), }; - self.transactions_sorted_tip_gas_ratio + self.executable_transactions_sorted_tip_gas_ratio .insert(Reverse(key), storage_id); + self.tx_id_to_creation_instant.insert( + stored_transaction.transaction.id(), + stored_transaction.creation_instant, + ); } Ok(()) } + fn get_less_worth_txs(&self) -> impl Iterator { + self.all_transactions_sorted_tip_gas_ratio.values().copied() + } + + fn on_stored_transaction( + &mut self, + transaction: &PoolTransaction, + creation_instant: Instant, + transaction_id: Self::StorageIndex, + ) -> Result<(), Error> { + let tip_gas_ratio = RatioTipGas::new(transaction.tip(), transaction.max_gas()); + let key = Key { + ratio: tip_gas_ratio, + creation_instant, + tx_id: transaction.id(), + }; + self.all_transactions_sorted_tip_gas_ratio + .insert(Reverse(key), transaction_id); + self.tx_id_to_creation_instant + .insert(transaction.id(), creation_instant); + Ok(()) + } + fn on_removed_transaction( &mut self, transaction: &PoolTransaction, ) -> Result<(), Error> { let tip_gas_ratio = RatioTipGas::new(transaction.tip(), transaction.max_gas()); + let creation_instant = *self + .tx_id_to_creation_instant + .get(&transaction.id()) + .ok_or(Error::TransactionNotFound( + "Expected the transaction to be in the tx_id_to_creation_instant map" + .to_string(), + ))?; let key = Key { ratio: tip_gas_ratio, - creation_instant: Instant::now(), + creation_instant, tx_id: transaction.id(), }; - self.transactions_sorted_tip_gas_ratio.remove(&Reverse(key)); + self.executable_transactions_sorted_tip_gas_ratio + .remove(&Reverse(key)); + self.all_transactions_sorted_tip_gas_ratio + .remove(&Reverse(key)); + self.tx_id_to_creation_instant.remove(&transaction.id()); Ok(()) } } diff --git a/crates/services/txpool_v2/src/service.rs b/crates/services/txpool_v2/src/service.rs index cce13d11b8f..ba6bb2b06e8 100644 --- a/crates/services/txpool_v2/src/service.rs +++ b/crates/services/txpool_v2/src/service.rs @@ -1,35 +1,229 @@ +use std::sync::Arc; + use fuel_core_services::{ RunnableService, RunnableTask, ServiceRunner, StateWatcher, }; -use fuel_core_types::fuel_tx::Transaction; +use fuel_core_types::{ + blockchain::consensus::Consensus, + entities::relayer::transaction, + fuel_tx::{ + Transaction, + TxId, + }, + fuel_types::BlockHeight, + fuel_vm::checked_transaction::CheckedTransaction, + services::txpool::PoolTransaction, +}; +use parking_lot::RwLock; + +use crate::{ + collision_manager::basic::BasicCollisionManager, + config::Config, + error::Error, + heavy_async_processing::HeavyAsyncProcessor, + pool::Pool, + ports::{ + AtomicView, + ConsensusParametersProvider, + GasPriceProvider as GasPriceProviderTrait, + MemoryPool as MemoryPoolTrait, + TxPoolPersistentStorage, + WasmChecker as WasmCheckerTrait, + }, + selection_algorithms::ratio_tip_gas::RatioTipGasSelection, + storage::graph::{ + GraphConfig, + GraphStorage, + }, + verifications::perform_all_verifications, +}; + +pub type RemovedTransactions = Vec; +pub type InsertionResult = Result; + +pub type TxPool = Arc< + RwLock< + Pool< + PSProvider, + GraphStorage, + BasicCollisionManager, + RatioTipGasSelection, + >, + >, +>; -#[derive(Clone)] -pub struct SharedState; +pub struct SharedState< + PSProvider, + ConsensusParamsProvider, + GasPriceProvider, + WasmChecker, + MemoryPool, +> { + pool: TxPool, + current_height: Arc>, + consensus_parameters_provider: Arc, + gas_price_provider: Arc, + wasm_checker: Arc, + memory: Arc, + heavy_async_processor: Arc, + utxo_validation: bool, +} + +impl Clone + for SharedState< + PSProvider, + ConsensusParamsProvider, + GasPriceProvider, + WasmChecker, + MemoryPool, + > +{ + fn clone(&self) -> Self { + SharedState { + pool: self.pool.clone(), + current_height: self.current_height.clone(), + consensus_parameters_provider: self.consensus_parameters_provider.clone(), + gas_price_provider: self.gas_price_provider.clone(), + wasm_checker: self.wasm_checker.clone(), + memory: self.memory.clone(), + heavy_async_processor: self.heavy_async_processor.clone(), + utxo_validation: self.utxo_validation, + } + } +} -impl SharedState { - // TODO: Implement conversion from `Transaction` to `PoolTransaction`. (with all the verifications that it implies): https://github.com/FuelLabs/fuel-core/issues/2186 - fn insert(&mut self, transactions: Vec) -> Vec<()> { - // Move verif of wasm there - vec![] +impl< + PSProvider, + PSView, + ConsensusParamsProvider, + GasPriceProvider, + WasmChecker, + MemoryPool, + > + SharedState< + PSProvider, + ConsensusParamsProvider, + GasPriceProvider, + WasmChecker, + MemoryPool, + > +where + PSProvider: AtomicView + 'static, + PSView: TxPoolPersistentStorage, + ConsensusParamsProvider: ConsensusParametersProvider + 'static, + GasPriceProvider: GasPriceProviderTrait + Send + Sync + 'static, + WasmChecker: WasmCheckerTrait + Send + Sync + 'static, + MemoryPool: MemoryPoolTrait + Send + Sync + 'static, +{ + async fn insert( + &self, + transactions: Vec, + ) -> Result, Error> { + let current_height = *self.current_height.read(); + let (version, params) = self + .consensus_parameters_provider + .latest_consensus_parameters(); + let mut results = vec![]; + for transaction in transactions { + self.heavy_async_processor.spawn({ + let shared_state = self.clone(); + let params = params.clone(); + async move { + // TODO: Return the error in the status update channel (see: https://github.com/FuelLabs/fuel-core/issues/2185) + let checked_tx = perform_all_verifications( + transaction, + shared_state.pool.clone(), + current_height, + ¶ms, + version, + shared_state.gas_price_provider.as_ref(), + shared_state.wasm_checker.as_ref(), + shared_state.memory.get_memory().await, + ) + .await + .unwrap(); + let result = { + let mut pool = shared_state.pool.write(); + // TODO: Return the result of the insertion (see: https://github.com/FuelLabs/fuel-core/issues/2185) + pool.insert(checked_tx) + }; + } + }); + } + Ok(results) } } -pub type Service = ServiceRunner; +pub type Service< + PSProvider, + ConsensusParamsProvider, + GasPriceProvider, + WasmChecker, + MemoryPool, +> = ServiceRunner< + Task, +>; -pub struct Task { - shared_state: SharedState, +pub struct Task< + PSProvider, + ConsensusParamsProvider, + GasPriceProvider, + WasmChecker, + MemoryPool, +> { + shared_state: SharedState< + PSProvider, + ConsensusParamsProvider, + GasPriceProvider, + WasmChecker, + MemoryPool, + >, } #[async_trait::async_trait] -impl RunnableService for Task { +impl< + PSProvider, + PSView, + ConsensusParamsProvider, + GasPriceProvider, + WasmChecker, + MemoryPool, + > RunnableService + for Task< + PSProvider, + ConsensusParamsProvider, + GasPriceProvider, + WasmChecker, + MemoryPool, + > +where + PSProvider: AtomicView, + PSView: TxPoolPersistentStorage, + ConsensusParamsProvider: ConsensusParametersProvider + Send + Sync, + GasPriceProvider: GasPriceProviderTrait + Send + Sync, + WasmChecker: WasmCheckerTrait + Send + Sync, + MemoryPool: MemoryPoolTrait + Send + Sync, +{ const NAME: &'static str = "TxPoolv2"; - type SharedData = SharedState; + type SharedData = SharedState< + PSProvider, + ConsensusParamsProvider, + GasPriceProvider, + WasmChecker, + MemoryPool, + >; - type Task = Task; + type Task = Task< + PSProvider, + ConsensusParamsProvider, + GasPriceProvider, + WasmChecker, + MemoryPool, + >; type TaskParams = (); @@ -47,7 +241,29 @@ impl RunnableService for Task { } #[async_trait::async_trait] -impl RunnableTask for Task { +impl< + PSProvider, + PSView, + ConsensusParamsProvider, + GasPriceProvider, + WasmChecker, + MemoryPool, + > RunnableTask + for Task< + PSProvider, + ConsensusParamsProvider, + GasPriceProvider, + WasmChecker, + MemoryPool, + > +where + PSProvider: AtomicView, + PSView: TxPoolPersistentStorage, + ConsensusParamsProvider: ConsensusParametersProvider + Send + Sync, + GasPriceProvider: GasPriceProviderTrait + Send + Sync, + WasmChecker: WasmCheckerTrait + Send + Sync, + MemoryPool: MemoryPoolTrait + Send + Sync, +{ async fn run(&mut self, watcher: &mut StateWatcher) -> anyhow::Result { let should_continue; tokio::select! { @@ -63,8 +279,54 @@ impl RunnableTask for Task { } } -pub fn new_service() -> Service { +pub fn new_service< + PSProvider, + PSView, + ConsensusParamsProvider, + GasPriceProvider, + WasmChecker, + MemoryPool, +>( + config: Config, + ps_provider: PSProvider, + consensus_parameters_provider: ConsensusParamsProvider, + current_height: BlockHeight, + gas_price_provider: GasPriceProvider, + wasm_checker: WasmChecker, + memory_pool: MemoryPool, +) -> Service +where + PSProvider: AtomicView, + PSView: TxPoolPersistentStorage, + ConsensusParamsProvider: ConsensusParametersProvider + Send + Sync, + GasPriceProvider: GasPriceProviderTrait + Send + Sync, + WasmChecker: WasmCheckerTrait + Send + Sync, + MemoryPool: MemoryPoolTrait + Send + Sync, +{ Service::new(Task { - shared_state: SharedState, + shared_state: SharedState { + consensus_parameters_provider: Arc::new(consensus_parameters_provider), + gas_price_provider: Arc::new(gas_price_provider), + wasm_checker: Arc::new(wasm_checker), + memory: Arc::new(memory_pool), + current_height: Arc::new(RwLock::new(current_height)), + utxo_validation: config.utxo_validation, + heavy_async_processor: Arc::new( + HeavyAsyncProcessor::new( + config.heavy_work.number_threads_to_verify_transactions, + config.heavy_work.size_of_verification_queue, + ) + .unwrap(), + ), + pool: Arc::new(RwLock::new(Pool::new( + ps_provider, + GraphStorage::new(GraphConfig { + max_txs_chain_count: config.max_txs_chain_count, + }), + BasicCollisionManager::new(), + RatioTipGasSelection::new(), + config, + ))), + }, }) } diff --git a/crates/services/txpool_v2/src/storage/graph.rs b/crates/services/txpool_v2/src/storage/graph.rs index 150d5b33ce6..9e4d25e49b8 100644 --- a/crates/services/txpool_v2/src/storage/graph.rs +++ b/crates/services/txpool_v2/src/storage/graph.rs @@ -1,5 +1,6 @@ use std::{ collections::{ + BTreeSet, HashMap, HashSet, }, @@ -29,17 +30,18 @@ use fuel_core_types::{ }, services::txpool::PoolTransaction, }; +use num_rational::Ratio; use petgraph::{ graph::NodeIndex, prelude::StableDiGraph, }; use crate::{ - collision_manager::{ - basic::BasicCollisionManagerStorage, + collision_manager::basic::BasicCollisionManagerStorage, + error::{ CollisionReason, + Error, }, - error::Error, ports::TxPoolPersistentStorage, selection_algorithms::ratio_tip_gas::RatioTipGasSelectionAlgorithmStorage, }; @@ -79,21 +81,6 @@ impl GraphStorage { } impl GraphStorage { - /// Remove a node and all its dependent sub-graph. - /// Edit the data of dependencies transactions accordingly. - /// Returns the removed transactions. - fn remove_node_and_dependent_sub_graph( - &mut self, - root_id: NodeIndex, - ) -> Vec { - let Some(root) = self.graph.node_weight(root_id) else { - debug_assert!(false, "Node with id {:?} not found", root_id); - return vec![]; - }; - let gas_reduction = root.dependents_cumulative_gas; - let tip_reduction = root.dependents_cumulative_tip; - self.remove_dependent_sub_graph(root_id) - } fn reduce_dependencies_cumulative_gas_tip_and_chain_count( &mut self, root_id: NodeIndex, @@ -124,7 +111,13 @@ impl GraphStorage { } } - fn remove_dependent_sub_graph(&mut self, root_id: NodeIndex) -> Vec { + /// Remove a node and all its dependent sub-graph. + /// Edit the data of dependencies transactions accordingly. + /// Returns the removed transactions. + fn remove_node_and_dependent_sub_graph( + &mut self, + root_id: NodeIndex, + ) -> Vec { let dependencies: Vec = self.get_dependencies(root_id).collect(); let dependents: Vec<_> = self .graph @@ -145,7 +138,8 @@ impl GraphStorage { self.clear_cache(root.transaction.outputs(), &root.transaction.id()); let mut removed_transactions = vec![root.transaction]; for dependent in dependents { - removed_transactions.extend(self.remove_dependent_sub_graph(dependent)); + removed_transactions + .extend(self.remove_node_and_dependent_sub_graph(dependent)); } removed_transactions } @@ -259,6 +253,26 @@ impl GraphStorage { self.graph .neighbors_directed(index, petgraph::Direction::Outgoing) } + + fn is_in_dependencies_subtrees( + &self, + index: NodeIndex, + transactions: &[NodeIndex], + ) -> Result { + let mut already_visited = HashSet::new(); + let mut to_check = transactions.to_vec(); + while let Some(node_id) = to_check.pop() { + if already_visited.contains(&node_id) { + continue; + } + if node_id == index { + return Ok(true); + } + already_visited.insert(node_id); + to_check.extend(self.get_dependencies(node_id)); + } + Ok(false) + } } impl Storage for GraphStorage { @@ -267,25 +281,21 @@ impl Storage for GraphStorage { fn store_transaction( &mut self, transaction: PoolTransaction, + creation_instant: Instant, dependencies: Vec, - collided_transactions: &HashSet, - ) -> Result<(Self::StorageIndex, RemovedTransactions), Error> { + ) -> Result { let tx_id = transaction.id(); // Add the new transaction to the graph and update the others in consequence let tip = transaction.tip(); let gas = transaction.max_gas(); + let size = transaction.metered_bytes_size(); let outputs = transaction.outputs().clone(); // Check if the dependency chain is too big let mut all_dependencies_recursively = HashSet::new(); let mut to_check = dependencies.clone(); while let Some(node_id) = to_check.pop() { - if collided_transactions.contains(&node_id) { - return Err(Error::Collided( - "Use a collided transaction as a dependency".to_string(), - )); - } // Already checked node if all_dependencies_recursively.contains(&node_id) { continue; @@ -308,17 +318,12 @@ impl Storage for GraphStorage { to_check.extend(self.get_dependencies(node_id)); } - // Remove collisions and their dependencies from the graph - let mut removed_transactions = vec![]; - for collision in collided_transactions { - removed_transactions - .extend(self.remove_node_and_dependent_sub_graph(*collision)); - } - let node = StorageData { dependents_cumulative_tip: tip, dependents_cumulative_gas: gas, + dependents_cumulative_bytes_size: size, transaction, + creation_instant, number_dependents_in_chain: 0, }; @@ -343,8 +348,48 @@ impl Storage for GraphStorage { node.dependents_cumulative_tip.saturating_add(tip); node.dependents_cumulative_gas = node.dependents_cumulative_gas.saturating_add(gas); + node.dependents_cumulative_bytes_size = + node.dependents_cumulative_bytes_size.saturating_add(size); + } + Ok(node_id) + } + + fn can_store_transaction( + &self, + transaction: &PoolTransaction, + dependencies: &[Self::StorageIndex], + colliding_transactions: &HashMap>, + ) -> Result<(), Error> { + for collision in colliding_transactions.keys() { + if self.is_in_dependencies_subtrees(*collision, dependencies)? { + return Err(Error::NotInsertedCollisionIsDependency); + } + } + let mut all_dependencies_recursively = HashSet::new(); + let mut to_check = dependencies.to_vec(); + while let Some(node_id) = to_check.pop() { + // Already checked node + if all_dependencies_recursively.contains(&node_id) { + continue; + } + let Some(dependency_node) = self.graph.node_weight(node_id) else { + return Err(Error::Storage(format!( + "Node with id {:?} not found", + node_id + ))); + }; + if dependency_node.number_dependents_in_chain + >= self.config.max_txs_chain_count + { + return Err(Error::NotInsertedChainDependencyTooBig); + } + all_dependencies_recursively.insert(node_id); + if all_dependencies_recursively.len() >= self.config.max_txs_chain_count { + return Err(Error::NotInsertedChainDependencyTooBig); + } + to_check.extend(self.get_dependencies(node_id)); } - Ok((node_id, removed_transactions)) + Ok(()) } fn get(&self, index: &Self::StorageIndex) -> Result<&StorageData, Error> { @@ -484,8 +529,15 @@ impl Storage for GraphStorage { }) } - fn count(&self) -> u64 { - self.graph.node_count() as u64 + fn remove_transaction_and_dependents_subtree( + &mut self, + index: Self::StorageIndex, + ) -> Result { + Ok(self.remove_node_and_dependent_sub_graph(index)) + } + + fn count(&self) -> usize { + self.graph.node_count() } } diff --git a/crates/services/txpool_v2/src/storage/mod.rs b/crates/services/txpool_v2/src/storage/mod.rs index aecadd4fa94..b7de06b0844 100644 --- a/crates/services/txpool_v2/src/storage/mod.rs +++ b/crates/services/txpool_v2/src/storage/mod.rs @@ -1,12 +1,17 @@ use std::{ - collections::HashSet, + collections::{ + HashMap, + HashSet, + }, fmt::Debug, time::Instant, }; use crate::{ - collision_manager::CollisionReason, - error::Error, + error::{ + CollisionReason, + Error, + }, ports::TxPoolPersistentStorage, }; use fuel_core_types::services::txpool::PoolTransaction; @@ -21,8 +26,12 @@ pub struct StorageData { pub dependents_cumulative_tip: u64, /// The cumulative gas of a transaction and all of its children. pub dependents_cumulative_gas: u64, + /// The cumulative of space used by a transaction and all of its children. + pub dependents_cumulative_bytes_size: usize, /// Number of dependents pub number_dependents_in_chain: usize, + /// The instant when the transaction was added to the pool. + pub creation_instant: Instant, } pub type RemovedTransactions = Vec; @@ -33,14 +42,23 @@ pub trait Storage { /// The index type used in the storage and allow other components to reference transactions. type StorageIndex: Copy + Debug; - /// Store a transaction in the storage according to the dependencies and collisions. - /// Returns the index of the stored transaction and the transactions that were removed from the storage in favor of the new transaction. + /// Store a transaction in the storage according to the dependencies. + /// Returns the index of the stored transaction. fn store_transaction( &mut self, transaction: PoolTransaction, + creation_instant: Instant, dependencies: Vec, - collided_transactions: &HashSet, - ) -> Result<(Self::StorageIndex, RemovedTransactions), Error>; + ) -> Result; + + /// Check if a transaction could be stored in the storage. This shouldn't be expected to be called before store_transaction. + /// Its just a way to perform some checks without storing the transaction. + fn can_store_transaction( + &self, + transaction: &PoolTransaction, + dependencies: &[Self::StorageIndex], + collisions: &HashMap>, + ) -> Result<(), Error>; /// Get the storage data by its index. fn get(&self, index: &Self::StorageIndex) -> Result<&StorageData, Error>; @@ -79,6 +97,12 @@ pub trait Storage { index: Self::StorageIndex, ) -> Result; + /// Remove a transaction along with its dependents subtree. + fn remove_transaction_and_dependents_subtree( + &mut self, + index: Self::StorageIndex, + ) -> Result; + /// Count the number of transactions in the storage. - fn count(&self) -> u64; + fn count(&self) -> usize; } diff --git a/crates/services/txpool_v2/src/tests/context.rs b/crates/services/txpool_v2/src/tests/context.rs index 047f12f26e9..ed1a8658846 100644 --- a/crates/services/txpool_v2/src/tests/context.rs +++ b/crates/services/txpool_v2/src/tests/context.rs @@ -1,3 +1,5 @@ +use std::sync::Arc; + use fuel_core_types::{ entities::{ coins::coin::{ @@ -25,7 +27,6 @@ use fuel_core_types::{ contract::Contract as ContractInput, Input, }, - Bytes32, ConsensusParameters, Contract, ContractId, @@ -41,160 +42,233 @@ use fuel_core_types::{ Word, }, fuel_vm::{ - checked_transaction::{ - Checked, - EstimatePredicates, - }, + checked_transaction::EstimatePredicates, interpreter::MemoryInstance, }, }; +use parking_lot::RwLock; use crate::{ collision_manager::basic::BasicCollisionManager, config::Config, error::Error, pool::Pool, - ports::{ - GasPriceProvider, - WasmChecker, - WasmValidityError, - }, selection_algorithms::ratio_tip_gas::RatioTipGasSelection, + service::{ + RemovedTransactions, + TxPool, + }, storage::graph::{ GraphConfig, GraphStorage, }, - tests::mock_db::{ + tests::mocks::{ MockDBProvider, MockDb, }, - transaction_conversion::check_single_tx, + verifications::perform_all_verifications, GasPrice, }; -// TDOO: Reorganize this file - -pub(crate) fn create_message_predicate_from_message( - amount: Word, - nonce: u64, -) -> (Message, Input) { - let predicate = vec![op::ret(1)].into_iter().collect::>(); - let message = MessageV1 { - sender: Default::default(), - recipient: Input::predicate_owner(&predicate), - nonce: nonce.into(), - amount, - data: vec![], - da_height: Default::default(), - }; - - ( - message.clone().into(), - Input::message_coin_predicate( - message.sender, - Input::predicate_owner(&predicate), - message.amount, - message.nonce, - Default::default(), - predicate, - Default::default(), - ) - .into_default_estimated(), - ) -} - -pub(crate) fn create_coin_output() -> Output { - Output::coin(Default::default(), Default::default(), Default::default()) -} - -pub(crate) fn create_contract_input( - tx_id: TxId, - output_index: u16, - contract_id: ContractId, -) -> Input { - Input::contract( - UtxoId::new(tx_id, output_index), - Default::default(), - Default::default(), - Default::default(), - contract_id, - ) -} -pub(crate) fn create_contract_output(contract_id: ContractId) -> Output { - Output::contract_created(contract_id, Contract::default_state_root()) -} +use super::mocks::{ + MockTxPoolGasPrice, + MockWasmChecker, +}; // use some arbitrary large amount, this shouldn't affect the txpool logic except for covering // the byte and gas price fees. pub const TEST_COIN_AMOUNT: u64 = 100_000_000u64; +pub const GAS_LIMIT: Word = 100000; -pub(crate) struct PoolContext { +pub struct TestPoolUniverse { mock_db: MockDb, rng: StdRng, - pub(crate) config: Config, + pub config: Config, + pool: Option>, } -impl Default for PoolContext { +impl Default for TestPoolUniverse { fn default() -> Self { Self { mock_db: MockDb::default(), rng: StdRng::seed_from_u64(0), config: Default::default(), + pool: None, } } } -impl PoolContext { - pub(crate) fn database_mut(&mut self) -> &mut MockDb { +impl TestPoolUniverse { + pub fn database_mut(&mut self) -> &mut MockDb { &mut self.mock_db } - pub(crate) fn config(self, config: Config) -> Self { + pub fn config(self, config: Config) -> Self { + if self.pool.is_some() { + panic!("Pool already built"); + } Self { config, ..self } } - pub(crate) fn build( - self, - ) -> Pool< - MockDBProvider, - GraphStorage, - BasicCollisionManager, - RatioTipGasSelection, - > { - Pool::new( - MockDBProvider(self.mock_db), + pub fn build_pool(&mut self) -> TxPool { + let pool = Arc::new(RwLock::new(Pool::new( + MockDBProvider(self.mock_db.clone()), GraphStorage::new(GraphConfig { max_txs_chain_count: self.config.max_txs_chain_count, }), BasicCollisionManager::new(), RatioTipGasSelection::new(), - self.config, - ) + self.config.clone(), + ))); + self.pool = Some(pool.clone()); + pool } - pub(crate) fn setup_coin(&mut self) -> (Coin, Input) { - setup_coin(&mut self.rng, Some(&self.mock_db)) + pub fn build_script_transaction( + &mut self, + inputs: Option>, + outputs: Option>, + tip: u64, + ) -> Transaction { + let mut inputs = inputs.unwrap_or_default(); + let (_, gas_coin) = self.setup_coin(); + inputs.push(gas_coin); + let outputs = outputs.unwrap_or_default(); + let mut tx_builder = TransactionBuilder::script(vec![], vec![]); + tx_builder.script_gas_limit(GAS_LIMIT); + for input in inputs { + tx_builder.add_input(input); + } + for output in outputs { + tx_builder.add_output(output); + } + tx_builder.tip(tip); + tx_builder.max_fee_limit(10000); + tx_builder.finalize().into() } - pub(crate) fn create_output_and_input( + pub async fn verify_and_insert( &mut self, - amount: Word, - ) -> (Output, UnsetInput) { - let input = self.random_predicate(AssetId::BASE, amount, None); - let output = Output::coin(*input.input_owner().unwrap(), amount, AssetId::BASE); + tx: Transaction, + ) -> Result { + if let Some(pool) = &self.pool { + let tx = perform_all_verifications( + tx, + pool.clone(), + Default::default(), + &ConsensusParameters::default(), + 0, + &MockTxPoolGasPrice::new(0), + &MockWasmChecker::new(Ok(())), + MemoryInstance::new(), + ) + .await?; + pool.write().insert(tx) + } else { + panic!("Pool needs to be built first"); + } + } + + pub async fn verify_and_insert_with_gas_price( + &mut self, + tx: Transaction, + gas_price: GasPrice, + ) -> Result { + if let Some(pool) = &self.pool { + let tx = perform_all_verifications( + tx, + pool.clone(), + Default::default(), + &ConsensusParameters::default(), + 0, + &MockTxPoolGasPrice::new(gas_price), + &MockWasmChecker::new(Ok(())), + MemoryInstance::new(), + ) + .await?; + pool.write().insert(tx) + } else { + panic!("Pool needs to be built first"); + } + } + + pub async fn verify_and_insert_with_consensus_params_wasm_checker( + &mut self, + tx: Transaction, + consensus_params: ConsensusParameters, + wasm_checker: MockWasmChecker, + ) -> Result { + if let Some(pool) = &self.pool { + let tx = perform_all_verifications( + tx, + pool.clone(), + Default::default(), + &consensus_params, + 0, + &MockTxPoolGasPrice::new(0), + &wasm_checker, + MemoryInstance::new(), + ) + .await?; + pool.write().insert(tx) + } else { + panic!("Pool needs to be built first"); + } + } + + pub fn get_pool(&self) -> TxPool { + self.pool.clone().unwrap() + } + + pub fn setup_coin(&mut self) -> (Coin, Input) { + let input = self.random_predicate(AssetId::BASE, TEST_COIN_AMOUNT, None); + + // add coin to the state + let mut coin = CompressedCoin::default(); + coin.set_owner(*input.input_owner().unwrap()); + coin.set_amount(TEST_COIN_AMOUNT); + coin.set_asset_id(*input.asset_id(&AssetId::BASE).unwrap()); + let utxo_id = *input.utxo_id().unwrap(); + self.mock_db + .data + .lock() + .unwrap() + .coins + .insert(utxo_id, coin.clone()); + (coin.uncompress(utxo_id), input) + } + + pub fn create_output_and_input(&mut self) -> (Output, UnsetInput) { + let input = self.random_predicate(AssetId::BASE, 1, None); + let output = Output::coin(*input.input_owner().unwrap(), 1, AssetId::BASE); (output, UnsetInput(input)) } - pub(crate) fn random_predicate( + pub fn random_predicate( &mut self, asset_id: AssetId, amount: Word, utxo_id: Option, ) -> Input { - random_predicate(&mut self.rng, asset_id, amount, utxo_id) + // use predicate inputs to avoid expensive cryptography for signatures + let mut predicate_code: Vec = vec![op::ret(1)].into_iter().collect(); + // append some randomizing bytes after the predicate has already returned. + predicate_code.push(self.rng.gen()); + let owner = Input::predicate_owner(&predicate_code); + Input::coin_predicate( + utxo_id.unwrap_or_else(|| self.rng.gen()), + owner, + amount, + asset_id, + Default::default(), + Default::default(), + predicate_code, + vec![], + ) + .into_default_estimated() } - pub(crate) fn custom_predicate( + pub fn custom_predicate( &mut self, asset_id: AssetId, amount: Word, @@ -215,63 +289,51 @@ impl PoolContext { } } -pub(crate) fn setup_coin(rng: &mut StdRng, mock_db: Option<&MockDb>) -> (Coin, Input) { - let input = random_predicate(rng, AssetId::BASE, TEST_COIN_AMOUNT, None); - add_coin_to_state(input, mock_db) -} +pub fn create_message_predicate_from_message( + amount: Word, + nonce: u64, +) -> (Message, Input) { + let predicate = vec![op::ret(1)].into_iter().collect::>(); + let message = MessageV1 { + sender: Default::default(), + recipient: Input::predicate_owner(&predicate), + nonce: nonce.into(), + amount, + data: vec![], + da_height: Default::default(), + }; -pub(crate) fn add_coin_to_state(input: Input, mock_db: Option<&MockDb>) -> (Coin, Input) { - let mut coin = CompressedCoin::default(); - coin.set_owner(*input.input_owner().unwrap()); - coin.set_amount(TEST_COIN_AMOUNT); - coin.set_asset_id(*input.asset_id(&AssetId::BASE).unwrap()); - let utxo_id = *input.utxo_id().unwrap(); - if let Some(mock_db) = mock_db { - mock_db - .data - .lock() - .unwrap() - .coins - .insert(utxo_id, coin.clone()); - } - (coin.uncompress(utxo_id), input) + ( + message.clone().into(), + Input::message_coin_predicate( + message.sender, + Input::predicate_owner(&predicate), + message.amount, + message.nonce, + Default::default(), + predicate, + Default::default(), + ) + .into_default_estimated(), + ) } -pub(crate) fn random_predicate( - rng: &mut StdRng, - asset_id: AssetId, - amount: Word, - utxo_id: Option, +pub fn create_contract_input( + tx_id: TxId, + output_index: u16, + contract_id: ContractId, ) -> Input { - // use predicate inputs to avoid expensive cryptography for signatures - let mut predicate_code: Vec = vec![op::ret(1)].into_iter().collect(); - // append some randomizing bytes after the predicate has already returned. - predicate_code.push(rng.gen()); - let owner = Input::predicate_owner(&predicate_code); - Input::coin_predicate( - utxo_id.unwrap_or_else(|| rng.gen()), - owner, - amount, - asset_id, + Input::contract( + UtxoId::new(tx_id, output_index), Default::default(), Default::default(), - predicate_code, - vec![], + Default::default(), + contract_id, ) - .into_default_estimated() } -pub struct MockWasmChecker { - pub result: Result<(), WasmValidityError>, -} - -impl WasmChecker for MockWasmChecker { - fn validate_uploaded_wasm( - &self, - _wasm_root: &Bytes32, - ) -> Result<(), WasmValidityError> { - self.result - } +pub fn create_contract_output(contract_id: ContractId) -> Output { + Output::contract_created(contract_id, Contract::default_state_root()) } pub struct UnsetInput(Input); @@ -311,72 +373,3 @@ impl IntoEstimated for Input { tx.inputs()[0].clone() } } - -#[derive(Debug, Clone)] -pub struct MockTxPoolGasPrice { - pub gas_price: Option, -} - -impl MockTxPoolGasPrice { - pub fn new(gas_price: GasPrice) -> Self { - Self { - gas_price: Some(gas_price), - } - } -} - -#[async_trait::async_trait] -impl GasPriceProvider for MockTxPoolGasPrice { - async fn next_gas_price(&self) -> Result { - self.gas_price - .ok_or(Error::GasPriceNotFound("Gas price not found".to_string())) - } -} - -pub async fn check_unwrap_tx(tx: Transaction, config: &Config) -> Checked { - let gas_price = 0; - check_unwrap_tx_with_gas_price(tx, config, gas_price).await -} - -pub async fn check_unwrap_tx_with_gas_price( - tx: Transaction, - config: &Config, - gas_price: GasPrice, -) -> Checked { - let gas_price_provider = MockTxPoolGasPrice::new(gas_price); - check_single_tx( - tx, - Default::default(), - config.utxo_validation, - &ConsensusParameters::default(), - &gas_price_provider, - MemoryInstance::new(), - ) - .await - .expect("Transaction should be checked") -} - -pub async fn check_tx( - tx: Transaction, - config: &Config, -) -> Result, Error> { - let gas_price = 0; - check_tx_with_gas_price(tx, config, gas_price).await -} - -pub async fn check_tx_with_gas_price( - tx: Transaction, - config: &Config, - gas_price: GasPrice, -) -> Result, Error> { - let gas_price_provider = MockTxPoolGasPrice::new(gas_price); - check_single_tx( - tx, - Default::default(), - config.utxo_validation, - &ConsensusParameters::default(), - &gas_price_provider, - MemoryInstance::new(), - ) - .await -} diff --git a/crates/services/txpool_v2/src/tests/mock_db.rs b/crates/services/txpool_v2/src/tests/mocks.rs similarity index 67% rename from crates/services/txpool_v2/src/tests/mock_db.rs rename to crates/services/txpool_v2/src/tests/mocks.rs index 91ff57cf08c..2c3c5e3d28d 100644 --- a/crates/services/txpool_v2/src/tests/mock_db.rs +++ b/crates/services/txpool_v2/src/tests/mocks.rs @@ -1,6 +1,13 @@ -use crate::ports::{ - AtomicView, - TxPoolPersistentStorage, +use crate::{ + error::Error, + ports::{ + AtomicView, + GasPriceProvider, + TxPoolPersistentStorage, + WasmChecker, + WasmValidityError, + }, + GasPrice, }; use fuel_core_storage::Result as StorageResult; use fuel_core_types::{ @@ -13,6 +20,7 @@ use fuel_core_types::{ }, fuel_tx::{ BlobId, + Bytes32, Contract, ContractId, UtxoId, @@ -107,3 +115,47 @@ impl AtomicView for MockDBProvider { Ok(self.0.clone()) } } + +#[derive(Debug, Clone)] +pub struct MockTxPoolGasPrice { + pub gas_price: Option, +} + +impl MockTxPoolGasPrice { + pub fn new(gas_price: GasPrice) -> Self { + Self { + gas_price: Some(gas_price), + } + } + + pub fn new_none() -> Self { + Self { gas_price: None } + } +} + +#[async_trait::async_trait] +impl GasPriceProvider for MockTxPoolGasPrice { + async fn next_gas_price(&self) -> Result { + self.gas_price + .ok_or(Error::GasPriceNotFound("Gas price not found".to_string())) + } +} + +pub struct MockWasmChecker { + pub result: Result<(), WasmValidityError>, +} + +impl MockWasmChecker { + pub fn new(result: Result<(), WasmValidityError>) -> Self { + Self { result } + } +} + +impl WasmChecker for MockWasmChecker { + fn validate_uploaded_wasm( + &self, + _wasm_root: &Bytes32, + ) -> Result<(), WasmValidityError> { + self.result + } +} diff --git a/crates/services/txpool_v2/src/tests/mod.rs b/crates/services/txpool_v2/src/tests/mod.rs index 71f7da9ba9c..5d39023e6aa 100644 --- a/crates/services/txpool_v2/src/tests/mod.rs +++ b/crates/services/txpool_v2/src/tests/mod.rs @@ -1,3 +1,3 @@ mod context; -mod mock_db; +mod mocks; mod pool; diff --git a/crates/services/txpool_v2/src/tests/pool.rs b/crates/services/txpool_v2/src/tests/pool.rs index ed334787797..8e156efd2f0 100644 --- a/crates/services/txpool_v2/src/tests/pool.rs +++ b/crates/services/txpool_v2/src/tests/pool.rs @@ -1,28 +1,32 @@ #![allow(non_snake_case)] use crate::{ - config::Config, - error::Error, - tests::context::{ - check_tx, - check_tx_with_gas_price, - check_unwrap_tx, - check_unwrap_tx_with_gas_price, - create_coin_output, - create_contract_input, - create_contract_output, - create_message_predicate_from_message, - IntoEstimated, - PoolContext, - TEST_COIN_AMOUNT, + config::{ + Config, + PoolLimits, + }, + error::{ + CollisionReason, + Error, + }, + ports::WasmValidityError, + tests::{ + context::{ + create_contract_input, + create_contract_output, + create_message_predicate_from_message, + IntoEstimated, + TestPoolUniverse, + GAS_LIMIT, + TEST_COIN_AMOUNT, + }, + mocks::MockWasmChecker, }, }; use fuel_core_types::{ - blockchain::header::ConsensusParametersVersion, fuel_asm::{ op, RegId, - Word, }, fuel_tx::{ input::coin::CoinPredicate, @@ -31,241 +35,161 @@ use fuel_core_types::{ BlobBody, BlobId, BlobIdExt, + Bytes32, + Chargeable, ConsensusParameters, Contract, - Finalizable, Input, Output, + PanicReason, PredicateParameters, - Transaction, TransactionBuilder, TxParameters, UniqueIdentifier, + UpgradePurpose, UtxoId, }, fuel_types::ChainId, - fuel_vm::checked_transaction::{ - CheckError, - Checked, - CheckedTransaction, - IntoChecked, + fuel_vm::{ + checked_transaction::{ + CheckError, + CheckedTransaction, + IntoChecked, + }, + PredicateVerificationFailed, }, - services::txpool::PoolTransaction, }; use std::vec; -const GAS_LIMIT: Word = 100000; - -fn check_tx_to_pool(checked_tx: Checked) -> PoolTransaction { - match checked_tx.into() { - CheckedTransaction::Blob(tx) => { - PoolTransaction::Blob(tx, ConsensusParametersVersion::MIN) - } - CheckedTransaction::Create(tx) => { - PoolTransaction::Create(tx, ConsensusParametersVersion::MIN) - } - CheckedTransaction::Script(tx) => { - PoolTransaction::Script(tx, ConsensusParametersVersion::MIN) - } - CheckedTransaction::Upgrade(tx) => { - PoolTransaction::Upgrade(tx, ConsensusParametersVersion::MIN) - } - CheckedTransaction::Upload(tx) => { - PoolTransaction::Upload(tx, ConsensusParametersVersion::MIN) - } - _ => panic!("Unexpected transaction type"), - } -} - #[tokio::test] -async fn insert_simple_tx_succeeds() { - let mut context = PoolContext::default(); +async fn insert_one_tx_succeeds() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); - let (_, gas_coin) = context.setup_coin(); - let tx = TransactionBuilder::script(vec![], vec![]) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .finalize_as_transaction() - .into_checked_basic(Default::default(), &ConsensusParameters::default()) - .unwrap(); + // Given + let tx = universe.build_script_transaction(None, None, 0); - let mut txpool = context.build(); + // When + let result = universe.verify_and_insert(tx).await; - for result in txpool.insert(vec![check_tx_to_pool(tx)]).unwrap() { - result.expect("Tx should be Ok, got Err"); - } + // Then + assert!(result.is_ok()); } #[tokio::test] -async fn insert_simple_tx_with_blacklisted_utxo_id_fails() { - let mut context = PoolContext::default(); - - let (_, gas_coin) = context.setup_coin(); - let tx = TransactionBuilder::script(vec![], vec![]) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin.clone()) - .finalize_as_transaction(); +async fn insert__tx_with_blacklisted_utxo_id() { + let mut universe = TestPoolUniverse::default(); // Given - let tx = check_unwrap_tx(tx, &context.config).await; - let utxo_id = *gas_coin.utxo_id().unwrap(); - context.config.black_list.coins.insert(utxo_id); - - let mut txpool = context.build(); + let coin = universe.setup_coin().1; + let utxo_id = *coin.utxo_id().unwrap(); + universe.config.black_list.coins.insert(utxo_id); + universe.build_pool(); + let tx = universe.build_script_transaction(Some(vec![coin]), None, 0); // When - let results = txpool.insert(vec![check_tx_to_pool(tx)]).unwrap(); + let err = universe.verify_and_insert(tx).await.unwrap_err(); // Then - for result in results { - assert!(result.is_err()); - assert!(result - .unwrap_err() - .to_string() - .contains(format!("The UTXO `{}` is blacklisted", utxo_id).as_str())); - } + assert!(matches!(err, Error::BlacklistedUTXO(id) if id == utxo_id)); } #[tokio::test] -async fn insert_simple_tx_with_blacklisted_owner_fails() { - let mut context = PoolContext::default(); - - let (_, gas_coin) = context.setup_coin(); - let tx = TransactionBuilder::script(vec![], vec![]) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin.clone()) - .finalize_as_transaction(); +async fn insert__tx_with_blacklisted_owner() { + let mut universe = TestPoolUniverse::default(); // Given - let tx = check_unwrap_tx(tx, &context.config).await; - let owner = *gas_coin.input_owner().unwrap(); - context.config.black_list.owners.insert(owner); - - let mut txpool = context.build(); + let coin = universe.setup_coin().1; + let owner_addr = *coin.input_owner().unwrap(); + universe.config.black_list.owners.insert(owner_addr); + universe.build_pool(); + let tx = universe.build_script_transaction(Some(vec![coin]), None, 0); // When - let results = txpool.insert(vec![check_tx_to_pool(tx)]).unwrap(); + let err = universe.verify_and_insert(tx).await.unwrap_err(); // Then - for result in results { - assert!(result.is_err()); - assert!(result - .unwrap_err() - .to_string() - .contains(format!("The owner `{}` is blacklisted", owner).as_str())); - } + assert!(matches!(err, Error::BlacklistedOwner(id) if id == owner_addr)); } #[tokio::test] -async fn insert_simple_tx_with_blacklisted_contract_fails() { - let mut context = PoolContext::default(); +async fn insert__tx_with_blacklisted_contract() { + let mut universe = TestPoolUniverse::default(); let contract_id = Contract::EMPTY_CONTRACT_ID; - let (_, gas_coin) = context.setup_coin(); - let tx = TransactionBuilder::script(vec![], vec![]) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin.clone()) - .add_input(create_contract_input( + // Given + universe.config.black_list.contracts.insert(contract_id); + universe.build_pool(); + let tx = universe.build_script_transaction( + Some(vec![create_contract_input( Default::default(), Default::default(), contract_id, - )) - .add_output(Output::contract(1, Default::default(), Default::default())) - .finalize_as_transaction(); - // Given - let tx = check_unwrap_tx(tx, &context.config).await; - context.config.black_list.contracts.insert(contract_id); - - let mut txpool = context.build(); + )]), + Some(vec![Output::contract( + 0, + Default::default(), + Default::default(), + )]), + 0, + ); // When - let results = txpool.insert(vec![check_tx_to_pool(tx)]).unwrap(); + let err = universe.verify_and_insert(tx).await.unwrap_err(); // Then - for result in results { - assert!(result.is_err()); - assert!(result - .unwrap_err() - .to_string() - .contains(format!("The contract `{}` is blacklisted", contract_id).as_str())); - } + assert!(matches!(err, Error::BlacklistedContract(id) if id == contract_id)); } #[tokio::test] -async fn insert_simple_tx_with_blacklisted_message_fails() { - let mut context = PoolContext::default(); - let (message, input) = create_message_predicate_from_message(5000, 0); +async fn insert__tx_with_blacklisted_message() { + let mut universe = TestPoolUniverse::default(); - let tx = TransactionBuilder::script(vec![], vec![]) - .script_gas_limit(GAS_LIMIT) - .add_input(input) - .finalize_as_transaction(); - - let nonce = *message.nonce(); // Given - let tx = check_unwrap_tx(tx, &context.config).await; - context.config.black_list.messages.insert(nonce); - - let mut txpool = context.build(); + let (message, input) = create_message_predicate_from_message(5000, 0); + let nonce = *message.nonce(); + universe.config.black_list.messages.insert(nonce); + universe.build_pool(); + let tx = universe.build_script_transaction(Some(vec![input]), None, 0); // When - let results = txpool.insert(vec![check_tx_to_pool(tx)]).unwrap(); + let err = universe.verify_and_insert(tx).await.unwrap_err(); // Then - for result in results { - assert!(result.is_err()); - assert!(result - .unwrap_err() - .to_string() - .contains(format!("The message `{}` is blacklisted", nonce).as_str())); - } + assert!(matches!(err, Error::BlacklistedMessage(id) if id == nonce)); } #[tokio::test] -async fn insert_simple_tx_dependency_chain_succeeds() { - let mut context = PoolContext::default(); +async fn insert_tx2_dependent_tx1() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); - let (_, gas_coin) = context.setup_coin(); - let (output, unset_input) = context.create_output_and_input(1); - let tx1 = TransactionBuilder::script(vec![], vec![]) - .tip(1) - .max_fee_limit(1) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .add_output(output) - .finalize_as_transaction(); + // Given + let (output, unset_input) = universe.create_output_and_input(); + let tx1 = universe.build_script_transaction(None, Some(vec![output]), 0); - let (_, gas_coin) = context.setup_coin(); - let input = unset_input.into_input(UtxoId::new(tx1.id(&Default::default()), 0)); - let tx2 = TransactionBuilder::script(vec![], vec![]) - .tip(1) - .max_fee_limit(1) - .script_gas_limit(GAS_LIMIT) - .add_input(input) - .add_input(gas_coin) - .finalize_as_transaction(); + let input = unset_input.into_input(UtxoId::new(tx1.id(&ChainId::default()), 0)); + let tx2 = universe.build_script_transaction(Some(vec![input]), None, 0); - let mut txpool = context.build(); - let tx1 = check_unwrap_tx(tx1, &txpool.config).await; - let tx2 = check_unwrap_tx(tx2, &txpool.config).await; + // When + let result1 = universe.verify_and_insert(tx1).await; + let result2 = universe.verify_and_insert(tx2).await; - for result in txpool - .insert(vec![check_tx_to_pool(tx1), check_tx_to_pool(tx2)]) - .unwrap() - { - result.expect("Tx should be Ok, got Err"); - } + // Then + assert!(result1.is_ok()); + assert!(result2.is_ok()); } #[tokio::test] -async fn faulty_tx2_collided_on_contract_id_from_tx1() { - let mut context = PoolContext::default(); +async fn insert__tx2_collided_on_contract_id() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); let contract_id = Contract::EMPTY_CONTRACT_ID; // contract creation tx - let (_, gas_coin) = context.setup_coin(); - let (output, unset_input) = context.create_output_and_input(10); + let (_, gas_coin) = universe.setup_coin(); + let (output, unset_input) = universe.create_output_and_input(); let tx = TransactionBuilder::create( Default::default(), Default::default(), @@ -278,9 +202,10 @@ async fn faulty_tx2_collided_on_contract_id_from_tx1() { .add_output(output) .finalize_as_transaction(); - let (_, gas_coin) = context.setup_coin(); + let (_, gas_coin) = universe.setup_coin(); let input = unset_input.into_input(UtxoId::new(tx.id(&Default::default()), 1)); + // Given // attempt to insert a different creation tx with a valid dependency on the first tx, // but with a conflicting output contract id let tx_faulty = TransactionBuilder::create( @@ -296,26 +221,27 @@ async fn faulty_tx2_collided_on_contract_id_from_tx1() { .add_output(output) .finalize_as_transaction(); - let mut txpool = context.build(); - let tx = check_unwrap_tx(tx, &txpool.config).await; - let tx_faulty = check_unwrap_tx(tx_faulty, &txpool.config).await; + // When + let result1 = universe.verify_and_insert(tx).await; + let result2 = universe.verify_and_insert(tx_faulty).await; - let results = txpool - .insert(vec![check_tx_to_pool(tx), check_tx_to_pool(tx_faulty)]) - .unwrap(); - assert_eq!(results.len(), 2); - assert!(results[0].is_ok()); - let err = results[1].as_ref().expect_err("Tx2 should be Err, got Ok"); - assert!(matches!(err, Error::Collided(_))); + assert!(result1.is_ok()); + + // Then + let err = result2.unwrap_err(); + assert!( + matches!(err, Error::Collided(CollisionReason::ContractCreation(id)) if id == contract_id) + ); } #[tokio::test] -async fn fail_to_insert_tx_with_dependency_on_invalid_utxo_type() { - let mut context = PoolContext::default(); - +async fn insert__tx_with_dependency_on_invalid_utxo_type() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); let contract_id = Contract::EMPTY_CONTRACT_ID; - let (_, gas_coin) = context.setup_coin(); - let tx_faulty = TransactionBuilder::create( + + let gas_coin = universe.setup_coin().1; + let tx = TransactionBuilder::create( Default::default(), Default::default(), Default::default(), @@ -323,176 +249,120 @@ async fn fail_to_insert_tx_with_dependency_on_invalid_utxo_type() { .add_input(gas_coin) .add_output(create_contract_output(contract_id)) .finalize_as_transaction(); + let utxo_id = UtxoId::new(tx.id(&Default::default()), 0); + // Given // create a second transaction with utxo id referring to // the wrong type of utxo (contract instead of coin) - let tx = TransactionBuilder::script(vec![], vec![]) - .tip(1) - .max_fee_limit(1) - .max_fee_limit(1) - .script_gas_limit(GAS_LIMIT) - .add_input(context.random_predicate( - AssetId::BASE, - TEST_COIN_AMOUNT, - Some(UtxoId::new(tx_faulty.id(&Default::default()), 0)), - )) - .finalize_as_transaction(); + let random_predicate = + universe.random_predicate(AssetId::BASE, TEST_COIN_AMOUNT, Some(utxo_id)); + let tx_faulty = + universe.build_script_transaction(Some(vec![random_predicate]), None, 0); - let mut txpool = context.build(); - let tx_faulty_id = tx_faulty.id(&ChainId::default()); - let tx_faulty = check_unwrap_tx(tx_faulty, &txpool.config).await; - let tx = check_unwrap_tx(tx, &txpool.config).await; + // When + let result1 = universe.verify_and_insert(tx).await; + let result2 = universe.verify_and_insert(tx_faulty).await; - let results = txpool - .insert(vec![check_tx_to_pool(tx_faulty), check_tx_to_pool(tx)]) - .unwrap(); - assert_eq!(results.len(), 2); - assert!(results[0].is_ok()); - let err = results[1].as_ref().expect_err("Tx should be Err, got Ok"); - assert!(matches!( - err, - Error::UtxoNotFound(id) if id == &UtxoId::new(tx_faulty_id, 0) - )); + // Then + assert!(result1.is_ok()); + let err = result2.unwrap_err(); + + assert!(matches!(err, Error::UtxoNotFound(id) if id == utxo_id)); } #[tokio::test] -async fn not_inserted_known_tx() { - let config = Config { +async fn insert__already_known_tx() { + let mut universe = TestPoolUniverse::default().config(Config { utxo_validation: false, ..Default::default() - }; - let context = PoolContext::default().config(config); - let mut txpool = context.build(); + }); + universe.build_pool(); - let tx = TransactionBuilder::script(vec![], vec![]) - .add_random_fee_input() - .finalize() - .into(); - let tx = check_unwrap_tx(tx, &txpool.config).await; + // Given + let tx = universe.build_script_transaction(None, None, 0); - let results = txpool - .insert(vec![check_tx_to_pool(tx.clone()), check_tx_to_pool(tx)]) - .unwrap(); - assert_eq!(results.len(), 2); - assert!(results[0].is_ok()); - let err = results[1].as_ref().expect_err("Tx should be Err, got Ok"); - assert!(matches!(err, Error::Collided(_))); + // When + let result1 = universe.verify_and_insert(tx.clone()).await; + let result2 = universe.verify_and_insert(tx).await; + + // Then + assert!(result1.is_ok()); + let err = result2.unwrap_err(); + assert!(matches!(err, Error::Collided(CollisionReason::Utxo(_)))); } #[tokio::test] -async fn try_to_insert_tx2_missing_utxo() { - let mut context = PoolContext::default(); +async fn insert__unknown_utxo() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); - let input = context.random_predicate(AssetId::BASE, TEST_COIN_AMOUNT, None); - let tx = TransactionBuilder::script(vec![], vec![]) - .tip(10) - .max_fee_limit(10) - .script_gas_limit(GAS_LIMIT) - .add_input(input) - .finalize_as_transaction(); + // Given + let input = universe.random_predicate(AssetId::BASE, TEST_COIN_AMOUNT, None); + let utxo_id = input.utxo_id().cloned().unwrap(); + let tx = universe.build_script_transaction(Some(vec![input]), None, 0); - let mut txpool = context.build(); - let tx = check_unwrap_tx(tx, &txpool.config).await; + // When + let result = universe.verify_and_insert(tx).await; - let results = txpool.insert(vec![check_tx_to_pool(tx)]).unwrap(); - assert_eq!(results.len(), 1); - let err = results[0].as_ref().expect_err("Tx should be Err, got Ok"); - assert!(matches!(err, Error::UtxoNotFound(_))); + // Then + let err = result.unwrap_err(); + assert!(matches!(err, Error::UtxoNotFound(id) if id == utxo_id)); } #[tokio::test] -async fn higher_priced_tx_removes_lower_priced_tx() { - let mut context = PoolContext::default(); - - let (_, coin_input) = context.setup_coin(); +async fn insert_higher_priced_tx_removes_lower_priced_tx() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); - let tx1 = TransactionBuilder::script(vec![], vec![]) - .tip(10) - .max_fee_limit(10) - .script_gas_limit(GAS_LIMIT) - .add_input(coin_input.clone()) - .finalize_as_transaction(); + // Given + let common_coin = universe.setup_coin().1; + let tx1 = + universe.build_script_transaction(Some(vec![common_coin.clone()]), None, 10); + let tx_id = tx1.id(&ChainId::default()); + let tx2 = universe.build_script_transaction(Some(vec![common_coin]), None, 20); - let tx2 = TransactionBuilder::script(vec![], vec![]) - .tip(20) - .max_fee_limit(20) - .script_gas_limit(GAS_LIMIT) - .add_input(coin_input) - .finalize_as_transaction(); + // When + universe.verify_and_insert(tx1).await.unwrap(); + let result = universe.verify_and_insert(tx2).await.unwrap(); - let tx1_id = tx1.id(&ChainId::default()); - let mut txpool = context.build(); - let tx1 = check_unwrap_tx(tx1, &txpool.config).await; - let tx2 = check_unwrap_tx(tx2, &txpool.config).await; - - let results = txpool - .insert(vec![ - check_tx_to_pool(tx1.clone()), - check_tx_to_pool(tx2.clone()), - ]) - .unwrap(); - assert_eq!(results.len(), 2); - assert!(results[0].is_ok()); - let vec = results[1].as_ref().unwrap(); - assert_eq!(vec[0].id(), tx1_id, "Tx1 id should be removed"); + // Then + assert_eq!(result[0].id(), tx_id); } #[tokio::test] -async fn underpriced_tx1_not_included_coin_collision() { - let mut context = PoolContext::default(); +async fn insert__colliding_dependent_underpriced() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); - let (_, gas_coin) = context.setup_coin(); - let (output, unset_input) = context.create_output_and_input(20); - let tx1 = TransactionBuilder::script(vec![], vec![]) - .tip(20) - .max_fee_limit(20) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .add_output(output) - .finalize_as_transaction(); - - let input = unset_input.into_input(UtxoId::new(tx1.id(&Default::default()), 0)); - - let tx2 = TransactionBuilder::script(vec![], vec![]) - .tip(20) - .max_fee_limit(20) - .script_gas_limit(GAS_LIMIT) - .add_input(input.clone()) - .finalize_as_transaction(); + let (output, unset_input) = universe.create_output_and_input(); + let tx1 = universe.build_script_transaction(None, Some(vec![output]), 20); + let utxo_id = UtxoId::new(tx1.id(&ChainId::default()), 0); + let input = unset_input.into_input(utxo_id); - let tx3 = TransactionBuilder::script(vec![], vec![]) - .tip(10) - .max_fee_limit(10) - .script_gas_limit(GAS_LIMIT) - .add_input(input) - .finalize_as_transaction(); + // Given + let tx2 = universe.build_script_transaction(Some(vec![input.clone()]), None, 20); + let tx3 = universe.build_script_transaction(Some(vec![input]), None, 10); - let mut txpool = context.build(); - let tx1_checked = check_unwrap_tx(tx1.clone(), &txpool.config).await; - let tx2_checked = check_unwrap_tx(tx2.clone(), &txpool.config).await; - let tx3_checked = check_unwrap_tx(tx3, &txpool.config).await; - - let results = txpool - .insert(vec![ - check_tx_to_pool(tx1_checked), - check_tx_to_pool(tx2_checked), - check_tx_to_pool(tx3_checked), - ]) - .unwrap(); - assert_eq!(results.len(), 3); - assert!(results[0].is_ok()); - assert!(results[1].is_ok()); - let err = results[2].as_ref().expect_err("Tx3 should be Err, got Ok"); + // When + let result1 = universe.verify_and_insert(tx1).await; + let result2 = universe.verify_and_insert(tx2).await; + let result3 = universe.verify_and_insert(tx3).await; - assert!(matches!(err, Error::Collided(_))); + // Then + assert!(result1.is_ok()); + assert!(result2.is_ok()); + let err = result3.unwrap_err(); + assert!(matches!(err, Error::Collided(CollisionReason::Utxo(id)) if id == utxo_id)); } #[tokio::test] -async fn dependent_contract_input_inserted() { - let mut context = PoolContext::default(); - +async fn insert_dependent_contract_creation() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); let contract_id = Contract::EMPTY_CONTRACT_ID; - let (_, gas_funds) = context.setup_coin(); + + // Given + let (_, gas_funds) = universe.setup_coin(); let tx1 = TransactionBuilder::create( Default::default(), Default::default(), @@ -504,272 +374,254 @@ async fn dependent_contract_input_inserted() { .add_output(create_contract_output(contract_id)) .finalize_as_transaction(); - let (_, gas_funds) = context.setup_coin(); - let tx2 = TransactionBuilder::script(vec![], vec![]) - .tip(10) - .max_fee_limit(10) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_funds) - .add_input(create_contract_input( + let tx2 = universe.build_script_transaction( + Some(vec![create_contract_input( Default::default(), Default::default(), contract_id, - )) - .add_output(Output::contract(1, Default::default(), Default::default())) - .finalize_as_transaction(); + )]), + Some(vec![Output::contract( + 0, + Default::default(), + Default::default(), + )]), + 10, + ); - let mut txpool = context.build(); - let tx1 = check_unwrap_tx(tx1, &txpool.config).await; - let tx2 = check_unwrap_tx(tx2, &txpool.config).await; + // When + let result1 = universe.verify_and_insert(tx1).await; + let result2 = universe.verify_and_insert(tx2).await; - let results = txpool - .insert(vec![check_tx_to_pool(tx1), check_tx_to_pool(tx2)]) - .unwrap(); - assert_eq!(results.len(), 2); - assert!(results[0].is_ok()); - assert!(results[1].is_ok()); + // Then + assert!(result1.is_ok()); + assert!(result2.is_ok()); } #[tokio::test] -async fn more_priced_tx3_removes_tx1_and_dependent_tx2() { - let mut context = PoolContext::default(); +async fn insert_more_priced_tx3_removes_tx1_and_dependent_tx2() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); - let (_, gas_coin) = context.setup_coin(); + // Given + let common_coin = universe.setup_coin().1; + let (output, unset_input) = universe.create_output_and_input(); - let (output, unset_input) = context.create_output_and_input(10); - let tx1 = TransactionBuilder::script(vec![], vec![]) - .tip(10) - .max_fee_limit(10) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin.clone()) - .add_output(output) - .finalize_as_transaction(); + let tx1 = universe.build_script_transaction( + Some(vec![common_coin.clone()]), + Some(vec![output]), + 10, + ); + let tx1_id = tx1.id(&ChainId::default()); + let input = unset_input.into_input(UtxoId::new(tx1_id, 0)); - let input = unset_input.into_input(UtxoId::new(tx1.id(&Default::default()), 0)); + let tx2 = universe.build_script_transaction(Some(vec![input.clone()]), None, 10); + let tx2_id = tx2.id(&ChainId::default()); - let tx2 = TransactionBuilder::script(vec![], vec![]) - .tip(9) - .max_fee_limit(9) - .script_gas_limit(GAS_LIMIT) - .add_input(input) - .finalize_as_transaction(); + let tx3 = universe.build_script_transaction(Some(vec![common_coin]), None, 20); - let tx3 = TransactionBuilder::script(vec![], vec![]) - .tip(20) - .max_fee_limit(20) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .finalize_as_transaction(); + // When + let result1 = universe.verify_and_insert(tx1).await; + let result2 = universe.verify_and_insert(tx2).await; + let result3 = universe.verify_and_insert(tx3).await; + + // Then + assert!(result1.is_ok()); + assert!(result2.is_ok()); + assert!(result3.is_ok()); + let removed_txs = result3.unwrap(); + assert_eq!(removed_txs.len(), 2); + assert_eq!(removed_txs[0].id(), tx1_id); + assert_eq!(removed_txs[1].id(), tx2_id); +} + +#[tokio::test] +async fn insert_more_priced_tx2_removes_tx1_and_more_priced_tx3_removes_tx2() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); + + // Given + let common_coin = universe.setup_coin().1; + let tx1 = + universe.build_script_transaction(Some(vec![common_coin.clone()]), None, 10); let tx1_id = tx1.id(&ChainId::default()); + + let tx2 = + universe.build_script_transaction(Some(vec![common_coin.clone()]), None, 11); let tx2_id = tx2.id(&ChainId::default()); - let mut txpool = context.build(); - let tx1 = check_unwrap_tx(tx1, &txpool.config).await; - let tx2 = check_unwrap_tx(tx2, &txpool.config).await; - let tx3 = check_unwrap_tx(tx3, &txpool.config).await; - - let results = txpool - .insert(vec![ - check_tx_to_pool(tx1.clone()), - check_tx_to_pool(tx2.clone()), - check_tx_to_pool(tx3.clone()), - ]) - .unwrap(); - assert_eq!(results.len(), 3); - assert!(results[0].is_ok()); - assert!(results[1].is_ok()); - let removed_transactions = results[2].as_ref().unwrap(); - assert_eq!(removed_transactions.len(), 2); - assert_eq!( - removed_transactions[0].id(), - tx1_id, - "Tx1 id should be removed" - ); - assert_eq!( - removed_transactions[1].id(), - tx2_id, - "Tx2 id should be removed" - ); + + let tx3 = universe.build_script_transaction(Some(vec![common_coin]), None, 12); + + // When + let result1 = universe.verify_and_insert(tx1).await; + let result2 = universe.verify_and_insert(tx2).await; + let result3 = universe.verify_and_insert(tx3).await; + + // Then + assert!(result1.is_ok()); + assert!(result2.is_ok()); + let removed_txs = result2.unwrap(); + assert_eq!(removed_txs.len(), 1); + assert_eq!(removed_txs[0].id(), tx1_id); + assert!(result3.is_ok()); + let removed_txs = result3.unwrap(); + assert_eq!(removed_txs.len(), 1); + assert_eq!(removed_txs[0].id(), tx2_id); } #[tokio::test] -async fn more_priced_tx2_removes_tx1_and_more_priced_tx3_removes_tx2() { - let mut context = PoolContext::default(); +async fn insert__tx_limit_hit() { + let mut universe = TestPoolUniverse::default().config(Config { + pool_limits: PoolLimits { + max_txs: 1, + max_bytes_size: 1000000000, + max_gas: 100_000_000_000, + }, + ..Default::default() + }); + universe.build_pool(); - let (_, gas_coin) = context.setup_coin(); + // Given + let tx1 = universe.build_script_transaction(None, None, 10); + let tx2 = universe.build_script_transaction(None, None, 0); - let tx1 = TransactionBuilder::script(vec![], vec![]) - .tip(10) - .max_fee_limit(10) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin.clone()) - .finalize_as_transaction(); + // When + let result1 = universe.verify_and_insert(tx1).await; + let result2 = universe.verify_and_insert(tx2).await; - let tx2 = TransactionBuilder::script(vec![], vec![]) - .tip(11) - .max_fee_limit(11) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin.clone()) - .finalize_as_transaction(); + // Then + assert!(result1.is_ok()); + let err = result2.unwrap_err(); + assert!(matches!(err, Error::NotInsertedLimitHit)); +} - let tx3 = TransactionBuilder::script(vec![], vec![]) - .tip(12) - .max_fee_limit(12) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .finalize_as_transaction(); +#[tokio::test] +async fn insert__tx_gas_limit() { + // Given + let mut universe = TestPoolUniverse::default(); + let tx1 = universe.build_script_transaction(None, None, 10); + let checked_tx: CheckedTransaction = tx1 + .clone() + .into_checked_basic(Default::default(), &ConsensusParameters::default()) + .unwrap() + .into(); + let max_gas = match checked_tx { + CheckedTransaction::Script(tx) => tx.metadata().max_gas, + _ => panic!("Expected script transaction"), + }; + let tx2 = universe.build_script_transaction(None, None, 0); + universe = universe.config(Config { + pool_limits: PoolLimits { + max_txs: 10000, + max_bytes_size: 1000000000, + max_gas: max_gas + 10, + }, + ..Default::default() + }); + universe.build_pool(); - let mut txpool = context.build(); - let tx1 = check_unwrap_tx(tx1, &txpool.config).await; - let tx2 = check_unwrap_tx(tx2, &txpool.config).await; - let tx3 = check_unwrap_tx(tx3, &txpool.config).await; - - let results = txpool - .insert(vec![ - check_tx_to_pool(tx1.clone()), - check_tx_to_pool(tx2.clone()), - check_tx_to_pool(tx3.clone()), - ]) - .unwrap(); - assert_eq!(results.len(), 3); - assert!(results[0].is_ok()); - let removed_transactions = results[1].as_ref().unwrap(); - assert_eq!(removed_transactions.len(), 1); - let removed_transactions = results[2].as_ref().unwrap(); - assert_eq!(removed_transactions.len(), 1); + // When + let result1 = universe.verify_and_insert(tx1).await; + let result2 = universe.verify_and_insert(tx2).await; + + // Then + assert!(result1.is_ok()); + let err = result2.unwrap_err(); + assert!(matches!(err, Error::NotInsertedLimitHit)); } #[tokio::test] -async fn tx_limit_hit() { - let mut context = PoolContext::default().config(Config { - max_txs: 1, +async fn insert__tx_bytes_limit() { + // Given + let mut universe = TestPoolUniverse::default(); + let tx1 = universe.build_script_transaction(None, None, 10); + let checked_tx: CheckedTransaction = tx1 + .clone() + .into_checked_basic(Default::default(), &ConsensusParameters::default()) + .unwrap() + .into(); + let max_bytes = match checked_tx { + CheckedTransaction::Script(tx) => tx.transaction().metered_bytes_size(), + _ => panic!("Expected script transaction"), + }; + let tx2 = universe.build_script_transaction(None, None, 0); + universe = universe.config(Config { + pool_limits: PoolLimits { + max_txs: 10000, + max_bytes_size: max_bytes + 10, + max_gas: 100_000_000_000, + }, ..Default::default() }); + universe.build_pool(); - let (_, gas_coin) = context.setup_coin(); - let tx1 = TransactionBuilder::script(vec![], vec![]) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .add_output(create_coin_output()) - .finalize_as_transaction(); - - let (_, gas_coin) = context.setup_coin(); - let tx2 = TransactionBuilder::script(vec![], vec![]) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .finalize_as_transaction(); + // When + let result1 = universe.verify_and_insert(tx1).await; + let result2 = universe.verify_and_insert(tx2).await; - let mut txpool = context.build(); - let tx1 = check_unwrap_tx(tx1, &txpool.config).await; - let tx2 = check_unwrap_tx(tx2, &txpool.config).await; - let results = txpool - .insert(vec![check_tx_to_pool(tx1), check_tx_to_pool(tx2)]) - .unwrap(); - assert_eq!(results.len(), 2); - assert!(results[0].is_ok()); - let err = results[1].as_ref().expect_err("Tx2 should be Err, got Ok"); + // Then + assert!(result1.is_ok()); + let err = result2.unwrap_err(); assert!(matches!(err, Error::NotInsertedLimitHit)); } #[tokio::test] -async fn tx_chain_length_hit() { - let mut context = PoolContext::default().config(Config { +async fn insert__dependency_chain_length_hit() { + let mut universe = TestPoolUniverse::default().config(Config { max_txs_chain_count: 2, ..Default::default() }); + universe.build_pool(); - let (_, gas_coin) = context.setup_coin(); - let (output, unset_input) = context.create_output_and_input(10_000); - let tx1 = TransactionBuilder::script(vec![], vec![]) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .add_output(output) - .finalize_as_transaction(); - + // Given + let (output, unset_input) = universe.create_output_and_input(); + let tx1 = universe.build_script_transaction(None, Some(vec![output]), 0); let input = unset_input.into_input(UtxoId::new(tx1.id(&Default::default()), 0)); - let (output, unset_input) = context.create_output_and_input(5_000); - let tx2 = TransactionBuilder::script(vec![], vec![]) - .script_gas_limit(GAS_LIMIT) - .add_input(input) - .add_output(output) - .finalize_as_transaction(); + let (output, unset_input) = universe.create_output_and_input(); + let tx2 = universe.build_script_transaction(Some(vec![input]), Some(vec![output]), 0); let input = unset_input.into_input(UtxoId::new(tx2.id(&Default::default()), 0)); - let tx3 = TransactionBuilder::script(vec![], vec![]) - .script_gas_limit(GAS_LIMIT) - .add_input(input) - .finalize_as_transaction(); - let mut txpool = context.build(); - let tx1 = check_unwrap_tx(tx1, &txpool.config).await; - let tx2 = check_unwrap_tx(tx2, &txpool.config).await; - let tx3 = check_unwrap_tx(tx3, &txpool.config).await; - - let results = txpool - .insert(vec![ - check_tx_to_pool(tx1.clone()), - check_tx_to_pool(tx2.clone()), - check_tx_to_pool(tx3.clone()), - ]) - .unwrap(); - assert_eq!(results.len(), 3); - assert!(results[0].is_ok()); - assert!(results[1].is_ok()); - let err = results[2].as_ref().expect_err("Tx3 should be Err, got Ok"); + let tx3 = universe.build_script_transaction(Some(vec![input]), None, 0); + + // When + let result1 = universe.verify_and_insert(tx1).await; + let result2 = universe.verify_and_insert(tx2).await; + let result3 = universe.verify_and_insert(tx3).await; + + // Then + assert!(result1.is_ok()); + assert!(result2.is_ok()); + let err = result3.unwrap_err(); assert!(matches!(err, Error::NotInsertedChainDependencyTooBig)); } #[tokio::test] -async fn sorted_out_tx1_2_3() { - let mut context = PoolContext::default(); - - let (_, gas_coin) = context.setup_coin(); - let tx1 = TransactionBuilder::script(vec![], vec![]) - .tip(10) - .max_fee_limit(10) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .finalize_as_transaction(); +async fn get_sorted_out_tx1_2_3() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); - let (_, gas_coin) = context.setup_coin(); - let tx2 = TransactionBuilder::script(vec![], vec![]) - .tip(9) - .max_fee_limit(9) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .finalize_as_transaction(); - - let (_, gas_coin) = context.setup_coin(); - let tx3 = TransactionBuilder::script(vec![], vec![]) - .tip(20) - .max_fee_limit(20) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .finalize_as_transaction(); + // Given + let tx1 = universe.build_script_transaction(None, None, 10); + let tx2 = universe.build_script_transaction(None, None, 9); + let tx3 = universe.build_script_transaction(None, None, 20); let tx1_id = tx1.id(&ChainId::default()); let tx2_id = tx2.id(&ChainId::default()); let tx3_id = tx3.id(&ChainId::default()); - let mut txpool = context.build(); - let tx1 = check_unwrap_tx(tx1, &txpool.config).await; - let tx2 = check_unwrap_tx(tx2, &txpool.config).await; - let tx3 = check_unwrap_tx(tx3, &txpool.config).await; - - let results = txpool - .insert(vec![ - check_tx_to_pool(tx1.clone()), - check_tx_to_pool(tx2.clone()), - check_tx_to_pool(tx3.clone()), - ]) - .unwrap(); - assert_eq!(results.len(), 3); - assert!(results[0].is_ok()); - assert!(results[1].is_ok()); - assert!(results[2].is_ok()); + universe.verify_and_insert(tx1).await.unwrap(); + universe.verify_and_insert(tx2).await.unwrap(); + universe.verify_and_insert(tx3).await.unwrap(); - let txs = txpool.extract_transactions_for_block().unwrap(); + // When + let txs = universe + .get_pool() + .write() + .extract_transactions_for_block() + .unwrap(); + // Then assert_eq!(txs.len(), 3, "Should have 3 txs"); assert_eq!(txs[0].id(), tx3_id, "First should be tx3"); assert_eq!(txs[1].id(), tx1_id, "Second should be tx1"); @@ -777,10 +629,12 @@ async fn sorted_out_tx1_2_3() { } #[tokio::test] -async fn sorted_out_tx_same_tips() { - let mut context = PoolContext::default(); +async fn get_sorted_out_tx_same_tips() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); - let (_, gas_coin) = context.setup_coin(); + // Given + let gas_coin = universe.setup_coin().1; let tx1 = TransactionBuilder::script(vec![], vec![]) .tip(10) .max_fee_limit(10) @@ -788,7 +642,7 @@ async fn sorted_out_tx_same_tips() { .add_input(gas_coin) .finalize_as_transaction(); - let (_, gas_coin) = context.setup_coin(); + let (_, gas_coin) = universe.setup_coin(); let tx2 = TransactionBuilder::script(vec![], vec![]) .tip(10) .max_fee_limit(10) @@ -796,7 +650,7 @@ async fn sorted_out_tx_same_tips() { .add_input(gas_coin) .finalize_as_transaction(); - let (_, gas_coin) = context.setup_coin(); + let (_, gas_coin) = universe.setup_coin(); let tx3 = TransactionBuilder::script(vec![], vec![]) .tip(10) .max_fee_limit(10) @@ -808,36 +662,31 @@ async fn sorted_out_tx_same_tips() { let tx2_id = tx2.id(&ChainId::default()); let tx3_id = tx3.id(&ChainId::default()); - let mut txpool = context.build(); - let tx1 = check_unwrap_tx(tx1, &txpool.config).await; - let tx2 = check_unwrap_tx(tx2, &txpool.config).await; - let tx3 = check_unwrap_tx(tx3, &txpool.config).await; - - let results = txpool - .insert(vec![ - check_tx_to_pool(tx1.clone()), - check_tx_to_pool(tx2.clone()), - check_tx_to_pool(tx3.clone()), - ]) + universe.verify_and_insert(tx1).await.unwrap(); + universe.verify_and_insert(tx2).await.unwrap(); + universe.verify_and_insert(tx3).await.unwrap(); + + // When + let txs = universe + .get_pool() + .write() + .extract_transactions_for_block() .unwrap(); - assert_eq!(results.len(), 3); - assert!(results[0].is_ok()); - assert!(results[1].is_ok()); - assert!(results[2].is_ok()); - let txs = txpool.extract_transactions_for_block().unwrap(); + // Then assert_eq!(txs.len(), 3, "Should have 3 txs"); - assert_eq!(txs[0].id(), tx3_id, "First should be tx3"); assert_eq!(txs[1].id(), tx2_id, "Second should be tx2"); assert_eq!(txs[2].id(), tx1_id, "Third should be tx1"); } #[tokio::test] -async fn sorted_out_tx_profitable_ratios() { - let mut context = PoolContext::default(); +async fn get_sorted_out_tx_profitable_ratios() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); - let (_, gas_coin) = context.setup_coin(); + // Given + let gas_coin = universe.setup_coin().1; let tx1 = TransactionBuilder::script(vec![], vec![]) .tip(4) .max_fee_limit(4) @@ -845,7 +694,7 @@ async fn sorted_out_tx_profitable_ratios() { .add_input(gas_coin) .finalize_as_transaction(); - let (_, gas_coin) = context.setup_coin(); + let (_, gas_coin) = universe.setup_coin(); let tx2 = TransactionBuilder::script(vec![], vec![]) .tip(2) .max_fee_limit(2) @@ -853,7 +702,7 @@ async fn sorted_out_tx_profitable_ratios() { .add_input(gas_coin) .finalize_as_transaction(); - let (_, gas_coin) = context.setup_coin(); + let (_, gas_coin) = universe.setup_coin(); let tx3 = TransactionBuilder::script(vec![], vec![]) .tip(1) .max_fee_limit(1) @@ -865,94 +714,52 @@ async fn sorted_out_tx_profitable_ratios() { let tx2_id = tx2.id(&ChainId::default()); let tx3_id = tx3.id(&ChainId::default()); - let mut txpool = context.build(); - let tx1 = check_unwrap_tx(tx1, &txpool.config).await; - let tx2 = check_unwrap_tx(tx2, &txpool.config).await; - let tx3 = check_unwrap_tx(tx3, &txpool.config).await; - - let results = txpool - .insert(vec![ - check_tx_to_pool(tx1.clone()), - check_tx_to_pool(tx2.clone()), - check_tx_to_pool(tx3.clone()), - ]) + universe.verify_and_insert(tx1).await.unwrap(); + universe.verify_and_insert(tx2).await.unwrap(); + universe.verify_and_insert(tx3).await.unwrap(); + + // When + let txs = universe + .get_pool() + .write() + .extract_transactions_for_block() .unwrap(); - assert_eq!(results.len(), 3); - assert!(results[0].is_ok()); - assert!(results[1].is_ok()); - assert!(results[2].is_ok()); - let txs = txpool.extract_transactions_for_block().unwrap(); + // Then assert_eq!(txs.len(), 3, "Should have 3 txs"); - assert_eq!(txs[0].id(), tx3_id, "First should be tx3"); assert_eq!(txs[1].id(), tx2_id, "Second should be tx2"); assert_eq!(txs[2].id(), tx1_id, "Third should be tx1"); } #[tokio::test] -async fn sorted_out_tx_by_creation_instant() { - let mut context = PoolContext::default(); - - let (_, gas_coin) = context.setup_coin(); - let tx1 = TransactionBuilder::script(vec![], vec![]) - .tip(4) - .max_fee_limit(4) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .finalize_as_transaction(); - - let (_, gas_coin) = context.setup_coin(); - let tx2 = TransactionBuilder::script(vec![], vec![]) - .tip(4) - .max_fee_limit(4) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .finalize_as_transaction(); - - let (_, gas_coin) = context.setup_coin(); - let tx3 = TransactionBuilder::script(vec![], vec![]) - .tip(4) - .max_fee_limit(4) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .finalize_as_transaction(); - - let (_, gas_coin) = context.setup_coin(); - let tx4 = TransactionBuilder::script(vec![], vec![]) - .tip(4) - .max_fee_limit(4) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .finalize_as_transaction(); +async fn get_sorted_out_tx_by_creation_instant() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); + // Given + let tx1 = universe.build_script_transaction(None, None, 0); + let tx2 = universe.build_script_transaction(None, None, 0); + let tx3 = universe.build_script_transaction(None, None, 0); + let tx4 = universe.build_script_transaction(None, None, 0); let tx1_id = tx1.id(&ChainId::default()); let tx2_id = tx2.id(&ChainId::default()); let tx3_id = tx3.id(&ChainId::default()); let tx4_id = tx4.id(&ChainId::default()); - let mut txpool = context.build(); - let tx1 = check_unwrap_tx(tx1, &txpool.config).await; - let tx2 = check_unwrap_tx(tx2, &txpool.config).await; - let tx3 = check_unwrap_tx(tx3, &txpool.config).await; - let tx4 = check_unwrap_tx(tx4, &txpool.config).await; - - let results = txpool - .insert(vec![ - check_tx_to_pool(tx1.clone()), - check_tx_to_pool(tx2.clone()), - check_tx_to_pool(tx3.clone()), - check_tx_to_pool(tx4.clone()), - ]) - .unwrap(); - assert_eq!(results.len(), 4); - assert!(results[0].is_ok()); - assert!(results[1].is_ok()); - assert!(results[2].is_ok()); - assert!(results[3].is_ok()); + universe.verify_and_insert(tx1).await.unwrap(); + universe.verify_and_insert(tx2).await.unwrap(); + universe.verify_and_insert(tx3).await.unwrap(); + universe.verify_and_insert(tx4).await.unwrap(); - let txs = txpool.extract_transactions_for_block().unwrap(); + // When + let txs = universe + .get_pool() + .write() + .extract_transactions_for_block() + .unwrap(); + // Then // This order doesn't match the lexicographical order of the tx ids // and so it verifies that the txs are sorted by creation instant // The newest tx should be first @@ -964,51 +771,43 @@ async fn sorted_out_tx_by_creation_instant() { } #[tokio::test] -async fn tx_at_least_min_gas_price_is_insertable() { +async fn insert_tx_at_least_min_gas_price() { + // Given let gas_price = 10; - let mut context = PoolContext::default().config(Config { + let mut universe = TestPoolUniverse::default().config(Config { ..Default::default() }); + universe.build_pool(); - let (_, gas_coin) = context.setup_coin(); - let tx = TransactionBuilder::script(vec![], vec![]) - .tip(10) - .max_fee_limit(1000) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .finalize_as_transaction(); - - let mut txpool = context.build(); - let tx = check_unwrap_tx_with_gas_price(tx, &txpool.config, gas_price).await; - - let results = txpool.insert(vec![check_tx_to_pool(tx)]).unwrap(); - assert_eq!(results.len(), 1); - assert!(results[0].is_ok()); + let tx = universe.build_script_transaction(None, None, gas_price); + // When + universe.verify_and_insert_with_gas_price(tx, gas_price).await + // Then + .unwrap(); } #[tokio::test] -async fn tx_below_min_gas_price_is_not_insertable() { - let mut context = PoolContext::default(); +async fn insert__tx_below_min_gas_price() { + // Given + let gas_price = 11; + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); - let gas_coin = context.random_predicate(AssetId::BASE, TEST_COIN_AMOUNT, None); + let gas_coin = universe.setup_coin().1; let tx = TransactionBuilder::script(vec![], vec![]) .tip(10) .max_fee_limit(10) .script_gas_limit(GAS_LIMIT) .add_input(gas_coin) .finalize_as_transaction(); - let gas_price = 11; - let err = check_tx_with_gas_price( - tx, - &Config { - ..Default::default() - }, - gas_price, - ) - .await - .expect_err("expected insertion failure"); + // When + let err = universe + .verify_and_insert_with_gas_price(tx, gas_price) + .await + .unwrap_err(); + // Then assert!(matches!( err, Error::ConsensusValidity(CheckError::InsufficientMaxFee { .. }) @@ -1016,241 +815,183 @@ async fn tx_below_min_gas_price_is_not_insertable() { } #[tokio::test] -async fn tx_inserted_into_pool_when_input_message_id_exists_in_db() { - let mut context = PoolContext::default(); - let (message, input) = create_message_predicate_from_message(5000, 0); - - let tx = TransactionBuilder::script(vec![], vec![]) - .script_gas_limit(GAS_LIMIT) - .add_input(input) - .finalize_as_transaction(); - - context.database_mut().insert_message(message); - - let tx1_id = tx.id(&ChainId::default()); - let mut txpool = context.build(); - - let tx = check_unwrap_tx(tx, &txpool.config).await; +async fn insert_tx_when_input_message_id_exists_in_db() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); - let results = txpool.insert(vec![check_tx_to_pool(tx)]).unwrap(); - assert_eq!(results.len(), 1); - assert!(results[0].is_ok()); + // Given + let (message, input) = create_message_predicate_from_message(5000, 0); + universe.database_mut().insert_message(message); + let tx = universe.build_script_transaction(Some(vec![input]), None, 0); - let tx = txpool.find_one(&tx1_id).unwrap(); - assert_eq!(tx.id(), tx1_id); + // When + universe.verify_and_insert(tx).await + // Then + .unwrap(); } #[tokio::test] -async fn tx_rejected_from_pool_when_input_message_id_does_not_exist_in_db() { - let context = PoolContext::default(); - let (message, input) = create_message_predicate_from_message(5000, 0); - let tx = TransactionBuilder::script(vec![], vec![]) - .script_gas_limit(GAS_LIMIT) - .add_input(input) - .finalize_as_transaction(); +async fn insert__tx_when_input_message_id_do_not_exists_in_db() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); - // Do not insert any messages into the DB to ensure there is no matching message for the - // tx. - let mut txpool = context.build(); - let tx = check_unwrap_tx(tx, &txpool.config).await; + // Given + let (message, input) = create_message_predicate_from_message(5000, 0); + let tx = universe.build_script_transaction(Some(vec![input]), None, 0); - let results = txpool.insert(vec![check_tx_to_pool(tx)]).unwrap(); - assert_eq!(results.len(), 1); + // When + let err = universe.verify_and_insert(tx).await.unwrap_err(); - let err = results[0].as_ref().expect_err("Tx should be Err, got Ok"); - // check error + // Then assert!(matches!( err, - Error::NotInsertedInputMessageUnknown(msg_id) if msg_id == message.id() + Error::NotInsertedInputMessageUnknown(msg_id) if msg_id == *message.id() )); } #[tokio::test] -async fn tx_rejected_from_pool_when_gas_price_is_lower_than_another_tx_with_same_message_id( -) { - let mut context = PoolContext::default(); - let message_amount = 10_000; - let max_fee_limit = 10u64; - let gas_price_high = 2u64; - let gas_price_low = 1u64; - let (message, conflicting_message_input) = - create_message_predicate_from_message(message_amount, 0); - - let tx_high = TransactionBuilder::script(vec![], vec![]) - .tip(gas_price_high) - .max_fee_limit(max_fee_limit) - .script_gas_limit(GAS_LIMIT) - .add_input(conflicting_message_input.clone()) - .finalize_as_transaction(); +async fn insert__tx_tip_lower_than_another_tx_with_same_message_id() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); - let tx_low = TransactionBuilder::script(vec![], vec![]) - .tip(gas_price_low) - .max_fee_limit(max_fee_limit) - .script_gas_limit(GAS_LIMIT) - .add_input(conflicting_message_input) - .finalize_as_transaction(); - - context.database_mut().insert_message(message.clone()); - - let mut txpool = context.build(); - - let _tx_high_id = tx_high.id(&ChainId::default()); - let tx_high = - check_unwrap_tx_with_gas_price(tx_high, &txpool.config, gas_price_high).await; + let tip_high = 2u64; + let tip_low = 1u64; + let (message, conflicting_message_input) = + create_message_predicate_from_message(10_000, 0); + universe.database_mut().insert_message(message.clone()); - // Insert a tx for the message id with a high gas amount - let results = txpool.insert(vec![check_tx_to_pool(tx_high)]).unwrap(); - assert_eq!(results.len(), 1); - assert!(results[0].is_ok()); + // Given + let tx_high = universe.build_script_transaction( + Some(vec![conflicting_message_input.clone()]), + None, + tip_high, + ); + let tx_low = universe.build_script_transaction( + Some(vec![conflicting_message_input]), + None, + tip_low, + ); - let tx_low = - check_unwrap_tx_with_gas_price(tx_low, &txpool.config, gas_price_low).await; - // Insert a tx for the message id with a low gas amount - // Because the new transaction's id matches an existing transaction, we compare the gas - // prices of both the new and existing transactions. Since the existing transaction's gas - // price is higher, we must now reject the new transaction. - let results = txpool.insert(vec![check_tx_to_pool(tx_low)]).unwrap(); - assert_eq!(results.len(), 1); - let err = results[0].as_ref().expect_err("Tx should be Err, got Ok"); + // When + universe.verify_and_insert(tx_high).await.unwrap(); + let err = universe.verify_and_insert(tx_low).await.unwrap_err(); - // check error - assert!(matches!(err, Error::Collided(_))); + // Then + assert!( + matches!(err, Error::Collided(CollisionReason::Message(msg_id)) if msg_id == *message.id()) + ); } #[tokio::test] -async fn higher_priced_tx_squeezes_out_lower_priced_tx_with_same_message_id() { - let mut context = PoolContext::default(); - let message_amount = 10_000; - let gas_price_high = 2u64; - let max_fee_limit = 10u64; - let gas_price_low = 1u64; +async fn insert_tx_tip_higher_than_another_tx_with_same_message_id() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); + + let tip_low = 1u64; + let tip_high = 2u64; let (message, conflicting_message_input) = - create_message_predicate_from_message(message_amount, 0); + create_message_predicate_from_message(10_000, 0); + universe.database_mut().insert_message(message.clone()); - // Insert a tx for the message id with a low gas amount - let tx_low = TransactionBuilder::script(vec![], vec![]) - .tip(gas_price_low) - .max_fee_limit(max_fee_limit) - .script_gas_limit(GAS_LIMIT) - .add_input(conflicting_message_input.clone()) - .finalize_as_transaction(); + // Given + let tx_high = universe.build_script_transaction( + Some(vec![conflicting_message_input.clone()]), + None, + tip_low, + ); + let tx_high_id = tx_high.id(&ChainId::default()); + let tx_low = universe.build_script_transaction( + Some(vec![conflicting_message_input]), + None, + tip_high, + ); - context.database_mut().insert_message(message); - - let mut txpool = context.build(); - let tx_low_id = tx_low.id(&ChainId::default()); - let tx_low = - check_unwrap_tx_with_gas_price(tx_low, &txpool.config, gas_price_low).await; - let results = txpool.insert(vec![check_tx_to_pool(tx_low)]).unwrap(); - assert_eq!(results.len(), 1); - assert!(results[0].is_ok()); - - // Insert a tx for the message id with a high gas amount - // Because the new transaction's id matches an existing transaction, we compare the gas - // prices of both the new and existing transactions. Since the existing transaction's gas - // price is lower, we accept the new transaction and squeeze out the old transaction. - let tx_high = TransactionBuilder::script(vec![], vec![]) - .tip(gas_price_high) - .max_fee_limit(max_fee_limit) - .script_gas_limit(GAS_LIMIT) - .add_input(conflicting_message_input) - .finalize_as_transaction(); - let tx_high = - check_unwrap_tx_with_gas_price(tx_high, &txpool.config, gas_price_high).await; - let results = txpool.insert(vec![check_tx_to_pool(tx_high)]).unwrap(); - assert_eq!(results.len(), 1); - assert!(results[0].is_ok()); - let squeezed_out_txs = results[0].as_ref().unwrap(); - - assert_eq!(squeezed_out_txs.len(), 1); - assert_eq!(squeezed_out_txs[0].id(), tx_low_id,); + // When + let result1 = universe.verify_and_insert(tx_high).await; + let result2 = universe.verify_and_insert(tx_low).await; + + // Then + assert!(result1.is_ok()); + assert!(result2.is_ok()); + let removed_txs = result2.unwrap(); + assert_eq!(removed_txs.len(), 1); + assert_eq!(removed_txs[0].id(), tx_high_id); } #[tokio::test] -async fn message_of_squeezed_out_tx_can_be_resubmitted_at_lower_gas_price() { - // tx1 (message 1, message 2) gas_price 2 - // tx2 (message 1) gas_price 3 - // squeezes tx1 with higher gas price - // tx3 (message 2) gas_price 1 - // works since tx1 is no longer part of txpool state even though gas price is less - - let mut context = PoolContext::default(); +async fn insert_again_message_after_squeeze_with_even_lower_tip() { + // tx1 (message 1, message 2) tip 2 + // tx2 (message 1) tip 3 + // squeezes tx1 with higher tip + // tx3 (message 2) tip 1 + // works since tx1 is no longer part of txpool state even though tip is less + + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); let (message_1, message_input_1) = create_message_predicate_from_message(10_000, 0); let (message_2, message_input_2) = create_message_predicate_from_message(20_000, 1); + universe.database_mut().insert_message(message_1.clone()); + universe.database_mut().insert_message(message_2.clone()); - // Insert a tx for the message id with a low gas amount - let tx1 = TransactionBuilder::script(vec![], vec![]) - .tip(2) - .max_fee_limit(2) - .script_gas_limit(GAS_LIMIT) - .add_input(message_input_1.clone()) - .add_input(message_input_2.clone()) - .finalize_as_transaction(); - - let tx2 = TransactionBuilder::script(vec![], vec![]) - .tip(3) - .max_fee_limit(3) - .script_gas_limit(GAS_LIMIT) - .add_input(message_input_1) - .finalize_as_transaction(); - - let tx3 = TransactionBuilder::script(vec![], vec![]) - .tip(1) - .max_fee_limit(1) - .script_gas_limit(GAS_LIMIT) - .add_input(message_input_2) - .finalize_as_transaction(); - - context.database_mut().insert_message(message_1); - context.database_mut().insert_message(message_2); - let mut txpool = context.build(); + // Given + let tx1 = universe.build_script_transaction( + Some(vec![message_input_1.clone(), message_input_2.clone()]), + None, + 2, + ); + let tx2 = universe.build_script_transaction(Some(vec![message_input_1]), None, 3); + let tx3 = universe.build_script_transaction(Some(vec![message_input_2]), None, 1); - let tx1 = check_unwrap_tx(tx1, &txpool.config).await; - let tx2 = check_unwrap_tx(tx2, &txpool.config).await; - let tx3 = check_unwrap_tx(tx3, &txpool.config).await; + // When + let result1 = universe.verify_and_insert(tx1).await; + let result2 = universe.verify_and_insert(tx2).await; + let result3 = universe.verify_and_insert(tx3).await; - let results = txpool - .insert(vec![ - check_tx_to_pool(tx1.clone()), - check_tx_to_pool(tx2.clone()), - check_tx_to_pool(tx3.clone()), - ]) - .unwrap(); - assert_eq!(results.len(), 3); - assert!(results[0].is_ok()); - assert!(results[1].is_ok()); - assert!(results[2].is_ok()); + // Then + assert!(result1.is_ok()); + assert!(result2.is_ok()); + assert!(result3.is_ok()); } #[tokio::test] -async fn predicates_with_incorrect_owner_fails() { - let mut context = PoolContext::default(); - let mut coin = context.random_predicate(AssetId::BASE, TEST_COIN_AMOUNT, None); +async fn insert__tx_with_predicates_incorrect_owner() { + let mut universe = TestPoolUniverse::default().config(Config { + utxo_validation: false, + ..Default::default() + }); + universe.build_pool(); + + // Given + let mut coin = universe.random_predicate(AssetId::BASE, TEST_COIN_AMOUNT, None); if let Input::CoinPredicate(CoinPredicate { owner, .. }) = &mut coin { *owner = Address::zeroed(); } - let tx = TransactionBuilder::script(vec![], vec![]) - .script_gas_limit(GAS_LIMIT) - .add_input(coin) - .finalize_as_transaction(); + let tx = universe.build_script_transaction(Some(vec![coin]), None, 0); - let err = check_tx(tx, &Default::default()) - .await - .expect_err("Transaction should be err, got ok"); + // When + let err = universe.verify_and_insert(tx).await.unwrap_err(); - assert!( - format!("{err:?}").contains("InputPredicateOwner"), - "unexpected error: {err:?}", - ) + // Then + assert!(matches!( + err, + Error::ConsensusValidity(CheckError::PredicateVerificationFailed( + PredicateVerificationFailed::InvalidOwner + )) + )); } #[tokio::test] -async fn predicate_without_enough_gas_returns_out_of_gas() { - let mut context = PoolContext::default(); +async fn insert__tx_with_predicate_without_enough_gas() { + let mut universe = TestPoolUniverse::default().config(Config { + utxo_validation: false, + ..Default::default() + }); + universe.build_pool(); let gas_limit = 10000; + // Given let mut consensus_parameters = ConsensusParameters::default(); consensus_parameters .set_tx_params(TxParameters::default().with_max_gas_per_tx(gas_limit)); @@ -1258,7 +999,7 @@ async fn predicate_without_enough_gas_returns_out_of_gas() { PredicateParameters::default().with_max_gas_per_predicate(gas_limit), ); - let coin = context + let coin = universe .custom_predicate( AssetId::BASE, TEST_COIN_AMOUNT, @@ -1268,26 +1009,30 @@ async fn predicate_without_enough_gas_returns_out_of_gas() { ) .into_estimated(&consensus_parameters); - let tx = TransactionBuilder::script(vec![], vec![]) - .script_gas_limit(GAS_LIMIT) - .add_input(coin) - .finalize_as_transaction(); + let tx = universe.build_script_transaction(Some(vec![coin]), None, 0); - let err = check_tx(tx, &Default::default()) - .await - .expect_err("Transaction should be err, got ok"); + // When + let err = universe.verify_and_insert(tx).await.unwrap_err(); - assert!( - err.to_string() - .contains("PredicateVerificationFailed(OutOfGas)"), - "unexpected error: {err}", - ) + // Then + assert!(matches!( + err, + Error::ConsensusValidity(CheckError::PredicateVerificationFailed( + PredicateVerificationFailed::OutOfGas + )) + )); } #[tokio::test] -async fn predicate_that_returns_false_is_invalid() { - let mut context = PoolContext::default(); - let coin = context +async fn insert__tx_with_predicate_that_returns_false() { + let mut universe = TestPoolUniverse::default().config(Config { + utxo_validation: false, + ..Default::default() + }); + universe.build_pool(); + + // Given + let coin = universe .custom_predicate( AssetId::BASE, TEST_COIN_AMOUNT, @@ -1297,23 +1042,29 @@ async fn predicate_that_returns_false_is_invalid() { ) .into_default_estimated(); - let tx = TransactionBuilder::script(vec![], vec![]) - .script_gas_limit(GAS_LIMIT) - .add_input(coin) - .finalize_as_transaction(); + let tx = universe.build_script_transaction(Some(vec![coin]), None, 0); - let err = check_tx(tx, &Default::default()) - .await - .expect_err("Transaction should be err, got ok"); + // When + let err = universe.verify_and_insert(tx).await.unwrap_err(); - assert!( - err.to_string().contains("PredicateVerificationFailed"), - "unexpected error: {err}", - ) + // Then + assert!(matches!( + err, + Error::ConsensusValidity(CheckError::PredicateVerificationFailed( + PredicateVerificationFailed::Panic(PanicReason::PredicateReturnedNonOne) + )) + )); } #[tokio::test] -async fn insert_single__blob_tx_works() { +async fn insert_tx_with_blob() { + let mut universe = TestPoolUniverse::default().config(Config { + utxo_validation: false, + ..Default::default() + }); + universe.build_pool(); + + // Given let program = vec![123; 123]; let tx = TransactionBuilder::blob(BlobBody { id: BlobId::compute(program.as_slice()), @@ -1323,27 +1074,21 @@ async fn insert_single__blob_tx_works() { .add_random_fee_input() .finalize_as_transaction(); - let config = Config { - utxo_validation: false, - ..Default::default() - }; - let context = PoolContext::default().config(config); - let mut txpool = context.build(); - - // Given - let tx = check_unwrap_tx(tx, &txpool.config).await; - let id = tx.id(); - // When - let results = txpool.insert(vec![check_tx_to_pool(tx)]).unwrap(); - assert_eq!(results.len(), 1); - assert!(results[0].is_ok()); - - assert!(txpool.find_one(&id).is_some(), "Should find tx in pool"); + universe.verify_and_insert(tx).await + // Then + .unwrap(); } #[tokio::test] -async fn insert_single__blob_tx_fails_if_blob_already_inserted_and_lower_tip() { +async fn insert__tx_with_blob_already_inserted_at_higher_tip() { + let mut universe = TestPoolUniverse::default().config(Config { + utxo_validation: false, + ..Default::default() + }); + universe.build_pool(); + + // Given let program = vec![123; 123]; let blob_id = BlobId::compute(program.as_slice()); let tx = TransactionBuilder::blob(BlobBody { @@ -1354,18 +1099,7 @@ async fn insert_single__blob_tx_fails_if_blob_already_inserted_and_lower_tip() { .add_random_fee_input() .finalize_as_transaction(); - let config = Config { - utxo_validation: false, - ..Default::default() - }; - let context = PoolContext::default().config(config); - let mut txpool = context.build(); - let tx = check_unwrap_tx(tx, &txpool.config).await; - - // Given - let results = txpool.insert(vec![check_tx_to_pool(tx.clone())]).unwrap(); - assert_eq!(results.len(), 1); - assert!(results[0].is_ok()); + universe.verify_and_insert(tx).await.unwrap(); let same_blob_tx = TransactionBuilder::blob(BlobBody { id: blob_id, @@ -1374,21 +1108,23 @@ async fn insert_single__blob_tx_fails_if_blob_already_inserted_and_lower_tip() { .add_random_fee_input() .add_witness(program.into()) .finalize_as_transaction(); - let same_blob_tx = check_unwrap_tx(same_blob_tx, &txpool.config).await; // When - let results = txpool - .insert(vec![check_tx_to_pool(same_blob_tx.clone())]) - .unwrap(); - assert_eq!(results.len(), 1); - let err = results[0].as_ref().expect_err("Tx should be Err, got Ok"); + let err = universe.verify_and_insert(same_blob_tx).await.unwrap_err(); // Then - assert!(matches!(err, Error::Collided(_))); + assert!(matches!(err, Error::Collided(CollisionReason::Blob(b)) if b == blob_id)); } #[tokio::test] -async fn insert_single__blob_tx_succeeds_if_blob_already_inserted_but_higher_tip() { +async fn insert_tx_with_blob_already_insert_at_lower_tip() { + let mut universe = TestPoolUniverse::default().config(Config { + utxo_validation: false, + ..Default::default() + }); + universe.build_pool(); + + // Given let program = vec![123; 123]; let blob_id = BlobId::compute(program.as_slice()); let tx = TransactionBuilder::blob(BlobBody { @@ -1399,18 +1135,7 @@ async fn insert_single__blob_tx_succeeds_if_blob_already_inserted_but_higher_tip .add_random_fee_input() .finalize_as_transaction(); - let config = Config { - utxo_validation: false, - ..Default::default() - }; - let context = PoolContext::default().config(config); - let mut txpool = context.build(); - let tx = check_unwrap_tx(tx, &txpool.config).await; - - // Given - let results = txpool.insert(vec![check_tx_to_pool(tx.clone())]).unwrap(); - assert_eq!(results.len(), 1); - assert!(results[0].is_ok()); + universe.verify_and_insert(tx).await.unwrap(); let same_blob_tx = TransactionBuilder::blob(BlobBody { id: blob_id, @@ -1421,20 +1146,21 @@ async fn insert_single__blob_tx_succeeds_if_blob_already_inserted_but_higher_tip .tip(100) .max_fee_limit(100) .finalize_as_transaction(); - let same_blob_tx = check_unwrap_tx(same_blob_tx, &txpool.config).await; // When - let results = txpool - .insert(vec![check_tx_to_pool(same_blob_tx.clone())]) - .unwrap(); - assert_eq!(results.len(), 1); + let result = universe.verify_and_insert(same_blob_tx).await; // Then - assert!(results[0].is_ok()); + assert!(result.is_ok()); } #[tokio::test] -async fn insert_single__blob_tx_fails_if_blob_already_exists_in_database() { +async fn insert__tx_blob_already_in_db() { + let mut universe = TestPoolUniverse::default().config(Config { + utxo_validation: false, + ..Default::default() + }); + universe.build_pool(); let program = vec![123; 123]; let blob_id = BlobId::compute(program.as_slice()); let tx = TransactionBuilder::blob(BlobBody { @@ -1445,142 +1171,91 @@ async fn insert_single__blob_tx_fails_if_blob_already_exists_in_database() { .add_random_fee_input() .finalize_as_transaction(); - let config = Config { - utxo_validation: false, - ..Default::default() - }; - let mut context = PoolContext::default().config(config); // Given - context.database_mut().insert_dummy_blob(blob_id); - let mut txpool = context.build(); - let tx = check_unwrap_tx(tx, &txpool.config).await; + universe.database_mut().insert_dummy_blob(blob_id); // When - let result = txpool.insert(vec![check_tx_to_pool(tx.clone())]).unwrap(); - assert_eq!(result.len(), 1); - let err = result[0].as_ref().expect_err("Tx should be Err, got Ok"); + let err = universe.verify_and_insert(tx).await.unwrap_err(); // Then assert!(matches!( err, - Error::NotInsertedBlobIdAlreadyTaken(b) if *b == blob_id + Error::NotInsertedBlobIdAlreadyTaken(b) if b == blob_id )); } #[tokio::test] -async fn insert__if_tx3_depends_and_collides_witg_tx2() { - let mut context = PoolContext::default(); +async fn insert__if_tx3_depends_and_collides_with_tx2() { + let mut universe = TestPoolUniverse::default(); + universe.build_pool(); // tx1 {inputs: {}, outputs: {coinA}, tip: 1} - let (_, gas_coin) = context.setup_coin(); - let (output_a, unset_input) = context.create_output_and_input(1); - let tx1 = TransactionBuilder::script(vec![], vec![]) - .tip(1) - .max_fee_limit(1) - .script_gas_limit(GAS_LIMIT) - .add_input(gas_coin) - .add_output(output_a) - .finalize_as_transaction(); - + let (output_a, unset_input) = universe.create_output_and_input(); + let tx1 = universe.build_script_transaction(None, Some(vec![output_a]), 1); // tx2 {inputs: {coinA}, outputs: {coinB}, tip: 1} - let (_, gas_coin) = context.setup_coin(); let input_a = unset_input.into_input(UtxoId::new(tx1.id(&Default::default()), 0)); - let (output_b, unset_input) = context.create_output_and_input(1); - let tx2 = TransactionBuilder::script(vec![], vec![]) - .tip(1) - .max_fee_limit(1) - .script_gas_limit(GAS_LIMIT) - .add_input(input_a.clone()) - .add_input(gas_coin) - .add_output(output_b) - .finalize_as_transaction(); - + let (output_b, unset_input) = universe.create_output_and_input(); + let tx2 = universe.build_script_transaction( + Some(vec![input_a.clone()]), + Some(vec![output_b]), + 1, + ); // Given // tx3 {inputs: {coinA, coinB}, outputs:{}, tip: 20} - let (_, gas_coin) = context.setup_coin(); let input_b = unset_input.into_input(UtxoId::new(tx2.id(&Default::default()), 0)); - let tx3 = TransactionBuilder::script(vec![], vec![]) - .tip(20) - .max_fee_limit(20) - .script_gas_limit(GAS_LIMIT) - .add_input(input_a) - .add_input(input_b.clone()) - .add_input(gas_coin) - .finalize_as_transaction(); + let tx3 = universe.build_script_transaction(Some(vec![input_a, input_b]), None, 20); + + // When + universe.verify_and_insert(tx1).await.unwrap(); + universe.verify_and_insert(tx2).await.unwrap(); + let err = universe.verify_and_insert(tx3).await.unwrap_err(); + + // Then + assert!(matches!(err, Error::NotInsertedCollisionIsDependency)); +} - let (_, gas_coin) = context.setup_coin(); +#[tokio::test] +async fn insert__tx_upgrade_with_invalid_wasm() { + let mut universe = TestPoolUniverse::default().config(Config { + utxo_validation: false, + ..Default::default() + }); + universe.build_pool(); - let mut txpool = context.build(); - let tx1 = check_unwrap_tx(tx1, &txpool.config).await; - let tx2 = check_unwrap_tx(tx2, &txpool.config).await; - let tx3 = check_unwrap_tx(tx3, &txpool.config).await; + let predicate = vec![op::ret(1)].into_iter().collect::>(); + let privileged_address = Input::predicate_owner(predicate.clone()); + + // Given + let tx = TransactionBuilder::upgrade(UpgradePurpose::StateTransition { + root: Bytes32::new([1; 32]), + }) + .add_input(Input::coin_predicate( + UtxoId::new(Bytes32::new([1; 32]), 0), + privileged_address, + 1_000_000_000, + AssetId::BASE, + Default::default(), + Default::default(), + predicate, + vec![], + )) + .finalize_as_transaction(); + let mut params = ConsensusParameters::default(); + params.set_privileged_address(privileged_address); // When - let results = txpool - .insert(vec![ - check_tx_to_pool(tx1.clone()), - check_tx_to_pool(tx2.clone()), - check_tx_to_pool(tx3.clone()), - ]) - .unwrap(); - assert_eq!(results.len(), 3); - assert!(results[0].is_ok()); - assert!(results[1].is_ok()); + let result = universe + .verify_and_insert_with_consensus_params_wasm_checker( + tx, + params, + MockWasmChecker::new(Err(WasmValidityError::NotEnabled)), + ) + .await + .unwrap_err(); // Then - let err = results[2].as_ref().expect_err("Tx3 should be Err, got Ok"); - assert!(matches!(err, Error::Collided(_))); + assert!(matches!( + result, + Error::WasmValidity(WasmValidityError::NotEnabled) + )); } - -// TODO: Reinstantiatte when https://github.com/FuelLabs/fuel-core/issues/2186 is implemented -// #[tokio::test] -// async fn insert_inner__rejects_upgrade_tx_with_invalid_wasm() { -// let predicate = vec![op::ret(1)].into_iter().collect::>(); -// let privileged_address = Input::predicate_owner(predicate.clone()); - -// let config = Config { -// utxo_validation: false, -// ..Default::default() -// }; -// let context = PoolContext::default() -// .config(config) -// .wasm_checker(MockWasmChecker { -// result: Err(WasmValidityError::NotValid), -// }); -// let mut txpool = context.build(); -// let gas_price_provider = MockTxPoolGasPrice::new(0); - -// // Given -// let tx = TransactionBuilder::upgrade(UpgradePurpose::StateTransition { -// root: Bytes32::new([1; 32]), -// }) -// .add_input(Input::coin_predicate( -// UtxoId::new(Bytes32::new([1; 32]), 0), -// privileged_address, -// 1_000_000_000, -// AssetId::BASE, -// Default::default(), -// Default::default(), -// predicate, -// vec![], -// )) -// .finalize_as_transaction(); -// let mut params = ConsensusParameters::default(); -// params.set_privileged_address(privileged_address); -// let tx = check_single_tx( -// tx, -// Default::default(), -// false, -// ¶ms, -// &gas_price_provider, -// MemoryInstance::new(), -// ) -// .await -// .expect("Transaction should be checked"); - -// // When -// let result = txpool.insert_single(tx); - -// // Then -// assert_eq!(result, Err(Error::NotInsertedInvalidWasm)); -// } diff --git a/crates/services/txpool_v2/src/transaction_conversion.rs b/crates/services/txpool_v2/src/transaction_conversion.rs deleted file mode 100644 index 071b45fb3bd..00000000000 --- a/crates/services/txpool_v2/src/transaction_conversion.rs +++ /dev/null @@ -1,139 +0,0 @@ -use std::sync::Arc; - -use fuel_core_types::{ - fuel_tx::{ - ConsensusParameters, - Transaction, - }, - fuel_types::BlockHeight, - fuel_vm::{ - checked_transaction::{ - CheckPredicateParams, - CheckPredicates, - Checked, - CheckedTransaction, - Checks, - IntoChecked, - }, - interpreter::Memory, - }, -}; - -use crate::{ - error::Error, - ports::{ - GasPriceProvider, - MemoryPool, - }, - GasPrice, -}; - -pub async fn check_transactions( - txs: Vec, - current_height: BlockHeight, - utxp_validation: bool, - consensus_params: &ConsensusParameters, - gas_price_provider: &Provider, - memory_pool: Arc, -) -> Vec, Error>> -where - Provider: GasPriceProvider, - MP: MemoryPool, -{ - let mut checked_txs = Vec::with_capacity(txs.len()); - - for tx in txs.into_iter() { - checked_txs.push( - check_single_tx( - tx, - current_height, - utxp_validation, - consensus_params, - gas_price_provider, - memory_pool.get_memory().await, - ) - .await, - ); - } - - checked_txs -} - -pub async fn check_single_tx( - tx: Transaction, - current_height: BlockHeight, - utxo_validation: bool, - consensus_params: &ConsensusParameters, - gas_price_provider: &GasPrice, - memory: M, -) -> Result, Error> -where - GasPrice: GasPriceProvider, - M: Memory + Send + Sync + 'static, -{ - if tx.is_mint() { - return Err(Error::NotSupportedTransactionType); - } - - let tx: Checked = if utxo_validation { - let tx = tx - .into_checked_basic(current_height, consensus_params)? - .check_signatures(&consensus_params.chain_id())?; - - let parameters = CheckPredicateParams::from(consensus_params); - let tx = - tokio_rayon::spawn_fifo(move || tx.check_predicates(¶meters, memory)) - .await?; - - debug_assert!(tx.checks().contains(Checks::all())); - - tx - } else { - tx.into_checked_basic(current_height, consensus_params)? - }; - - let gas_price = gas_price_provider.next_gas_price().await?; - - let tx = verify_tx_min_gas_price(tx, consensus_params, gas_price)?; - - Ok(tx) -} - -fn verify_tx_min_gas_price( - tx: Checked, - consensus_params: &ConsensusParameters, - gas_price: GasPrice, -) -> Result, Error> { - let tx: CheckedTransaction = tx.into(); - let gas_costs = consensus_params.gas_costs(); - let fee_parameters = consensus_params.fee_params(); - let ready_tx = match tx { - CheckedTransaction::Script(script) => { - let ready = script.into_ready(gas_price, gas_costs, fee_parameters)?; - let (_, checked) = ready.decompose(); - CheckedTransaction::Script(checked) - } - CheckedTransaction::Create(create) => { - let ready = create.into_ready(gas_price, gas_costs, fee_parameters)?; - let (_, checked) = ready.decompose(); - CheckedTransaction::Create(checked) - } - CheckedTransaction::Upgrade(tx) => { - let ready = tx.into_ready(gas_price, gas_costs, fee_parameters)?; - let (_, checked) = ready.decompose(); - CheckedTransaction::Upgrade(checked) - } - CheckedTransaction::Upload(tx) => { - let ready = tx.into_ready(gas_price, gas_costs, fee_parameters)?; - let (_, checked) = ready.decompose(); - CheckedTransaction::Upload(checked) - } - CheckedTransaction::Blob(tx) => { - let ready = tx.into_ready(gas_price, gas_costs, fee_parameters)?; - let (_, checked) = ready.decompose(); - CheckedTransaction::Blob(checked) - } - CheckedTransaction::Mint(_) => return Err(Error::NotSupportedTransactionType), - }; - Ok(ready_tx.into()) -} diff --git a/crates/services/txpool_v2/src/verifications.rs b/crates/services/txpool_v2/src/verifications.rs new file mode 100644 index 00000000000..997f1b35afa --- /dev/null +++ b/crates/services/txpool_v2/src/verifications.rs @@ -0,0 +1,225 @@ +use fuel_core_types::{ + blockchain::header::ConsensusParametersVersion, + fuel_tx::{ + field::UpgradePurpose as _, + ConsensusParameters, + Transaction, + UpgradePurpose, + }, + fuel_types::BlockHeight, + fuel_vm::{ + checked_transaction::{ + CheckPredicateParams, + CheckPredicates, + Checked, + CheckedTransaction, + Checks, + IntoChecked, + }, + interpreter::Memory, + }, + services::txpool::PoolTransaction, +}; + +use crate::{ + error::Error, + ports::{ + AtomicView, + GasPriceProvider, + TxPoolPersistentStorage, + WasmChecker, + }, + service::TxPool, + GasPrice, +}; + +#[derive(Debug, Clone, Eq, PartialEq, Hash)] +pub struct UnverifiedTx(Transaction); + +#[derive(Debug, Clone, Eq, PartialEq, Hash)] +pub struct BasicVerifiedTx(CheckedTransaction); + +#[derive(Debug, Clone, Eq, PartialEq, Hash)] +pub struct InputDependenciesVerifiedTx(Checked); + +#[derive(Debug, Clone, Eq, PartialEq, Hash)] +pub struct InputComputationVerifiedTx(Checked); + +#[derive(Debug, Clone, Eq, PartialEq, Hash)] +pub struct FullyVerifiedTx(Checked); + +pub enum TransactionVerifState { + Unverified(UnverifiedTx), + BasicVerified(BasicVerifiedTx), + InputDependenciesVerified(InputDependenciesVerifiedTx), + InputComputationVerified(InputComputationVerifiedTx), + FullyVerified(FullyVerifiedTx), +} + +impl UnverifiedTx { + pub async fn perform_basic_verifications( + self, + current_height: BlockHeight, + consensus_params: &ConsensusParameters, + gas_price_provider: &impl GasPriceProvider, + ) -> Result { + if self.0.is_mint() { + return Err(Error::NotSupportedTransactionType); + } + let tx = self + .0 + .into_checked_basic(current_height, consensus_params)?; + let gas_price = gas_price_provider.next_gas_price().await?; + let tx = verify_tx_min_gas_price(tx, consensus_params, gas_price)?; + Ok(BasicVerifiedTx(tx)) + } +} + +impl BasicVerifiedTx { + pub fn perform_inputs_verifications( + self, + pool: TxPool, + version: ConsensusParametersVersion, + ) -> Result + where + PSProvider: AtomicView, + PSView: TxPoolPersistentStorage, + { + let pool_tx = checked_tx_into_pool(self.0, version)?; + + pool.read().can_insert_transaction(&pool_tx)?; + let checked_transaction: CheckedTransaction = pool_tx.into(); + Ok(InputDependenciesVerifiedTx(checked_transaction.into())) + } +} + +impl InputDependenciesVerifiedTx { + pub fn perform_input_computation_verifications( + self, + consensus_params: &ConsensusParameters, + wasm_checker: &impl WasmChecker, + memory: M, + ) -> Result + where + M: Memory + Send + Sync + 'static, + { + if let Transaction::Upgrade(upgrade) = self.0.transaction() { + if let UpgradePurpose::StateTransition { root } = upgrade.upgrade_purpose() { + wasm_checker + .validate_uploaded_wasm(root) + .map_err(Error::WasmValidity)?; + } + } + + let parameters = CheckPredicateParams::from(consensus_params); + let tx = self.0.check_predicates(¶meters, memory)?; + + Ok(InputComputationVerifiedTx(tx)) + } +} + +impl InputComputationVerifiedTx { + pub fn perform_final_verifications( + self, + consensus_params: &ConsensusParameters, + ) -> Result { + let tx = self.0.check_signatures(&consensus_params.chain_id())?; + debug_assert!(tx.checks().contains(Checks::all())); + Ok(FullyVerifiedTx(tx)) + } +} + +impl FullyVerifiedTx { + pub fn into_pool_transaction( + self, + version: ConsensusParametersVersion, + ) -> Result { + checked_tx_into_pool(self.0.into(), version) + } +} + +#[allow(clippy::too_many_arguments)] +pub async fn perform_all_verifications( + tx: Transaction, + pool: TxPool, + current_height: BlockHeight, + consensus_params: &ConsensusParameters, + consensus_params_version: ConsensusParametersVersion, + gas_price_provider: &impl GasPriceProvider, + wasm_checker: &impl WasmChecker, + memory: M, +) -> Result +where + M: Memory + Send + Sync + 'static, + PSProvider: AtomicView, + PSView: TxPoolPersistentStorage, +{ + let unverified = UnverifiedTx(tx); + let basically_verified_tx = unverified + .perform_basic_verifications(current_height, consensus_params, gas_price_provider) + .await?; + let inputs_verified_tx = basically_verified_tx + .perform_inputs_verifications(pool, consensus_params_version)?; + let input_computation_verified_tx = inputs_verified_tx + .perform_input_computation_verifications( + consensus_params, + wasm_checker, + memory, + )?; + let fully_verified_tx = + input_computation_verified_tx.perform_final_verifications(consensus_params)?; + fully_verified_tx.into_pool_transaction(consensus_params_version) +} + +fn verify_tx_min_gas_price( + tx: Checked, + consensus_params: &ConsensusParameters, + gas_price: GasPrice, +) -> Result { + let tx: CheckedTransaction = tx.into(); + let gas_costs = consensus_params.gas_costs(); + let fee_parameters = consensus_params.fee_params(); + let read = match tx { + CheckedTransaction::Script(script) => { + let ready = script.into_ready(gas_price, gas_costs, fee_parameters)?; + let (_, checked) = ready.decompose(); + CheckedTransaction::Script(checked) + } + CheckedTransaction::Create(create) => { + let ready = create.into_ready(gas_price, gas_costs, fee_parameters)?; + let (_, checked) = ready.decompose(); + CheckedTransaction::Create(checked) + } + CheckedTransaction::Upgrade(tx) => { + let ready = tx.into_ready(gas_price, gas_costs, fee_parameters)?; + let (_, checked) = ready.decompose(); + CheckedTransaction::Upgrade(checked) + } + CheckedTransaction::Upload(tx) => { + let ready = tx.into_ready(gas_price, gas_costs, fee_parameters)?; + let (_, checked) = ready.decompose(); + CheckedTransaction::Upload(checked) + } + CheckedTransaction::Blob(tx) => { + let ready = tx.into_ready(gas_price, gas_costs, fee_parameters)?; + let (_, checked) = ready.decompose(); + CheckedTransaction::Blob(checked) + } + CheckedTransaction::Mint(_) => return Err(Error::NotSupportedTransactionType), + }; + Ok(read) +} + +pub fn checked_tx_into_pool( + tx: CheckedTransaction, + version: ConsensusParametersVersion, +) -> Result { + match tx { + CheckedTransaction::Script(tx) => Ok(PoolTransaction::Script(tx, version)), + CheckedTransaction::Create(tx) => Ok(PoolTransaction::Create(tx, version)), + CheckedTransaction::Mint(_) => Err(Error::MintIsDisallowed), + CheckedTransaction::Upgrade(tx) => Ok(PoolTransaction::Upgrade(tx, version)), + CheckedTransaction::Upload(tx) => Ok(PoolTransaction::Upload(tx, version)), + CheckedTransaction::Blob(tx) => Ok(PoolTransaction::Blob(tx, version)), + } +} diff --git a/crates/types/src/services/txpool.rs b/crates/types/src/services/txpool.rs index e1b047e27ff..22ed0d9eade 100644 --- a/crates/types/src/services/txpool.rs +++ b/crates/types/src/services/txpool.rs @@ -160,6 +160,40 @@ impl PoolTransaction { } } +impl From for Transaction { + fn from(tx: PoolTransaction) -> Self { + match tx { + PoolTransaction::Script(tx, _) => { + Transaction::Script(Into::<(Script, _)>::into(tx).0) + } + PoolTransaction::Create(tx, _) => { + Transaction::Create(Into::<(Create, _)>::into(tx).0) + } + PoolTransaction::Upgrade(tx, _) => { + Transaction::Upgrade(Into::<(Upgrade, _)>::into(tx).0) + } + PoolTransaction::Upload(tx, _) => { + Transaction::Upload(Into::<(Upload, _)>::into(tx).0) + } + PoolTransaction::Blob(tx, _) => { + Transaction::Blob(Into::<(Blob, _)>::into(tx).0) + } + } + } +} + +impl From for CheckedTransaction { + fn from(tx: PoolTransaction) -> Self { + match tx { + PoolTransaction::Script(tx, _) => CheckedTransaction::Script(tx), + PoolTransaction::Create(tx, _) => CheckedTransaction::Create(tx), + PoolTransaction::Upgrade(tx, _) => CheckedTransaction::Upgrade(tx), + PoolTransaction::Upload(tx, _) => CheckedTransaction::Upload(tx), + PoolTransaction::Blob(tx, _) => CheckedTransaction::Blob(tx), + } + } +} + impl From<&PoolTransaction> for Transaction { fn from(tx: &PoolTransaction) -> Self { match tx {