diff --git a/consensus/core/src/tx.rs b/consensus/core/src/tx.rs index 1d8dd8334..70f9da09e 100644 --- a/consensus/core/src/tx.rs +++ b/consensus/core/src/tx.rs @@ -8,6 +8,7 @@ pub use script_public_key::{ scriptvec, ScriptPublicKey, ScriptPublicKeyT, ScriptPublicKeyVersion, ScriptPublicKeys, ScriptVec, SCRIPT_VECTOR_SIZE, }; use serde::{Deserialize, Serialize}; +use std::collections::HashSet; use std::sync::atomic::AtomicU64; use std::sync::atomic::Ordering::SeqCst; use std::{ @@ -437,6 +438,37 @@ impl> MutableTransaction { None } } + + /// A function for estimating the amount of memory bytes used by this transaction (dedicated to mempool usage). + /// We need consistency between estimation calls so only this function should be used for this purpose since + /// `estimate_mem_bytes` is sensitive to pointer wrappers such as Arc + pub fn mempool_estimated_bytes(&self) -> usize { + self.estimate_mem_bytes() + } + + pub fn has_parent(&self, possible_parent: TransactionId) -> bool { + self.tx.as_ref().inputs.iter().any(|x| x.previous_outpoint.transaction_id == possible_parent) + } + + pub fn has_parent_in_set(&self, possible_parents: &HashSet) -> bool { + self.tx.as_ref().inputs.iter().any(|x| possible_parents.contains(&x.previous_outpoint.transaction_id)) + } +} + +impl> MemSizeEstimator for MutableTransaction { + fn estimate_mem_bytes(&self) -> usize { + size_of::() + + self + .entries + .iter() + .map(|op| { + // size_of::>() already counts SCRIPT_VECTOR_SIZE bytes within, so we only add the delta + size_of::>() + + op.as_ref().map_or(0, |e| e.script_public_key.script().len().saturating_sub(SCRIPT_VECTOR_SIZE)) + }) + .sum::() + + self.tx.as_ref().estimate_mem_bytes() + } } impl> AsRef for MutableTransaction { diff --git a/mining/errors/src/mempool.rs b/mining/errors/src/mempool.rs index be8ff389a..319aaa484 100644 --- a/mining/errors/src/mempool.rs +++ b/mining/errors/src/mempool.rs @@ -33,9 +33,9 @@ pub enum RuleError { #[error("replace by fee found more than one double spending transaction in the mempool")] RejectRbfTooManyDoubleSpendingTransactions, - /// New behavior: a transaction is rejected if the mempool is full - #[error("number of high-priority transactions in mempool ({0}) has reached the maximum allowed ({1})")] - RejectMempoolIsFull(usize, u64), + /// a transaction is rejected if the mempool is full + #[error("transaction could not be added to the mempool because it's full with transactions with higher priority")] + RejectMempoolIsFull, /// An error emitted by mining\src\mempool\check_transaction_standard.rs #[error("transaction {0} is not standard: {1}")] diff --git a/mining/src/manager.rs b/mining/src/manager.rs index 081854698..a8a51f3cc 100644 --- a/mining/src/manager.rs +++ b/mining/src/manager.rs @@ -836,6 +836,11 @@ impl MiningManager { pub fn unknown_transactions(&self, transactions: Vec) -> Vec { self.mempool.read().unknown_transactions(transactions) } + + #[cfg(test)] + pub(crate) fn get_estimated_size(&self) -> usize { + self.mempool.read().get_estimated_size() + } } /// Async proxy for the mining manager diff --git a/mining/src/manager_tests.rs b/mining/src/manager_tests.rs index 0a2f04faf..6ddc86e45 100644 --- a/mining/src/manager_tests.rs +++ b/mining/src/manager_tests.rs @@ -10,7 +10,7 @@ mod tests { model::frontier::selectors::TakeAllSelector, tx::{Orphan, Priority, RbfPolicy}, }, - model::tx_query::TransactionQuery, + model::{tx_insert::TransactionInsertion, tx_query::TransactionQuery}, testutils::consensus_mock::ConsensusMock, MiningCounters, }; @@ -35,6 +35,7 @@ mod tests { pay_to_address_script, pay_to_script_hash_signature_script, test_helpers::{create_transaction, create_transaction_with_change, op_true_script}, }; + use kaspa_utils::mem_size::MemSizeEstimator; use std::{iter::once, sync::Arc}; use tokio::sync::mpsc::{error::TryRecvError, unbounded_channel}; @@ -1116,6 +1117,71 @@ mod tests { // TODO: extend the test according to the golang scenario } + // This is a sanity test for the mempool eviction policy. We check that if the mempool reached to its maximum + // (in bytes) a high paying transaction will evict as much transactions as needed so it can enter the + // mempool. + // TODO: Add a test where we try to add a heavy transaction with fee rate that's higher than some of the mempool + // transactions, but not enough, so the transaction will be rejected nonetheless. + #[test] + fn test_evict() { + const TX_COUNT: usize = 10; + let txs = (0..TX_COUNT).map(|i| create_transaction_with_utxo_entry(i as u32, 0)).collect_vec(); + + let consensus = Arc::new(ConsensusMock::new()); + let counters = Arc::new(MiningCounters::default()); + let mut config = Config::build_default(TARGET_TIME_PER_BLOCK, false, MAX_BLOCK_MASS); + let tx_size = txs[0].mempool_estimated_bytes(); + let size_limit = TX_COUNT * tx_size; + config.mempool_size_limit = size_limit; + let mining_manager = MiningManager::with_config(config, None, counters); + + for tx in txs { + validate_and_insert_mutable_transaction(&mining_manager, consensus.as_ref(), tx).unwrap(); + } + assert_eq!(mining_manager.get_all_transactions(TransactionQuery::TransactionsOnly).0.len(), TX_COUNT); + + let heavy_tx_low_fee = { + let mut heavy_tx = create_transaction_with_utxo_entry(TX_COUNT as u32, 0); + let mut inner_tx = (*(heavy_tx.tx)).clone(); + inner_tx.payload = vec![0u8; TX_COUNT / 2 * tx_size - inner_tx.estimate_mem_bytes()]; + heavy_tx.tx = inner_tx.into(); + heavy_tx.calculated_fee = Some(2081); + heavy_tx + }; + assert!(validate_and_insert_mutable_transaction(&mining_manager, consensus.as_ref(), heavy_tx_low_fee.clone()).is_err()); + assert_eq!(mining_manager.get_all_transactions(TransactionQuery::TransactionsOnly).0.len(), TX_COUNT); + + let heavy_tx_high_fee = { + let mut heavy_tx = create_transaction_with_utxo_entry(TX_COUNT as u32 + 1, 0); + let mut inner_tx = (*(heavy_tx.tx)).clone(); + inner_tx.payload = vec![0u8; TX_COUNT / 2 * tx_size - inner_tx.estimate_mem_bytes()]; + heavy_tx.tx = inner_tx.into(); + heavy_tx.calculated_fee = Some(500_000); + heavy_tx + }; + validate_and_insert_mutable_transaction(&mining_manager, consensus.as_ref(), heavy_tx_high_fee.clone()).unwrap(); + assert_eq!(mining_manager.get_all_transactions(TransactionQuery::TransactionsOnly).0.len(), TX_COUNT - 5); + assert!(mining_manager.get_estimated_size() <= size_limit); + + let too_big_tx = { + let mut heavy_tx = create_transaction_with_utxo_entry(TX_COUNT as u32 + 2, 0); + let mut inner_tx = (*(heavy_tx.tx)).clone(); + inner_tx.payload = vec![0u8; size_limit]; + heavy_tx.tx = inner_tx.into(); + heavy_tx.calculated_fee = Some(500_000); + heavy_tx + }; + assert!(validate_and_insert_mutable_transaction(&mining_manager, consensus.as_ref(), too_big_tx.clone()).is_err()); + } + + fn validate_and_insert_mutable_transaction( + mining_manager: &MiningManager, + consensus: &dyn ConsensusApi, + tx: MutableTransaction, + ) -> Result { + mining_manager.validate_and_insert_mutable_transaction(consensus, tx, Priority::Low, Orphan::Allowed, RbfPolicy::Forbidden) + } + fn sweep_compare_modified_template_to_built( consensus: &dyn ConsensusApi, address_prefix: Prefix, diff --git a/mining/src/mempool/config.rs b/mining/src/mempool/config.rs index 419a4362a..b90a36577 100644 --- a/mining/src/mempool/config.rs +++ b/mining/src/mempool/config.rs @@ -1,9 +1,10 @@ use kaspa_consensus_core::constants::TX_VERSION; -pub(crate) const DEFAULT_MAXIMUM_TRANSACTION_COUNT: u32 = 1_000_000; +pub(crate) const DEFAULT_MAXIMUM_TRANSACTION_COUNT: usize = 1_000_000; +pub(crate) const DEFAULT_MEMPOOL_SIZE_LIMIT: usize = 1_000_000_000; pub(crate) const DEFAULT_MAXIMUM_BUILD_BLOCK_TEMPLATE_ATTEMPTS: u64 = 5; -pub(crate) const DEFAULT_TRANSACTION_EXPIRE_INTERVAL_SECONDS: u64 = 60; +pub(crate) const DEFAULT_TRANSACTION_EXPIRE_INTERVAL_SECONDS: u64 = 24 * 60 * 60; pub(crate) const DEFAULT_TRANSACTION_EXPIRE_SCAN_INTERVAL_SECONDS: u64 = 10; pub(crate) const DEFAULT_ACCEPTED_TRANSACTION_EXPIRE_INTERVAL_SECONDS: u64 = 120; pub(crate) const DEFAULT_ACCEPTED_TRANSACTION_EXPIRE_SCAN_INTERVAL_SECONDS: u64 = 10; @@ -11,9 +12,7 @@ pub(crate) const DEFAULT_ORPHAN_EXPIRE_INTERVAL_SECONDS: u64 = 60; pub(crate) const DEFAULT_ORPHAN_EXPIRE_SCAN_INTERVAL_SECONDS: u64 = 10; pub(crate) const DEFAULT_MAXIMUM_ORPHAN_TRANSACTION_MASS: u64 = 100_000; - -// TODO: when rusty-kaspa nodes run most of the network, consider increasing this value -pub(crate) const DEFAULT_MAXIMUM_ORPHAN_TRANSACTION_COUNT: u64 = 50; +pub(crate) const DEFAULT_MAXIMUM_ORPHAN_TRANSACTION_COUNT: u64 = 500; /// DEFAULT_MINIMUM_RELAY_TRANSACTION_FEE specifies the minimum transaction fee for a transaction to be accepted to /// the mempool and relayed. It is specified in sompi per 1kg (or 1000 grams) of transaction mass. @@ -28,7 +27,8 @@ pub(crate) const DEFAULT_MAXIMUM_STANDARD_TRANSACTION_VERSION: u16 = TX_VERSION; #[derive(Clone, Debug)] pub struct Config { - pub maximum_transaction_count: u32, + pub maximum_transaction_count: usize, + pub mempool_size_limit: usize, pub maximum_build_block_template_attempts: u64, pub transaction_expire_interval_daa_score: u64, pub transaction_expire_scan_interval_daa_score: u64, @@ -51,7 +51,8 @@ pub struct Config { impl Config { #[allow(clippy::too_many_arguments)] pub fn new( - maximum_transaction_count: u32, + maximum_transaction_count: usize, + mempool_size_limit: usize, maximum_build_block_template_attempts: u64, transaction_expire_interval_daa_score: u64, transaction_expire_scan_interval_daa_score: u64, @@ -72,6 +73,7 @@ impl Config { ) -> Self { Self { maximum_transaction_count, + mempool_size_limit, maximum_build_block_template_attempts, transaction_expire_interval_daa_score, transaction_expire_scan_interval_daa_score, @@ -97,6 +99,7 @@ impl Config { pub const fn build_default(target_milliseconds_per_block: u64, relay_non_std_transactions: bool, max_block_mass: u64) -> Self { Self { maximum_transaction_count: DEFAULT_MAXIMUM_TRANSACTION_COUNT, + mempool_size_limit: DEFAULT_MEMPOOL_SIZE_LIMIT, maximum_build_block_template_attempts: DEFAULT_MAXIMUM_BUILD_BLOCK_TEMPLATE_ATTEMPTS, transaction_expire_interval_daa_score: DEFAULT_TRANSACTION_EXPIRE_INTERVAL_SECONDS * 1000 / target_milliseconds_per_block, transaction_expire_scan_interval_daa_score: DEFAULT_TRANSACTION_EXPIRE_SCAN_INTERVAL_SECONDS * 1000 @@ -121,7 +124,7 @@ impl Config { } pub fn apply_ram_scale(mut self, ram_scale: f64) -> Self { - self.maximum_transaction_count = (self.maximum_transaction_count as f64 * ram_scale.min(1.0)) as u32; // Allow only scaling down + self.maximum_transaction_count = (self.maximum_transaction_count as f64 * ram_scale.min(1.0)) as usize; // Allow only scaling down self } diff --git a/mining/src/mempool/mod.rs b/mining/src/mempool/mod.rs index e5cd7dbeb..845b16aa6 100644 --- a/mining/src/mempool/mod.rs +++ b/mining/src/mempool/mod.rs @@ -162,6 +162,11 @@ impl Mempool { .filter(|transaction_id| !(self.transaction_pool.has(transaction_id) || self.orphan_pool.has(transaction_id))); self.accepted_transactions.unaccepted(&mut not_in_pools_txs) } + + #[cfg(test)] + pub(crate) fn get_estimated_size(&self) -> usize { + self.transaction_pool.get_estimated_size() + } } pub mod tx { diff --git a/mining/src/mempool/model/frontier.rs b/mining/src/mempool/model/frontier.rs index e8d2b54ab..3a7a41a07 100644 --- a/mining/src/mempool/model/frontier.rs +++ b/mining/src/mempool/model/frontier.rs @@ -5,12 +5,12 @@ use crate::{ }; use feerate_key::FeerateTransactionKey; -use kaspa_consensus_core::block::TemplateTransactionSelector; +use kaspa_consensus_core::{block::TemplateTransactionSelector, tx::Transaction}; use kaspa_core::trace; use rand::{distributions::Uniform, prelude::Distribution, Rng}; use search_tree::SearchTree; use selectors::{SequenceSelector, SequenceSelectorInput, TakeAllSelector}; -use std::collections::HashSet; +use std::{collections::HashSet, iter::FusedIterator, sync::Arc}; pub(crate) mod feerate_key; pub(crate) mod search_tree; @@ -254,6 +254,11 @@ impl Frontier { } estimator } + + /// Returns an iterator to the transactions in the frontier in increasing feerate order + pub fn ascending_iter(&self) -> impl DoubleEndedIterator> + ExactSizeIterator + FusedIterator { + self.search_tree.ascending_iter().map(|key| &key.tx) + } } #[cfg(test)] diff --git a/mining/src/mempool/model/transactions_pool.rs b/mining/src/mempool/model/transactions_pool.rs index bc3469409..5449606c4 100644 --- a/mining/src/mempool/model/transactions_pool.rs +++ b/mining/src/mempool/model/transactions_pool.rs @@ -11,16 +11,17 @@ use crate::{ }, tx::Priority, }, - model::topological_index::TopologicalIndex, + model::{topological_index::TopologicalIndex, TransactionIdSet}, Policy, }; use kaspa_consensus_core::{ block::TemplateTransactionSelector, tx::{MutableTransaction, TransactionId, TransactionOutpoint}, }; -use kaspa_core::{time::unix_now, trace, warn}; +use kaspa_core::{debug, time::unix_now, trace}; use std::{ collections::{hash_map::Keys, hash_set::Iter}, + iter::once, sync::Arc, }; @@ -64,6 +65,8 @@ pub(crate) struct TransactionsPool { /// last expire scan time in milliseconds last_expire_scan_time: u64, + estimated_size: usize, + /// Store of UTXOs utxo_set: MempoolUtxoSet, } @@ -79,6 +82,7 @@ impl TransactionsPool { last_expire_scan_daa_score: 0, last_expire_scan_time: unix_now(), utxo_set: MempoolUtxoSet::new(), + estimated_size: 0, } } @@ -88,15 +92,16 @@ impl TransactionsPool { transaction: MutableTransaction, virtual_daa_score: u64, priority: Priority, + transaction_size: usize, ) -> RuleResult<&MempoolTransaction> { let transaction = MempoolTransaction::new(transaction, priority, virtual_daa_score); let id = transaction.id(); - self.add_mempool_transaction(transaction)?; + self.add_mempool_transaction(transaction, transaction_size)?; Ok(self.get(&id).unwrap()) } /// Add a mempool transaction to the pool - pub(crate) fn add_mempool_transaction(&mut self, transaction: MempoolTransaction) -> RuleResult<()> { + pub(crate) fn add_mempool_transaction(&mut self, transaction: MempoolTransaction, transaction_size: usize) -> RuleResult<()> { let id = transaction.id(); assert!(!self.all_transactions.contains_key(&id), "transaction {id} to be added already exists in the transactions pool"); @@ -117,6 +122,7 @@ impl TransactionsPool { } self.utxo_set.add_transaction(&transaction.mtx); + self.estimated_size += transaction_size; self.all_transactions.insert(id, transaction); trace!("Added transaction {}", id); Ok(()) @@ -159,6 +165,7 @@ impl TransactionsPool { // Remove the transaction from the mempool UTXO set self.utxo_set.remove_transaction(&removed_tx.mtx, &parent_ids); + self.estimated_size -= removed_tx.mtx.mempool_estimated_bytes(); Ok(removed_tx) } @@ -181,70 +188,73 @@ impl TransactionsPool { self.ready_transactions.build_feerate_estimator(args) } - /// Is the mempool transaction identified by `transaction_id` unchained, thus having no successor? - pub(crate) fn transaction_is_unchained(&self, transaction_id: &TransactionId) -> bool { - if self.all_transactions.contains_key(transaction_id) { - if let Some(chains) = self.chained_transactions.get(transaction_id) { - return chains.is_empty(); - } - return true; - } - false - } - /// Returns the exceeding low-priority transactions having the lowest fee rates in order - /// to have room for at least `free_slots` new transactions. The returned transactions + /// to make room for `transaction`. The returned transactions /// are guaranteed to be unchained (no successor in mempool) and to not be parent of /// `transaction`. /// - /// An error is returned if the mempool is filled with high priority transactions. + /// An error is returned if the mempool is filled with high priority transactions, or + /// there are not enough lower feerate transactions that can be removed to accommodate `transaction` pub(crate) fn limit_transaction_count( &self, - free_slots: usize, transaction: &MutableTransaction, + transaction_size: usize, ) -> RuleResult> { - assert!(free_slots > 0); - // Returns a vector of transactions to be removed that the caller has to remove actually. - // The caller is golang validateAndInsertTransaction equivalent. - // This behavior differs from golang impl. - let trim_size = self.len() + free_slots - usize::min(self.len() + free_slots, self.config.maximum_transaction_count as usize); - let mut transactions_to_remove = Vec::with_capacity(trim_size); - if trim_size > 0 { - // TODO: consider introducing an index on all_transactions low-priority items instead. - // - // Sorting this vector here may be sub-optimal compared with maintaining a sorted - // index of all_transactions low-priority items if the proportion of low-priority txs - // in all_transactions is important. - let low_priority_txs = self - .all_transactions - .values() - .filter(|x| x.priority == Priority::Low && self.transaction_is_unchained(&x.id()) && !x.is_parent_of(transaction)); - - if trim_size == 1 { - // This is the most likely case. Here we just search the minimum, thus avoiding the need to sort altogether. - if let Some(tx) = low_priority_txs.min_by(|a, b| a.fee_rate().partial_cmp(&b.fee_rate()).unwrap()) { - transactions_to_remove.push(tx); - } - } else { - let mut low_priority_txs = low_priority_txs.collect::>(); - if low_priority_txs.len() > trim_size { - low_priority_txs.sort_by(|a, b| a.fee_rate().partial_cmp(&b.fee_rate()).unwrap()); - transactions_to_remove.extend_from_slice(&low_priority_txs[0..usize::min(trim_size, low_priority_txs.len())]); - } else { - transactions_to_remove = low_priority_txs; - } - } + // No eviction needed -- return + if self.len() < self.config.maximum_transaction_count + && self.estimated_size + transaction_size <= self.config.mempool_size_limit + { + return Ok(Default::default()); } - // An error is returned if the mempool is filled with high priority and other unremovable transactions. - let tx_count = self.len() + free_slots - transactions_to_remove.len(); - if tx_count as u64 > self.config.maximum_transaction_count as u64 { - let err = RuleError::RejectMempoolIsFull(tx_count - free_slots, self.config.maximum_transaction_count as u64); - warn!("{}", err.to_string()); - return Err(err); + // Returns a vector of transactions to be removed (the caller has to actually remove) + let feerate_threshold = transaction.calculated_feerate().unwrap(); + let mut txs_to_remove = Vec::with_capacity(1); // Normally we expect a single removal + let mut selection_overall_size = 0; + for tx in self + .ready_transactions + .ascending_iter() + .map(|tx| self.all_transactions.get(&tx.id()).unwrap()) + .filter(|mtx| mtx.priority == Priority::Low) + { + // TODO (optimization): inline the `has_parent_in_set` check within the redeemer traversal and exit early if possible + let redeemers = self.get_redeemer_ids_in_pool(&tx.id()).into_iter().chain(once(tx.id())).collect::(); + if transaction.has_parent_in_set(&redeemers) { + continue; + } + + // We are iterating ready txs by ascending feerate so the pending tx has lower feerate than all remaining txs + if tx.fee_rate() > feerate_threshold { + let err = RuleError::RejectMempoolIsFull; + debug!("Transaction {} with feerate {} has been rejected: {}", transaction.id(), feerate_threshold, err); + return Err(err); + } + + txs_to_remove.push(tx.id()); + selection_overall_size += tx.mtx.mempool_estimated_bytes(); + + if self.len() + 1 - txs_to_remove.len() <= self.config.maximum_transaction_count + && self.estimated_size + transaction_size - selection_overall_size <= self.config.mempool_size_limit + { + return Ok(txs_to_remove); + } } - Ok(transactions_to_remove.iter().map(|x| x.id()).collect()) + // We could not find sufficient space for the pending transaction + debug!( + "Mempool is filled with high-priority/ancestor txs (count: {}, bytes: {}). Transaction {} with feerate {} and size {} has been rejected: {}", + self.len(), + self.estimated_size, + transaction.id(), + feerate_threshold, + transaction_size, + RuleError::RejectMempoolIsFull + ); + Err(RuleError::RejectMempoolIsFull) + } + + pub(crate) fn get_estimated_size(&self) -> usize { + self.estimated_size } pub(crate) fn all_transaction_ids_with_priority(&self, priority: Priority) -> Vec { diff --git a/mining/src/mempool/model/tx.rs b/mining/src/mempool/model/tx.rs index 9b65faeb2..27bb87d09 100644 --- a/mining/src/mempool/model/tx.rs +++ b/mining/src/mempool/model/tx.rs @@ -27,11 +27,6 @@ impl MempoolTransaction { assert!(contextual_mass > 0, "expected to be called for validated txs only"); self.mtx.calculated_fee.unwrap() as f64 / contextual_mass as f64 } - - pub(crate) fn is_parent_of(&self, transaction: &MutableTransaction) -> bool { - let parent_id = self.id(); - transaction.tx.inputs.iter().any(|x| x.previous_outpoint.transaction_id == parent_id) - } } impl RbfPolicy { diff --git a/mining/src/mempool/validate_and_insert_transaction.rs b/mining/src/mempool/validate_and_insert_transaction.rs index bcfedc2db..3eddac459 100644 --- a/mining/src/mempool/validate_and_insert_transaction.rs +++ b/mining/src/mempool/validate_and_insert_transaction.rs @@ -22,7 +22,7 @@ impl Mempool { rbf_policy: RbfPolicy, ) -> RuleResult { self.validate_transaction_unacceptance(&transaction)?; - // Populate mass in the beginning, it will be used in multiple places throughout the validation and insertion. + // Populate mass and estimated_size in the beginning, it will be used in multiple places throughout the validation and insertion. transaction.calculated_compute_mass = Some(consensus.calculate_transaction_compute_mass(&transaction.tx)); self.validate_transaction_in_isolation(&transaction)?; let feerate_threshold = self.get_replace_by_fee_constraint(&transaction, rbf_policy)?; @@ -67,19 +67,57 @@ impl Mempool { } } + // Perform mempool in-context validations prior to possible RBF replacements + self.validate_transaction_in_context(&transaction)?; + // Check double spends and try to remove them if the RBF policy requires it let removed_transaction = self.execute_replace_by_fee(&transaction, rbf_policy)?; - self.validate_transaction_in_context(&transaction)?; + // + // Note: there exists a case below where `limit_transaction_count` returns an error signaling that + // this tx should be rejected due to mempool size limits (rather than evicting others). However, + // if this tx happened to be an RBF tx, it might have already caused an eviction in the line + // above. We choose to ignore this rare case for now, as it essentially means that even the increased + // feerate of the replacement tx is very low relative to the mempool overall. + // // Before adding the transaction, check if there is room in the pool - self.transaction_pool.limit_transaction_count(1, &transaction)?.iter().try_for_each(|x| { - self.remove_transaction(x, true, TxRemovalReason::MakingRoom, format!(" for {}", transaction_id).as_str()) - })?; + let transaction_size = transaction.mempool_estimated_bytes(); + let txs_to_remove = self.transaction_pool.limit_transaction_count(&transaction, transaction_size)?; + for x in txs_to_remove.iter() { + self.remove_transaction(x, true, TxRemovalReason::MakingRoom, format!(" for {}", transaction_id).as_str())?; + // self.transaction_pool.limit_transaction_count(&transaction) returns the + // smallest prefix of `ready_transactions` (sorted by ascending fee-rate) + // that makes enough room for `transaction`, but since each call to `self.remove_transaction` + // also removes all transactions dependant on `x` we might already have sufficient space, so + // we constantly check the break condition. + // + // Note that self.transaction_pool.len() < self.config.maximum_transaction_count means we have + // at least one available slot in terms of the count limit + if self.transaction_pool.len() < self.config.maximum_transaction_count + && self.transaction_pool.get_estimated_size() + transaction_size <= self.config.mempool_size_limit + { + break; + } + } + + assert!( + self.transaction_pool.len() < self.config.maximum_transaction_count + && self.transaction_pool.get_estimated_size() + transaction_size <= self.config.mempool_size_limit, + "Transactions in mempool: {}, max: {}, mempool bytes size: {}, max: {}", + self.transaction_pool.len() + 1, + self.config.maximum_transaction_count, + self.transaction_pool.get_estimated_size() + transaction_size, + self.config.mempool_size_limit, + ); // Add the transaction to the mempool as a MempoolTransaction and return a clone of the embedded Arc - let accepted_transaction = - self.transaction_pool.add_transaction(transaction, consensus.get_virtual_daa_score(), priority)?.mtx.tx.clone(); + let accepted_transaction = self + .transaction_pool + .add_transaction(transaction, consensus.get_virtual_daa_score(), priority, transaction_size)? + .mtx + .tx + .clone(); Ok(TransactionPostValidation { removed: removed_transaction, accepted: Some(accepted_transaction) }) } @@ -98,6 +136,7 @@ impl Mempool { if self.transaction_pool.has(&transaction_id) { return Err(RuleError::RejectDuplicate(transaction_id)); } + if !self.config.accept_non_standard { self.check_transaction_standard_in_isolation(transaction)?; } diff --git a/mining/src/testutils/consensus_mock.rs b/mining/src/testutils/consensus_mock.rs index d68e062a0..28d3f5897 100644 --- a/mining/src/testutils/consensus_mock.rs +++ b/mining/src/testutils/consensus_mock.rs @@ -133,11 +133,14 @@ impl ConsensusApi for ConsensusMock { // At this point we know all UTXO entries are populated, so we can safely calculate the fee let total_in: u64 = mutable_tx.entries.iter().map(|x| x.as_ref().unwrap().amount).sum(); let total_out: u64 = mutable_tx.tx.outputs.iter().map(|x| x.value).sum(); - let calculated_fee = total_in - total_out; mutable_tx .tx .set_mass(self.calculate_transaction_storage_mass(mutable_tx).unwrap() + mutable_tx.calculated_compute_mass.unwrap()); - mutable_tx.calculated_fee = Some(calculated_fee); + + if mutable_tx.calculated_fee.is_none() { + let calculated_fee = total_in - total_out; + mutable_tx.calculated_fee = Some(calculated_fee); + } Ok(()) }