From e65581991fd1b92b6456f7a8f9b5da4614f7c0cc Mon Sep 17 00:00:00 2001 From: yukang Date: Mon, 15 May 2023 18:19:55 +0800 Subject: [PATCH 01/58] begin refactor tx_pool --- tx-pool/src/component/commit_txs_scanner.rs | 7 +- tx-pool/src/pool.rs | 634 +++++++++++++++++++- 2 files changed, 615 insertions(+), 26 deletions(-) diff --git a/tx-pool/src/component/commit_txs_scanner.rs b/tx-pool/src/component/commit_txs_scanner.rs index 8f97432ae8..af277a1b0a 100644 --- a/tx-pool/src/component/commit_txs_scanner.rs +++ b/tx-pool/src/component/commit_txs_scanner.rs @@ -2,6 +2,7 @@ use crate::component::{container::AncestorsScoreSortKey, entry::TxEntry, propose use ckb_types::{core::Cycle, packed::ProposalShortId}; use ckb_util::LinkedHashMap; use std::collections::{BTreeSet, HashMap, HashSet}; +use crate::pool::MultiIndexPoolEntryMap; // A template data struct used to store modified entries when package txs #[derive(Default)] @@ -49,6 +50,7 @@ const MAX_CONSECUTIVE_FAILURES: usize = 500; /// find txs to package into commitment pub struct CommitTxsScanner<'a> { proposed_pool: &'a ProposedPool, + pool_entries: &'a MultiIndexPoolEntryMap, entries: Vec, // modified_entries will store sorted packages after they are modified // because some of their txs are already in the block @@ -60,10 +62,11 @@ pub struct CommitTxsScanner<'a> { } impl<'a> CommitTxsScanner<'a> { - pub fn new(proposed_pool: &'a ProposedPool) -> CommitTxsScanner<'a> { + pub fn new(proposed_pool: &'a ProposedPool, pool_entries: &'a MultiIndexPoolEntryMap) -> CommitTxsScanner<'a> { CommitTxsScanner { proposed_pool, entries: Vec::new(), + pool_entries: pool_entries, modified_entries: TxModifiedEntries::default(), fetched_txs: HashSet::default(), failed_txs: HashSet::default(), @@ -80,7 +83,7 @@ impl<'a> CommitTxsScanner<'a> { let mut cycles: Cycle = 0; let mut consecutive_failed = 0; - let mut iter = self.proposed_pool.score_sorted_iter().peekable(); + let mut iter = self.pool_entries.score_sorted_iter().peekable(); loop { let mut using_modified = false; diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 8fb2da50c6..5ee73cfbb8 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -1,6 +1,10 @@ //! Top-level Pool type, methods, and tests +extern crate rustc_hash; +extern crate slab; use super::component::{commit_txs_scanner::CommitTxsScanner, TxEntry}; use crate::callback::Callbacks; +use crate::component::container::AncestorsScoreSortKey; +use crate::component::entry::EvictKey; use crate::component::pending::PendingQueue; use crate::component::proposed::ProposedPool; use crate::component::recent_reject::RecentReject; @@ -10,9 +14,18 @@ use ckb_app_config::TxPoolConfig; use ckb_logger::{debug, error, trace, warn}; use ckb_snapshot::Snapshot; use ckb_store::ChainStore; +use ckb_types::core::error::OutPointError; +use ckb_types::packed::OutPoint; +use ckb_types::{ + core::cell::{CellMetaBuilder, CellProvider, CellStatus}, + prelude::*, +}; use ckb_types::{ core::{ - cell::{resolve_transaction, OverlayCellChecker, OverlayCellProvider, ResolvedTransaction}, + cell::{ + resolve_transaction, CellChecker, OverlayCellChecker, OverlayCellProvider, + ResolvedTransaction, + }, tx_pool::{TxPoolEntryInfo, TxPoolIds}, Cycle, TransactionView, UncleBlockView, }, @@ -20,7 +33,10 @@ use ckb_types::{ }; use ckb_verification::{cache::CacheEntry, TxVerifyEnv}; use lru::LruCache; -use std::collections::HashSet; +use multi_index_map::MultiIndexMap; +use std::collections::hash_map::Entry; +use std::collections::HashMap; +use std::collections::{HashSet, VecDeque}; use std::sync::Arc; const COMMITTED_HASH_CACHE_SIZE: usize = 100_000; @@ -51,6 +67,40 @@ macro_rules! evict_for_trim_size { }; } +type ConflictEntry = (TxEntry, Reject); + +#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub enum Status { + Pending, + Gap, + Proposed, +} + +#[derive(MultiIndexMap, Clone)] +pub struct PoolEntry { + #[multi_index(hashed_unique)] + pub id: ProposalShortId, + #[multi_index(ordered_non_unique)] + pub score: AncestorsScoreSortKey, + #[multi_index(ordered_non_unique)] + pub status: Status, + #[multi_index(ordered_non_unique)] + pub evict_key: EvictKey, + + pub inner: TxEntry, + // other sort key +} + +impl MultiIndexPoolEntryMap { + /// sorted by ancestor score from higher to lower + pub fn score_sorted_iter(&self) -> impl Iterator { + // Note: multi_index don't support reverse order iteration now + // so we need to collect and reverse + let entries = self.iter_by_score().collect::>(); + entries.into_iter().rev().map(move |entry| &entry.inner) + } +} + /// Tx-pool implementation pub struct TxPool { pub(crate) config: TxPoolConfig, @@ -60,6 +110,18 @@ pub struct TxPool { pub(crate) gap: PendingQueue, /// Tx pool that finely for commit pub(crate) proposed: ProposedPool, + + /// The pool entries with different kinds of sort strategies + pub(crate) entries: MultiIndexPoolEntryMap, + /// dep-set map represent in-pool tx's header deps + pub(crate) header_deps: HashMap>, + /// dep-set map represent in-pool tx's deps + pub(crate) deps: HashMap>, + /// input-set map represent in-pool tx's inputs + pub(crate) inputs: HashMap>, + pub(crate) outputs: HashMap>, + pub(crate) max_ancestors_count: usize, + /// cache for committed transactions hash pub(crate) committed_txs_hash_cache: LruCache, // sum of all tx_pool tx's virtual sizes. @@ -77,12 +139,18 @@ pub struct TxPool { impl TxPool { /// Create new TxPool pub fn new(config: TxPoolConfig, snapshot: Arc) -> TxPool { - let recent_reject = build_recent_reject(&config); + let recent_reject = Self::build_recent_reject(&config); let expiry = config.expiry_hours as u64 * 60 * 60 * 1000; TxPool { pending: PendingQueue::new(), gap: PendingQueue::new(), proposed: ProposedPool::new(config.max_ancestors_count), + entries: MultiIndexPoolEntryMap::default(), + header_deps: HashMap::default(), + deps: Default::default(), + inputs: Default::default(), + outputs: Default::default(), + max_ancestors_count: config.max_ancestors_count, committed_txs_hash_cache: LruCache::new(COMMITTED_HASH_CACHE_SIZE), total_tx_size: 0, total_tx_cycles: 0, @@ -135,6 +203,24 @@ impl TxPool { self.total_tx_cycles = total_tx_cycles; } + fn add_poolentry(&mut self, entry: TxEntry, status: Status) -> bool { + let short_id = entry.proposal_short_id(); + if self.entries.get_by_id(&short_id).is_some() { + return false; + } + trace!("add_{:?} {}", status, entry.transaction().hash()); + let score = entry.as_score_key(); + let evict_key = entry.as_evict_key(); + self.entries.insert(PoolEntry { + id: short_id, + score, + status, + inner: entry, + evict_key, + }); + true + } + /// Add tx to pending pool /// If did have this value present, false is returned. pub fn add_pending(&mut self, entry: TxEntry) -> bool { @@ -145,6 +231,10 @@ impl TxPool { self.pending.add_entry(entry) } + pub fn add_pending_v2(&mut self, entry: TxEntry) -> bool { + self.add_poolentry(entry, Status::Pending) + } + /// Add tx which proposed but still uncommittable to gap pool pub fn add_gap(&mut self, entry: TxEntry) -> bool { if self.proposed.contains_key(&entry.proposal_short_id()) { @@ -154,17 +244,29 @@ impl TxPool { self.gap.add_entry(entry) } + pub fn add_gap_v2(&mut self, entry: TxEntry) -> bool { + self.add_poolentry(entry, Status::Gap) + } + /// Add tx to proposed pool pub fn add_proposed(&mut self, entry: TxEntry) -> Result { trace!("add_proposed {}", entry.transaction().hash()); self.proposed.add_entry(entry) } + pub fn add_proposed_v2(&mut self, entry: TxEntry) -> bool { + self.add_poolentry(entry, Status::Proposed) + } + /// Returns true if the tx-pool contains a tx with specified id. pub fn contains_proposal_id(&self, id: &ProposalShortId) -> bool { self.pending.contains_key(id) || self.gap.contains_key(id) || self.proposed.contains_key(id) } + pub fn contains_proposal_id_v2(&self, id: &ProposalShortId) -> bool { + self.entries.get_by_id(id).is_some() + } + /// Returns tx with cycles corresponding to the id. pub fn get_tx_with_cycles(&self, id: &ProposalShortId) -> Option<(TransactionView, Cycle)> { self.pending @@ -182,6 +284,12 @@ impl TxPool { }) } + pub fn get_tx_with_cycles_v2(&self, id: &ProposalShortId) -> Option<(TransactionView, Cycle)> { + self.entries + .get_by_id(id) + .map(|entry| (entry.inner.transaction().clone(), entry.inner.cycles)) + } + /// Returns tx corresponding to the id. pub fn get_tx(&self, id: &ProposalShortId) -> Option<&TransactionView> { self.pending @@ -190,11 +298,28 @@ impl TxPool { .or_else(|| self.proposed.get_tx(id)) } + pub fn get_tx_v2(&self, id: &ProposalShortId) -> Option<&TransactionView> { + self.entries + .get_by_id(id) + .map(|entry| entry.inner.transaction()) + } + /// Returns tx from pending and gap corresponding to the id. RPC pub fn get_entry_from_pending_or_gap(&self, id: &ProposalShortId) -> Option<&TxEntry> { self.pending.get(id).or_else(|| self.gap.get(id)) } + pub fn get_entry_from_pending_or_gap_v2(&self, id: &ProposalShortId) -> Option<&TxEntry> { + if let Some(entry) = self.entries.get_by_id(id) { + match entry.status { + Status::Pending | Status::Gap => return Some(&entry.inner), + _ => return None, + } + } else { + return None; + } + } + pub(crate) fn proposed(&self) -> &ProposedPool { &self.proposed } @@ -209,6 +334,15 @@ impl TxPool { .or_else(|| self.pending.get_tx(id)) } + pub(crate) fn get_tx_from_proposed_and_others_v2( + &self, + id: &ProposalShortId, + ) -> Option<&TransactionView> { + self.entries + .get_by_id(id) + .map(|entry| entry.inner.transaction()) + } + pub(crate) fn remove_committed_txs<'a>( &mut self, txs: impl Iterator, @@ -245,6 +379,129 @@ impl TxPool { } } + pub(crate) fn resolve_conflict_header_dep_v2( + &mut self, + detached_headers: &HashSet, + callbacks: &Callbacks, + ) { + for (entry, reject) in self.__resolve_conflict_header_dep_v2(detached_headers) { + callbacks.call_reject(self, &entry, reject); + } + } + + pub(crate) fn get_descendants(&self, entry: &TxEntry) -> HashSet { + let mut entries: VecDeque<&TxEntry> = VecDeque::new(); + entries.push_back(entry); + + let mut descendants = HashSet::new(); + while let Some(entry) = entries.pop_front() { + let outputs = entry.transaction().output_pts(); + + for output in outputs { + if let Some(ids) = self.outputs.get(&output) { + for id in ids { + if descendants.insert(id.clone()) { + if let Some(entry) = self.entries.get_by_id(id) { + entries.push_back(&entry.inner); + } + } + } + } + } + } + descendants + } + + pub(crate) fn remove_entry_relation(&mut self, entry: &TxEntry) { + let inputs = entry.transaction().input_pts_iter(); + let tx_short_id = entry.proposal_short_id(); + let outputs = entry.transaction().output_pts(); + + for i in inputs { + if let Entry::Occupied(mut occupied) = self.inputs.entry(i) { + let empty = { + let ids = occupied.get_mut(); + ids.remove(&tx_short_id); + ids.is_empty() + }; + if empty { + occupied.remove(); + } + } + } + + // remove dep + for d in entry.related_dep_out_points().cloned() { + if let Entry::Occupied(mut occupied) = self.deps.entry(d) { + let empty = { + let ids = occupied.get_mut(); + ids.remove(&tx_short_id); + ids.is_empty() + }; + if empty { + occupied.remove(); + } + } + } + + for o in outputs { + self.outputs.remove(&o); + } + + self.header_deps.remove(&tx_short_id); + } + + fn remove_entry(&mut self, id: &ProposalShortId) -> Option { + let removed = self.entries.remove_by_id(id); + + if let Some(ref entry) = removed { + self.remove_entry_relation(&entry.inner); + } + removed.map(|e| e.inner) + } + + fn remove_entry_and_descendants(&mut self, id: &ProposalShortId) -> Vec { + let mut removed = Vec::new(); + if let Some(entry) = self.entries.remove_by_id(id) { + let descendants = self.get_descendants(&entry.inner); + self.remove_entry_relation(&entry.inner); + removed.push(entry.inner); + for id in descendants { + if let Some(entry) = self.remove_entry(&id) { + removed.push(entry); + } + } + } + removed + } + + fn __resolve_conflict_header_dep_v2( + &mut self, + headers: &HashSet, + ) -> Vec { + let mut conflicts = Vec::new(); + + // invalid header deps + let mut ids = Vec::new(); + for (tx_id, deps) in self.header_deps.iter() { + for hash in deps { + if headers.contains(hash) { + ids.push((hash.clone(), tx_id.clone())); + break; + } + } + } + + for (blk_hash, id) in ids { + let entries = self.remove_entry_and_descendants(&id); + for entry in entries { + let reject = Reject::Resolve(OutPointError::InvalidHeader(blk_hash.to_owned())); + conflicts.push((entry, reject)); + } + } + conflicts + } + pub(crate) fn remove_committed_tx(&mut self, tx: &TransactionView, callbacks: &Callbacks) { let hash = tx.hash(); let short_id = tx.proposal_short_id(); @@ -287,6 +544,50 @@ impl TxPool { } } + fn resolve_conflict(&mut self, tx: &TransactionView) -> Vec { + let inputs = tx.input_pts_iter(); + let mut conflicts = Vec::new(); + + for i in inputs { + if let Some(ids) = self.inputs.remove(&i) { + for id in ids { + let entries = self.remove_entry_and_descendants(&id); + for entry in entries { + let reject = Reject::Resolve(OutPointError::Dead(i.clone())); + conflicts.push((entry, reject)); + } + } + } + + // deps consumed + if let Some(ids) = self.deps.remove(&i) { + for id in ids { + let entries = self.remove_entry_and_descendants(&id); + for entry in entries { + let reject = Reject::Resolve(OutPointError::Dead(i.clone())); + conflicts.push((entry, reject)); + } + } + } + } + conflicts + } + + pub(crate) fn remove_committed_tx_v2(&mut self, tx: &TransactionView, callbacks: &Callbacks) { + let hash = tx.hash(); + let short_id = tx.proposal_short_id(); + if let Some(entry) = self.remove_entry(&short_id) { + debug!("remove_committed_tx from gap {}", hash); + callbacks.call_committed(self, &entry) + } + { + let conflicts = self.resolve_conflict(tx); + for (entry, reject) in conflicts { + callbacks.call_reject(self, &entry, reject); + } + } + } + // Expire all transaction (and their dependencies) in the pool. pub(crate) fn remove_expired(&mut self, callbacks: &Callbacks) { let now_ms = ckb_systemtime::unix_time_as_millis(); @@ -318,6 +619,42 @@ impl TxPool { } } + fn remove_entries_by_filter bool>( + &mut self, + mut predicate: P, + ) -> Vec { + let mut removed = Vec::new(); + for (_, entry) in self.entries.iter() { + if predicate(&entry.id, &entry.inner) { + removed.push(entry.inner.clone()); + } + } + for entry in &removed { + self.remove_entry(&entry.proposal_short_id()); + } + + removed + } + + // Expire all transaction (and their dependencies) in the pool. + pub(crate) fn remove_expired_v2(&mut self, callbacks: &Callbacks) { + let now_ms = ckb_systemtime::unix_time_as_millis(); + let removed: Vec<_> = self + .entries + .iter() + .filter(|&(_, entry)| self.expiry + entry.inner.timestamp < now_ms) + .map(|(_, entry)| entry.inner.clone()) + .collect(); + + for entry in removed { + self.remove_entry(&entry.proposal_short_id()); + let tx_hash = entry.transaction().hash(); + debug!("remove_expired {} timestamp({})", tx_hash, entry.timestamp); + let reject = Reject::Expiry(entry.timestamp); + callbacks.call_reject(self, &entry, reject); + } + } + // Remove transactions from the pool until total size < size_limit. pub(crate) fn limit_size(&mut self, callbacks: &Callbacks) { while self.total_tx_size > self.config.max_tx_pool_size { @@ -331,6 +668,31 @@ impl TxPool { } } + pub(crate) fn limit_size_v2(&mut self, callbacks: &Callbacks) { + while self.total_tx_size > self.config.max_tx_pool_size { + if let Some(id) = self + .entries + .iter_by_evict_key() + .next() + .map(|entry| entry.id.clone()) + { + let removed = self.remove_entry_and_descendants(&id); + for entry in removed { + let tx_hash = entry.transaction().hash(); + debug!( + "removed by size limit {} timestamp({})", + tx_hash, entry.timestamp + ); + let reject = Reject::Full(format!( + "the fee_rate for this transaction is: {}", + entry.fee_rate() + )); + callbacks.call_reject(self, &entry, reject); + } + } + } + } + // remove transaction with detached proposal from gap and proposed // try re-put to pending pub(crate) fn remove_by_detached_proposal<'a>( @@ -360,6 +722,34 @@ impl TxPool { } } + // remove transaction with detached proposal from gap and proposed + // try re-put to pending + pub(crate) fn remove_by_detached_proposal_v2<'a>( + &mut self, + ids: impl Iterator, + ) { + for id in ids { + if let Some(e) = self.entries.get_by_id(id) { + let status = e.status; + // TODO: double check this + if status == Status::Pending { + continue; + } + let mut entries = self.remove_entry_and_descendants(id); + entries.sort_unstable_by_key(|entry| entry.ancestors_count); + for mut entry in entries { + let tx_hash = entry.transaction().hash(); + entry.reset_ancestors_state(); + let ret = self.add_pending(entry); + debug!( + "remove_by_detached_proposal from {:?} {} add_pending {}", status, + tx_hash, ret + ); + } + } + } + } + pub(crate) fn remove_tx(&mut self, id: &ProposalShortId) -> bool { let entries = self.proposed.remove_entry_and_descendants(id); if !entries.is_empty() { @@ -382,6 +772,14 @@ impl TxPool { false } + pub(crate) fn remove_tx_v2(&mut self, id: &ProposalShortId) -> bool { + if let Some(entry) = self.remove_entry(id) { + self.update_statics_for_remove_tx(entry.size, entry.cycles); + return true; + } + false + } + pub(crate) fn resolve_tx_from_pending_and_proposed( &self, tx: TransactionView, @@ -402,6 +800,18 @@ impl TxPool { .map_err(Reject::Resolve) } + pub(crate) fn resolve_tx_from_pending_and_proposed_v2( + &self, + tx: TransactionView, + ) -> Result, Reject> { + let snapshot = self.snapshot(); + let provider = OverlayCellProvider::new(&self.entries, snapshot); + let mut seen_inputs = HashSet::new(); + resolve_transaction(tx, &mut seen_inputs, &provider, snapshot) + .map(Arc::new) + .map_err(Reject::Resolve) + } + pub(crate) fn check_rtx_from_pending_and_proposed( &self, rtx: &ResolvedTransaction, @@ -416,6 +826,17 @@ impl TxPool { .map_err(Reject::Resolve) } + pub(crate) fn check_rtx_from_pending_and_proposed_v2( + &self, + rtx: &ResolvedTransaction, + ) -> Result<(), Reject> { + let snapshot = self.snapshot(); + let checker = OverlayCellChecker::new(&self.entries, snapshot); + let mut seen_inputs = HashSet::new(); + rtx.check(&mut seen_inputs, &checker, snapshot) + .map_err(Reject::Resolve) + } + pub(crate) fn resolve_tx_from_proposed( &self, tx: TransactionView, @@ -428,6 +849,17 @@ impl TxPool { .map_err(Reject::Resolve) } + pub(crate) fn resolve_tx_from_proposed_v2( + &self, + rtx: &ResolvedTransaction, + ) -> Result<(), Reject> { + let snapshot = self.snapshot(); + let checker = OverlayCellChecker::new(&self.entries, snapshot); + let mut seen_inputs = HashSet::new(); + rtx.check(&mut seen_inputs, &checker, snapshot) + .map_err(Reject::Resolve) + } + pub(crate) fn check_rtx_from_proposed(&self, rtx: &ResolvedTransaction) -> Result<(), Reject> { let snapshot = self.snapshot(); let cell_checker = OverlayCellChecker::new(&self.proposed, snapshot); @@ -436,6 +868,17 @@ impl TxPool { .map_err(Reject::Resolve) } + pub(crate) fn check_rtx_from_proposed_v2( + &self, + rtx: &ResolvedTransaction, + ) -> Result<(), Reject> { + let snapshot = self.snapshot(); + let cell_checker = OverlayCellChecker::new(&self.entries, snapshot); + let mut seen_inputs = HashSet::new(); + rtx.check(&mut seen_inputs, &cell_checker, snapshot) + .map_err(Reject::Resolve) + } + pub(crate) fn gap_rtx( &mut self, cache_entry: CacheEntry, @@ -498,6 +941,24 @@ impl TxPool { } } + // fill proposal txs + pub fn fill_proposals( + &self, + limit: usize, + exclusion: &HashSet, + proposals: &mut HashSet, + status: &Status, + ) { + for entry in self.entries.get_by_status(status) { + if proposals.len() == limit { + break; + } + if !exclusion.contains(&entry.id) { + proposals.insert(entry.id.clone()); + } + } + } + /// Get to-be-proposal transactions that may be included in the next block. pub fn get_proposals( &self, @@ -511,6 +972,18 @@ impl TxPool { proposals } + /// Get to-be-proposal transactions that may be included in the next block. + pub fn get_proposals_v2( + &self, + limit: usize, + exclusion: &HashSet, + ) -> HashSet { + let mut proposals = HashSet::with_capacity(limit); + self.fill_proposals(limit, exclusion, &mut proposals, &Status::Pending); + self.fill_proposals(limit, exclusion, &mut proposals, &Status::Gap); + proposals + } + /// Returns tx from tx-pool or storage corresponding to the id. pub fn get_tx_from_pool_or_store( &self, @@ -542,6 +1015,25 @@ impl TxPool { TxPoolIds { pending, proposed } } + // This is for RPC request, performance is not critical + pub(crate) fn get_ids_v2(&self) -> TxPoolIds { + let pending: Vec = self + .entries + .get_by_status(&Status::Pending) + .iter() + .chain(self.entries.get_by_status(&Status::Gap).iter()) + .map(|entry| entry.inner.transaction().hash()) + .collect(); + + let proposed: Vec = self + .proposed + .iter() + .map(|(_, entry)| entry.transaction().hash()) + .collect(); + + TxPoolIds { pending, proposed } + } + pub(crate) fn get_all_entry_info(&self) -> TxPoolEntryInfo { let pending = self .pending @@ -563,8 +1055,27 @@ impl TxPool { TxPoolEntryInfo { pending, proposed } } + pub(crate) fn get_all_entry_info_v2(&self) -> TxPoolEntryInfo { + let pending = self + .entries + .get_by_status(&Status::Pending) + .iter() + .chain(self.entries.get_by_status(&Status::Gap).iter()) + .map(|entry| (entry.inner.transaction().hash(), entry.inner.to_info())) + .collect(); + + let proposed = self + .entries + .get_by_status(&Status::Proposed) + .iter() + .map(|entry| (entry.inner.transaction().hash(), entry.inner.to_info())) + .collect(); + + TxPoolEntryInfo { pending, proposed } + } + pub(crate) fn drain_all_transactions(&mut self) -> Vec { - let mut txs = CommitTxsScanner::new(&self.proposed) + let mut txs = CommitTxsScanner::new(&self.proposed, &self.entries) .txs_to_commit(self.total_tx_size, self.total_tx_cycles) .0 .into_iter() @@ -579,10 +1090,47 @@ impl TxPool { txs } + pub(crate) fn drain_all_transactions_v2(&mut self) -> Vec { + let mut txs = CommitTxsScanner::new(&self.proposed, &self.entries) + .txs_to_commit(self.total_tx_size, self.total_tx_cycles) + .0 + .into_iter() + .map(|tx_entry| tx_entry.into_transaction()) + .collect::>(); + self.proposed.clear(); + let mut pending = self + .entries + .remove_by_status(&Status::Pending) + .into_iter() + .map(|e| e.inner.into_transaction()) + .collect::>(); + txs.append(&mut pending); + let mut gap = self + .entries + .remove_by_status(&Status::Gap) + .into_iter() + .map(|e| e.inner.into_transaction()) + .collect::>(); + txs.append(&mut gap); + self.total_tx_size = 0; + self.total_tx_cycles = 0; + self.deps.clear(); + self.inputs.clear(); + self.header_deps.clear(); + self.outputs.clear(); + // self.touch_last_txs_updated_at(); + txs + } + pub(crate) fn clear(&mut self, snapshot: Arc) { self.pending = PendingQueue::new(); self.gap = PendingQueue::new(); self.proposed = ProposedPool::new(self.config.max_ancestors_count); + self.entries = MultiIndexPoolEntryMap::default(); + self.header_deps = HashMap::default(); + self.deps = HashMap::default(); + self.inputs = HashMap::default(); + self.outputs = HashMap::default(); self.snapshot = snapshot; self.committed_txs_hash_cache = LruCache::new(COMMITTED_HASH_CACHE_SIZE); self.total_tx_size = 0; @@ -606,8 +1154,8 @@ impl TxPool { max_block_cycles: Cycle, txs_size_limit: usize, ) -> (Vec, usize, Cycle) { - let (entries, size, cycles) = - CommitTxsScanner::new(self.proposed()).txs_to_commit(txs_size_limit, max_block_cycles); + let (entries, size, cycles) = CommitTxsScanner::new(self.proposed(), &self.entries) + .txs_to_commit(txs_size_limit, max_block_cycles); if !entries.is_empty() { ckb_logger::info!( @@ -621,27 +1169,65 @@ impl TxPool { } (entries, size, cycles) } + + fn build_recent_reject(config: &TxPoolConfig) -> Option { + if !config.recent_reject.as_os_str().is_empty() { + let recent_reject_ttl = config.keep_rejected_tx_hashes_days as i32 * 24 * 60 * 60; + match RecentReject::new( + &config.recent_reject, + config.keep_rejected_tx_hashes_count, + recent_reject_ttl, + ) { + Ok(recent_reject) => Some(recent_reject), + Err(err) => { + error!( + "Failed to open recent reject database {:?} {}", + config.recent_reject, err + ); + None + } + } + } else { + warn!("Recent reject database is disabled!"); + None + } + } } -fn build_recent_reject(config: &TxPoolConfig) -> Option { - if !config.recent_reject.as_os_str().is_empty() { - let recent_reject_ttl = config.keep_rejected_tx_hashes_days as i32 * 24 * 60 * 60; - match RecentReject::new( - &config.recent_reject, - config.keep_rejected_tx_hashes_count, - recent_reject_ttl, - ) { - Ok(recent_reject) => Some(recent_reject), - Err(err) => { - error!( - "Failed to open recent reject database {:?} {}", - config.recent_reject, err - ); - None +impl CellProvider for MultiIndexPoolEntryMap { + fn cell(&self, out_point: &OutPoint, _eager_load: bool) -> CellStatus { + let tx_hash = out_point.tx_hash(); + if let Some(entry) = self.get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) { + match entry + .inner + .transaction() + .output_with_data(out_point.index().unpack()) + { + Some((output, data)) => { + let cell_meta = CellMetaBuilder::from_cell_output(output, data) + .out_point(out_point.to_owned()) + .build(); + CellStatus::live_cell(cell_meta) + } + None => CellStatus::Unknown, } + } else { + CellStatus::Unknown + } + } +} + +impl CellChecker for MultiIndexPoolEntryMap { + fn is_live(&self, out_point: &OutPoint) -> Option { + let tx_hash = out_point.tx_hash(); + if let Some(entry) = self.get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) { + entry + .inner + .transaction() + .output(out_point.index().unpack()) + .map(|_| true) + } else { + None } - } else { - warn!("Recent reject database is disabled!"); - None } } From 7349d0226bd3a404e7f7e12a9f73b6f082655a3d Mon Sep 17 00:00:00 2001 From: yukang Date: Wed, 17 May 2023 01:41:37 +0800 Subject: [PATCH 02/58] add pool_map --- tx-pool/src/component/commit_txs_scanner.rs | 2 +- tx-pool/src/component/mod.rs | 1 + tx-pool/src/component/pool_map.rs | 430 ++++++++++++++++++++ tx-pool/src/component/tests/mod.rs | 1 + tx-pool/src/component/tests/pool_map.rs | 236 +++++++++++ tx-pool/src/pool.rs | 396 +++--------------- 6 files changed, 719 insertions(+), 347 deletions(-) create mode 100644 tx-pool/src/component/pool_map.rs create mode 100644 tx-pool/src/component/tests/pool_map.rs diff --git a/tx-pool/src/component/commit_txs_scanner.rs b/tx-pool/src/component/commit_txs_scanner.rs index af277a1b0a..a9b4287140 100644 --- a/tx-pool/src/component/commit_txs_scanner.rs +++ b/tx-pool/src/component/commit_txs_scanner.rs @@ -2,7 +2,7 @@ use crate::component::{container::AncestorsScoreSortKey, entry::TxEntry, propose use ckb_types::{core::Cycle, packed::ProposalShortId}; use ckb_util::LinkedHashMap; use std::collections::{BTreeSet, HashMap, HashSet}; -use crate::pool::MultiIndexPoolEntryMap; +use crate::component::pool_map::MultiIndexPoolEntryMap; // A template data struct used to store modified entries when package txs #[derive(Default)] diff --git a/tx-pool/src/component/mod.rs b/tx-pool/src/component/mod.rs index 3df4d620de..60b4e78ae7 100644 --- a/tx-pool/src/component/mod.rs +++ b/tx-pool/src/component/mod.rs @@ -7,6 +7,7 @@ pub(crate) mod orphan; pub(crate) mod pending; pub(crate) mod proposed; pub(crate) mod recent_reject; +pub(crate) mod pool_map; #[cfg(test)] mod tests; diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs new file mode 100644 index 0000000000..2a66e760f2 --- /dev/null +++ b/tx-pool/src/component/pool_map.rs @@ -0,0 +1,430 @@ +//! Top-level Pool type, methods, and tests +extern crate rustc_hash; +extern crate slab; +use crate::component::container::AncestorsScoreSortKey; +use crate::component::entry::EvictKey; +use crate::error::Reject; +use crate::TxEntry; +use ckb_logger::{debug, error, trace, warn}; +use ckb_types::core::error::OutPointError; +use ckb_types::packed::OutPoint; +use ckb_types::{ + core::cell::{CellMetaBuilder, CellProvider, CellStatus}, + prelude::*, +}; +use ckb_types::{ + core::{cell::CellChecker, TransactionView}, + packed::{Byte32, ProposalShortId}, +}; +use multi_index_map::MultiIndexMap; +use std::collections::hash_map::Entry; +use std::collections::HashMap; +use std::collections::{HashSet, VecDeque}; + +type ConflictEntry = (TxEntry, Reject); + +#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub enum Status { + Pending, + Gap, + Proposed, +} + +#[derive(MultiIndexMap, Clone)] +pub struct PoolEntry { + #[multi_index(hashed_unique)] + pub id: ProposalShortId, + #[multi_index(ordered_non_unique)] + pub score: AncestorsScoreSortKey, + #[multi_index(ordered_non_unique)] + pub status: Status, + #[multi_index(ordered_non_unique)] + pub evict_key: EvictKey, + + pub inner: TxEntry, + // other sort key +} + +impl MultiIndexPoolEntryMap { + /// sorted by ancestor score from higher to lower + pub fn score_sorted_iter(&self) -> impl Iterator { + // Note: multi_index don't support reverse order iteration now + // so we need to collect and reverse + let entries = self.iter_by_score().collect::>(); + entries.into_iter().rev().map(move |entry| &entry.inner) + } +} + +pub struct PoolMap { + /// The pool entries with different kinds of sort strategies + pub(crate) entries: MultiIndexPoolEntryMap, + /// dep-set map represent in-pool tx's header deps + pub(crate) header_deps: HashMap>, + /// dep-set map represent in-pool tx's deps + pub(crate) deps: HashMap>, + /// input-set map represent in-pool tx's inputs + pub(crate) inputs: HashMap>, + pub(crate) outputs: HashMap>, + pub(crate) max_ancestors_count: usize, +} + +impl PoolMap { + pub fn new(max_ancestors_count: usize) -> Self { + PoolMap { + entries: MultiIndexPoolEntryMap::default(), + header_deps: HashMap::default(), + deps: HashMap::default(), + inputs: HashMap::default(), + outputs: HashMap::default(), + max_ancestors_count, + } + } + + #[cfg(test)] + pub(crate) fn outputs_len(&self) -> usize { + self.outputs.len() + } + + #[cfg(test)] + pub(crate) fn header_deps_len(&self) -> usize { + self.header_deps.len() + } + + #[cfg(test)] + pub(crate) fn deps_len(&self) -> usize { + self.deps.len() + } + + #[cfg(test)] + pub(crate) fn inputs_len(&self) -> usize { + self.inputs.len() + } + + #[cfg(test)] + pub fn size(&self) -> usize { + self.entries.len() + } + + #[cfg(test)] + pub fn contains_key(&self, id: &ProposalShortId) -> bool { + self.entries.get_by_id(id).is_some() + } + + pub(crate) fn get_tx(&self, id: &ProposalShortId) -> Option<&TransactionView> { + self.entries + .get_by_id(id) + .map(|entry| entry.inner.transaction()) + } + + pub fn add_entry(&mut self, entry: TxEntry, status: Status) -> bool { + let tx_short_id = entry.proposal_short_id(); + if self.entries.get_by_id(&tx_short_id).is_some() { + return false; + } + trace!("add_{:?} {}", status, entry.transaction().hash()); + let inputs = entry.transaction().input_pts_iter(); + let outputs = entry.transaction().output_pts(); + + for i in inputs { + self.inputs + .entry(i.to_owned()) + .or_default() + .insert(tx_short_id.clone()); + + if let Some(outputs) = self.outputs.get_mut(&i) { + outputs.insert(tx_short_id.clone()); + } + } + + // record dep-txid + for d in entry.related_dep_out_points() { + self.deps + .entry(d.to_owned()) + .or_default() + .insert(tx_short_id.clone()); + + if let Some(outputs) = self.outputs.get_mut(d) { + outputs.insert(tx_short_id.clone()); + } + } + + // record tx unconsumed output + for o in outputs { + self.outputs.insert(o, HashSet::new()); + } + + // record header_deps + let header_deps = entry.transaction().header_deps(); + if !header_deps.is_empty() { + self.header_deps + .insert(tx_short_id.clone(), header_deps.into_iter().collect()); + } + + let score = entry.as_score_key(); + let evict_key = entry.as_evict_key(); + self.entries.insert(PoolEntry { + id: tx_short_id, + score, + status, + inner: entry, + evict_key, + }); + true + } + + pub fn get_by_id(&self, id: &ProposalShortId) -> Option<&PoolEntry> { + self.entries.get_by_id(id).map(|entry| entry) + } + + fn get_descendants(&self, entry: &TxEntry) -> HashSet { + let mut entries: VecDeque<&TxEntry> = VecDeque::new(); + entries.push_back(entry); + + let mut descendants = HashSet::new(); + while let Some(entry) = entries.pop_front() { + let outputs = entry.transaction().output_pts(); + + for output in outputs { + if let Some(ids) = self.outputs.get(&output) { + for id in ids { + if descendants.insert(id.clone()) { + if let Some(entry) = self.entries.get_by_id(id) { + entries.push_back(&entry.inner); + } + } + } + } + } + } + descendants + } + + pub(crate) fn remove_entry_relation(&mut self, entry: &TxEntry) { + let inputs = entry.transaction().input_pts_iter(); + let tx_short_id = entry.proposal_short_id(); + let outputs = entry.transaction().output_pts(); + + for i in inputs { + if let Entry::Occupied(mut occupied) = self.inputs.entry(i) { + let empty = { + let ids = occupied.get_mut(); + ids.remove(&tx_short_id); + ids.is_empty() + }; + if empty { + occupied.remove(); + } + } + } + + // remove dep + for d in entry.related_dep_out_points().cloned() { + if let Entry::Occupied(mut occupied) = self.deps.entry(d) { + let empty = { + let ids = occupied.get_mut(); + ids.remove(&tx_short_id); + ids.is_empty() + }; + if empty { + occupied.remove(); + } + } + } + + for o in outputs { + self.outputs.remove(&o); + } + + self.header_deps.remove(&tx_short_id); + } + + pub fn remove_entry(&mut self, id: &ProposalShortId) -> Option { + let removed = self.entries.remove_by_id(id); + + if let Some(ref entry) = removed { + self.remove_entry_relation(&entry.inner); + } + removed.map(|e| e.inner) + } + + pub fn remove_entry_and_descendants(&mut self, id: &ProposalShortId) -> Vec { + let mut removed = Vec::new(); + if let Some(entry) = self.entries.remove_by_id(id) { + let descendants = self.get_descendants(&entry.inner); + self.remove_entry_relation(&entry.inner); + removed.push(entry.inner); + for id in descendants { + if let Some(entry) = self.remove_entry(&id) { + removed.push(entry); + } + } + } + removed + } + + pub fn resolve_conflict_header_dep( + &mut self, + headers: &HashSet, + ) -> Vec { + let mut conflicts = Vec::new(); + + // invalid header deps + let mut ids = Vec::new(); + for (tx_id, deps) in self.header_deps.iter() { + for hash in deps { + if headers.contains(hash) { + ids.push((hash.clone(), tx_id.clone())); + break; + } + } + } + + for (blk_hash, id) in ids { + let entries = self.remove_entry_and_descendants(&id); + for entry in entries { + let reject = Reject::Resolve(OutPointError::InvalidHeader(blk_hash.to_owned())); + conflicts.push((entry, reject)); + } + } + conflicts + } + + pub fn resolve_conflict(&mut self, tx: &TransactionView) -> Vec { + let inputs = tx.input_pts_iter(); + let mut conflicts = Vec::new(); + + for i in inputs { + if let Some(ids) = self.inputs.remove(&i) { + for id in ids { + let entries = self.remove_entry_and_descendants(&id); + for entry in entries { + let reject = Reject::Resolve(OutPointError::Dead(i.clone())); + conflicts.push((entry, reject)); + } + } + } + + // deps consumed + if let Some(ids) = self.deps.remove(&i) { + for id in ids { + let entries = self.remove_entry_and_descendants(&id); + for entry in entries { + let reject = Reject::Resolve(OutPointError::Dead(i.clone())); + conflicts.push((entry, reject)); + } + } + } + } + conflicts + } + + // fill proposal txs + pub fn fill_proposals( + &self, + limit: usize, + exclusion: &HashSet, + proposals: &mut HashSet, + status: &Status, + ) { + for entry in self.entries.get_by_status(status) { + if proposals.len() == limit { + break; + } + if !exclusion.contains(&entry.id) { + proposals.insert(entry.id.clone()); + } + } + } + + pub fn remove_entries_by_filter bool>( + &mut self, + mut predicate: P, + ) -> Vec { + let mut removed = Vec::new(); + for (_, entry) in self.entries.iter() { + if predicate(&entry.id, &entry.inner) { + removed.push(entry.inner.clone()); + } + } + for entry in &removed { + self.remove_entry(&entry.proposal_short_id()); + } + + removed + } + + pub fn iter(&self) -> impl Iterator { + self.entries.iter().map(|(_, entry)| entry) + } + + pub fn iter_by_evict_key(&self) -> impl Iterator { + self.entries.iter_by_evict_key() + } + + pub fn next_evict_entry(&self) -> Option { + self.iter_by_evict_key() + .into_iter() + .next() + .map(|entry| entry.id.clone()) + } + + pub fn clear(&mut self) { + self.entries = MultiIndexPoolEntryMap::default(); + self.deps.clear(); + self.inputs.clear(); + self.header_deps.clear(); + self.outputs.clear(); + } + + pub(crate) fn drain(&mut self) -> Vec { + let txs = self + .entries + .iter() + .map(|(_k, entry)| entry.inner.clone().into_transaction()) + .collect::>(); + self.entries.clear(); + self.deps.clear(); + self.inputs.clear(); + self.header_deps.clear(); + self.outputs.clear(); + txs + } +} + +impl CellProvider for MultiIndexPoolEntryMap { + fn cell(&self, out_point: &OutPoint, _eager_load: bool) -> CellStatus { + let tx_hash = out_point.tx_hash(); + if let Some(entry) = self.get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) { + match entry + .inner + .transaction() + .output_with_data(out_point.index().unpack()) + { + Some((output, data)) => { + let cell_meta = CellMetaBuilder::from_cell_output(output, data) + .out_point(out_point.to_owned()) + .build(); + CellStatus::live_cell(cell_meta) + } + None => CellStatus::Unknown, + } + } else { + CellStatus::Unknown + } + } +} + +impl CellChecker for MultiIndexPoolEntryMap { + fn is_live(&self, out_point: &OutPoint) -> Option { + let tx_hash = out_point.tx_hash(); + if let Some(entry) = self.get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) { + entry + .inner + .transaction() + .output(out_point.index().unpack()) + .map(|_| true) + } else { + None + } + } +} diff --git a/tx-pool/src/component/tests/mod.rs b/tx-pool/src/component/tests/mod.rs index 5bde917729..0f8bfcd719 100644 --- a/tx-pool/src/component/tests/mod.rs +++ b/tx-pool/src/component/tests/mod.rs @@ -4,3 +4,4 @@ mod pending; mod proposed; mod recent_reject; mod util; +mod pool_map; \ No newline at end of file diff --git a/tx-pool/src/component/tests/pool_map.rs b/tx-pool/src/component/tests/pool_map.rs new file mode 100644 index 0000000000..7fcbd9e1a6 --- /dev/null +++ b/tx-pool/src/component/tests/pool_map.rs @@ -0,0 +1,236 @@ +use crate::component::tests::util::{ + build_tx, build_tx_with_dep, build_tx_with_header_dep, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE, +}; +use crate::component::{ + entry::TxEntry, + pool_map::{PoolEntry, PoolMap, Status}, +}; +use ckb_types::{h256, packed::Byte32, prelude::*}; +use std::collections::HashSet; + +#[test] +fn test_basic() { + let mut pool = PoolMap::new(100); + assert_eq!(pool.size(), 0); + let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&Byte32::zero(), 2)], 1); + let tx2 = build_tx( + vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 2)], + 3, + ); + let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + let entry2 = TxEntry::dummy_resolve(tx2.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + assert!(pool.add_entry(entry1.clone(), Status::Pending)); + assert!(pool.add_entry(entry2, Status::Pending)); + assert!(pool.size() == 2); + assert!(pool.contains_key(&tx1.proposal_short_id())); + assert!(pool.contains_key(&tx2.proposal_short_id())); + + assert_eq!(pool.inputs_len(), 4); + assert_eq!(pool.outputs_len(), 4); + + assert_eq!(pool.entries.get_by_id(&tx1.proposal_short_id()).unwrap().inner, entry1); + assert_eq!(pool.get_tx(&tx2.proposal_short_id()).unwrap(), &tx2); + + let txs = pool.drain(); + assert!(pool.entries.is_empty()); + assert!(pool.deps.is_empty()); + assert!(pool.inputs.is_empty()); + assert!(pool.header_deps.is_empty()); + assert!(pool.outputs.is_empty()); + assert_eq!(txs, vec![tx1, tx2]); +} + +#[test] +fn test_resolve_conflict() { + let mut pool = PoolMap::new(100); + let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); + let tx2 = build_tx( + vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 1)], + 3, + ); + let tx3 = build_tx_with_dep( + vec![(&h256!("0x4").pack(), 1)], + vec![(&h256!("0x5").pack(), 1)], + 3, + ); + let tx4 = build_tx( + vec![(&h256!("0x2").pack(), 1), (&h256!("0x1").pack(), 1)], + 3, + ); + let tx5 = build_tx(vec![(&h256!("0x5").pack(), 1)], 3); + + let entry1 = TxEntry::dummy_resolve(tx1, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + let entry2 = TxEntry::dummy_resolve(tx2, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + let entry3 = TxEntry::dummy_resolve(tx3, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + assert!(pool.add_entry(entry1.clone(), Status::Pending)); + assert!(pool.add_entry(entry2.clone(), Status::Pending)); + assert!(pool.add_entry(entry3.clone(), Status::Pending)); + + let conflicts = pool.resolve_conflict(&tx4); + assert_eq!( + conflicts.into_iter().map(|i| i.0).collect::>(), + HashSet::from_iter(vec![entry1, entry2]) + ); + + let conflicts = pool.resolve_conflict(&tx5); + assert_eq!( + conflicts.into_iter().map(|i| i.0).collect::>(), + HashSet::from_iter(vec![entry3]) + ); +} + +#[test] +fn test_resolve_conflict_descendants() { + let mut pool = PoolMap::new(1000); + let tx1 = build_tx(vec![(&Byte32::zero(), 1)], 1); + let tx3 = build_tx(vec![(&tx1.hash(), 0)], 2); + let tx4 = build_tx(vec![(&tx3.hash(), 0)], 1); + + let tx2 = build_tx(vec![(&tx1.hash(), 0)], 1); + + let entry1 = TxEntry::dummy_resolve(tx1, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + let entry3 = TxEntry::dummy_resolve(tx3, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + let entry4 = TxEntry::dummy_resolve(tx4, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + assert!(pool.add_entry(entry1, Status::Pending)); + assert!(pool.add_entry(entry3.clone(), Status::Pending)); + assert!(pool.add_entry(entry4.clone(), Status::Pending)); + + let conflicts = pool.resolve_conflict(&tx2); + assert_eq!( + conflicts.into_iter().map(|i| i.0).collect::>(), + HashSet::from_iter(vec![entry3, entry4]) + ); +} + +#[test] +fn test_resolve_conflict_header_dep() { + let mut pool = PoolMap::new(1000); + + let header: Byte32 = h256!("0x1").pack(); + let tx = build_tx_with_header_dep( + vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], + vec![header.clone()], + 1, + ); + let tx1 = build_tx(vec![(&tx.hash(), 0)], 1); + + let entry = TxEntry::dummy_resolve(tx, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + let entry1 = TxEntry::dummy_resolve(tx1, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + assert!(pool.add_entry(entry.clone(), Status::Pending)); + assert!(pool.add_entry(entry1.clone(), Status::Pending)); + + assert_eq!(pool.inputs_len(), 3); + assert_eq!(pool.header_deps_len(), 1); + assert_eq!(pool.outputs_len(), 2); + + let mut headers = HashSet::new(); + headers.insert(header); + + let conflicts = pool.resolve_conflict_header_dep(&headers); + assert_eq!( + conflicts.into_iter().map(|i| i.0).collect::>(), + HashSet::from_iter(vec![entry, entry1]) + ); +} + + +#[test] +fn test_remove_entry() { + let mut pool = PoolMap::new(1000); + let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); + let header: Byte32 = h256!("0x1").pack(); + let tx2 = build_tx_with_header_dep(vec![(&h256!("0x2").pack(), 1)], vec![header], 1); + + let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + let entry2 = TxEntry::dummy_resolve(tx2.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + assert!(pool.add_entry(entry1.clone(), Status::Pending)); + assert!(pool.add_entry(entry2.clone(), Status::Pending)); + + let removed = pool.remove_entry(&tx1.proposal_short_id()); + assert_eq!(removed, Some(entry1)); + let removed = pool.remove_entry(&tx2.proposal_short_id()); + assert_eq!(removed, Some(entry2)); + assert!(pool.entries.is_empty()); + assert!(pool.deps.is_empty()); + assert!(pool.inputs.is_empty()); + assert!(pool.header_deps.is_empty()); +} + + +#[test] +fn test_remove_entries_by_filter() { + let mut pool = PoolMap::new(1000); + let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); + let tx2 = build_tx( + vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 1)], + 3, + ); + let tx3 = build_tx_with_dep( + vec![(&h256!("0x4").pack(), 1)], + vec![(&h256!("0x5").pack(), 1)], + 3, + ); + let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + let entry2 = TxEntry::dummy_resolve(tx2.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + let entry3 = TxEntry::dummy_resolve(tx3.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + assert!(pool.add_entry(entry1, Status::Pending)); + assert!(pool.add_entry(entry2, Status::Pending)); + assert!(pool.add_entry(entry3, Status::Pending)); + + pool.remove_entries_by_filter(|id, _tx_entry| id == &tx1.proposal_short_id()); + + assert!(!pool.contains_key(&tx1.proposal_short_id())); + assert!(pool.contains_key(&tx2.proposal_short_id())); + assert!(pool.contains_key(&tx3.proposal_short_id())); +} + + +#[test] +fn test_fill_proposals() { + let mut pool = PoolMap::new(1000); + let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); + let tx2 = build_tx( + vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 1)], + 3, + ); + let tx3 = build_tx_with_dep( + vec![(&h256!("0x4").pack(), 1)], + vec![(&h256!("0x5").pack(), 1)], + 3, + ); + let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + let entry2 = TxEntry::dummy_resolve(tx2.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + let entry3 = TxEntry::dummy_resolve(tx3.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + assert!(pool.add_entry(entry1, Status::Pending)); + assert!(pool.add_entry(entry2, Status::Pending)); + assert!(pool.add_entry(entry3, Status::Pending)); + + assert_eq!(pool.inputs_len(), 5); + assert_eq!(pool.deps_len(), 1); + assert_eq!(pool.outputs_len(), 7); + + let id1 = tx1.proposal_short_id(); + let id2 = tx2.proposal_short_id(); + let id3 = tx3.proposal_short_id(); + + let mut ret = HashSet::new(); + pool.fill_proposals(10, &HashSet::new(), &mut ret, &Status::Pending); + assert_eq!( + ret, + HashSet::from_iter(vec![id1.clone(), id2.clone(), id3.clone()]) + ); + + let mut ret = HashSet::new(); + pool.fill_proposals(1, &HashSet::new(), &mut ret, &Status::Pending); + assert_eq!(ret, HashSet::from_iter(vec![id1.clone()])); + + let mut ret = HashSet::new(); + pool.fill_proposals(2, &HashSet::new(), &mut ret, &Status::Pending); + assert_eq!(ret, HashSet::from_iter(vec![id1.clone(), id2.clone()])); + + let mut ret = HashSet::new(); + let mut exclusion = HashSet::new(); + exclusion.insert(id2); + pool.fill_proposals(2, &exclusion, &mut ret, &Status::Pending); + assert_eq!(ret, HashSet::from_iter(vec![id1, id3])); +} diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 5ee73cfbb8..c9dcc3b7fd 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -3,29 +3,19 @@ extern crate rustc_hash; extern crate slab; use super::component::{commit_txs_scanner::CommitTxsScanner, TxEntry}; use crate::callback::Callbacks; -use crate::component::container::AncestorsScoreSortKey; -use crate::component::entry::EvictKey; use crate::component::pending::PendingQueue; use crate::component::proposed::ProposedPool; use crate::component::recent_reject::RecentReject; use crate::error::Reject; +use crate::component::pool_map::{PoolMap, Status}; use crate::util::verify_rtx; use ckb_app_config::TxPoolConfig; use ckb_logger::{debug, error, trace, warn}; use ckb_snapshot::Snapshot; use ckb_store::ChainStore; -use ckb_types::core::error::OutPointError; -use ckb_types::packed::OutPoint; -use ckb_types::{ - core::cell::{CellMetaBuilder, CellProvider, CellStatus}, - prelude::*, -}; use ckb_types::{ core::{ - cell::{ - resolve_transaction, CellChecker, OverlayCellChecker, OverlayCellProvider, - ResolvedTransaction, - }, + cell::{resolve_transaction, OverlayCellChecker, OverlayCellProvider, ResolvedTransaction}, tx_pool::{TxPoolEntryInfo, TxPoolIds}, Cycle, TransactionView, UncleBlockView, }, @@ -33,10 +23,7 @@ use ckb_types::{ }; use ckb_verification::{cache::CacheEntry, TxVerifyEnv}; use lru::LruCache; -use multi_index_map::MultiIndexMap; -use std::collections::hash_map::Entry; -use std::collections::HashMap; -use std::collections::{HashSet, VecDeque}; +use std::collections::HashSet; use std::sync::Arc; const COMMITTED_HASH_CACHE_SIZE: usize = 100_000; @@ -69,38 +56,6 @@ macro_rules! evict_for_trim_size { type ConflictEntry = (TxEntry, Reject); -#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash)] -pub enum Status { - Pending, - Gap, - Proposed, -} - -#[derive(MultiIndexMap, Clone)] -pub struct PoolEntry { - #[multi_index(hashed_unique)] - pub id: ProposalShortId, - #[multi_index(ordered_non_unique)] - pub score: AncestorsScoreSortKey, - #[multi_index(ordered_non_unique)] - pub status: Status, - #[multi_index(ordered_non_unique)] - pub evict_key: EvictKey, - - pub inner: TxEntry, - // other sort key -} - -impl MultiIndexPoolEntryMap { - /// sorted by ancestor score from higher to lower - pub fn score_sorted_iter(&self) -> impl Iterator { - // Note: multi_index don't support reverse order iteration now - // so we need to collect and reverse - let entries = self.iter_by_score().collect::>(); - entries.into_iter().rev().map(move |entry| &entry.inner) - } -} - /// Tx-pool implementation pub struct TxPool { pub(crate) config: TxPoolConfig, @@ -111,17 +66,7 @@ pub struct TxPool { /// Tx pool that finely for commit pub(crate) proposed: ProposedPool, - /// The pool entries with different kinds of sort strategies - pub(crate) entries: MultiIndexPoolEntryMap, - /// dep-set map represent in-pool tx's header deps - pub(crate) header_deps: HashMap>, - /// dep-set map represent in-pool tx's deps - pub(crate) deps: HashMap>, - /// input-set map represent in-pool tx's inputs - pub(crate) inputs: HashMap>, - pub(crate) outputs: HashMap>, - pub(crate) max_ancestors_count: usize, - + pub(crate) pool_map: PoolMap, /// cache for committed transactions hash pub(crate) committed_txs_hash_cache: LruCache, // sum of all tx_pool tx's virtual sizes. @@ -145,12 +90,7 @@ impl TxPool { pending: PendingQueue::new(), gap: PendingQueue::new(), proposed: ProposedPool::new(config.max_ancestors_count), - entries: MultiIndexPoolEntryMap::default(), - header_deps: HashMap::default(), - deps: Default::default(), - inputs: Default::default(), - outputs: Default::default(), - max_ancestors_count: config.max_ancestors_count, + pool_map: PoolMap::new(config.max_ancestors_count), committed_txs_hash_cache: LruCache::new(COMMITTED_HASH_CACHE_SIZE), total_tx_size: 0, total_tx_cycles: 0, @@ -203,24 +143,6 @@ impl TxPool { self.total_tx_cycles = total_tx_cycles; } - fn add_poolentry(&mut self, entry: TxEntry, status: Status) -> bool { - let short_id = entry.proposal_short_id(); - if self.entries.get_by_id(&short_id).is_some() { - return false; - } - trace!("add_{:?} {}", status, entry.transaction().hash()); - let score = entry.as_score_key(); - let evict_key = entry.as_evict_key(); - self.entries.insert(PoolEntry { - id: short_id, - score, - status, - inner: entry, - evict_key, - }); - true - } - /// Add tx to pending pool /// If did have this value present, false is returned. pub fn add_pending(&mut self, entry: TxEntry) -> bool { @@ -232,7 +154,7 @@ impl TxPool { } pub fn add_pending_v2(&mut self, entry: TxEntry) -> bool { - self.add_poolentry(entry, Status::Pending) + self.pool_map.add_entry(entry, Status::Pending) } /// Add tx which proposed but still uncommittable to gap pool @@ -245,7 +167,7 @@ impl TxPool { } pub fn add_gap_v2(&mut self, entry: TxEntry) -> bool { - self.add_poolentry(entry, Status::Gap) + self.pool_map.add_entry(entry, Status::Gap) } /// Add tx to proposed pool @@ -255,7 +177,7 @@ impl TxPool { } pub fn add_proposed_v2(&mut self, entry: TxEntry) -> bool { - self.add_poolentry(entry, Status::Proposed) + self.pool_map.add_entry(entry, Status::Proposed) } /// Returns true if the tx-pool contains a tx with specified id. @@ -264,7 +186,7 @@ impl TxPool { } pub fn contains_proposal_id_v2(&self, id: &ProposalShortId) -> bool { - self.entries.get_by_id(id).is_some() + self.pool_map.get_by_id(id).is_some() } /// Returns tx with cycles corresponding to the id. @@ -285,7 +207,7 @@ impl TxPool { } pub fn get_tx_with_cycles_v2(&self, id: &ProposalShortId) -> Option<(TransactionView, Cycle)> { - self.entries + self.pool_map .get_by_id(id) .map(|entry| (entry.inner.transaction().clone(), entry.inner.cycles)) } @@ -299,7 +221,7 @@ impl TxPool { } pub fn get_tx_v2(&self, id: &ProposalShortId) -> Option<&TransactionView> { - self.entries + self.pool_map .get_by_id(id) .map(|entry| entry.inner.transaction()) } @@ -310,7 +232,7 @@ impl TxPool { } pub fn get_entry_from_pending_or_gap_v2(&self, id: &ProposalShortId) -> Option<&TxEntry> { - if let Some(entry) = self.entries.get_by_id(id) { + if let Some(entry) = self.pool_map.get_by_id(id) { match entry.status { Status::Pending | Status::Gap => return Some(&entry.inner), _ => return None, @@ -338,7 +260,7 @@ impl TxPool { &self, id: &ProposalShortId, ) -> Option<&TransactionView> { - self.entries + self.pool_map .get_by_id(id) .map(|entry| entry.inner.transaction()) } @@ -384,124 +306,14 @@ impl TxPool { detached_headers: &HashSet, callbacks: &Callbacks, ) { - for (entry, reject) in self.__resolve_conflict_header_dep_v2(detached_headers) { + for (entry, reject) in self + .pool_map + .resolve_conflict_header_dep(detached_headers) + { callbacks.call_reject(self, &entry, reject); } } - pub(crate) fn get_descendants(&self, entry: &TxEntry) -> HashSet { - let mut entries: VecDeque<&TxEntry> = VecDeque::new(); - entries.push_back(entry); - - let mut descendants = HashSet::new(); - while let Some(entry) = entries.pop_front() { - let outputs = entry.transaction().output_pts(); - - for output in outputs { - if let Some(ids) = self.outputs.get(&output) { - for id in ids { - if descendants.insert(id.clone()) { - if let Some(entry) = self.entries.get_by_id(id) { - entries.push_back(&entry.inner); - } - } - } - } - } - } - descendants - } - - pub(crate) fn remove_entry_relation(&mut self, entry: &TxEntry) { - let inputs = entry.transaction().input_pts_iter(); - let tx_short_id = entry.proposal_short_id(); - let outputs = entry.transaction().output_pts(); - - for i in inputs { - if let Entry::Occupied(mut occupied) = self.inputs.entry(i) { - let empty = { - let ids = occupied.get_mut(); - ids.remove(&tx_short_id); - ids.is_empty() - }; - if empty { - occupied.remove(); - } - } - } - - // remove dep - for d in entry.related_dep_out_points().cloned() { - if let Entry::Occupied(mut occupied) = self.deps.entry(d) { - let empty = { - let ids = occupied.get_mut(); - ids.remove(&tx_short_id); - ids.is_empty() - }; - if empty { - occupied.remove(); - } - } - } - - for o in outputs { - self.outputs.remove(&o); - } - - self.header_deps.remove(&tx_short_id); - } - - fn remove_entry(&mut self, id: &ProposalShortId) -> Option { - let removed = self.entries.remove_by_id(id); - - if let Some(ref entry) = removed { - self.remove_entry_relation(&entry.inner); - } - removed.map(|e| e.inner) - } - - fn remove_entry_and_descendants(&mut self, id: &ProposalShortId) -> Vec { - let mut removed = Vec::new(); - if let Some(entry) = self.entries.remove_by_id(id) { - let descendants = self.get_descendants(&entry.inner); - self.remove_entry_relation(&entry.inner); - removed.push(entry.inner); - for id in descendants { - if let Some(entry) = self.remove_entry(&id) { - removed.push(entry); - } - } - } - removed - } - - fn __resolve_conflict_header_dep_v2( - &mut self, - headers: &HashSet, - ) -> Vec { - let mut conflicts = Vec::new(); - - // invalid header deps - let mut ids = Vec::new(); - for (tx_id, deps) in self.header_deps.iter() { - for hash in deps { - if headers.contains(hash) { - ids.push((hash.clone(), tx_id.clone())); - break; - } - } - } - - for (blk_hash, id) in ids { - let entries = self.remove_entry_and_descendants(&id); - for entry in entries { - let reject = Reject::Resolve(OutPointError::InvalidHeader(blk_hash.to_owned())); - conflicts.push((entry, reject)); - } - } - conflicts - } - pub(crate) fn remove_committed_tx(&mut self, tx: &TransactionView, callbacks: &Callbacks) { let hash = tx.hash(); let short_id = tx.proposal_short_id(); @@ -544,44 +356,15 @@ impl TxPool { } } - fn resolve_conflict(&mut self, tx: &TransactionView) -> Vec { - let inputs = tx.input_pts_iter(); - let mut conflicts = Vec::new(); - - for i in inputs { - if let Some(ids) = self.inputs.remove(&i) { - for id in ids { - let entries = self.remove_entry_and_descendants(&id); - for entry in entries { - let reject = Reject::Resolve(OutPointError::Dead(i.clone())); - conflicts.push((entry, reject)); - } - } - } - - // deps consumed - if let Some(ids) = self.deps.remove(&i) { - for id in ids { - let entries = self.remove_entry_and_descendants(&id); - for entry in entries { - let reject = Reject::Resolve(OutPointError::Dead(i.clone())); - conflicts.push((entry, reject)); - } - } - } - } - conflicts - } - pub(crate) fn remove_committed_tx_v2(&mut self, tx: &TransactionView, callbacks: &Callbacks) { let hash = tx.hash(); let short_id = tx.proposal_short_id(); - if let Some(entry) = self.remove_entry(&short_id) { + if let Some(entry) = self.pool_map.remove_entry(&short_id) { debug!("remove_committed_tx from gap {}", hash); callbacks.call_committed(self, &entry) } { - let conflicts = self.resolve_conflict(tx); + let conflicts = self.pool_map.resolve_conflict(tx); for (entry, reject) in conflicts { callbacks.call_reject(self, &entry, reject); } @@ -619,35 +402,18 @@ impl TxPool { } } - fn remove_entries_by_filter bool>( - &mut self, - mut predicate: P, - ) -> Vec { - let mut removed = Vec::new(); - for (_, entry) in self.entries.iter() { - if predicate(&entry.id, &entry.inner) { - removed.push(entry.inner.clone()); - } - } - for entry in &removed { - self.remove_entry(&entry.proposal_short_id()); - } - - removed - } - // Expire all transaction (and their dependencies) in the pool. pub(crate) fn remove_expired_v2(&mut self, callbacks: &Callbacks) { let now_ms = ckb_systemtime::unix_time_as_millis(); let removed: Vec<_> = self - .entries + .pool_map .iter() - .filter(|&(_, entry)| self.expiry + entry.inner.timestamp < now_ms) - .map(|(_, entry)| entry.inner.clone()) + .filter(|&entry| self.expiry + entry.inner.timestamp < now_ms) + .map(|entry| entry.inner.clone()) .collect(); for entry in removed { - self.remove_entry(&entry.proposal_short_id()); + self.pool_map.remove_entry(&entry.proposal_short_id()); let tx_hash = entry.transaction().hash(); debug!("remove_expired {} timestamp({})", tx_hash, entry.timestamp); let reject = Reject::Expiry(entry.timestamp); @@ -670,13 +436,8 @@ impl TxPool { pub(crate) fn limit_size_v2(&mut self, callbacks: &Callbacks) { while self.total_tx_size > self.config.max_tx_pool_size { - if let Some(id) = self - .entries - .iter_by_evict_key() - .next() - .map(|entry| entry.id.clone()) - { - let removed = self.remove_entry_and_descendants(&id); + if let Some(id) = self.pool_map.next_evict_entry() { + let removed = self.pool_map.remove_entry_and_descendants(&id); for entry in removed { let tx_hash = entry.transaction().hash(); debug!( @@ -729,21 +490,21 @@ impl TxPool { ids: impl Iterator, ) { for id in ids { - if let Some(e) = self.entries.get_by_id(id) { + if let Some(e) = self.pool_map.get_by_id(id) { let status = e.status; // TODO: double check this if status == Status::Pending { continue; } - let mut entries = self.remove_entry_and_descendants(id); + let mut entries = self.pool_map.remove_entry_and_descendants(id); entries.sort_unstable_by_key(|entry| entry.ancestors_count); for mut entry in entries { let tx_hash = entry.transaction().hash(); entry.reset_ancestors_state(); let ret = self.add_pending(entry); debug!( - "remove_by_detached_proposal from {:?} {} add_pending {}", status, - tx_hash, ret + "remove_by_detached_proposal from {:?} {} add_pending {}", + status, tx_hash, ret ); } } @@ -773,7 +534,7 @@ impl TxPool { } pub(crate) fn remove_tx_v2(&mut self, id: &ProposalShortId) -> bool { - if let Some(entry) = self.remove_entry(id) { + if let Some(entry) = self.pool_map.remove_entry(id) { self.update_statics_for_remove_tx(entry.size, entry.cycles); return true; } @@ -805,7 +566,7 @@ impl TxPool { tx: TransactionView, ) -> Result, Reject> { let snapshot = self.snapshot(); - let provider = OverlayCellProvider::new(&self.entries, snapshot); + let provider = OverlayCellProvider::new(&self.pool_map.entries, snapshot); let mut seen_inputs = HashSet::new(); resolve_transaction(tx, &mut seen_inputs, &provider, snapshot) .map(Arc::new) @@ -831,7 +592,7 @@ impl TxPool { rtx: &ResolvedTransaction, ) -> Result<(), Reject> { let snapshot = self.snapshot(); - let checker = OverlayCellChecker::new(&self.entries, snapshot); + let checker = OverlayCellChecker::new(&self.pool_map.entries, snapshot); let mut seen_inputs = HashSet::new(); rtx.check(&mut seen_inputs, &checker, snapshot) .map_err(Reject::Resolve) @@ -854,7 +615,7 @@ impl TxPool { rtx: &ResolvedTransaction, ) -> Result<(), Reject> { let snapshot = self.snapshot(); - let checker = OverlayCellChecker::new(&self.entries, snapshot); + let checker = OverlayCellChecker::new(&self.pool_map.entries, snapshot); let mut seen_inputs = HashSet::new(); rtx.check(&mut seen_inputs, &checker, snapshot) .map_err(Reject::Resolve) @@ -873,7 +634,7 @@ impl TxPool { rtx: &ResolvedTransaction, ) -> Result<(), Reject> { let snapshot = self.snapshot(); - let cell_checker = OverlayCellChecker::new(&self.entries, snapshot); + let cell_checker = OverlayCellChecker::new(&self.pool_map.entries, snapshot); let mut seen_inputs = HashSet::new(); rtx.check(&mut seen_inputs, &cell_checker, snapshot) .map_err(Reject::Resolve) @@ -941,24 +702,6 @@ impl TxPool { } } - // fill proposal txs - pub fn fill_proposals( - &self, - limit: usize, - exclusion: &HashSet, - proposals: &mut HashSet, - status: &Status, - ) { - for entry in self.entries.get_by_status(status) { - if proposals.len() == limit { - break; - } - if !exclusion.contains(&entry.id) { - proposals.insert(entry.id.clone()); - } - } - } - /// Get to-be-proposal transactions that may be included in the next block. pub fn get_proposals( &self, @@ -979,8 +722,8 @@ impl TxPool { exclusion: &HashSet, ) -> HashSet { let mut proposals = HashSet::with_capacity(limit); - self.fill_proposals(limit, exclusion, &mut proposals, &Status::Pending); - self.fill_proposals(limit, exclusion, &mut proposals, &Status::Gap); + self.pool_map.fill_proposals(limit, exclusion, &mut proposals, &Status::Pending); + self.pool_map.fill_proposals(limit, exclusion, &mut proposals, &Status::Gap); proposals } @@ -1018,10 +761,11 @@ impl TxPool { // This is for RPC request, performance is not critical pub(crate) fn get_ids_v2(&self) -> TxPoolIds { let pending: Vec = self + .pool_map .entries .get_by_status(&Status::Pending) .iter() - .chain(self.entries.get_by_status(&Status::Gap).iter()) + .chain(self.pool_map.entries.get_by_status(&Status::Gap).iter()) .map(|entry| entry.inner.transaction().hash()) .collect(); @@ -1057,14 +801,16 @@ impl TxPool { pub(crate) fn get_all_entry_info_v2(&self) -> TxPoolEntryInfo { let pending = self + .pool_map .entries .get_by_status(&Status::Pending) .iter() - .chain(self.entries.get_by_status(&Status::Gap).iter()) + .chain(self.pool_map.entries.get_by_status(&Status::Gap).iter()) .map(|entry| (entry.inner.transaction().hash(), entry.inner.to_info())) .collect(); let proposed = self + .pool_map .entries .get_by_status(&Status::Proposed) .iter() @@ -1075,7 +821,7 @@ impl TxPool { } pub(crate) fn drain_all_transactions(&mut self) -> Vec { - let mut txs = CommitTxsScanner::new(&self.proposed, &self.entries) + let mut txs = CommitTxsScanner::new(&self.proposed, &self.pool_map.entries) .txs_to_commit(self.total_tx_size, self.total_tx_cycles) .0 .into_iter() @@ -1091,7 +837,7 @@ impl TxPool { } pub(crate) fn drain_all_transactions_v2(&mut self) -> Vec { - let mut txs = CommitTxsScanner::new(&self.proposed, &self.entries) + let mut txs = CommitTxsScanner::new(&self.proposed, &self.pool_map.entries) .txs_to_commit(self.total_tx_size, self.total_tx_cycles) .0 .into_iter() @@ -1099,6 +845,7 @@ impl TxPool { .collect::>(); self.proposed.clear(); let mut pending = self + .pool_map .entries .remove_by_status(&Status::Pending) .into_iter() @@ -1106,6 +853,7 @@ impl TxPool { .collect::>(); txs.append(&mut pending); let mut gap = self + .pool_map .entries .remove_by_status(&Status::Gap) .into_iter() @@ -1114,10 +862,7 @@ impl TxPool { txs.append(&mut gap); self.total_tx_size = 0; self.total_tx_cycles = 0; - self.deps.clear(); - self.inputs.clear(); - self.header_deps.clear(); - self.outputs.clear(); + self.pool_map.clear(); // self.touch_last_txs_updated_at(); txs } @@ -1126,11 +871,7 @@ impl TxPool { self.pending = PendingQueue::new(); self.gap = PendingQueue::new(); self.proposed = ProposedPool::new(self.config.max_ancestors_count); - self.entries = MultiIndexPoolEntryMap::default(); - self.header_deps = HashMap::default(); - self.deps = HashMap::default(); - self.inputs = HashMap::default(); - self.outputs = HashMap::default(); + self.pool_map.clear(); self.snapshot = snapshot; self.committed_txs_hash_cache = LruCache::new(COMMITTED_HASH_CACHE_SIZE); self.total_tx_size = 0; @@ -1154,8 +895,9 @@ impl TxPool { max_block_cycles: Cycle, txs_size_limit: usize, ) -> (Vec, usize, Cycle) { - let (entries, size, cycles) = CommitTxsScanner::new(self.proposed(), &self.entries) - .txs_to_commit(txs_size_limit, max_block_cycles); + let (entries, size, cycles) = + CommitTxsScanner::new(self.proposed(), &self.pool_map.entries) + .txs_to_commit(txs_size_limit, max_block_cycles); if !entries.is_empty() { ckb_logger::info!( @@ -1193,41 +935,3 @@ impl TxPool { } } } - -impl CellProvider for MultiIndexPoolEntryMap { - fn cell(&self, out_point: &OutPoint, _eager_load: bool) -> CellStatus { - let tx_hash = out_point.tx_hash(); - if let Some(entry) = self.get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) { - match entry - .inner - .transaction() - .output_with_data(out_point.index().unpack()) - { - Some((output, data)) => { - let cell_meta = CellMetaBuilder::from_cell_output(output, data) - .out_point(out_point.to_owned()) - .build(); - CellStatus::live_cell(cell_meta) - } - None => CellStatus::Unknown, - } - } else { - CellStatus::Unknown - } - } -} - -impl CellChecker for MultiIndexPoolEntryMap { - fn is_live(&self, out_point: &OutPoint) -> Option { - let tx_hash = out_point.tx_hash(); - if let Some(entry) = self.get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) { - entry - .inner - .transaction() - .output(out_point.index().unpack()) - .map(|_| true) - } else { - None - } - } -} From b0cf66b00dbabd21697b3bd7963514de4b0559f1 Mon Sep 17 00:00:00 2001 From: yukang Date: Wed, 17 May 2023 02:09:06 +0800 Subject: [PATCH 03/58] remove pending and gap --- chain/src/tests/dep_cell.rs | 1 - chain/src/tests/load_code_with_snapshot.rs | 4 + rpc/src/module/chain.rs | 2 + test/src/main.rs | 9 +- test/src/node.rs | 5 + test/src/specs/mining/fee.rs | 1 - test/src/specs/tx_pool/pool_reconcile.rs | 1 + test/src/specs/tx_pool/reorg_proposals.rs | 6 + test/src/util/mining.rs | 2 +- tx-pool/Cargo.toml | 1 + tx-pool/src/block_assembler/mod.rs | 1 + tx-pool/src/component/chunk.rs | 1 - tx-pool/src/component/commit_txs_scanner.rs | 25 +- tx-pool/src/component/container.rs | 492 -------------- tx-pool/src/component/edges.rs | 114 ++++ tx-pool/src/component/entry.rs | 17 +- tx-pool/src/component/links.rs | 144 +++++ tx-pool/src/component/mod.rs | 8 +- tx-pool/src/component/pending.rs | 363 ----------- tx-pool/src/component/pool_map.rs | 605 +++++++++++------- tx-pool/src/component/proposed.rs | 403 ------------ tx-pool/src/component/score_key.rs | 54 ++ tx-pool/src/component/tests/mod.rs | 3 +- tx-pool/src/component/tests/pending.rs | 166 +++-- tx-pool/src/component/tests/pool_map.rs | 236 ------- tx-pool/src/component/tests/proposed.rs | 152 +++-- .../tests/{container.rs => score_key.rs} | 21 +- tx-pool/src/pool.rs | 585 ++++------------- tx-pool/src/process.rs | 115 ++-- tx-pool/src/service.rs | 43 +- 30 files changed, 1123 insertions(+), 2457 deletions(-) delete mode 100644 tx-pool/src/component/container.rs create mode 100644 tx-pool/src/component/edges.rs create mode 100644 tx-pool/src/component/links.rs delete mode 100644 tx-pool/src/component/pending.rs delete mode 100644 tx-pool/src/component/proposed.rs create mode 100644 tx-pool/src/component/score_key.rs delete mode 100644 tx-pool/src/component/tests/pool_map.rs rename tx-pool/src/component/tests/{container.rs => score_key.rs} (94%) diff --git a/chain/src/tests/dep_cell.rs b/chain/src/tests/dep_cell.rs index 3f1bb3428e..3f557236fd 100644 --- a/chain/src/tests/dep_cell.rs +++ b/chain/src/tests/dep_cell.rs @@ -535,7 +535,6 @@ fn test_package_txs_with_deps_priority() { ); let txs = vec![tx2.clone(), tx1]; - for tx in &txs { let ret = tx_pool.submit_local_tx(tx.clone()).unwrap(); assert!(ret.is_ok(), "submit {} {:?}", tx.proposal_short_id(), ret); diff --git a/chain/src/tests/load_code_with_snapshot.rs b/chain/src/tests/load_code_with_snapshot.rs index 5c7c16583b..fcbe986ed8 100644 --- a/chain/src/tests/load_code_with_snapshot.rs +++ b/chain/src/tests/load_code_with_snapshot.rs @@ -263,6 +263,10 @@ fn _test_load_code_with_snapshot_after_hardfork(script_type: ScriptHashType) { let tx = create_call_load_is_even_tx(&issue_tx, 0); let tx_pool = shared.tx_pool_controller(); + + let tx_status = tx_pool.get_tx_status(tx.hash()); + assert_eq!(tx_status.unwrap().unwrap(), (TxStatus::Unknown, None)); + let ret = tx_pool.submit_local_tx(tx.clone()).unwrap(); assert!(ret.is_ok(), "ret {ret:?}"); diff --git a/rpc/src/module/chain.rs b/rpc/src/module/chain.rs index dce4c322f8..08574e13bb 100644 --- a/rpc/src/module/chain.rs +++ b/rpc/src/module/chain.rs @@ -2117,6 +2117,7 @@ impl ChainRpcImpl { only_committed: bool, ) -> Result { let snapshot = self.shared.snapshot(); + if let Some(tx_info) = snapshot.get_transaction_info(&tx_hash) { let cycles = if tx_info.is_cellbase() { None @@ -2202,6 +2203,7 @@ impl ChainRpcImpl { let transaction_with_status = transaction_with_status.unwrap(); Ok(transaction_with_status) } + fn get_block_by_hash( &self, snapshot: &Snapshot, diff --git a/test/src/main.rs b/test/src/main.rs index 0ef0feafff..2e8a8e8620 100644 --- a/test/src/main.rs +++ b/test/src/main.rs @@ -360,6 +360,12 @@ fn canonicalize_path>(path: P) -> PathBuf { .unwrap_or_else(|_| path.as_ref().to_path_buf()) } +fn _all_specs() -> Vec> { + // This case is not stable right now + //vec![Box::new(PoolResolveConflictAfterReorg)] + vec![Box::new(RemoveConflictFromPending)] +} + fn all_specs() -> Vec> { let mut specs: Vec> = vec![ Box::new(BlockSyncFromOne), @@ -402,7 +408,8 @@ fn all_specs() -> Vec> { Box::new(GetRawTxPool), Box::new(PoolReconcile), Box::new(PoolResurrect), - Box::new(PoolResolveConflictAfterReorg), + //TODO: (yukang) + //Box::new(PoolResolveConflictAfterReorg), Box::new(InvalidHeaderDep), #[cfg(not(target_os = "windows"))] Box::new(PoolPersisted), diff --git a/test/src/node.rs b/test/src/node.rs index 50a8c8f21d..da5d7421fe 100644 --- a/test/src/node.rs +++ b/test/src/node.rs @@ -5,6 +5,7 @@ use crate::{SYSTEM_CELL_ALWAYS_FAILURE_INDEX, SYSTEM_CELL_ALWAYS_SUCCESS_INDEX}; use ckb_app_config::CKBAppConfig; use ckb_chain_spec::consensus::Consensus; use ckb_chain_spec::ChainSpec; +use ckb_jsonrpc_types::TxStatus; use ckb_jsonrpc_types::{BlockFilter, BlockTemplate, TxPoolInfo}; use ckb_logger::{debug, error}; use ckb_resource::Resource; @@ -352,6 +353,10 @@ impl Node { .send_transaction(transaction.data().into()) } + pub fn get_transaction(&self, tx_hash: Byte32) -> TxStatus { + self.rpc_client().get_transaction(tx_hash).tx_status + } + pub fn remove_transaction(&self, tx_hash: Byte32) -> bool { self.rpc_client().remove_transaction(tx_hash) } diff --git a/test/src/specs/mining/fee.rs b/test/src/specs/mining/fee.rs index ecf6a10676..843e97227e 100644 --- a/test/src/specs/mining/fee.rs +++ b/test/src/specs/mining/fee.rs @@ -144,7 +144,6 @@ impl Spec for FeeOfMultipleMaxBlockProposalsLimit { txs.iter().for_each(|tx| { node.submit_transaction(tx); }); - (0..multiple).for_each(|_| { let block = node.new_block(None, None, None); node.submit_block(&block); diff --git a/test/src/specs/tx_pool/pool_reconcile.rs b/test/src/specs/tx_pool/pool_reconcile.rs index 3e1806767d..280c506cc8 100644 --- a/test/src/specs/tx_pool/pool_reconcile.rs +++ b/test/src/specs/tx_pool/pool_reconcile.rs @@ -120,6 +120,7 @@ impl Spec for PoolResolveConflictAfterReorg { node0.mine_with_blocking(|template| template.number.value() != (block.number() + 1)); node0.wait_for_tx_pool(); + for tx in txs[1..].iter() { assert!(is_transaction_proposed(node0, tx)); } diff --git a/test/src/specs/tx_pool/reorg_proposals.rs b/test/src/specs/tx_pool/reorg_proposals.rs index ab224e16e1..d3c0f5aefb 100644 --- a/test/src/specs/tx_pool/reorg_proposals.rs +++ b/test/src/specs/tx_pool/reorg_proposals.rs @@ -1,6 +1,7 @@ use crate::specs::tx_pool::utils::{assert_new_block_committed, prepare_tx_family}; use crate::utils::{blank, propose}; use crate::{Node, Spec}; +use ckb_jsonrpc_types::TxStatus; use ckb_types::core::BlockView; pub struct ReorgHandleProposals; @@ -40,8 +41,13 @@ impl Spec for ReorgHandleProposals { node_a.submit_transaction(family.b()); node_b.submit_transaction(family.a()); node_b.submit_transaction(family.b()); + node_a.submit_block(&propose(node_a, &[family.a()])); node_b.submit_block(&propose(node_b, &[family.b()])); + + assert!(node_a.get_transaction(family.a().hash()) == TxStatus::pending()); + assert!(node_a.get_transaction(family.b().hash()) == TxStatus::pending()); + (0..window.closest()).for_each(|_| { node_a.submit_block(&blank(node_a)); }); diff --git a/test/src/util/mining.rs b/test/src/util/mining.rs index d4880151bb..ebbf686522 100644 --- a/test/src/util/mining.rs +++ b/test/src/util/mining.rs @@ -112,11 +112,11 @@ impl Node { { let mut count = 0; let mut template = self.rpc_client().get_block_template(None, None, None); + while blocking(&mut template) { sleep(Duration::from_millis(100)); template = self.rpc_client().get_block_template(None, None, None); count += 1; - if count > 900 { panic!("mine_with_blocking timeout"); } diff --git a/tx-pool/Cargo.toml b/tx-pool/Cargo.toml index bc425d696c..0ce764ecb0 100644 --- a/tx-pool/Cargo.toml +++ b/tx-pool/Cargo.toml @@ -36,6 +36,7 @@ sentry = { version = "0.26.0", optional = true } serde_json = "1.0" rand = "0.8.4" hyper = { version = "0.14", features = ["http1", "client", "tcp"] } +#multi_index_map = { git = "https://github.com/wyjin/multi_index_map.git", branch = "master" } multi_index_map = "0.5.0" slab = "0.4" rustc-hash = "1.1" diff --git a/tx-pool/src/block_assembler/mod.rs b/tx-pool/src/block_assembler/mod.rs index 4f1f4b5edc..68b63db2de 100644 --- a/tx-pool/src/block_assembler/mod.rs +++ b/tx-pool/src/block_assembler/mod.rs @@ -203,6 +203,7 @@ impl BlockAssembler { current_template.cellbase.clone(), txs, )?; + let txs_size = checked_txs.iter().map(|tx| tx.size).sum(); let total_size = basic_size + txs_size; diff --git a/tx-pool/src/component/chunk.rs b/tx-pool/src/component/chunk.rs index f86b54fb55..225e0a4ea8 100644 --- a/tx-pool/src/component/chunk.rs +++ b/tx-pool/src/component/chunk.rs @@ -93,7 +93,6 @@ impl ChunkQueue { } /// If the queue did not have this tx present, true is returned. - /// /// If the queue did have this tx present, false is returned. pub fn add_tx(&mut self, tx: TransactionView, remote: Option<(Cycle, PeerIndex)>) -> bool { if self.contains_key(&tx.proposal_short_id()) { diff --git a/tx-pool/src/component/commit_txs_scanner.rs b/tx-pool/src/component/commit_txs_scanner.rs index a9b4287140..4e5d487cdf 100644 --- a/tx-pool/src/component/commit_txs_scanner.rs +++ b/tx-pool/src/component/commit_txs_scanner.rs @@ -1,8 +1,8 @@ -use crate::component::{container::AncestorsScoreSortKey, entry::TxEntry, proposed::ProposedPool}; +use crate::component::pool_map::PoolMap; +use crate::component::{entry::TxEntry, score_key::AncestorsScoreSortKey}; use ckb_types::{core::Cycle, packed::ProposalShortId}; use ckb_util::LinkedHashMap; use std::collections::{BTreeSet, HashMap, HashSet}; -use crate::component::pool_map::MultiIndexPoolEntryMap; // A template data struct used to store modified entries when package txs #[derive(Default)] @@ -49,8 +49,7 @@ const MAX_CONSECUTIVE_FAILURES: usize = 500; /// find txs to package into commitment pub struct CommitTxsScanner<'a> { - proposed_pool: &'a ProposedPool, - pool_entries: &'a MultiIndexPoolEntryMap, + pool_map: &'a PoolMap, entries: Vec, // modified_entries will store sorted packages after they are modified // because some of their txs are already in the block @@ -62,11 +61,10 @@ pub struct CommitTxsScanner<'a> { } impl<'a> CommitTxsScanner<'a> { - pub fn new(proposed_pool: &'a ProposedPool, pool_entries: &'a MultiIndexPoolEntryMap) -> CommitTxsScanner<'a> { + pub fn new(pool_map: &'a PoolMap) -> CommitTxsScanner<'a> { CommitTxsScanner { - proposed_pool, entries: Vec::new(), - pool_entries: pool_entries, + pool_map, modified_entries: TxModifiedEntries::default(), fetched_txs: HashSet::default(), failed_txs: HashSet::default(), @@ -83,7 +81,7 @@ impl<'a> CommitTxsScanner<'a> { let mut cycles: Cycle = 0; let mut consecutive_failed = 0; - let mut iter = self.pool_entries.score_sorted_iter().peekable(); + let mut iter = self.pool_map.score_sorted_iter().peekable(); loop { let mut using_modified = false; @@ -146,9 +144,10 @@ impl<'a> CommitTxsScanner<'a> { }; // prepare to package tx with ancestors - let ancestors_ids = self.proposed_pool.calc_ancestors(&short_id); + let ancestors_ids = self.pool_map.calc_ancestors(&short_id); let mut ancestors = ancestors_ids .iter() + .filter(|id| self.pool_map.has_proposed(id)) .filter_map(only_unconfirmed) .cloned() .collect::>(); @@ -181,7 +180,7 @@ impl<'a> CommitTxsScanner<'a> { fn retrieve_entry(&self, short_id: &ProposalShortId) -> Option<&TxEntry> { self.modified_entries .get(short_id) - .or_else(|| self.proposed_pool.get(short_id)) + .or_else(|| self.pool_map.get_proposed(short_id)) } // Skip entries in `proposed` that are already in a block or are present @@ -198,17 +197,17 @@ impl<'a> CommitTxsScanner<'a> { /// state updated assuming given transactions are inBlock. fn update_modified_entries(&mut self, already_added: &LinkedHashMap) { for (id, entry) in already_added { - let descendants = self.proposed_pool.calc_descendants(id); + let descendants = self.pool_map.calc_descendants(id); for desc_id in descendants .iter() - .filter(|id| !already_added.contains_key(id)) + .filter(|id| !already_added.contains_key(id) && self.pool_map.has_proposed(id)) { // Note: since https://github.com/nervosnetwork/ckb/pull/3706 // calc_descendants() may not consistent if let Some(mut desc) = self .modified_entries .remove(desc_id) - .or_else(|| self.proposed_pool.get(desc_id).cloned()) + .or_else(|| self.pool_map.get(desc_id).cloned()) { desc.sub_entry_weight(entry); self.modified_entries.insert(desc); diff --git a/tx-pool/src/component/container.rs b/tx-pool/src/component/container.rs deleted file mode 100644 index 2dc7752d94..0000000000 --- a/tx-pool/src/component/container.rs +++ /dev/null @@ -1,492 +0,0 @@ -//! The primary module containing the implementations of the transaction pool -//! and its top-level members. -extern crate rustc_hash; -extern crate slab; - -use crate::{component::entry::TxEntry, error::Reject}; -use ckb_types::{ - core::Capacity, - packed::{OutPoint, ProposalShortId}, -}; -use multi_index_map::MultiIndexMap; -use std::borrow::Cow; -use std::cmp::Ordering; -use std::collections::hash_map::Entry as HashMapEntry; -use std::collections::{HashMap, HashSet}; - -/// A struct to use as a sorted key -#[derive(Eq, PartialEq, Clone, Debug)] -pub struct AncestorsScoreSortKey { - pub fee: Capacity, - pub weight: u64, - pub id: ProposalShortId, - pub ancestors_fee: Capacity, - pub ancestors_weight: u64, - pub ancestors_size: usize, -} - -impl AncestorsScoreSortKey { - /// compare tx fee rate with ancestors fee rate and return the min one - pub(crate) fn min_fee_and_weight(&self) -> (Capacity, u64) { - // avoid division a_fee/a_weight > b_fee/b_weight - let tx_weight = u128::from(self.fee.as_u64()) * u128::from(self.ancestors_weight); - let ancestors_weight = u128::from(self.ancestors_fee.as_u64()) * u128::from(self.weight); - - if tx_weight < ancestors_weight { - (self.fee, self.weight) - } else { - (self.ancestors_fee, self.ancestors_weight) - } - } -} - -impl PartialOrd for AncestorsScoreSortKey { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} - -impl Ord for AncestorsScoreSortKey { - fn cmp(&self, other: &Self) -> Ordering { - // avoid division a_fee/a_weight > b_fee/b_weight - let (fee, weight) = self.min_fee_and_weight(); - let (other_fee, other_weight) = other.min_fee_and_weight(); - let self_weight = u128::from(fee.as_u64()) * u128::from(other_weight); - let other_weight = u128::from(other_fee.as_u64()) * u128::from(weight); - if self_weight == other_weight { - // if fee rate weight is same, then compare with ancestor weight - if self.ancestors_weight == other.ancestors_weight { - self.id.raw_data().cmp(&other.id.raw_data()) - } else { - self.ancestors_weight.cmp(&other.ancestors_weight) - } - } else { - self_weight.cmp(&other_weight) - } - } -} - -#[derive(Default, Debug, Clone)] -pub struct TxLinks { - pub parents: HashSet, - pub children: HashSet, -} - -#[derive(Clone, Copy)] -enum Relation { - Parents, - Children, -} - -impl TxLinks { - fn get_direct_ids(&self, relation: Relation) -> &HashSet { - match relation { - Relation::Parents => &self.parents, - Relation::Children => &self.children, - } - } -} - -#[derive(Default, Debug, Clone)] -pub struct TxLinksMap { - pub(crate) inner: HashMap, -} - -impl TxLinksMap { - fn new() -> Self { - TxLinksMap { - inner: Default::default(), - } - } - - fn calc_relative_ids( - &self, - short_id: &ProposalShortId, - relation: Relation, - ) -> HashSet { - let direct = self - .inner - .get(short_id) - .map(|link| link.get_direct_ids(relation)) - .cloned() - .unwrap_or_default(); - - self.calc_relation_ids(Cow::Owned(direct), relation) - } - - fn calc_relation_ids( - &self, - stage: Cow>, - relation: Relation, - ) -> HashSet { - let mut stage = stage.into_owned(); - let mut relation_ids = HashSet::with_capacity(stage.len()); - - while let Some(id) = stage.iter().next().cloned() { - relation_ids.insert(id.clone()); - stage.remove(&id); - - //recursively - for id in self - .inner - .get(&id) - .map(|link| link.get_direct_ids(relation)) - .cloned() - .unwrap_or_default() - { - if !relation_ids.contains(&id) { - stage.insert(id); - } - } - } - relation_ids - } - - fn calc_ancestors(&self, short_id: &ProposalShortId) -> HashSet { - self.calc_relative_ids(short_id, Relation::Parents) - } - - fn calc_descendants(&self, short_id: &ProposalShortId) -> HashSet { - self.calc_relative_ids(short_id, Relation::Children) - } - - pub fn get_children(&self, short_id: &ProposalShortId) -> Option<&HashSet> { - self.inner.get(short_id).map(|link| &link.children) - } - - pub fn get_parents(&self, short_id: &ProposalShortId) -> Option<&HashSet> { - self.inner.get(short_id).map(|link| &link.parents) - } - - pub fn remove(&mut self, short_id: &ProposalShortId) -> Option { - self.inner.remove(short_id) - } - - fn remove_child( - &mut self, - short_id: &ProposalShortId, - child: &ProposalShortId, - ) -> Option { - self.inner - .get_mut(short_id) - .map(|links| links.children.remove(child)) - } - - fn remove_parent( - &mut self, - short_id: &ProposalShortId, - parent: &ProposalShortId, - ) -> Option { - self.inner - .get_mut(short_id) - .map(|links| links.parents.remove(parent)) - } - - fn add_child(&mut self, short_id: &ProposalShortId, child: ProposalShortId) -> Option { - self.inner - .get_mut(short_id) - .map(|links| links.children.insert(child)) - } - - fn add_parent(&mut self, short_id: &ProposalShortId, parent: ProposalShortId) -> Option { - self.inner - .get_mut(short_id) - .map(|links| links.parents.insert(parent)) - } - - fn clear(&mut self) { - self.inner.clear(); - } -} - -/// MultiIndexMap is used for multiple sort strategies, -/// to add any new sort strategy, you need to follow `AncestorsScoreSortKey` -/// and add logic to update the sort column in `insert_index_key` and `update_*_index_key` -#[derive(MultiIndexMap, Clone)] -pub struct IndexKey { - #[multi_index(hashed_unique)] - pub id: ProposalShortId, - #[multi_index(ordered_non_unique)] - pub score: AncestorsScoreSortKey, - // other sort key -} - -#[derive(Copy, Clone)] -enum EntryOp { - Add, - Remove, -} - -#[derive(Clone)] -pub(crate) struct SortedTxMap { - entries: HashMap, - pub(crate) sorted_index: MultiIndexIndexKeyMap, - deps: HashMap>, - /// A map track transaction ancestors and descendants - pub(crate) links: TxLinksMap, - max_ancestors_count: usize, -} - -impl SortedTxMap { - pub fn new(max_ancestors_count: usize) -> Self { - SortedTxMap { - entries: Default::default(), - sorted_index: MultiIndexIndexKeyMap::default(), - links: TxLinksMap::new(), - deps: Default::default(), - max_ancestors_count, - } - } - - pub fn size(&self) -> usize { - self.entries.len() - } - - pub fn iter(&self) -> impl Iterator { - self.entries.iter() - } - - fn insert_index_key(&mut self, entry: &TxEntry) { - self.sorted_index.insert(entry.as_index_key()); - } - - fn remove_sort_key(&mut self, entry: &TxEntry) { - self.sorted_index.remove_by_id(&entry.proposal_short_id()); - } - - fn update_descendants_index_key(&mut self, entry: &TxEntry, op: EntryOp) { - let descendants = self.calc_descendants(&entry.proposal_short_id()); - for desc_id in &descendants { - if let Some(desc_entry) = self.entries.get_mut(desc_id) { - let deleted = self - .sorted_index - .remove_by_id(&desc_entry.proposal_short_id()); - debug_assert!(deleted.is_some(), "pool inconsistent"); - - match op { - EntryOp::Remove => desc_entry.sub_entry_weight(entry), - EntryOp::Add => desc_entry.add_entry_weight(entry), - } - self.sorted_index.insert(desc_entry.as_index_key()); - } - } - } - - // Usually when a new transaction is added to the pool, it has no in-pool - // children (because any such children would be an orphan). So in add_entry(), we: - // - update a new entry's parents set to include all in-pool parents - // - update the new entry's parents to include the new tx as a child - // - update all ancestors of the transaction to include the new tx's size/fee - pub fn add_entry(&mut self, mut entry: TxEntry) -> Result { - let short_id = entry.proposal_short_id(); - - if self.contains_key(&short_id) { - return Ok(false); - }; - - // find in pool parents - let mut parents: HashSet = HashSet::with_capacity( - entry.transaction().inputs().len() + entry.transaction().cell_deps().len(), - ); - - for input in entry.transaction().inputs() { - let input_pt = input.previous_output(); - if let Some(deps) = self.deps.get(&input_pt) { - parents.extend(deps.iter().cloned()); - } - - let parent_hash = &input_pt.tx_hash(); - let id = ProposalShortId::from_tx_hash(parent_hash); - if self.links.inner.contains_key(&id) { - parents.insert(id); - } - } - for cell_dep in entry.transaction().cell_deps() { - let dep_pt = cell_dep.out_point(); - let id = ProposalShortId::from_tx_hash(&dep_pt.tx_hash()); - if self.links.inner.contains_key(&id) { - parents.insert(id); - } - } - - let ancestors = self - .links - .calc_relation_ids(Cow::Borrowed(&parents), Relation::Parents); - - // update parents references - for ancestor_id in &ancestors { - let ancestor = self.entries.get(ancestor_id).expect("pool consistent"); - entry.add_entry_weight(ancestor); - } - - if entry.ancestors_count > self.max_ancestors_count { - return Err(Reject::ExceededMaximumAncestorsCount); - } - - for cell_dep in entry.transaction().cell_deps() { - let dep_pt = cell_dep.out_point(); - // insert dep-ref map - self.deps - .entry(dep_pt) - .or_insert_with(HashSet::new) - .insert(short_id.clone()); - } - - for parent in &parents { - self.links.add_child(parent, short_id.clone()); - } - - // insert links - let links = TxLinks { - parents, - children: Default::default(), - }; - self.links.inner.insert(short_id.clone(), links); - self.insert_index_key(&entry); - self.entries.insert(short_id, entry); - Ok(true) - } - - // update_descendants_from_detached is used to update - // the descendants for a single transaction that has been added to the - // pool but may have child transactions in the pool, eg during a - // chain reorg. - pub fn update_descendants_from_detached( - &mut self, - id: &ProposalShortId, - children: HashSet, - ) { - if let Some(entry) = self.entries.get(id).cloned() { - for child in &children { - self.links.add_parent(child, id.clone()); - } - if let Some(links) = self.links.inner.get_mut(id) { - links.children.extend(children); - } - - self.update_descendants_index_key(&entry, EntryOp::Add); - } - } - - pub fn contains_key(&self, id: &ProposalShortId) -> bool { - self.entries.contains_key(id) - } - - pub fn get(&self, id: &ProposalShortId) -> Option<&TxEntry> { - self.entries.get(id) - } - - #[cfg(test)] - pub(crate) fn deps(&self) -> &HashMap> { - &self.deps - } - - fn update_deps_for_remove(&mut self, entry: &TxEntry) { - for cell_dep in entry.transaction().cell_deps() { - let dep_pt = cell_dep.out_point(); - if let HashMapEntry::Occupied(mut o) = self.deps.entry(dep_pt) { - let set = o.get_mut(); - if set.remove(&entry.proposal_short_id()) && set.is_empty() { - o.remove_entry(); - } - } - } - } - - fn update_children_for_remove(&mut self, id: &ProposalShortId) { - if let Some(children) = self.get_children(id).cloned() { - for child in children { - self.links.remove_parent(&child, id); - } - } - } - - fn update_parents_for_remove(&mut self, id: &ProposalShortId) { - if let Some(parents) = self.get_parents(id).cloned() { - for parent in parents { - self.links.remove_child(&parent, id); - } - } - } - - fn remove_unchecked(&mut self, id: &ProposalShortId) -> Option { - self.entries.remove(id).map(|entry| { - self.remove_sort_key(&entry); - self.update_deps_for_remove(&entry); - entry - }) - } - - pub fn remove_entry_and_descendants(&mut self, id: &ProposalShortId) -> Vec { - let mut removed_ids = vec![id.to_owned()]; - let mut removed = vec![]; - let descendants = self.calc_descendants(id); - removed_ids.extend(descendants); - - // update links state for remove - for id in &removed_ids { - self.update_parents_for_remove(id); - self.update_children_for_remove(id); - } - - for id in removed_ids { - if let Some(entry) = self.remove_unchecked(&id) { - self.links.remove(&id); - removed.push(entry); - } - } - removed - } - - // notice: - // we are sure that all in-pool ancestor have already been processed. - // otherwise `links` will differ from the set of parents we'd calculate by searching - pub fn remove_entry(&mut self, id: &ProposalShortId) -> Option { - self.remove_unchecked(id).map(|entry| { - // We're not recursively removing a tx and all its descendants - // So we need update statistics state - self.update_descendants_index_key(&entry, EntryOp::Remove); - self.update_parents_for_remove(id); - self.update_children_for_remove(id); - self.links.remove(id); - entry - }) - } - - /// calculate all ancestors from pool - pub fn calc_ancestors(&self, short_id: &ProposalShortId) -> HashSet { - self.links.calc_ancestors(short_id) - } - - /// calculate all descendants from pool - pub fn calc_descendants(&self, short_id: &ProposalShortId) -> HashSet { - self.links.calc_descendants(short_id) - } - - /// find children from pool - pub fn get_children(&self, short_id: &ProposalShortId) -> Option<&HashSet> { - self.links.get_children(short_id) - } - - /// find parents from pool - pub fn get_parents(&self, short_id: &ProposalShortId) -> Option<&HashSet> { - self.links.get_parents(short_id) - } - - /// sorted by ancestor score from higher to lower - pub fn score_sorted_iter(&self) -> impl Iterator { - // Note: multi_index don't support reverse order iteration now - // so we need to collect and reverse - let keys = self.sorted_index.iter_by_score().collect::>(); - keys.into_iter() - .rev() - .map(move |key| self.entries.get(&key.id).expect("consistent")) - } - - pub(crate) fn clear(&mut self) { - self.sorted_index.clear(); - self.deps.clear(); - self.links.clear(); - self.entries.clear(); - } -} diff --git a/tx-pool/src/component/edges.rs b/tx-pool/src/component/edges.rs new file mode 100644 index 0000000000..decf98c1a5 --- /dev/null +++ b/tx-pool/src/component/edges.rs @@ -0,0 +1,114 @@ +use ckb_types::packed::{Byte32, OutPoint, ProposalShortId}; +use std::collections::{hash_map::Entry, HashMap, HashSet}; + +#[derive(Debug, PartialEq, Clone)] +pub(crate) enum OutPointStatus { + UnConsumed, + Consumed(ProposalShortId), +} + +#[derive(Default, Debug, Clone)] +pub(crate) struct Edges { + /// input-txid map represent in-pool tx's inputs + pub(crate) inputs: HashMap, + /// output-op map represent in-pool tx's outputs + pub(crate) outputs: HashMap, + /// dep-set map represent in-pool tx's deps + pub(crate) deps: HashMap>, + /// dep-set map represent in-pool tx's header deps + pub(crate) header_deps: HashMap>, +} + +impl Edges { + #[cfg(test)] + pub(crate) fn outputs_len(&self) -> usize { + self.outputs.len() + } + + #[cfg(test)] + pub(crate) fn inputs_len(&self) -> usize { + self.inputs.len() + } + + #[cfg(test)] + pub(crate) fn header_deps_len(&self) -> usize { + self.header_deps.len() + } + + #[cfg(test)] + pub(crate) fn deps_len(&self) -> usize { + self.deps.len() + } + + pub(crate) fn insert_input(&mut self, out_point: OutPoint, txid: ProposalShortId) { + self.inputs.insert(out_point, txid); + } + + pub(crate) fn remove_input(&mut self, out_point: &OutPoint) -> Option { + self.inputs.remove(out_point) + } + + pub(crate) fn remove_output(&mut self, out_point: &OutPoint) -> Option { + match self.outputs.remove(out_point) { + Some(OutPointStatus::Consumed(id)) => Some(id), + _ => None, + } + } + + pub(crate) fn insert_unconsumed_output(&mut self, out_point: OutPoint) { + self.outputs.insert(out_point, OutPointStatus::UnConsumed); + } + + pub(crate) fn insert_consumed_output(&mut self, out_point: OutPoint, id: ProposalShortId) { + self.outputs.insert(out_point, OutPointStatus::Consumed(id)); + } + + pub(crate) fn get_input_ref(&self, out_point: &OutPoint) -> Option<&ProposalShortId> { + self.inputs.get(out_point) + } + + pub(crate) fn get_deps_ref(&self, out_point: &OutPoint) -> Option<&HashSet> { + self.deps.get(out_point) + } + + pub(crate) fn set_output_consumed( + &mut self, + out_point: &OutPoint, + tx_short_id: &ProposalShortId, + ) { + if let Some(status) = self.outputs.get_mut(out_point) { + *status = OutPointStatus::Consumed(tx_short_id.clone()); + } + } + + pub(crate) fn set_output_unconsumed(&mut self, out_point: &OutPoint) { + if let Some(status) = self.outputs.get_mut(out_point) { + *status = OutPointStatus::UnConsumed; + } + } + + pub(crate) fn get_output_ref(&self, out_point: &OutPoint) -> Option<&OutPointStatus> { + self.outputs.get(out_point) + } + + pub(crate) fn insert_deps(&mut self, out_point: OutPoint, txid: ProposalShortId) { + self.deps.entry(out_point).or_default().insert(txid); + } + + pub(crate) fn delete_txid_by_dep(&mut self, out_point: OutPoint, txid: &ProposalShortId) { + if let Entry::Occupied(mut occupied) = self.deps.entry(out_point) { + let ids = occupied.get_mut(); + ids.remove(txid); + if ids.is_empty() { + occupied.remove(); + } + } + } + + pub(crate) fn clear(&mut self) { + self.inputs.clear(); + self.outputs.clear(); + self.deps.clear(); + self.header_deps.clear(); + } +} diff --git a/tx-pool/src/component/entry.rs b/tx-pool/src/component/entry.rs index 9dc6a027c7..bcce6a2e16 100644 --- a/tx-pool/src/component/entry.rs +++ b/tx-pool/src/component/entry.rs @@ -1,5 +1,4 @@ -use crate::component::container::AncestorsScoreSortKey; -use crate::component::container::IndexKey; +use crate::component::score_key::AncestorsScoreSortKey; use ckb_systemtime::unix_time_as_millis; use ckb_types::{ core::{ @@ -100,11 +99,6 @@ impl TxEntry { EvictKey::from(self) } - /// Return a sort index - pub fn as_index_key(&self) -> IndexKey { - IndexKey::from(self) - } - /// Returns fee rate pub fn fee_rate(&self) -> FeeRate { let weight = get_transaction_weight(self.size, self.cycles); @@ -172,15 +166,6 @@ impl From<&TxEntry> for AncestorsScoreSortKey { } } -impl From<&TxEntry> for IndexKey { - fn from(entry: &TxEntry) -> Self { - IndexKey { - id: entry.proposal_short_id(), - score: entry.as_score_key(), - } - } -} - impl Hash for TxEntry { fn hash(&self, state: &mut H) { Hash::hash(self.transaction(), state); diff --git a/tx-pool/src/component/links.rs b/tx-pool/src/component/links.rs new file mode 100644 index 0000000000..520673b59d --- /dev/null +++ b/tx-pool/src/component/links.rs @@ -0,0 +1,144 @@ +use ckb_types::packed::ProposalShortId; +use std::borrow::Cow; +use std::collections::{HashMap, HashSet}; + +#[derive(Default, Debug, Clone)] +pub struct TxLinks { + pub parents: HashSet, + pub children: HashSet, +} + +#[derive(Clone, Copy)] +pub enum Relation { + Parents, + Children, +} + +impl TxLinks { + fn get_direct_ids(&self, relation: Relation) -> &HashSet { + match relation { + Relation::Parents => &self.parents, + Relation::Children => &self.children, + } + } +} + +#[derive(Default, Debug, Clone)] +pub struct TxLinksMap { + pub inner: HashMap, +} + +impl TxLinksMap { + pub fn new() -> Self { + TxLinksMap { + inner: Default::default(), + } + } + + fn calc_relative_ids( + &self, + short_id: &ProposalShortId, + relation: Relation, + ) -> HashSet { + let direct = self + .inner + .get(short_id) + .map(|link| link.get_direct_ids(relation)) + .cloned() + .unwrap_or_default(); + + self.calc_relation_ids(Cow::Owned(direct), relation) + } + + pub fn calc_relation_ids( + &self, + stage: Cow>, + relation: Relation, + ) -> HashSet { + let mut stage = stage.into_owned(); + let mut relation_ids = HashSet::with_capacity(stage.len()); + + while let Some(id) = stage.iter().next().cloned() { + relation_ids.insert(id.clone()); + stage.remove(&id); + + //recursively + for id in self + .inner + .get(&id) + .map(|link| link.get_direct_ids(relation)) + .cloned() + .unwrap_or_default() + { + if !relation_ids.contains(&id) { + stage.insert(id); + } + } + } + relation_ids + } + + pub fn calc_ancestors(&self, short_id: &ProposalShortId) -> HashSet { + self.calc_relative_ids(short_id, Relation::Parents) + } + + pub fn calc_descendants(&self, short_id: &ProposalShortId) -> HashSet { + self.calc_relative_ids(short_id, Relation::Children) + } + + pub fn get_children(&self, short_id: &ProposalShortId) -> Option<&HashSet> { + self.inner.get(short_id).map(|link| &link.children) + } + + pub fn get_parents(&self, short_id: &ProposalShortId) -> Option<&HashSet> { + self.inner.get(short_id).map(|link| &link.parents) + } + + pub fn remove(&mut self, short_id: &ProposalShortId) -> Option { + self.inner.remove(short_id) + } + + pub fn remove_child( + &mut self, + short_id: &ProposalShortId, + child: &ProposalShortId, + ) -> Option { + self.inner + .get_mut(short_id) + .map(|links| links.children.remove(child)) + } + + pub fn remove_parent( + &mut self, + short_id: &ProposalShortId, + parent: &ProposalShortId, + ) -> Option { + self.inner + .get_mut(short_id) + .map(|links| links.parents.remove(parent)) + } + + pub fn add_child( + &mut self, + short_id: &ProposalShortId, + child: ProposalShortId, + ) -> Option { + self.inner + .get_mut(short_id) + .map(|links| links.children.insert(child)) + } + + pub fn add_parent( + &mut self, + short_id: &ProposalShortId, + parent: ProposalShortId, + ) -> Option { + self.inner + .get_mut(short_id) + .map(|links| links.parents.insert(parent)) + } + + pub fn clear(&mut self) { + self.inner.clear(); + } +} diff --git a/tx-pool/src/component/mod.rs b/tx-pool/src/component/mod.rs index 60b4e78ae7..7f325424ba 100644 --- a/tx-pool/src/component/mod.rs +++ b/tx-pool/src/component/mod.rs @@ -2,12 +2,12 @@ pub mod commit_txs_scanner; pub mod entry; pub(crate) mod chunk; -pub(crate) mod container; +pub(crate) mod edges; +pub(crate) mod links; pub(crate) mod orphan; -pub(crate) mod pending; -pub(crate) mod proposed; -pub(crate) mod recent_reject; pub(crate) mod pool_map; +pub(crate) mod recent_reject; +pub(crate) mod score_key; #[cfg(test)] mod tests; diff --git a/tx-pool/src/component/pending.rs b/tx-pool/src/component/pending.rs deleted file mode 100644 index 3d0bd72383..0000000000 --- a/tx-pool/src/component/pending.rs +++ /dev/null @@ -1,363 +0,0 @@ -use crate::component::entry::TxEntry; -use ckb_types::{ - core::{ - cell::{CellChecker, CellMetaBuilder, CellProvider, CellStatus}, - error::OutPointError, - tx_pool::Reject, - TransactionView, - }, - packed::{Byte32, OutPoint, ProposalShortId}, - prelude::*, -}; -use ckb_util::{LinkedHashMap, LinkedHashMapEntries}; -use std::collections::{hash_map::Entry, HashMap, HashSet, VecDeque}; - -type ConflictEntry = (TxEntry, Reject); - -#[derive(Debug, Clone)] -pub(crate) struct PendingQueue { - pub(crate) inner: LinkedHashMap, - /// dep-set map represent in-pool tx's deps - pub(crate) deps: HashMap>, - /// input-set map represent in-pool tx's inputs - pub(crate) inputs: HashMap>, - /// dep-set map represent in-pool tx's header deps - pub(crate) header_deps: HashMap>, - // /// output-op map represent in-pool tx's outputs - pub(crate) outputs: HashMap>, -} - -impl PendingQueue { - pub(crate) fn new() -> Self { - PendingQueue { - inner: Default::default(), - deps: Default::default(), - inputs: Default::default(), - header_deps: Default::default(), - outputs: Default::default(), - } - } - - pub(crate) fn size(&self) -> usize { - self.inner.len() - } - - pub(crate) fn is_empty(&self) -> bool { - self.inner.len() == 0 - } - - #[cfg(test)] - pub(crate) fn outputs_len(&self) -> usize { - self.outputs.len() - } - - #[cfg(test)] - pub(crate) fn header_deps_len(&self) -> usize { - self.header_deps.len() - } - - #[cfg(test)] - pub(crate) fn deps_len(&self) -> usize { - self.deps.len() - } - - #[cfg(test)] - pub(crate) fn inputs_len(&self) -> usize { - self.inputs.len() - } - - pub(crate) fn add_entry(&mut self, entry: TxEntry) -> bool { - let tx_short_id = entry.proposal_short_id(); - if self.inner.contains_key(&tx_short_id) { - return false; - } - - let inputs = entry.transaction().input_pts_iter(); - let outputs = entry.transaction().output_pts(); - - for i in inputs { - self.inputs - .entry(i.to_owned()) - .or_default() - .insert(tx_short_id.clone()); - - if let Some(outputs) = self.outputs.get_mut(&i) { - outputs.insert(tx_short_id.clone()); - } - } - - // record dep-txid - for d in entry.related_dep_out_points() { - self.deps - .entry(d.to_owned()) - .or_default() - .insert(tx_short_id.clone()); - - if let Some(outputs) = self.outputs.get_mut(d) { - outputs.insert(tx_short_id.clone()); - } - } - - // record tx unconsumed output - for o in outputs { - self.outputs.insert(o, HashSet::new()); - } - - // record header_deps - let header_deps = entry.transaction().header_deps(); - if !header_deps.is_empty() { - self.header_deps - .insert(tx_short_id.clone(), header_deps.into_iter().collect()); - } - - self.inner.insert(tx_short_id, entry); - true - } - - pub(crate) fn resolve_conflict(&mut self, tx: &TransactionView) -> Vec { - let inputs = tx.input_pts_iter(); - let mut conflicts = Vec::new(); - - for i in inputs { - if let Some(ids) = self.inputs.remove(&i) { - for id in ids { - let entries = self.remove_entry_and_descendants(&id); - for entry in entries { - let reject = Reject::Resolve(OutPointError::Dead(i.clone())); - conflicts.push((entry, reject)); - } - } - } - - // deps consumed - if let Some(ids) = self.deps.remove(&i) { - for id in ids { - let entries = self.remove_entry_and_descendants(&id); - for entry in entries { - let reject = Reject::Resolve(OutPointError::Dead(i.clone())); - conflicts.push((entry, reject)); - } - } - } - } - conflicts - } - - pub(crate) fn resolve_conflict_header_dep( - &mut self, - headers: &HashSet, - ) -> Vec { - let mut conflicts = Vec::new(); - - // invalid header deps - let mut ids = Vec::new(); - for (tx_id, deps) in self.header_deps.iter() { - for hash in deps { - if headers.contains(hash) { - ids.push((hash.clone(), tx_id.clone())); - break; - } - } - } - - for (blk_hash, id) in ids { - let entries = self.remove_entry_and_descendants(&id); - for entry in entries { - let reject = Reject::Resolve(OutPointError::InvalidHeader(blk_hash.to_owned())); - conflicts.push((entry, reject)); - } - } - conflicts - } - - pub(crate) fn contains_key(&self, id: &ProposalShortId) -> bool { - self.inner.contains_key(id) - } - - pub fn iter(&self) -> impl Iterator { - self.inner.iter() - } - - pub(crate) fn get(&self, id: &ProposalShortId) -> Option<&TxEntry> { - self.inner.get(id) - } - - pub(crate) fn get_tx(&self, id: &ProposalShortId) -> Option<&TransactionView> { - self.inner.get(id).map(|entry| entry.transaction()) - } - - pub(crate) fn remove_entry(&mut self, id: &ProposalShortId) -> Option { - let removed = self.inner.remove(id); - - if let Some(ref entry) = removed { - self.remove_entry_relation(entry); - } - - removed - } - - pub(crate) fn remove_entry_and_descendants(&mut self, id: &ProposalShortId) -> Vec { - let mut removed = Vec::new(); - if let Some(entry) = self.inner.remove(id) { - let descendants = self.get_descendants(&entry); - self.remove_entry_relation(&entry); - removed.push(entry); - for id in descendants { - if let Some(entry) = self.remove_entry(&id) { - removed.push(entry); - } - } - } - removed - } - - pub(crate) fn get_descendants(&self, entry: &TxEntry) -> HashSet { - let mut entries: VecDeque<&TxEntry> = VecDeque::new(); - entries.push_back(entry); - - let mut descendants = HashSet::new(); - while let Some(entry) = entries.pop_front() { - let outputs = entry.transaction().output_pts(); - - for output in outputs { - if let Some(ids) = self.outputs.get(&output) { - for id in ids { - if descendants.insert(id.clone()) { - if let Some(entry) = self.inner.get(id) { - entries.push_back(entry); - } - } - } - } - } - } - descendants - } - - pub(crate) fn remove_entry_relation(&mut self, entry: &TxEntry) { - let inputs = entry.transaction().input_pts_iter(); - let tx_short_id = entry.proposal_short_id(); - let outputs = entry.transaction().output_pts(); - - for i in inputs { - if let Entry::Occupied(mut occupied) = self.inputs.entry(i) { - let empty = { - let ids = occupied.get_mut(); - ids.remove(&tx_short_id); - ids.is_empty() - }; - if empty { - occupied.remove(); - } - } - } - - // remove dep - for d in entry.related_dep_out_points().cloned() { - if let Entry::Occupied(mut occupied) = self.deps.entry(d) { - let empty = { - let ids = occupied.get_mut(); - ids.remove(&tx_short_id); - ids.is_empty() - }; - if empty { - occupied.remove(); - } - } - } - - for o in outputs { - self.outputs.remove(&o); - } - - self.header_deps.remove(&tx_short_id); - } - - pub(crate) fn remove_entries_by_filter bool>( - &mut self, - mut predicate: P, - ) -> Vec { - let entries = self.entries(); - let mut removed = Vec::new(); - for entry in entries { - if predicate(entry.key(), entry.get()) { - removed.push(entry.remove()); - } - } - for entry in &removed { - self.remove_entry_relation(entry); - } - - removed - } - - pub fn entries(&mut self) -> LinkedHashMapEntries { - self.inner.entries() - } - - // fill proposal txs - pub fn fill_proposals( - &self, - limit: usize, - exclusion: &HashSet, - proposals: &mut HashSet, - ) { - for id in self.inner.keys() { - if proposals.len() == limit { - break; - } - if !exclusion.contains(id) { - proposals.insert(id.clone()); - } - } - } - - pub(crate) fn drain(&mut self) -> Vec { - let txs = self - .inner - .drain() - .map(|(_k, entry)| entry.into_transaction()) - .collect::>(); - self.deps.clear(); - self.inputs.clear(); - self.header_deps.clear(); - self.outputs.clear(); - txs - } -} - -impl CellProvider for PendingQueue { - fn cell(&self, out_point: &OutPoint, _eager_load: bool) -> CellStatus { - let tx_hash = out_point.tx_hash(); - if let Some(entry) = self.inner.get(&ProposalShortId::from_tx_hash(&tx_hash)) { - match entry - .transaction() - .output_with_data(out_point.index().unpack()) - { - Some((output, data)) => { - let cell_meta = CellMetaBuilder::from_cell_output(output, data) - .out_point(out_point.to_owned()) - .build(); - CellStatus::live_cell(cell_meta) - } - None => CellStatus::Unknown, - } - } else { - CellStatus::Unknown - } - } -} - -impl CellChecker for PendingQueue { - fn is_live(&self, out_point: &OutPoint) -> Option { - let tx_hash = out_point.tx_hash(); - if let Some(entry) = self.inner.get(&ProposalShortId::from_tx_hash(&tx_hash)) { - entry - .transaction() - .output(out_point.index().unpack()) - .map(|_| true) - } else { - None - } - } -} diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index 2a66e760f2..f9b00b75e3 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -1,25 +1,30 @@ //! Top-level Pool type, methods, and tests extern crate rustc_hash; extern crate slab; -use crate::component::container::AncestorsScoreSortKey; +use crate::component::edges::{Edges, OutPointStatus}; use crate::component::entry::EvictKey; +use crate::component::links::{Relation, TxLinksMap}; +use crate::component::score_key::AncestorsScoreSortKey; use crate::error::Reject; use crate::TxEntry; -use ckb_logger::{debug, error, trace, warn}; + +use ckb_logger::trace; use ckb_types::core::error::OutPointError; use ckb_types::packed::OutPoint; use ckb_types::{ - core::cell::{CellMetaBuilder, CellProvider, CellStatus}, - prelude::*, + bytes::Bytes, + core::{cell::CellChecker, TransactionView}, + packed::{Byte32, CellOutput, ProposalShortId}, }; use ckb_types::{ - core::{cell::CellChecker, TransactionView}, - packed::{Byte32, ProposalShortId}, + core::cell::{CellMetaBuilder, CellProvider, CellStatus}, + prelude::*, }; use multi_index_map::MultiIndexMap; -use std::collections::hash_map::Entry; -use std::collections::HashMap; -use std::collections::{HashSet, VecDeque}; +use std::borrow::Cow; +use std::collections::HashSet; + +use super::links::TxLinks; type ConflictEntry = (TxEntry, Reject); @@ -30,6 +35,12 @@ pub enum Status { Proposed, } +#[derive(Copy, Clone)] +enum EntryOp { + Add, + Remove, +} + #[derive(MultiIndexMap, Clone)] pub struct PoolEntry { #[multi_index(hashed_unique)] @@ -40,9 +51,8 @@ pub struct PoolEntry { pub status: Status, #[multi_index(ordered_non_unique)] pub evict_key: EvictKey, - - pub inner: TxEntry, // other sort key + pub inner: TxEntry, } impl MultiIndexPoolEntryMap { @@ -50,7 +60,10 @@ impl MultiIndexPoolEntryMap { pub fn score_sorted_iter(&self) -> impl Iterator { // Note: multi_index don't support reverse order iteration now // so we need to collect and reverse - let entries = self.iter_by_score().collect::>(); + let entries = self + .iter_by_score() + .filter(|entry| entry.status == Status::Proposed) + .collect::>(); entries.into_iter().rev().map(move |entry| &entry.inner) } } @@ -58,13 +71,10 @@ impl MultiIndexPoolEntryMap { pub struct PoolMap { /// The pool entries with different kinds of sort strategies pub(crate) entries: MultiIndexPoolEntryMap, - /// dep-set map represent in-pool tx's header deps - pub(crate) header_deps: HashMap>, - /// dep-set map represent in-pool tx's deps - pub(crate) deps: HashMap>, - /// input-set map represent in-pool tx's inputs - pub(crate) inputs: HashMap>, - pub(crate) outputs: HashMap>, + /// All the deps, header_deps, inputs, outputs relationships + pub(crate) edges: Edges, + /// All the parent/children relationships + pub(crate) links: TxLinksMap, pub(crate) max_ancestors_count: usize, } @@ -72,197 +82,140 @@ impl PoolMap { pub fn new(max_ancestors_count: usize) -> Self { PoolMap { entries: MultiIndexPoolEntryMap::default(), - header_deps: HashMap::default(), - deps: HashMap::default(), - inputs: HashMap::default(), - outputs: HashMap::default(), + edges: Edges::default(), + links: TxLinksMap::new(), max_ancestors_count, } } #[cfg(test)] pub(crate) fn outputs_len(&self) -> usize { - self.outputs.len() + self.edges.outputs_len() } #[cfg(test)] pub(crate) fn header_deps_len(&self) -> usize { - self.header_deps.len() + self.edges.header_deps_len() } #[cfg(test)] pub(crate) fn deps_len(&self) -> usize { - self.deps.len() + self.edges.deps_len() } #[cfg(test)] pub(crate) fn inputs_len(&self) -> usize { - self.inputs.len() + self.edges.inputs_len() } #[cfg(test)] - pub fn size(&self) -> usize { + pub(crate) fn size(&self) -> usize { self.entries.len() } #[cfg(test)] - pub fn contains_key(&self, id: &ProposalShortId) -> bool { + pub(crate) fn contains_key(&self, id: &ProposalShortId) -> bool { self.entries.get_by_id(id).is_some() } + #[cfg(test)] pub(crate) fn get_tx(&self, id: &ProposalShortId) -> Option<&TransactionView> { self.entries .get_by_id(id) .map(|entry| entry.inner.transaction()) } - pub fn add_entry(&mut self, entry: TxEntry, status: Status) -> bool { - let tx_short_id = entry.proposal_short_id(); - if self.entries.get_by_id(&tx_short_id).is_some() { - return false; - } - trace!("add_{:?} {}", status, entry.transaction().hash()); - let inputs = entry.transaction().input_pts_iter(); - let outputs = entry.transaction().output_pts(); - - for i in inputs { - self.inputs - .entry(i.to_owned()) - .or_default() - .insert(tx_short_id.clone()); - - if let Some(outputs) = self.outputs.get_mut(&i) { - outputs.insert(tx_short_id.clone()); - } - } - - // record dep-txid - for d in entry.related_dep_out_points() { - self.deps - .entry(d.to_owned()) - .or_default() - .insert(tx_short_id.clone()); - - if let Some(outputs) = self.outputs.get_mut(d) { - outputs.insert(tx_short_id.clone()); - } - } + #[cfg(test)] + pub(crate) fn add_proposed(&mut self, entry: TxEntry) -> Result { + self.add_entry(entry, Status::Proposed) + } - // record tx unconsumed output - for o in outputs { - self.outputs.insert(o, HashSet::new()); - } + #[cfg(test)] + pub(crate) fn remove_committed_tx(&mut self, tx: &TransactionView) -> Option { + self.remove_entry(&tx.proposal_short_id()) + } - // record header_deps - let header_deps = entry.transaction().header_deps(); - if !header_deps.is_empty() { - self.header_deps - .insert(tx_short_id.clone(), header_deps.into_iter().collect()); - } + pub(crate) fn get_by_id(&self, id: &ProposalShortId) -> Option<&PoolEntry> { + self.entries.get_by_id(id) + } - let score = entry.as_score_key(); - let evict_key = entry.as_evict_key(); - self.entries.insert(PoolEntry { - id: tx_short_id, - score, - status, - inner: entry, - evict_key, - }); - true + pub(crate) fn pending_size(&self) -> usize { + self.entries.get_by_status(&Status::Pending).len() + + self.entries.get_by_status(&Status::Gap).len() } - pub fn get_by_id(&self, id: &ProposalShortId) -> Option<&PoolEntry> { - self.entries.get_by_id(id).map(|entry| entry) + pub(crate) fn proposed_size(&self) -> usize { + self.entries.get_by_status(&Status::Proposed).len() } - fn get_descendants(&self, entry: &TxEntry) -> HashSet { - let mut entries: VecDeque<&TxEntry> = VecDeque::new(); - entries.push_back(entry); + pub(crate) fn score_sorted_iter(&self) -> impl Iterator { + self.entries.score_sorted_iter() + } - let mut descendants = HashSet::new(); - while let Some(entry) = entries.pop_front() { - let outputs = entry.transaction().output_pts(); + pub(crate) fn get(&self, id: &ProposalShortId) -> Option<&TxEntry> { + self.get_by_id(id).map(|entry| &entry.inner) + } - for output in outputs { - if let Some(ids) = self.outputs.get(&output) { - for id in ids { - if descendants.insert(id.clone()) { - if let Some(entry) = self.entries.get_by_id(id) { - entries.push_back(&entry.inner); - } - } - } - } - } + pub(crate) fn get_proposed(&self, id: &ProposalShortId) -> Option<&TxEntry> { + match self.get_by_id(id) { + Some(entry) if entry.status == Status::Proposed => Some(&entry.inner), + _ => None, } - descendants } - pub(crate) fn remove_entry_relation(&mut self, entry: &TxEntry) { - let inputs = entry.transaction().input_pts_iter(); - let tx_short_id = entry.proposal_short_id(); - let outputs = entry.transaction().output_pts(); + pub(crate) fn has_proposed(&self, id: &ProposalShortId) -> bool { + self.get_proposed(id).is_some() + } - for i in inputs { - if let Entry::Occupied(mut occupied) = self.inputs.entry(i) { - let empty = { - let ids = occupied.get_mut(); - ids.remove(&tx_short_id); - ids.is_empty() - }; - if empty { - occupied.remove(); - } - } - } + /// calculate all ancestors from pool + pub(crate) fn calc_ancestors(&self, short_id: &ProposalShortId) -> HashSet { + self.links.calc_ancestors(short_id) + } - // remove dep - for d in entry.related_dep_out_points().cloned() { - if let Entry::Occupied(mut occupied) = self.deps.entry(d) { - let empty = { - let ids = occupied.get_mut(); - ids.remove(&tx_short_id); - ids.is_empty() - }; - if empty { - occupied.remove(); - } - } - } + /// calculate all descendants from pool + pub(crate) fn calc_descendants(&self, short_id: &ProposalShortId) -> HashSet { + self.links.calc_descendants(short_id) + } - for o in outputs { - self.outputs.remove(&o); - } + pub(crate) fn get_output_with_data(&self, out_point: &OutPoint) -> Option<(CellOutput, Bytes)> { + self.get(&ProposalShortId::from_tx_hash(&out_point.tx_hash())) + .and_then(|entry| { + entry + .transaction() + .output_with_data(out_point.index().unpack()) + }) + } - self.header_deps.remove(&tx_short_id); + pub(crate) fn remove_entry(&mut self, id: &ProposalShortId) -> Option { + if let Some(entry) = self.entries.remove_by_id(id) { + self.update_descendants_index_key(&entry.inner, EntryOp::Remove); + self.remove_entry_deps(&entry.inner); + self.remove_entry_edges(&entry.inner); + self.remove_entry_links(id); + return Some(entry.inner); + } + None } - pub fn remove_entry(&mut self, id: &ProposalShortId) -> Option { - let removed = self.entries.remove_by_id(id); + pub(crate) fn remove_entry_and_descendants(&mut self, id: &ProposalShortId) -> Vec { + let mut removed_ids = vec![id.to_owned()]; + let mut removed = vec![]; + removed_ids.extend(self.calc_descendants(id)); - if let Some(ref entry) = removed { - self.remove_entry_relation(&entry.inner); + // update links state for remove + for id in &removed_ids { + self.remove_entry_links(id); } - removed.map(|e| e.inner) - } - pub fn remove_entry_and_descendants(&mut self, id: &ProposalShortId) -> Vec { - let mut removed = Vec::new(); - if let Some(entry) = self.entries.remove_by_id(id) { - let descendants = self.get_descendants(&entry.inner); - self.remove_entry_relation(&entry.inner); - removed.push(entry.inner); - for id in descendants { - if let Some(entry) = self.remove_entry(&id) { - removed.push(entry); - } + for id in removed_ids { + if let Some(entry) = self.remove_entry(&id) { + removed.push(entry); } } removed } - pub fn resolve_conflict_header_dep( + pub(crate) fn resolve_conflict_header_dep( &mut self, headers: &HashSet, ) -> Vec { @@ -270,7 +223,7 @@ impl PoolMap { // invalid header deps let mut ids = Vec::new(); - for (tx_id, deps) in self.header_deps.iter() { + for (tx_id, deps) in self.edges.header_deps.iter() { for hash in deps { if headers.contains(hash) { ids.push((hash.clone(), tx_id.clone())); @@ -289,37 +242,45 @@ impl PoolMap { conflicts } - pub fn resolve_conflict(&mut self, tx: &TransactionView) -> Vec { - let inputs = tx.input_pts_iter(); + /// pending gap and proposed store the inputs and deps in edges, it's removed in `remove_entry` + /// here we use `input_pts_iter` and `related_dep_out_points` to find the conflict txs + pub(crate) fn resolve_conflict(&mut self, tx: &TransactionView) -> Vec { + let mut to_be_removed = Vec::new(); let mut conflicts = Vec::new(); - for i in inputs { - if let Some(ids) = self.inputs.remove(&i) { - for id in ids { - let entries = self.remove_entry_and_descendants(&id); - for entry in entries { - let reject = Reject::Resolve(OutPointError::Dead(i.clone())); - conflicts.push((entry, reject)); - } + for (_, entry) in self.entries.iter() { + let entry = &entry.inner; + let tx_id = entry.proposal_short_id(); + let tx_inputs = entry.transaction().input_pts_iter(); + let deps = entry.related_dep_out_points(); + + // tx input conflict + for i in tx_inputs { + if tx.input_pts_iter().any(|j| i == j) { + to_be_removed.push((tx_id.to_owned(), i.clone())); } } - // deps consumed - if let Some(ids) = self.deps.remove(&i) { - for id in ids { - let entries = self.remove_entry_and_descendants(&id); - for entry in entries { - let reject = Reject::Resolve(OutPointError::Dead(i.clone())); - conflicts.push((entry, reject)); - } + // tx deps conflict + for i in deps { + if tx.input_pts_iter().any(|j| *i == j) { + to_be_removed.push((tx_id.to_owned(), i.clone())); } } } + + for (tx_id, input) in to_be_removed.iter() { + let entries = self.remove_entry_and_descendants(tx_id); + let reject = Reject::Resolve(OutPointError::Dead(input.to_owned())); + let rejects = std::iter::repeat(reject).take(entries.len()); + conflicts.extend(entries.into_iter().zip(rejects)); + } + conflicts } // fill proposal txs - pub fn fill_proposals( + pub(crate) fn fill_proposals( &self, limit: usize, exclusion: &HashSet, @@ -336,12 +297,13 @@ impl PoolMap { } } - pub fn remove_entries_by_filter bool>( + pub(crate) fn remove_entries_by_filter bool>( &mut self, + status: &Status, mut predicate: P, ) -> Vec { let mut removed = Vec::new(); - for (_, entry) in self.entries.iter() { + for entry in self.entries.get_by_status(status) { if predicate(&entry.id, &entry.inner) { removed.push(entry.inner.clone()); } @@ -349,82 +311,283 @@ impl PoolMap { for entry in &removed { self.remove_entry(&entry.proposal_short_id()); } - removed } - pub fn iter(&self) -> impl Iterator { + pub(crate) fn iter(&self) -> impl Iterator { self.entries.iter().map(|(_, entry)| entry) } - pub fn iter_by_evict_key(&self) -> impl Iterator { + pub(crate) fn iter_by_evict_key(&self) -> impl Iterator { self.entries.iter_by_evict_key() } - pub fn next_evict_entry(&self) -> Option { + pub(crate) fn next_evict_entry(&self) -> Option { self.iter_by_evict_key() - .into_iter() .next() .map(|entry| entry.id.clone()) } - pub fn clear(&mut self) { + pub(crate) fn clear(&mut self) { self.entries = MultiIndexPoolEntryMap::default(); - self.deps.clear(); - self.inputs.clear(); - self.header_deps.clear(); - self.outputs.clear(); + self.edges.clear(); + self.links.clear(); } - pub(crate) fn drain(&mut self) -> Vec { - let txs = self - .entries - .iter() - .map(|(_k, entry)| entry.inner.clone().into_transaction()) - .collect::>(); - self.entries.clear(); - self.deps.clear(); - self.inputs.clear(); - self.header_deps.clear(); - self.outputs.clear(); - txs + fn remove_entry_links(&mut self, id: &ProposalShortId) { + if let Some(parents) = self.links.get_parents(id).cloned() { + for parent in parents { + self.links.remove_child(&parent, id); + } + } + if let Some(children) = self.links.get_children(id).cloned() { + for child in children { + self.links.remove_parent(&child, id); + } + } + self.links.remove(id); + } + + fn update_descendants_index_key(&mut self, parent: &TxEntry, op: EntryOp) { + let descendants: HashSet = + self.links.calc_descendants(&parent.proposal_short_id()); + for desc_id in &descendants { + // update child score + let entry = self.entries.get_by_id(desc_id).unwrap().clone(); + let mut child = entry.inner.clone(); + match op { + EntryOp::Remove => child.sub_entry_weight(parent), + EntryOp::Add => child.add_entry_weight(parent), + } + let short_id = child.proposal_short_id(); + //TODO: optimize it + self.entries.remove_by_id(&short_id); + self.insert_entry(&child, entry.status) + .expect("pool consistent"); + } + } + + fn record_entry_deps(&mut self, entry: &TxEntry) { + let tx_short_id: ProposalShortId = entry.proposal_short_id(); + let header_deps = entry.transaction().header_deps(); + let related_dep_out_points: Vec<_> = entry.related_dep_out_points().cloned().collect(); + + // record dep-txid + for d in related_dep_out_points { + self.edges.insert_deps(d.to_owned(), tx_short_id.clone()); + } + // record header_deps + if !header_deps.is_empty() { + self.edges + .header_deps + .insert(tx_short_id, header_deps.into_iter().collect()); + } + } + + fn record_entry_edges(&mut self, entry: &TxEntry) { + let tx_short_id: ProposalShortId = entry.proposal_short_id(); + let inputs = entry.transaction().input_pts_iter(); + let outputs = entry.transaction().output_pts(); + + let mut children = HashSet::new(); + // if input reference a in-pool output, connect it + // otherwise, record input for conflict check + for i in inputs { + self.edges.set_output_consumed(&i, &tx_short_id); + self.edges.insert_input(i.to_owned(), tx_short_id.clone()); + } + + // record tx output + for o in outputs { + if let Some(ids) = self.edges.get_deps_ref(&o).cloned() { + children.extend(ids); + } + if let Some(id) = self.edges.get_input_ref(&o).cloned() { + self.edges.insert_consumed_output(o, id.clone()); + children.insert(id); + } else { + self.edges.insert_unconsumed_output(o); + } + } + // update children + if !children.is_empty() { + self.update_descendants_from_detached(&tx_short_id, children); + } + } + + // update_descendants_from_detached is used to update + // the descendants for a single transaction that has been added to the + // pool but may have child transactions in the pool, eg during a + // chain reorg. + fn update_descendants_from_detached( + &mut self, + id: &ProposalShortId, + children: HashSet, + ) { + if let Some(entry) = self.get_by_id(id).cloned() { + for child in &children { + self.links.add_parent(child, id.clone()); + } + if let Some(links) = self.links.inner.get_mut(id) { + links.children.extend(children); + } + + self.update_descendants_index_key(&entry.inner, EntryOp::Add); + } + } + + /// Record the links for entry + fn record_entry_links(&mut self, entry: &mut TxEntry, status: &Status) -> Result { + // find in pool parents + let mut parents: HashSet = HashSet::with_capacity( + entry.transaction().inputs().len() + entry.transaction().cell_deps().len(), + ); + let short_id = entry.proposal_short_id(); + + for input in entry.transaction().inputs() { + let input_pt = input.previous_output(); + if let Some(deps) = self.edges.deps.get(&input_pt) { + parents.extend(deps.iter().cloned()); + } + + let parent_hash = &input_pt.tx_hash(); + let id = ProposalShortId::from_tx_hash(parent_hash); + if self.links.inner.contains_key(&id) { + parents.insert(id); + } + } + for cell_dep in entry.transaction().cell_deps() { + let dep_pt = cell_dep.out_point(); + let id = ProposalShortId::from_tx_hash(&dep_pt.tx_hash()); + if self.links.inner.contains_key(&id) { + parents.insert(id); + } + } + + let ancestors = self + .links + .calc_relation_ids(Cow::Borrowed(&parents), Relation::Parents); + + // update parents references + for ancestor_id in &ancestors { + let ancestor = self + .entries + .get_by_id(ancestor_id) + .expect("pool consistent"); + entry.add_entry_weight(&ancestor.inner); + } + if *status == Status::Proposed && entry.ancestors_count > self.max_ancestors_count { + return Err(Reject::ExceededMaximumAncestorsCount); + } + + for cell_dep in entry.transaction().cell_deps() { + let dep_pt = cell_dep.out_point(); + // insert dep-ref map + self.edges + .deps + .entry(dep_pt) + .or_insert_with(HashSet::new) + .insert(short_id.clone()); + } + + for parent in &parents { + self.links.add_child(parent, short_id.clone()); + } + + // insert links + let links = TxLinks { + parents, + children: Default::default(), + }; + self.links.inner.insert(short_id, links); + + Ok(true) + } + + fn remove_entry_edges(&mut self, entry: &TxEntry) { + let inputs = entry.transaction().input_pts_iter(); + let outputs = entry.transaction().output_pts(); + + for o in outputs { + self.edges.remove_output(&o); + } + + for i in inputs { + // release input record + self.edges.remove_input(&i); + self.edges.set_output_unconsumed(&i); + } + } + + fn remove_entry_deps(&mut self, entry: &TxEntry) { + let id = entry.proposal_short_id(); + for d in entry.related_dep_out_points().cloned() { + self.edges.delete_txid_by_dep(d, &id); + } + + self.edges.header_deps.remove(&id); + } + + pub(crate) fn add_entry(&mut self, mut entry: TxEntry, status: Status) -> Result { + let tx_short_id = entry.proposal_short_id(); + if self.entries.get_by_id(&tx_short_id).is_some() { + return Ok(false); + } + trace!("add_{:?} {}", status, entry.transaction().hash()); + self.record_entry_links(&mut entry, &status)?; + self.insert_entry(&entry, status)?; + self.record_entry_deps(&entry); + self.record_entry_edges(&entry); + Ok(true) + } + + fn insert_entry(&mut self, entry: &TxEntry, status: Status) -> Result { + let tx_short_id = entry.proposal_short_id(); + let score = entry.as_score_key(); + let evict_key = entry.as_evict_key(); + self.entries.insert(PoolEntry { + id: tx_short_id, + score, + status, + inner: entry.clone(), + evict_key, + }); + Ok(true) } } -impl CellProvider for MultiIndexPoolEntryMap { +impl CellProvider for PoolMap { fn cell(&self, out_point: &OutPoint, _eager_load: bool) -> CellStatus { - let tx_hash = out_point.tx_hash(); - if let Some(entry) = self.get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) { - match entry - .inner - .transaction() - .output_with_data(out_point.index().unpack()) - { - Some((output, data)) => { - let cell_meta = CellMetaBuilder::from_cell_output(output, data) - .out_point(out_point.to_owned()) - .build(); - CellStatus::live_cell(cell_meta) - } - None => CellStatus::Unknown, + if let Some(id) = self.edges.get_input_ref(out_point) { + if self.has_proposed(id) { + return CellStatus::Dead; + } + } + match self.edges.get_output_ref(out_point) { + Some(OutPointStatus::UnConsumed) => { + let (output, data) = self.get_output_with_data(out_point).expect("output"); + let cell_meta = CellMetaBuilder::from_cell_output(output, data) + .out_point(out_point.to_owned()) + .build(); + CellStatus::live_cell(cell_meta) } - } else { - CellStatus::Unknown + Some(OutPointStatus::Consumed(id)) if self.has_proposed(id) => CellStatus::Dead, + _ => CellStatus::Unknown, } } } -impl CellChecker for MultiIndexPoolEntryMap { +impl CellChecker for PoolMap { fn is_live(&self, out_point: &OutPoint) -> Option { - let tx_hash = out_point.tx_hash(); - if let Some(entry) = self.get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) { - entry - .inner - .transaction() - .output(out_point.index().unpack()) - .map(|_| true) - } else { - None + if let Some(id) = self.edges.get_input_ref(out_point) { + if self.has_proposed(id) { + return Some(false); + } + } + match self.edges.get_output_ref(out_point) { + Some(OutPointStatus::Consumed(id)) if self.has_proposed(id) => Some(false), + Some(OutPointStatus::UnConsumed) => Some(true), + _ => None, } } } diff --git a/tx-pool/src/component/proposed.rs b/tx-pool/src/component/proposed.rs deleted file mode 100644 index 2ac9114aa8..0000000000 --- a/tx-pool/src/component/proposed.rs +++ /dev/null @@ -1,403 +0,0 @@ -use crate::component::container::SortedTxMap; -use crate::component::entry::TxEntry; -use crate::error::Reject; -use ckb_types::{ - bytes::Bytes, - core::{ - cell::{CellChecker, CellMetaBuilder, CellProvider, CellStatus}, - error::OutPointError, - TransactionView, - }, - packed::{Byte32, CellOutput, OutPoint, ProposalShortId}, - prelude::*, -}; -use std::collections::{hash_map::Entry, HashMap, HashSet}; -use std::iter; - -type ConflictEntry = (TxEntry, Reject); - -#[derive(Default, Debug, Clone)] -pub(crate) struct Edges { - /// output-op map represent in-pool tx's outputs - pub(crate) outputs: HashMap>, - /// input-txid map represent in-pool tx's inputs - pub(crate) inputs: HashMap, - /// dep-set map represent in-pool tx's deps - pub(crate) deps: HashMap>, - /// dep-set map represent in-pool tx's header deps - pub(crate) header_deps: HashMap>, -} - -impl Edges { - #[cfg(test)] - pub(crate) fn outputs_len(&self) -> usize { - self.outputs.len() - } - - #[cfg(test)] - pub(crate) fn inputs_len(&self) -> usize { - self.inputs.len() - } - - pub(crate) fn insert_input(&mut self, out_point: OutPoint, txid: ProposalShortId) { - self.inputs.insert(out_point, txid); - } - - pub(crate) fn remove_input(&mut self, out_point: &OutPoint) -> Option { - self.inputs.remove(out_point) - } - - pub(crate) fn remove_output(&mut self, out_point: &OutPoint) -> Option { - self.outputs.remove(out_point).unwrap_or(None) - } - - pub(crate) fn insert_output(&mut self, out_point: OutPoint) { - self.outputs.insert(out_point, None); - } - - pub(crate) fn insert_consumed_output(&mut self, out_point: OutPoint, id: ProposalShortId) { - self.outputs.insert(out_point, Some(id)); - } - - pub(crate) fn get_output_ref(&self, out_point: &OutPoint) -> Option<&Option> { - self.outputs.get(out_point) - } - - pub(crate) fn get_input_ref(&self, out_point: &OutPoint) -> Option<&ProposalShortId> { - self.inputs.get(out_point) - } - - pub(crate) fn get_deps_ref(&self, out_point: &OutPoint) -> Option<&HashSet> { - self.deps.get(out_point) - } - - pub(crate) fn get_mut_output( - &mut self, - out_point: &OutPoint, - ) -> Option<&mut Option> { - self.outputs.get_mut(out_point) - } - - pub(crate) fn remove_deps(&mut self, out_point: &OutPoint) -> Option> { - self.deps.remove(out_point) - } - - pub(crate) fn insert_deps(&mut self, out_point: OutPoint, txid: ProposalShortId) { - self.deps.entry(out_point).or_default().insert(txid); - } - - pub(crate) fn delete_txid_by_dep(&mut self, out_point: OutPoint, txid: &ProposalShortId) { - if let Entry::Occupied(mut occupied) = self.deps.entry(out_point) { - let empty = { - let ids = occupied.get_mut(); - ids.remove(txid); - ids.is_empty() - }; - if empty { - occupied.remove(); - } - } - } - - pub(crate) fn clear(&mut self) { - self.outputs.clear(); - self.inputs.clear(); - self.deps.clear(); - self.header_deps.clear(); - } -} - -#[derive(Clone)] -pub struct ProposedPool { - pub(crate) edges: Edges, - inner: SortedTxMap, -} - -impl CellProvider for ProposedPool { - fn cell(&self, out_point: &OutPoint, _eager_load: bool) -> CellStatus { - if self.edges.get_input_ref(out_point).is_some() { - return CellStatus::Dead; - } - if let Some(x) = self.edges.get_output_ref(out_point) { - // output consumed - if x.is_some() { - return CellStatus::Dead; - } else { - let (output, data) = self.get_output_with_data(out_point).expect("output"); - let cell_meta = CellMetaBuilder::from_cell_output(output, data) - .out_point(out_point.to_owned()) - .build(); - return CellStatus::live_cell(cell_meta); - } - } - CellStatus::Unknown - } -} - -impl CellChecker for ProposedPool { - fn is_live(&self, out_point: &OutPoint) -> Option { - if self.edges.get_input_ref(out_point).is_some() { - return Some(false); - } - if let Some(x) = self.edges.get_output_ref(out_point) { - // output consumed - if x.is_some() { - return Some(false); - } else { - return Some(true); - } - } - None - } -} - -impl ProposedPool { - pub(crate) fn new(max_ancestors_count: usize) -> Self { - ProposedPool { - edges: Default::default(), - inner: SortedTxMap::new(max_ancestors_count), - } - } - - pub(crate) fn contains_key(&self, id: &ProposalShortId) -> bool { - self.inner.contains_key(id) - } - - pub fn get(&self, id: &ProposalShortId) -> Option<&TxEntry> { - self.inner.get(id) - } - - pub fn iter(&self) -> impl Iterator { - self.inner.iter() - } - - pub(crate) fn get_tx(&self, id: &ProposalShortId) -> Option<&TransactionView> { - self.get(id).map(|entry| entry.transaction()) - } - - pub fn size(&self) -> usize { - self.inner.size() - } - - pub(crate) fn get_output_with_data(&self, out_point: &OutPoint) -> Option<(CellOutput, Bytes)> { - self.inner - .get(&ProposalShortId::from_tx_hash(&out_point.tx_hash())) - .and_then(|entry| { - entry - .transaction() - .output_with_data(out_point.index().unpack()) - }) - } - - // remove entry and all it's descendants - pub(crate) fn remove_entry_and_descendants(&mut self, id: &ProposalShortId) -> Vec { - let removed_entries = self.inner.remove_entry_and_descendants(id); - for entry in &removed_entries { - let tx = entry.transaction(); - let inputs = tx.input_pts_iter(); - let outputs = tx.output_pts(); - for i in inputs { - self.edges.inputs.remove(&i); - if let Some(id) = self.edges.get_mut_output(&i) { - *id = None; - } - } - - for d in entry.related_dep_out_points().cloned() { - self.edges.delete_txid_by_dep(d, id); - } - - for o in outputs { - self.edges.remove_output(&o); - } - - self.edges.header_deps.remove(&entry.proposal_short_id()); - } - removed_entries - } - - pub(crate) fn remove_committed_tx(&mut self, tx: &TransactionView) -> Option { - let outputs = tx.output_pts(); - let inputs = tx.input_pts_iter(); - let id = tx.proposal_short_id(); - - if let Some(entry) = self.inner.remove_entry(&id) { - for o in outputs { - self.edges.remove_output(&o); - } - - for i in inputs { - // release input record - self.edges.remove_input(&i); - if let Some(id) = self.edges.get_mut_output(&i) { - *id = None; - } - } - - for d in entry.related_dep_out_points().cloned() { - self.edges.delete_txid_by_dep(d, &id); - } - - self.edges.header_deps.remove(&id); - - return Some(entry); - } - None - } - - // In the event of a reorg, the assumption that a newly added tx has no - // in-pool children is false. In particular, the pool is in an - // inconsistent state while new transactions are being added, because there may - // be descendant transactions of a tx coming from a disconnected block that are - // unreachable from just looking at transactions in the pool (the linking - // transactions may also be in the disconnected block, waiting to be added). - // Because of this, there's not much benefit in trying to search for in-pool - // children in add_entry(). Instead, in the special case of transactions - // being added from a disconnected block, out-of-block descendants for all the - // in-block transactions by calling update_descendants_from_detached(). Note that - // until this is called, the pool state is not consistent, and in particular - // TxLinks may not be correct (and therefore functions like - // calc_ancestors() and calc_descendants() that rely - // on them to walk the pool are not generally safe to use). - pub(crate) fn add_entry(&mut self, entry: TxEntry) -> Result { - let tx_short_id = entry.proposal_short_id(); - - if self.inner.contains_key(&tx_short_id) { - return Ok(false); - } - - let inputs = entry.transaction().input_pts_iter(); - let outputs = entry.transaction().output_pts(); - let related_dep_out_points: Vec<_> = entry.related_dep_out_points().cloned().collect(); - let header_deps = entry.transaction().header_deps(); - - self.inner.add_entry(entry).map(|inserted| { - if inserted { - let mut children = HashSet::new(); - // if input reference a in-pool output, connect it - // otherwise, record input for conflict check - for i in inputs { - if let Some(id) = self.edges.get_mut_output(&i) { - *id = Some(tx_short_id.clone()); - } - self.edges.insert_input(i.to_owned(), tx_short_id.clone()); - } - - // record dep-txid - for d in related_dep_out_points { - self.edges.insert_deps(d.to_owned(), tx_short_id.clone()); - } - - // record tx output - for o in outputs { - if let Some(ids) = self.edges.get_deps_ref(&o).cloned() { - children.extend(ids); - } - if let Some(id) = self.edges.get_input_ref(&o).cloned() { - self.edges.insert_consumed_output(o, id.clone()); - children.insert(id); - } else { - self.edges.insert_output(o); - } - } - - // record header_deps - if !header_deps.is_empty() { - self.edges - .header_deps - .insert(tx_short_id.clone(), header_deps.into_iter().collect()); - } - - if !children.is_empty() { - self.inner - .update_descendants_from_detached(&tx_short_id, children); - } - } - inserted - }) - } - - pub(crate) fn resolve_conflict(&mut self, tx: &TransactionView) -> Vec { - let inputs = tx.input_pts_iter(); - let mut conflicts = Vec::new(); - - for i in inputs { - if let Some(id) = self.edges.remove_input(&i) { - let entries = self.remove_entry_and_descendants(&id); - if !entries.is_empty() { - let reject = Reject::Resolve(OutPointError::Dead(i.clone())); - let rejects = iter::repeat(reject).take(entries.len()); - conflicts.extend(entries.into_iter().zip(rejects)); - } - } - - // deps consumed - if let Some(x) = self.edges.remove_deps(&i) { - for id in x { - let entries = self.remove_entry_and_descendants(&id); - if !entries.is_empty() { - let reject = Reject::Resolve(OutPointError::Dead(i.clone())); - let rejects = iter::repeat(reject).take(entries.len()); - conflicts.extend(entries.into_iter().zip(rejects)); - } - } - } - } - - conflicts - } - - pub(crate) fn resolve_conflict_header_dep( - &mut self, - headers: &HashSet, - ) -> Vec { - let mut conflicts = Vec::new(); - - // invalid header deps - let mut invalid_header_ids = Vec::new(); - for (tx_id, deps) in self.edges.header_deps.iter() { - for hash in deps { - if headers.contains(hash) { - invalid_header_ids.push((hash.clone(), tx_id.clone())); - break; - } - } - } - - for (blk_hash, id) in invalid_header_ids { - let entries = self.remove_entry_and_descendants(&id); - if !entries.is_empty() { - let reject = Reject::Resolve(OutPointError::InvalidHeader(blk_hash)); - let rejects = iter::repeat(reject).take(entries.len()); - conflicts.extend(entries.into_iter().zip(rejects)); - } - } - - conflicts - } - - /// sorted by ancestor score from higher to lower - pub fn score_sorted_iter(&self) -> impl Iterator { - self.inner.score_sorted_iter() - } - - /// find all ancestors from pool - pub fn calc_ancestors(&self, tx_short_id: &ProposalShortId) -> HashSet { - self.inner.calc_ancestors(tx_short_id) - } - - /// find all descendants from pool - pub fn calc_descendants(&self, tx_short_id: &ProposalShortId) -> HashSet { - self.inner.calc_descendants(tx_short_id) - } - - #[cfg(test)] - pub(crate) fn inner(&self) -> &SortedTxMap { - &self.inner - } - - pub(crate) fn clear(&mut self) { - self.edges.clear(); - self.inner.clear(); - } -} diff --git a/tx-pool/src/component/score_key.rs b/tx-pool/src/component/score_key.rs new file mode 100644 index 0000000000..011fb4000b --- /dev/null +++ b/tx-pool/src/component/score_key.rs @@ -0,0 +1,54 @@ +use ckb_types::{core::Capacity, packed::ProposalShortId}; +use std::cmp::Ordering; + +/// A struct to use as a sorted key +#[derive(Eq, PartialEq, Clone, Debug)] +pub struct AncestorsScoreSortKey { + pub fee: Capacity, + pub weight: u64, + pub id: ProposalShortId, + pub ancestors_fee: Capacity, + pub ancestors_weight: u64, + pub ancestors_size: usize, +} + +impl AncestorsScoreSortKey { + /// compare tx fee rate with ancestors fee rate and return the min one + pub(crate) fn min_fee_and_weight(&self) -> (Capacity, u64) { + // avoid division a_fee/a_weight > b_fee/b_weight + let tx_weight = u128::from(self.fee.as_u64()) * u128::from(self.ancestors_weight); + let ancestors_weight = u128::from(self.ancestors_fee.as_u64()) * u128::from(self.weight); + + if tx_weight < ancestors_weight { + (self.fee, self.weight) + } else { + (self.ancestors_fee, self.ancestors_weight) + } + } +} + +impl PartialOrd for AncestorsScoreSortKey { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for AncestorsScoreSortKey { + fn cmp(&self, other: &Self) -> Ordering { + // avoid division a_fee/a_weight > b_fee/b_weight + let (fee, weight) = self.min_fee_and_weight(); + let (other_fee, other_weight) = other.min_fee_and_weight(); + let self_weight = u128::from(fee.as_u64()) * u128::from(other_weight); + let other_weight = u128::from(other_fee.as_u64()) * u128::from(weight); + if self_weight == other_weight { + // if fee rate weight is same, then compare with ancestor weight + if self.ancestors_weight == other.ancestors_weight { + self.id.raw_data().cmp(&other.id.raw_data()) + } else { + self.ancestors_weight.cmp(&other.ancestors_weight) + } + } else { + self_weight.cmp(&other_weight) + } + } +} diff --git a/tx-pool/src/component/tests/mod.rs b/tx-pool/src/component/tests/mod.rs index 0f8bfcd719..d9a3529707 100644 --- a/tx-pool/src/component/tests/mod.rs +++ b/tx-pool/src/component/tests/mod.rs @@ -1,7 +1,6 @@ mod chunk; -mod container; mod pending; mod proposed; mod recent_reject; +mod score_key; mod util; -mod pool_map; \ No newline at end of file diff --git a/tx-pool/src/component/tests/pending.rs b/tx-pool/src/component/tests/pending.rs index d38ae3c1c0..b07e2e96e6 100644 --- a/tx-pool/src/component/tests/pending.rs +++ b/tx-pool/src/component/tests/pending.rs @@ -1,13 +1,19 @@ +use crate::component::edges::Edges; use crate::component::tests::util::{ build_tx, build_tx_with_dep, build_tx_with_header_dep, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE, }; -use crate::component::{entry::TxEntry, pending::PendingQueue}; +use crate::component::{ + entry::TxEntry, + pool_map::{PoolMap, Status}, +}; +use ckb_types::packed::OutPoint; use ckb_types::{h256, packed::Byte32, prelude::*}; use std::collections::HashSet; #[test] fn test_basic() { - let mut queue = PendingQueue::new(); + let mut pool = PoolMap::new(100); + assert_eq!(pool.size(), 0); let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&Byte32::zero(), 2)], 1); let tx2 = build_tx( vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 2)], @@ -15,30 +21,36 @@ fn test_basic() { ); let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); let entry2 = TxEntry::dummy_resolve(tx2.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(queue.add_entry(entry1.clone())); - assert!(queue.add_entry(entry2)); - assert!(queue.size() == 2); - assert!(queue.contains_key(&tx1.proposal_short_id())); - assert!(queue.contains_key(&tx2.proposal_short_id())); - - assert_eq!(queue.inputs_len(), 4); - assert_eq!(queue.outputs_len(), 4); - - assert_eq!(queue.get(&tx1.proposal_short_id()).unwrap(), &entry1); - assert_eq!(queue.get_tx(&tx2.proposal_short_id()).unwrap(), &tx2); - - let txs = queue.drain(); - assert!(queue.inner.is_empty()); - assert!(queue.deps.is_empty()); - assert!(queue.inputs.is_empty()); - assert!(queue.header_deps.is_empty()); - assert!(queue.outputs.is_empty()); - assert_eq!(txs, vec![tx1, tx2]); + assert!(pool.add_entry(entry1.clone(), Status::Pending).unwrap()); + assert!(pool.add_entry(entry2, Status::Pending).unwrap()); + assert!(pool.size() == 2); + assert!(pool.contains_key(&tx1.proposal_short_id())); + assert!(pool.contains_key(&tx2.proposal_short_id())); + + assert_eq!(pool.inputs_len(), 4); + assert_eq!(pool.outputs_len(), 4); + + assert_eq!( + pool.entries + .get_by_id(&tx1.proposal_short_id()) + .unwrap() + .inner, + entry1 + ); + assert_eq!(pool.get_tx(&tx2.proposal_short_id()).unwrap(), &tx2); + assert_eq!(pool.edges.deps.len(), 0); + + pool.clear(); + assert!(pool.entries.is_empty()); + assert!(pool.edges.deps.is_empty()); + assert!(pool.edges.inputs.is_empty()); + assert!(pool.edges.header_deps.is_empty()); + assert!(pool.edges.outputs.is_empty()); } #[test] fn test_resolve_conflict() { - let mut queue = PendingQueue::new(); + let mut pool = PoolMap::new(100); let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); let tx2 = build_tx( vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 1)], @@ -58,17 +70,17 @@ fn test_resolve_conflict() { let entry1 = TxEntry::dummy_resolve(tx1, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); let entry2 = TxEntry::dummy_resolve(tx2, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); let entry3 = TxEntry::dummy_resolve(tx3, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(queue.add_entry(entry1.clone())); - assert!(queue.add_entry(entry2.clone())); - assert!(queue.add_entry(entry3.clone())); + assert!(pool.add_entry(entry1.clone(), Status::Pending).unwrap()); + assert!(pool.add_entry(entry2.clone(), Status::Pending).unwrap()); + assert!(pool.add_entry(entry3.clone(), Status::Pending).unwrap()); - let conflicts = queue.resolve_conflict(&tx4); + let conflicts = pool.resolve_conflict(&tx4); assert_eq!( conflicts.into_iter().map(|i| i.0).collect::>(), HashSet::from_iter(vec![entry1, entry2]) ); - let conflicts = queue.resolve_conflict(&tx5); + let conflicts = pool.resolve_conflict(&tx5); assert_eq!( conflicts.into_iter().map(|i| i.0).collect::>(), HashSet::from_iter(vec![entry3]) @@ -77,7 +89,7 @@ fn test_resolve_conflict() { #[test] fn test_resolve_conflict_descendants() { - let mut queue = PendingQueue::new(); + let mut pool = PoolMap::new(1000); let tx1 = build_tx(vec![(&Byte32::zero(), 1)], 1); let tx3 = build_tx(vec![(&tx1.hash(), 0)], 2); let tx4 = build_tx(vec![(&tx3.hash(), 0)], 1); @@ -87,11 +99,11 @@ fn test_resolve_conflict_descendants() { let entry1 = TxEntry::dummy_resolve(tx1, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); let entry3 = TxEntry::dummy_resolve(tx3, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); let entry4 = TxEntry::dummy_resolve(tx4, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(queue.add_entry(entry1)); - assert!(queue.add_entry(entry3.clone())); - assert!(queue.add_entry(entry4.clone())); + assert!(pool.add_entry(entry1, Status::Pending).unwrap()); + assert!(pool.add_entry(entry3.clone(), Status::Pending).unwrap()); + assert!(pool.add_entry(entry4.clone(), Status::Pending).unwrap()); - let conflicts = queue.resolve_conflict(&tx2); + let conflicts = pool.resolve_conflict(&tx2); assert_eq!( conflicts.into_iter().map(|i| i.0).collect::>(), HashSet::from_iter(vec![entry3, entry4]) @@ -100,7 +112,7 @@ fn test_resolve_conflict_descendants() { #[test] fn test_resolve_conflict_header_dep() { - let mut queue = PendingQueue::new(); + let mut pool = PoolMap::new(1000); let header: Byte32 = h256!("0x1").pack(); let tx = build_tx_with_header_dep( @@ -112,17 +124,17 @@ fn test_resolve_conflict_header_dep() { let entry = TxEntry::dummy_resolve(tx, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); let entry1 = TxEntry::dummy_resolve(tx1, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(queue.add_entry(entry.clone())); - assert!(queue.add_entry(entry1.clone())); + assert!(pool.add_entry(entry.clone(), Status::Pending).unwrap()); + assert!(pool.add_entry(entry1.clone(), Status::Pending).unwrap()); - assert_eq!(queue.inputs_len(), 3); - assert_eq!(queue.header_deps_len(), 1); - assert_eq!(queue.outputs_len(), 2); + assert_eq!(pool.inputs_len(), 3); + assert_eq!(pool.header_deps_len(), 1); + assert_eq!(pool.outputs_len(), 2); let mut headers = HashSet::new(); headers.insert(header); - let conflicts = queue.resolve_conflict_header_dep(&headers); + let conflicts = pool.resolve_conflict_header_dep(&headers); assert_eq!( conflicts.into_iter().map(|i| i.0).collect::>(), HashSet::from_iter(vec![entry, entry1]) @@ -131,29 +143,29 @@ fn test_resolve_conflict_header_dep() { #[test] fn test_remove_entry() { - let mut queue = PendingQueue::new(); + let mut pool = PoolMap::new(1000); let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); let header: Byte32 = h256!("0x1").pack(); let tx2 = build_tx_with_header_dep(vec![(&h256!("0x2").pack(), 1)], vec![header], 1); let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); let entry2 = TxEntry::dummy_resolve(tx2.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(queue.add_entry(entry1.clone())); - assert!(queue.add_entry(entry2.clone())); + assert!(pool.add_entry(entry1.clone(), Status::Pending).unwrap()); + assert!(pool.add_entry(entry2.clone(), Status::Pending).unwrap()); - let removed = queue.remove_entry(&tx1.proposal_short_id()); + let removed = pool.remove_entry(&tx1.proposal_short_id()); assert_eq!(removed, Some(entry1)); - let removed = queue.remove_entry(&tx2.proposal_short_id()); + let removed = pool.remove_entry(&tx2.proposal_short_id()); assert_eq!(removed, Some(entry2)); - assert!(queue.inner.is_empty()); - assert!(queue.deps.is_empty()); - assert!(queue.inputs.is_empty()); - assert!(queue.header_deps.is_empty()); + assert!(pool.entries.is_empty()); + assert!(pool.edges.deps.is_empty()); + assert!(pool.edges.inputs.is_empty()); + assert!(pool.edges.header_deps.is_empty()); } #[test] fn test_remove_entries_by_filter() { - let mut queue = PendingQueue::new(); + let mut pool = PoolMap::new(1000); let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); let tx2 = build_tx( vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 1)], @@ -167,20 +179,22 @@ fn test_remove_entries_by_filter() { let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); let entry2 = TxEntry::dummy_resolve(tx2.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); let entry3 = TxEntry::dummy_resolve(tx3.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(queue.add_entry(entry1)); - assert!(queue.add_entry(entry2)); - assert!(queue.add_entry(entry3)); + assert!(pool.add_entry(entry1, Status::Pending).unwrap()); + assert!(pool.add_entry(entry2, Status::Pending).unwrap()); + assert!(pool.add_entry(entry3, Status::Pending).unwrap()); - queue.remove_entries_by_filter(|id, _tx_entry| id == &tx1.proposal_short_id()); + pool.remove_entries_by_filter(&Status::Pending, |id, _tx_entry| { + id == &tx1.proposal_short_id() + }); - assert!(!queue.contains_key(&tx1.proposal_short_id())); - assert!(queue.contains_key(&tx2.proposal_short_id())); - assert!(queue.contains_key(&tx3.proposal_short_id())); + assert!(!pool.contains_key(&tx1.proposal_short_id())); + assert!(pool.contains_key(&tx2.proposal_short_id())); + assert!(pool.contains_key(&tx3.proposal_short_id())); } #[test] fn test_fill_proposals() { - let mut queue = PendingQueue::new(); + let mut pool = PoolMap::new(1000); let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); let tx2 = build_tx( vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 1)], @@ -194,36 +208,54 @@ fn test_fill_proposals() { let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); let entry2 = TxEntry::dummy_resolve(tx2.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); let entry3 = TxEntry::dummy_resolve(tx3.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(queue.add_entry(entry1)); - assert!(queue.add_entry(entry2)); - assert!(queue.add_entry(entry3)); + assert!(pool.add_entry(entry1, Status::Pending).unwrap()); + assert!(pool.add_entry(entry2, Status::Pending).unwrap()); + assert!(pool.add_entry(entry3, Status::Pending).unwrap()); - assert_eq!(queue.inputs_len(), 5); - assert_eq!(queue.deps_len(), 1); - assert_eq!(queue.outputs_len(), 7); + assert_eq!(pool.inputs_len(), 5); + assert_eq!(pool.deps_len(), 1); + assert_eq!(pool.outputs_len(), 7); let id1 = tx1.proposal_short_id(); let id2 = tx2.proposal_short_id(); let id3 = tx3.proposal_short_id(); let mut ret = HashSet::new(); - queue.fill_proposals(10, &HashSet::new(), &mut ret); + pool.fill_proposals(10, &HashSet::new(), &mut ret, &Status::Pending); assert_eq!( ret, HashSet::from_iter(vec![id1.clone(), id2.clone(), id3.clone()]) ); let mut ret = HashSet::new(); - queue.fill_proposals(1, &HashSet::new(), &mut ret); + pool.fill_proposals(1, &HashSet::new(), &mut ret, &Status::Pending); assert_eq!(ret, HashSet::from_iter(vec![id1.clone()])); let mut ret = HashSet::new(); - queue.fill_proposals(2, &HashSet::new(), &mut ret); + pool.fill_proposals(2, &HashSet::new(), &mut ret, &Status::Pending); assert_eq!(ret, HashSet::from_iter(vec![id1.clone(), id2.clone()])); let mut ret = HashSet::new(); let mut exclusion = HashSet::new(); exclusion.insert(id2); - queue.fill_proposals(2, &exclusion, &mut ret); + pool.fill_proposals(2, &exclusion, &mut ret, &Status::Pending); assert_eq!(ret, HashSet::from_iter(vec![id1, id3])); } + +#[test] +fn test_edges() { + let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); + let tx2 = build_tx(vec![(&h256!("0x1").pack(), 1)], 1); + + let short_id1 = tx1.proposal_short_id(); + let short_id2 = tx2.proposal_short_id(); + let mut edges = Edges::default(); + let outpoint = OutPoint::default(); + edges.insert_deps(outpoint.clone(), short_id1.clone()); + edges.insert_deps(outpoint.clone(), short_id2.clone()); + assert!(edges.deps.contains_key(&outpoint)); + edges.delete_txid_by_dep(outpoint.clone(), &short_id1); + assert!(!edges.deps.is_empty()); + edges.delete_txid_by_dep(outpoint, &short_id2); + assert!(edges.deps.is_empty()); +} diff --git a/tx-pool/src/component/tests/pool_map.rs b/tx-pool/src/component/tests/pool_map.rs deleted file mode 100644 index 7fcbd9e1a6..0000000000 --- a/tx-pool/src/component/tests/pool_map.rs +++ /dev/null @@ -1,236 +0,0 @@ -use crate::component::tests::util::{ - build_tx, build_tx_with_dep, build_tx_with_header_dep, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE, -}; -use crate::component::{ - entry::TxEntry, - pool_map::{PoolEntry, PoolMap, Status}, -}; -use ckb_types::{h256, packed::Byte32, prelude::*}; -use std::collections::HashSet; - -#[test] -fn test_basic() { - let mut pool = PoolMap::new(100); - assert_eq!(pool.size(), 0); - let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&Byte32::zero(), 2)], 1); - let tx2 = build_tx( - vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 2)], - 3, - ); - let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - let entry2 = TxEntry::dummy_resolve(tx2.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(pool.add_entry(entry1.clone(), Status::Pending)); - assert!(pool.add_entry(entry2, Status::Pending)); - assert!(pool.size() == 2); - assert!(pool.contains_key(&tx1.proposal_short_id())); - assert!(pool.contains_key(&tx2.proposal_short_id())); - - assert_eq!(pool.inputs_len(), 4); - assert_eq!(pool.outputs_len(), 4); - - assert_eq!(pool.entries.get_by_id(&tx1.proposal_short_id()).unwrap().inner, entry1); - assert_eq!(pool.get_tx(&tx2.proposal_short_id()).unwrap(), &tx2); - - let txs = pool.drain(); - assert!(pool.entries.is_empty()); - assert!(pool.deps.is_empty()); - assert!(pool.inputs.is_empty()); - assert!(pool.header_deps.is_empty()); - assert!(pool.outputs.is_empty()); - assert_eq!(txs, vec![tx1, tx2]); -} - -#[test] -fn test_resolve_conflict() { - let mut pool = PoolMap::new(100); - let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); - let tx2 = build_tx( - vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 1)], - 3, - ); - let tx3 = build_tx_with_dep( - vec![(&h256!("0x4").pack(), 1)], - vec![(&h256!("0x5").pack(), 1)], - 3, - ); - let tx4 = build_tx( - vec![(&h256!("0x2").pack(), 1), (&h256!("0x1").pack(), 1)], - 3, - ); - let tx5 = build_tx(vec![(&h256!("0x5").pack(), 1)], 3); - - let entry1 = TxEntry::dummy_resolve(tx1, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - let entry2 = TxEntry::dummy_resolve(tx2, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - let entry3 = TxEntry::dummy_resolve(tx3, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(pool.add_entry(entry1.clone(), Status::Pending)); - assert!(pool.add_entry(entry2.clone(), Status::Pending)); - assert!(pool.add_entry(entry3.clone(), Status::Pending)); - - let conflicts = pool.resolve_conflict(&tx4); - assert_eq!( - conflicts.into_iter().map(|i| i.0).collect::>(), - HashSet::from_iter(vec![entry1, entry2]) - ); - - let conflicts = pool.resolve_conflict(&tx5); - assert_eq!( - conflicts.into_iter().map(|i| i.0).collect::>(), - HashSet::from_iter(vec![entry3]) - ); -} - -#[test] -fn test_resolve_conflict_descendants() { - let mut pool = PoolMap::new(1000); - let tx1 = build_tx(vec![(&Byte32::zero(), 1)], 1); - let tx3 = build_tx(vec![(&tx1.hash(), 0)], 2); - let tx4 = build_tx(vec![(&tx3.hash(), 0)], 1); - - let tx2 = build_tx(vec![(&tx1.hash(), 0)], 1); - - let entry1 = TxEntry::dummy_resolve(tx1, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - let entry3 = TxEntry::dummy_resolve(tx3, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - let entry4 = TxEntry::dummy_resolve(tx4, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(pool.add_entry(entry1, Status::Pending)); - assert!(pool.add_entry(entry3.clone(), Status::Pending)); - assert!(pool.add_entry(entry4.clone(), Status::Pending)); - - let conflicts = pool.resolve_conflict(&tx2); - assert_eq!( - conflicts.into_iter().map(|i| i.0).collect::>(), - HashSet::from_iter(vec![entry3, entry4]) - ); -} - -#[test] -fn test_resolve_conflict_header_dep() { - let mut pool = PoolMap::new(1000); - - let header: Byte32 = h256!("0x1").pack(); - let tx = build_tx_with_header_dep( - vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], - vec![header.clone()], - 1, - ); - let tx1 = build_tx(vec![(&tx.hash(), 0)], 1); - - let entry = TxEntry::dummy_resolve(tx, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - let entry1 = TxEntry::dummy_resolve(tx1, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(pool.add_entry(entry.clone(), Status::Pending)); - assert!(pool.add_entry(entry1.clone(), Status::Pending)); - - assert_eq!(pool.inputs_len(), 3); - assert_eq!(pool.header_deps_len(), 1); - assert_eq!(pool.outputs_len(), 2); - - let mut headers = HashSet::new(); - headers.insert(header); - - let conflicts = pool.resolve_conflict_header_dep(&headers); - assert_eq!( - conflicts.into_iter().map(|i| i.0).collect::>(), - HashSet::from_iter(vec![entry, entry1]) - ); -} - - -#[test] -fn test_remove_entry() { - let mut pool = PoolMap::new(1000); - let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); - let header: Byte32 = h256!("0x1").pack(); - let tx2 = build_tx_with_header_dep(vec![(&h256!("0x2").pack(), 1)], vec![header], 1); - - let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - let entry2 = TxEntry::dummy_resolve(tx2.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(pool.add_entry(entry1.clone(), Status::Pending)); - assert!(pool.add_entry(entry2.clone(), Status::Pending)); - - let removed = pool.remove_entry(&tx1.proposal_short_id()); - assert_eq!(removed, Some(entry1)); - let removed = pool.remove_entry(&tx2.proposal_short_id()); - assert_eq!(removed, Some(entry2)); - assert!(pool.entries.is_empty()); - assert!(pool.deps.is_empty()); - assert!(pool.inputs.is_empty()); - assert!(pool.header_deps.is_empty()); -} - - -#[test] -fn test_remove_entries_by_filter() { - let mut pool = PoolMap::new(1000); - let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); - let tx2 = build_tx( - vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 1)], - 3, - ); - let tx3 = build_tx_with_dep( - vec![(&h256!("0x4").pack(), 1)], - vec![(&h256!("0x5").pack(), 1)], - 3, - ); - let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - let entry2 = TxEntry::dummy_resolve(tx2.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - let entry3 = TxEntry::dummy_resolve(tx3.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(pool.add_entry(entry1, Status::Pending)); - assert!(pool.add_entry(entry2, Status::Pending)); - assert!(pool.add_entry(entry3, Status::Pending)); - - pool.remove_entries_by_filter(|id, _tx_entry| id == &tx1.proposal_short_id()); - - assert!(!pool.contains_key(&tx1.proposal_short_id())); - assert!(pool.contains_key(&tx2.proposal_short_id())); - assert!(pool.contains_key(&tx3.proposal_short_id())); -} - - -#[test] -fn test_fill_proposals() { - let mut pool = PoolMap::new(1000); - let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); - let tx2 = build_tx( - vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 1)], - 3, - ); - let tx3 = build_tx_with_dep( - vec![(&h256!("0x4").pack(), 1)], - vec![(&h256!("0x5").pack(), 1)], - 3, - ); - let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - let entry2 = TxEntry::dummy_resolve(tx2.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - let entry3 = TxEntry::dummy_resolve(tx3.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(pool.add_entry(entry1, Status::Pending)); - assert!(pool.add_entry(entry2, Status::Pending)); - assert!(pool.add_entry(entry3, Status::Pending)); - - assert_eq!(pool.inputs_len(), 5); - assert_eq!(pool.deps_len(), 1); - assert_eq!(pool.outputs_len(), 7); - - let id1 = tx1.proposal_short_id(); - let id2 = tx2.proposal_short_id(); - let id3 = tx3.proposal_short_id(); - - let mut ret = HashSet::new(); - pool.fill_proposals(10, &HashSet::new(), &mut ret, &Status::Pending); - assert_eq!( - ret, - HashSet::from_iter(vec![id1.clone(), id2.clone(), id3.clone()]) - ); - - let mut ret = HashSet::new(); - pool.fill_proposals(1, &HashSet::new(), &mut ret, &Status::Pending); - assert_eq!(ret, HashSet::from_iter(vec![id1.clone()])); - - let mut ret = HashSet::new(); - pool.fill_proposals(2, &HashSet::new(), &mut ret, &Status::Pending); - assert_eq!(ret, HashSet::from_iter(vec![id1.clone(), id2.clone()])); - - let mut ret = HashSet::new(); - let mut exclusion = HashSet::new(); - exclusion.insert(id2); - pool.fill_proposals(2, &exclusion, &mut ret, &Status::Pending); - assert_eq!(ret, HashSet::from_iter(vec![id1, id3])); -} diff --git a/tx-pool/src/component/tests/proposed.rs b/tx-pool/src/component/tests/proposed.rs index 7c784b36b4..ff5fa67866 100644 --- a/tx-pool/src/component/tests/proposed.rs +++ b/tx-pool/src/component/tests/proposed.rs @@ -2,7 +2,7 @@ use crate::component::tests::util::{ build_tx, build_tx_with_dep, build_tx_with_header_dep, DEFAULT_MAX_ANCESTORS_COUNT, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE, }; -use crate::component::{entry::TxEntry, proposed::ProposedPool}; +use crate::component::{entry::TxEntry, pool_map::PoolMap}; use ckb_types::{ bytes::Bytes, core::{ @@ -49,16 +49,16 @@ fn test_add_entry() { let tx1_hash = tx1.hash(); let tx2 = build_tx(vec![(&tx1_hash, 0)], 1); - let mut pool = ProposedPool::new(DEFAULT_MAX_ANCESTORS_COUNT); + let mut pool = PoolMap::new(DEFAULT_MAX_ANCESTORS_COUNT); - pool.add_entry(TxEntry::new( + pool.add_proposed(TxEntry::new( dummy_resolve(tx1.clone(), |_| None), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE, )) .unwrap(); - pool.add_entry(TxEntry::new( + pool.add_proposed(TxEntry::new( dummy_resolve(tx2, |_| None), MOCK_CYCLES, MOCK_FEE, @@ -91,21 +91,20 @@ fn test_add_entry_from_detached() { let id2 = entry2.proposal_short_id(); let id3 = entry3.proposal_short_id(); - let mut pool = ProposedPool::new(DEFAULT_MAX_ANCESTORS_COUNT); - pool.add_entry(entry1.clone()).unwrap(); - pool.add_entry(entry2.clone()).unwrap(); - pool.add_entry(entry3).unwrap(); + let mut pool = PoolMap::new(DEFAULT_MAX_ANCESTORS_COUNT); + pool.add_proposed(entry1.clone()).unwrap(); + pool.add_proposed(entry2.clone()).unwrap(); + pool.add_proposed(entry3).unwrap(); assert_eq!(pool.size(), 3); assert_eq!(pool.edges.outputs_len(), 3); assert_eq!(pool.edges.inputs_len(), 4); - assert_eq!(pool.inner().sorted_index.len(), 3); + assert_eq!(pool.size(), 3); let expected = vec![(id1.clone(), 1), (id2.clone(), 2), (id3.clone(), 3)]; let got = pool - .inner() - .sorted_index + .entries .iter() .map(|(_, key)| (key.id.clone(), key.score.ancestors_size)) .collect::>(); @@ -114,81 +113,78 @@ fn test_add_entry_from_detached() { // check link { - assert!(pool.inner().links.get_parents(&id1).unwrap().is_empty()); + assert!(pool.links.get_parents(&id1).unwrap().is_empty()); assert_eq!( - pool.inner().links.get_children(&id1).unwrap(), + pool.links.get_children(&id1).unwrap(), &HashSet::from_iter(vec![id2.clone()].into_iter()) ); assert_eq!( - pool.inner().links.get_parents(&id2).unwrap(), + pool.links.get_parents(&id2).unwrap(), &HashSet::from_iter(vec![id1.clone()].into_iter()) ); assert_eq!( - pool.inner() - .links + pool.links .get_children(&entry2.proposal_short_id()) .unwrap(), &HashSet::from_iter(vec![id3.clone()].into_iter()) ); assert_eq!( - pool.inner().links.get_parents(&id3).unwrap(), + pool.links.get_parents(&id3).unwrap(), &HashSet::from_iter(vec![id2.clone()].into_iter()) ); - assert!(pool.inner().links.get_children(&id3).unwrap().is_empty()); + assert!(pool.links.get_children(&id3).unwrap().is_empty()); } pool.remove_committed_tx(&tx1); assert_eq!(pool.edges.outputs_len(), 2); assert_eq!(pool.edges.inputs_len(), 2); - assert_eq!(pool.inner().sorted_index.len(), 2); + assert_eq!(pool.entries.len(), 2); let left = vec![(id2.clone(), 1), (id3.clone(), 2)]; let got = pool - .inner() - .sorted_index + .entries .iter() .map(|(_, key)| (key.id.clone(), key.score.ancestors_size)) .collect::>(); assert_eq!(left, got); assert!(pool - .inner() .links .get_parents(&entry2.proposal_short_id()) .unwrap() .is_empty()); - assert!(pool.add_entry(entry1).unwrap()); - for (idx, (_, key)) in pool.inner().sorted_index.iter().enumerate() { - assert_eq!(key.id, expected[idx].0); - assert_eq!(key.score.ancestors_size, expected[idx].1); + assert!(pool.add_proposed(entry1).unwrap()); + + for (idx, (_, entry)) in pool.entries.iter().enumerate() { + assert_eq!(entry.id, expected[idx].0); + assert_eq!(entry.score.ancestors_size, expected[idx].1); } { - assert!(pool.inner().links.get_parents(&id1).unwrap().is_empty()); + assert!(pool.links.get_parents(&id1).unwrap().is_empty()); assert_eq!( - pool.inner().links.get_children(&id1).unwrap(), + pool.links.get_children(&id1).unwrap(), &HashSet::from_iter(vec![id2.clone()].into_iter()) ); assert_eq!( - pool.inner().links.get_parents(&id2).unwrap(), + pool.links.get_parents(&id2).unwrap(), &HashSet::from_iter(vec![id1].into_iter()) ); assert_eq!( - pool.inner() - .links + pool.links .get_children(&entry2.proposal_short_id()) .unwrap(), &HashSet::from_iter(vec![id3.clone()].into_iter()) ); assert_eq!( - pool.inner().links.get_parents(&id3).unwrap(), + pool.links.get_parents(&id3).unwrap(), &HashSet::from_iter(vec![id2].into_iter()) ); - assert!(pool.inner().links.get_children(&id3).unwrap().is_empty()); + assert!(pool.links.get_children(&id3).unwrap().is_empty()); } } @@ -200,16 +196,16 @@ fn test_add_roots() { 3, ); - let mut pool = ProposedPool::new(DEFAULT_MAX_ANCESTORS_COUNT); + let mut pool = PoolMap::new(DEFAULT_MAX_ANCESTORS_COUNT); - pool.add_entry(TxEntry::new( + pool.add_proposed(TxEntry::new( dummy_resolve(tx1.clone(), |_| None), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE, )) .unwrap(); - pool.add_entry(TxEntry::new( + pool.add_proposed(TxEntry::new( dummy_resolve(tx2, |_| None), MOCK_CYCLES, MOCK_FEE, @@ -240,37 +236,37 @@ fn test_add_no_roots() { let tx3_hash = tx3.hash(); let tx5 = build_tx(vec![(&tx1_hash, 2), (&tx3_hash, 0)], 2); - let mut pool = ProposedPool::new(DEFAULT_MAX_ANCESTORS_COUNT); + let mut pool = PoolMap::new(DEFAULT_MAX_ANCESTORS_COUNT); - pool.add_entry(TxEntry::new( + pool.add_proposed(TxEntry::new( dummy_resolve(tx1.clone(), |_| None), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE, )) .unwrap(); - pool.add_entry(TxEntry::new( + pool.add_proposed(TxEntry::new( dummy_resolve(tx2, |_| None), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE, )) .unwrap(); - pool.add_entry(TxEntry::new( + pool.add_proposed(TxEntry::new( dummy_resolve(tx3, |_| None), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE, )) .unwrap(); - pool.add_entry(TxEntry::new( + pool.add_proposed(TxEntry::new( dummy_resolve(tx4, |_| None), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE, )) .unwrap(); - pool.add_entry(TxEntry::new( + pool.add_proposed(TxEntry::new( dummy_resolve(tx5, |_| None), MOCK_CYCLES, MOCK_FEE, @@ -293,26 +289,26 @@ fn test_sorted_by_tx_fee_rate() { let tx2 = build_tx(vec![(&Byte32::zero(), 2)], 1); let tx3 = build_tx(vec![(&Byte32::zero(), 3)], 1); - let mut pool = ProposedPool::new(DEFAULT_MAX_ANCESTORS_COUNT); + let mut pool = PoolMap::new(DEFAULT_MAX_ANCESTORS_COUNT); let cycles = 5_000_000; let size = 200; - pool.add_entry(TxEntry::dummy_resolve( + pool.add_proposed(TxEntry::dummy_resolve( tx1.clone(), cycles, Capacity::shannons(100), size, )) .unwrap(); - pool.add_entry(TxEntry::dummy_resolve( + pool.add_proposed(TxEntry::dummy_resolve( tx2.clone(), cycles, Capacity::shannons(300), size, )) .unwrap(); - pool.add_entry(TxEntry::dummy_resolve( + pool.add_proposed(TxEntry::dummy_resolve( tx3.clone(), cycles, Capacity::shannons(200), @@ -337,33 +333,33 @@ fn test_sorted_by_ancestors_score() { let tx3 = build_tx(vec![(&tx1_hash, 2)], 1); let tx4 = build_tx(vec![(&tx2_hash, 1)], 1); - let mut pool = ProposedPool::new(DEFAULT_MAX_ANCESTORS_COUNT); + let mut pool = PoolMap::new(DEFAULT_MAX_ANCESTORS_COUNT); let cycles = 5_000_000; let size = 200; - pool.add_entry(TxEntry::dummy_resolve( + pool.add_proposed(TxEntry::dummy_resolve( tx1.clone(), cycles, Capacity::shannons(100), size, )) .unwrap(); - pool.add_entry(TxEntry::dummy_resolve( + pool.add_proposed(TxEntry::dummy_resolve( tx2.clone(), cycles, Capacity::shannons(300), size, )) .unwrap(); - pool.add_entry(TxEntry::dummy_resolve( + pool.add_proposed(TxEntry::dummy_resolve( tx3.clone(), cycles, Capacity::shannons(200), size, )) .unwrap(); - pool.add_entry(TxEntry::dummy_resolve( + pool.add_proposed(TxEntry::dummy_resolve( tx4.clone(), cycles, Capacity::shannons(400), @@ -395,7 +391,7 @@ fn test_sorted_by_ancestors_score_competitive() { let tx2_3_hash = tx2_3.hash(); let tx2_4 = build_tx(vec![(&tx2_3_hash, 0)], 1); - let mut pool = ProposedPool::new(DEFAULT_MAX_ANCESTORS_COUNT); + let mut pool = PoolMap::new(DEFAULT_MAX_ANCESTORS_COUNT); // Choose 5_000_839, so the weight is 853.0001094046, which will not lead to carry when // calculating the weight for a package. @@ -403,7 +399,7 @@ fn test_sorted_by_ancestors_score_competitive() { let size = 200; for &tx in &[&tx1, &tx2, &tx3, &tx2_1, &tx2_2, &tx2_3, &tx2_4] { - pool.add_entry(TxEntry::dummy_resolve( + pool.add_proposed(TxEntry::dummy_resolve( tx.clone(), cycles, Capacity::shannons(200), @@ -430,33 +426,33 @@ fn test_get_ancestors() { let tx3 = build_tx(vec![(&tx1_hash, 1)], 1); let tx4 = build_tx(vec![(&tx2_hash, 0)], 1); - let mut pool = ProposedPool::new(DEFAULT_MAX_ANCESTORS_COUNT); + let mut pool = PoolMap::new(DEFAULT_MAX_ANCESTORS_COUNT); let cycles = 5_000_000; let size = 200; - pool.add_entry(TxEntry::dummy_resolve( + pool.add_proposed(TxEntry::dummy_resolve( tx1.clone(), cycles, Capacity::shannons(100), size, )) .unwrap(); - pool.add_entry(TxEntry::dummy_resolve( + pool.add_proposed(TxEntry::dummy_resolve( tx2.clone(), cycles, Capacity::shannons(300), size, )) .unwrap(); - pool.add_entry(TxEntry::dummy_resolve( + pool.add_proposed(TxEntry::dummy_resolve( tx3.clone(), cycles, Capacity::shannons(200), size, )) .unwrap(); - pool.add_entry(TxEntry::dummy_resolve( + pool.add_proposed(TxEntry::dummy_resolve( tx4.clone(), cycles, Capacity::shannons(400), @@ -561,9 +557,9 @@ fn test_dep_group() { } }; - let mut pool = ProposedPool::new(DEFAULT_MAX_ANCESTORS_COUNT); + let mut pool = PoolMap::new(DEFAULT_MAX_ANCESTORS_COUNT); for tx in &[&tx1, &tx2, &tx3] { - pool.add_entry(TxEntry::new( + pool.add_proposed(TxEntry::new( dummy_resolve((*tx).clone(), get_cell_data), MOCK_CYCLES, MOCK_FEE, @@ -572,7 +568,7 @@ fn test_dep_group() { .unwrap(); } - let get_deps_len = |pool: &ProposedPool, out_point: &OutPoint| -> usize { + let get_deps_len = |pool: &PoolMap, out_point: &OutPoint| -> usize { pool.edges .deps .get(out_point) @@ -591,7 +587,7 @@ fn test_dep_group() { #[test] fn test_resolve_conflict_header_dep() { - let mut pool = ProposedPool::new(DEFAULT_MAX_ANCESTORS_COUNT); + let mut pool = PoolMap::new(DEFAULT_MAX_ANCESTORS_COUNT); let header: Byte32 = h256!("0x1").pack(); let tx = build_tx_with_header_dep( @@ -602,7 +598,7 @@ fn test_resolve_conflict_header_dep() { let entry = TxEntry::dummy_resolve(tx, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(pool.add_entry(entry.clone()).is_ok()); + assert!(pool.add_proposed(entry.clone()).is_ok()); let mut headers = HashSet::new(); headers.insert(header); @@ -633,10 +629,10 @@ fn test_disordered_remove_committed_tx() { MOCK_SIZE, ); - let mut pool = ProposedPool::new(DEFAULT_MAX_ANCESTORS_COUNT); + let mut pool = PoolMap::new(DEFAULT_MAX_ANCESTORS_COUNT); - pool.add_entry(entry1).unwrap(); - pool.add_entry(entry2).unwrap(); + pool.add_proposed(entry1).unwrap(); + pool.add_proposed(entry2).unwrap(); assert_eq!(pool.edges.outputs_len(), 2); assert_eq!(pool.edges.inputs_len(), 2); @@ -650,7 +646,7 @@ fn test_disordered_remove_committed_tx() { #[test] fn test_max_ancestors() { - let mut pool = ProposedPool::new(1); + let mut pool = PoolMap::new(1); let tx1 = build_tx(vec![(&Byte32::zero(), 0)], 1); let tx1_id = tx1.proposal_short_id(); let tx1_hash = tx1.hash(); @@ -659,15 +655,15 @@ fn test_max_ancestors() { let entry1 = TxEntry::dummy_resolve(tx1, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); let entry2 = TxEntry::dummy_resolve(tx2, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(pool.add_entry(entry1).is_ok()); - assert!(pool.add_entry(entry2).is_err()); + assert!(pool.add_proposed(entry1).is_ok()); + assert!(pool.add_proposed(entry2).is_err()); assert_eq!( - pool.inner() + pool.links .get_children(&tx1_id) .map(|children| children.is_empty()), Some(true) ); - assert!(pool.inner().calc_descendants(&tx1_id).is_empty()); + assert!(pool.calc_descendants(&tx1_id).is_empty()); assert_eq!(pool.edges.inputs_len(), 1); assert_eq!(pool.edges.outputs_len(), 1); @@ -675,7 +671,7 @@ fn test_max_ancestors() { #[test] fn test_max_ancestors_with_dep() { - let mut pool = ProposedPool::new(1); + let mut pool = PoolMap::new(1); let tx1 = build_tx_with_dep( vec![(&Byte32::zero(), 0)], vec![(&h256!("0x1").pack(), 0)], @@ -687,14 +683,14 @@ fn test_max_ancestors_with_dep() { let entry1 = TxEntry::dummy_resolve(tx1, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); let entry2 = TxEntry::dummy_resolve(tx2, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(pool.add_entry(entry1).is_ok()); - assert!(pool.add_entry(entry2).is_err()); - assert_eq!(pool.inner().deps().len(), 1); + assert!(pool.add_proposed(entry1).is_ok()); + assert!(pool.add_proposed(entry2).is_err()); + assert_eq!(pool.edges.deps.len(), 1); assert!(pool - .inner() - .deps() + .edges + .deps .contains_key(&OutPoint::new(h256!("0x1").pack(), 0))); - assert!(pool.inner().calc_descendants(&tx1_id).is_empty()); + assert!(pool.calc_descendants(&tx1_id).is_empty()); assert_eq!(pool.edges.inputs_len(), 1); assert_eq!(pool.edges.outputs_len(), 1); diff --git a/tx-pool/src/component/tests/container.rs b/tx-pool/src/component/tests/score_key.rs similarity index 94% rename from tx-pool/src/component/tests/container.rs rename to tx-pool/src/component/tests/score_key.rs index 5c06e9601f..22da657f84 100644 --- a/tx-pool/src/component/tests/container.rs +++ b/tx-pool/src/component/tests/score_key.rs @@ -6,10 +6,7 @@ use ckb_types::{ }; use std::mem::size_of; -use crate::component::{ - container::{AncestorsScoreSortKey, SortedTxMap}, - entry::TxEntry, -}; +use crate::component::{entry::TxEntry, pool_map::PoolMap, score_key::AncestorsScoreSortKey}; const DEFAULT_MAX_ANCESTORS_COUNT: usize = 125; @@ -98,7 +95,7 @@ fn test_ancestors_sorted_key_order() { #[test] fn test_remove_entry() { - let mut map = SortedTxMap::new(DEFAULT_MAX_ANCESTORS_COUNT); + let mut map = PoolMap::new(DEFAULT_MAX_ANCESTORS_COUNT); let tx1 = TxEntry::dummy_resolve( TransactionBuilder::default().build(), 100, @@ -144,9 +141,9 @@ fn test_remove_entry() { let tx1_id = tx1.proposal_short_id(); let tx2_id = tx2.proposal_short_id(); let tx3_id = tx3.proposal_short_id(); - map.add_entry(tx1).unwrap(); - map.add_entry(tx2).unwrap(); - map.add_entry(tx3).unwrap(); + map.add_proposed(tx1).unwrap(); + map.add_proposed(tx2).unwrap(); + map.add_proposed(tx3).unwrap(); let descendants_set = map.calc_descendants(&tx1_id); assert!(descendants_set.contains(&tx2_id)); assert!(descendants_set.contains(&tx3_id)); @@ -171,7 +168,7 @@ fn test_remove_entry() { #[test] fn test_remove_entry_and_descendants() { - let mut map = SortedTxMap::new(DEFAULT_MAX_ANCESTORS_COUNT); + let mut map = PoolMap::new(DEFAULT_MAX_ANCESTORS_COUNT); let tx1 = TxEntry::dummy_resolve( TransactionBuilder::default().build(), 100, @@ -217,9 +214,9 @@ fn test_remove_entry_and_descendants() { let tx1_id = tx1.proposal_short_id(); let tx2_id = tx2.proposal_short_id(); let tx3_id = tx3.proposal_short_id(); - map.add_entry(tx1).unwrap(); - map.add_entry(tx2).unwrap(); - map.add_entry(tx3).unwrap(); + map.add_proposed(tx1).unwrap(); + map.add_proposed(tx2).unwrap(); + map.add_proposed(tx3).unwrap(); let descendants_set = map.calc_descendants(&tx1_id); assert!(descendants_set.contains(&tx2_id)); assert!(descendants_set.contains(&tx3_id)); diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index c9dcc3b7fd..c01ad22b64 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -3,23 +3,23 @@ extern crate rustc_hash; extern crate slab; use super::component::{commit_txs_scanner::CommitTxsScanner, TxEntry}; use crate::callback::Callbacks; -use crate::component::pending::PendingQueue; -use crate::component::proposed::ProposedPool; +use crate::component::pool_map::{PoolEntry, PoolMap, Status}; use crate::component::recent_reject::RecentReject; use crate::error::Reject; -use crate::component::pool_map::{PoolMap, Status}; use crate::util::verify_rtx; use ckb_app_config::TxPoolConfig; -use ckb_logger::{debug, error, trace, warn}; +use ckb_logger::{debug, error, warn}; use ckb_snapshot::Snapshot; use ckb_store::ChainStore; use ckb_types::{ core::{ cell::{resolve_transaction, OverlayCellChecker, OverlayCellProvider, ResolvedTransaction}, + cell::{CellChecker, CellMetaBuilder, CellProvider, CellStatus}, tx_pool::{TxPoolEntryInfo, TxPoolIds}, Cycle, TransactionView, UncleBlockView, }, - packed::{Byte32, ProposalShortId}, + packed::{Byte32, OutPoint, ProposalShortId}, + prelude::*, }; use ckb_verification::{cache::CacheEntry, TxVerifyEnv}; use lru::LruCache; @@ -28,44 +28,9 @@ use std::sync::Arc; const COMMITTED_HASH_CACHE_SIZE: usize = 100_000; -// limit the size of the pool by sorting out tx based on EvictKey. -macro_rules! evict_for_trim_size { - ($self:ident, $pool:expr, $callbacks:expr) => { - if let Some(id) = $pool - .iter() - .min_by_key(|(_id, entry)| entry.as_evict_key()) - .map(|(id, _)| id) - .cloned() - { - let removed = $pool.remove_entry_and_descendants(&id); - for entry in removed { - let tx_hash = entry.transaction().hash(); - debug!( - "removed by size limit {} timestamp({})", - tx_hash, entry.timestamp - ); - let reject = Reject::Full(format!( - "the fee_rate for this transaction is: {}", - entry.fee_rate() - )); - $callbacks.call_reject($self, &entry, reject); - } - } - }; -} - -type ConflictEntry = (TxEntry, Reject); - /// Tx-pool implementation pub struct TxPool { pub(crate) config: TxPoolConfig, - /// The short id that has not been proposed - pub(crate) pending: PendingQueue, - /// The proposal gap - pub(crate) gap: PendingQueue, - /// Tx pool that finely for commit - pub(crate) proposed: ProposedPool, - pub(crate) pool_map: PoolMap, /// cache for committed transactions hash pub(crate) committed_txs_hash_cache: LruCache, @@ -87,9 +52,6 @@ impl TxPool { let recent_reject = Self::build_recent_reject(&config); let expiry = config.expiry_hours as u64 * 60 * 60 * 1000; TxPool { - pending: PendingQueue::new(), - gap: PendingQueue::new(), - proposed: ProposedPool::new(config.max_ancestors_count), pool_map: PoolMap::new(config.max_ancestors_count), committed_txs_hash_cache: LruCache::new(COMMITTED_HASH_CACHE_SIZE), total_tx_size: 0, @@ -102,18 +64,22 @@ impl TxPool { } /// Tx-pool owned snapshot, it may not consistent with chain cause tx-pool update snapshot asynchronously - pub fn snapshot(&self) -> &Snapshot { + pub(crate) fn snapshot(&self) -> &Snapshot { &self.snapshot } /// Makes a clone of the `Arc` - pub fn cloned_snapshot(&self) -> Arc { + pub(crate) fn cloned_snapshot(&self) -> Arc { Arc::clone(&self.snapshot) } - /// Whether Tx-pool reach size limit - pub fn reach_size_limit(&self, tx_size: usize) -> bool { - (self.total_tx_size + tx_size) > self.config.max_tx_pool_size + fn get_by_status(&self, status: &Status) -> Vec<&PoolEntry> { + self.pool_map.entries.get_by_status(status) + } + + /// Get tx-pool size + pub fn status_size(&self, status: &Status) -> usize { + self.get_by_status(status).len() } /// Update size and cycles statics for add tx @@ -143,123 +109,38 @@ impl TxPool { self.total_tx_cycles = total_tx_cycles; } - /// Add tx to pending pool + /// Add tx with pending status /// If did have this value present, false is returned. - pub fn add_pending(&mut self, entry: TxEntry) -> bool { - if self.gap.contains_key(&entry.proposal_short_id()) { - return false; - } - trace!("add_pending {}", entry.transaction().hash()); - self.pending.add_entry(entry) - } - - pub fn add_pending_v2(&mut self, entry: TxEntry) -> bool { + pub(crate) fn add_pending(&mut self, entry: TxEntry) -> Result { self.pool_map.add_entry(entry, Status::Pending) } - /// Add tx which proposed but still uncommittable to gap pool - pub fn add_gap(&mut self, entry: TxEntry) -> bool { - if self.proposed.contains_key(&entry.proposal_short_id()) { - return false; - } - trace!("add_gap {}", entry.transaction().hash()); - self.gap.add_entry(entry) - } - - pub fn add_gap_v2(&mut self, entry: TxEntry) -> bool { + /// Add tx which proposed but still uncommittable to gap + pub(crate) fn add_gap(&mut self, entry: TxEntry) -> Result { self.pool_map.add_entry(entry, Status::Gap) } - /// Add tx to proposed pool - pub fn add_proposed(&mut self, entry: TxEntry) -> Result { - trace!("add_proposed {}", entry.transaction().hash()); - self.proposed.add_entry(entry) - } - - pub fn add_proposed_v2(&mut self, entry: TxEntry) -> bool { + /// Add tx with proposed status + pub(crate) fn add_proposed(&mut self, entry: TxEntry) -> Result { self.pool_map.add_entry(entry, Status::Proposed) } /// Returns true if the tx-pool contains a tx with specified id. - pub fn contains_proposal_id(&self, id: &ProposalShortId) -> bool { - self.pending.contains_key(id) || self.gap.contains_key(id) || self.proposed.contains_key(id) - } - - pub fn contains_proposal_id_v2(&self, id: &ProposalShortId) -> bool { + pub(crate) fn contains_proposal_id(&self, id: &ProposalShortId) -> bool { self.pool_map.get_by_id(id).is_some() } /// Returns tx with cycles corresponding to the id. - pub fn get_tx_with_cycles(&self, id: &ProposalShortId) -> Option<(TransactionView, Cycle)> { - self.pending - .get(id) - .map(|entry| (entry.transaction().clone(), entry.cycles)) - .or_else(|| { - self.gap - .get(id) - .map(|entry| (entry.transaction().clone(), entry.cycles)) - }) - .or_else(|| { - self.proposed - .get(id) - .map(|entry| (entry.transaction().clone(), entry.cycles)) - }) - } - - pub fn get_tx_with_cycles_v2(&self, id: &ProposalShortId) -> Option<(TransactionView, Cycle)> { + pub(crate) fn get_tx_with_cycles( + &self, + id: &ProposalShortId, + ) -> Option<(TransactionView, Cycle)> { self.pool_map .get_by_id(id) .map(|entry| (entry.inner.transaction().clone(), entry.inner.cycles)) } - /// Returns tx corresponding to the id. - pub fn get_tx(&self, id: &ProposalShortId) -> Option<&TransactionView> { - self.pending - .get_tx(id) - .or_else(|| self.gap.get_tx(id)) - .or_else(|| self.proposed.get_tx(id)) - } - - pub fn get_tx_v2(&self, id: &ProposalShortId) -> Option<&TransactionView> { - self.pool_map - .get_by_id(id) - .map(|entry| entry.inner.transaction()) - } - - /// Returns tx from pending and gap corresponding to the id. RPC - pub fn get_entry_from_pending_or_gap(&self, id: &ProposalShortId) -> Option<&TxEntry> { - self.pending.get(id).or_else(|| self.gap.get(id)) - } - - pub fn get_entry_from_pending_or_gap_v2(&self, id: &ProposalShortId) -> Option<&TxEntry> { - if let Some(entry) = self.pool_map.get_by_id(id) { - match entry.status { - Status::Pending | Status::Gap => return Some(&entry.inner), - _ => return None, - } - } else { - return None; - } - } - - pub(crate) fn proposed(&self) -> &ProposedPool { - &self.proposed - } - - pub(crate) fn get_tx_from_proposed_and_others( - &self, - id: &ProposalShortId, - ) -> Option<&TransactionView> { - self.proposed - .get_tx(id) - .or_else(|| self.gap.get_tx(id)) - .or_else(|| self.pending.get_tx(id)) - } - - pub(crate) fn get_tx_from_proposed_and_others_v2( - &self, - id: &ProposalShortId, - ) -> Option<&TransactionView> { + pub(crate) fn get_tx_from_pool(&self, id: &ProposalShortId) -> Option<&TransactionView> { self.pool_map .get_by_id(id) .map(|entry| entry.inner.transaction()) @@ -273,7 +154,6 @@ impl TxPool { ) { for tx in txs { let tx_hash = tx.hash(); - debug!("try remove_committed_tx {}", tx_hash); self.remove_committed_tx(tx, callbacks); self.committed_txs_hash_cache @@ -290,77 +170,14 @@ impl TxPool { detached_headers: &HashSet, callbacks: &Callbacks, ) { - for (entry, reject) in self.proposed.resolve_conflict_header_dep(detached_headers) { - callbacks.call_reject(self, &entry, reject); - } - for (entry, reject) in self.gap.resolve_conflict_header_dep(detached_headers) { - callbacks.call_reject(self, &entry, reject); - } - for (entry, reject) in self.pending.resolve_conflict_header_dep(detached_headers) { - callbacks.call_reject(self, &entry, reject); - } - } - - pub(crate) fn resolve_conflict_header_dep_v2( - &mut self, - detached_headers: &HashSet, - callbacks: &Callbacks, - ) { - for (entry, reject) in self - .pool_map - .resolve_conflict_header_dep(detached_headers) - { + for (entry, reject) in self.pool_map.resolve_conflict_header_dep(detached_headers) { callbacks.call_reject(self, &entry, reject); } } pub(crate) fn remove_committed_tx(&mut self, tx: &TransactionView, callbacks: &Callbacks) { - let hash = tx.hash(); - let short_id = tx.proposal_short_id(); - // try remove committed tx from proposed - // proposed tx should not contain conflict, if exists just skip resolve conflict - if let Some(entry) = self.proposed.remove_committed_tx(tx) { - debug!("remove_committed_tx from proposed {}", hash); - callbacks.call_committed(self, &entry) - } else { - let conflicts = self.proposed.resolve_conflict(tx); - - for (entry, reject) in conflicts { - callbacks.call_reject(self, &entry, reject); - } - } - - // pending and gap should resolve conflict no matter exists or not - if let Some(entry) = self.gap.remove_entry(&short_id) { - debug!("remove_committed_tx from gap {}", hash); - callbacks.call_committed(self, &entry) - } - { - let conflicts = self.gap.resolve_conflict(tx); - - for (entry, reject) in conflicts { - callbacks.call_reject(self, &entry, reject); - } - } - - if let Some(entry) = self.pending.remove_entry(&short_id) { - debug!("remove_committed_tx from pending {}", hash); - callbacks.call_committed(self, &entry) - } - { - let conflicts = self.pending.resolve_conflict(tx); - - for (entry, reject) in conflicts { - callbacks.call_reject(self, &entry, reject); - } - } - } - - pub(crate) fn remove_committed_tx_v2(&mut self, tx: &TransactionView, callbacks: &Callbacks) { - let hash = tx.hash(); let short_id = tx.proposal_short_id(); if let Some(entry) = self.pool_map.remove_entry(&short_id) { - debug!("remove_committed_tx from gap {}", hash); callbacks.call_committed(self, &entry) } { @@ -373,37 +190,6 @@ impl TxPool { // Expire all transaction (and their dependencies) in the pool. pub(crate) fn remove_expired(&mut self, callbacks: &Callbacks) { - let now_ms = ckb_systemtime::unix_time_as_millis(); - let expired = - |_id: &ProposalShortId, tx_entry: &TxEntry| self.expiry + tx_entry.timestamp < now_ms; - let mut removed = self.pending.remove_entries_by_filter(expired); - removed.extend(self.gap.remove_entries_by_filter(expired)); - let removed_proposed_ids: Vec<_> = self - .proposed - .iter() - .filter_map(|(id, tx_entry)| { - if self.expiry + tx_entry.timestamp < now_ms { - Some(id) - } else { - None - } - }) - .cloned() - .collect(); - for id in removed_proposed_ids { - removed.extend(self.proposed.remove_entry_and_descendants(&id)) - } - - for entry in removed { - let tx_hash = entry.transaction().hash(); - debug!("remove_expired {} timestamp({})", tx_hash, entry.timestamp); - let reject = Reject::Expiry(entry.timestamp); - callbacks.call_reject(self, &entry, reject); - } - } - - // Expire all transaction (and their dependencies) in the pool. - pub(crate) fn remove_expired_v2(&mut self, callbacks: &Callbacks) { let now_ms = ckb_systemtime::unix_time_as_millis(); let removed: Vec<_> = self .pool_map @@ -414,8 +200,6 @@ impl TxPool { for entry in removed { self.pool_map.remove_entry(&entry.proposal_short_id()); - let tx_hash = entry.transaction().hash(); - debug!("remove_expired {} timestamp({})", tx_hash, entry.timestamp); let reject = Reject::Expiry(entry.timestamp); callbacks.call_reject(self, &entry, reject); } @@ -423,18 +207,6 @@ impl TxPool { // Remove transactions from the pool until total size < size_limit. pub(crate) fn limit_size(&mut self, callbacks: &Callbacks) { - while self.total_tx_size > self.config.max_tx_pool_size { - if !self.pending.is_empty() { - evict_for_trim_size!(self, self.pending, callbacks) - } else if !self.gap.is_empty() { - evict_for_trim_size!(self, self.gap, callbacks) - } else { - evict_for_trim_size!(self, self.proposed, callbacks) - } - } - } - - pub(crate) fn limit_size_v2(&mut self, callbacks: &Callbacks) { while self.total_tx_size > self.config.max_tx_pool_size { if let Some(id) = self.pool_map.next_evict_entry() { let removed = self.pool_map.remove_entry_and_descendants(&id); @@ -459,40 +231,10 @@ impl TxPool { pub(crate) fn remove_by_detached_proposal<'a>( &mut self, ids: impl Iterator, - ) { - for id in ids { - if let Some(entry) = self.gap.remove_entry(id) { - let tx_hash = entry.transaction().hash(); - let ret = self.add_pending(entry); - debug!( - "remove_by_detached_proposal from gap {} add_pending {}", - tx_hash, ret - ); - } - let mut entries = self.proposed.remove_entry_and_descendants(id); - entries.sort_unstable_by_key(|entry| entry.ancestors_count); - for mut entry in entries { - let tx_hash = entry.transaction().hash(); - entry.reset_ancestors_state(); - let ret = self.add_pending(entry); - debug!( - "remove_by_detached_proposal from proposed {} add_pending {}", - tx_hash, ret - ); - } - } - } - - // remove transaction with detached proposal from gap and proposed - // try re-put to pending - pub(crate) fn remove_by_detached_proposal_v2<'a>( - &mut self, - ids: impl Iterator, ) { for id in ids { if let Some(e) = self.pool_map.get_by_id(id) { let status = e.status; - // TODO: double check this if status == Status::Pending { continue; } @@ -503,7 +245,7 @@ impl TxPool { entry.reset_ancestors_state(); let ret = self.add_pending(entry); debug!( - "remove_by_detached_proposal from {:?} {} add_pending {}", + "remove_by_detached_proposal from {:?} {} add_pending {:?}", status, tx_hash, ret ); } @@ -512,7 +254,7 @@ impl TxPool { } pub(crate) fn remove_tx(&mut self, id: &ProposalShortId) -> bool { - let entries = self.proposed.remove_entry_and_descendants(id); + let entries = self.pool_map.remove_entry_and_descendants(id); if !entries.is_empty() { for entry in entries { self.update_statics_for_remove_tx(entry.size, entry.cycles); @@ -520,20 +262,6 @@ impl TxPool { return true; } - if let Some(entry) = self.gap.remove_entry(id) { - self.update_statics_for_remove_tx(entry.size, entry.cycles); - return true; - } - - if let Some(entry) = self.pending.remove_entry(id) { - self.update_statics_for_remove_tx(entry.size, entry.cycles); - return true; - } - - false - } - - pub(crate) fn remove_tx_v2(&mut self, id: &ProposalShortId) -> bool { if let Some(entry) = self.pool_map.remove_entry(id) { self.update_statics_for_remove_tx(entry.size, entry.cycles); return true; @@ -541,102 +269,48 @@ impl TxPool { false } - pub(crate) fn resolve_tx_from_pending_and_proposed( - &self, - tx: TransactionView, - ) -> Result, Reject> { - let snapshot = self.snapshot(); - let proposed_provider = OverlayCellProvider::new(&self.proposed, snapshot); - let gap_and_proposed_provider = OverlayCellProvider::new(&self.gap, &proposed_provider); - let pending_and_proposed_provider = - OverlayCellProvider::new(&self.pending, &gap_and_proposed_provider); - let mut seen_inputs = HashSet::new(); - resolve_transaction( - tx, - &mut seen_inputs, - &pending_and_proposed_provider, - snapshot, - ) - .map(Arc::new) - .map_err(Reject::Resolve) - } - - pub(crate) fn resolve_tx_from_pending_and_proposed_v2( - &self, - tx: TransactionView, - ) -> Result, Reject> { - let snapshot = self.snapshot(); - let provider = OverlayCellProvider::new(&self.pool_map.entries, snapshot); - let mut seen_inputs = HashSet::new(); - resolve_transaction(tx, &mut seen_inputs, &provider, snapshot) - .map(Arc::new) - .map_err(Reject::Resolve) - } - pub(crate) fn check_rtx_from_pending_and_proposed( &self, rtx: &ResolvedTransaction, ) -> Result<(), Reject> { let snapshot = self.snapshot(); - let proposed_checker = OverlayCellChecker::new(&self.proposed, snapshot); - let gap_and_proposed_checker = OverlayCellChecker::new(&self.gap, &proposed_checker); - let pending_and_proposed_checker = - OverlayCellChecker::new(&self.pending, &gap_and_proposed_checker); + let proposal_checker = OverlayCellChecker::new(&self.pool_map, snapshot); + let checker = OverlayCellChecker::new(self, &proposal_checker); let mut seen_inputs = HashSet::new(); - rtx.check(&mut seen_inputs, &pending_and_proposed_checker, snapshot) + rtx.check(&mut seen_inputs, &checker, snapshot) .map_err(Reject::Resolve) } - pub(crate) fn check_rtx_from_pending_and_proposed_v2( - &self, - rtx: &ResolvedTransaction, - ) -> Result<(), Reject> { + pub(crate) fn check_rtx_from_proposed(&self, rtx: &ResolvedTransaction) -> Result<(), Reject> { let snapshot = self.snapshot(); - let checker = OverlayCellChecker::new(&self.pool_map.entries, snapshot); + let proposal_checker = OverlayCellChecker::new(&self.pool_map, snapshot); let mut seen_inputs = HashSet::new(); - rtx.check(&mut seen_inputs, &checker, snapshot) + rtx.check(&mut seen_inputs, &proposal_checker, snapshot) .map_err(Reject::Resolve) } - pub(crate) fn resolve_tx_from_proposed( + pub(crate) fn resolve_tx_from_pending_and_proposed( &self, tx: TransactionView, ) -> Result, Reject> { let snapshot = self.snapshot(); - let cell_provider = OverlayCellProvider::new(&self.proposed, snapshot); + let proposed_provider = OverlayCellProvider::new(&self.pool_map, snapshot); + let provider = OverlayCellProvider::new(self, &proposed_provider); let mut seen_inputs = HashSet::new(); - resolve_transaction(tx, &mut seen_inputs, &cell_provider, snapshot) + resolve_transaction(tx, &mut seen_inputs, &provider, snapshot) .map(Arc::new) .map_err(Reject::Resolve) } - pub(crate) fn resolve_tx_from_proposed_v2( - &self, - rtx: &ResolvedTransaction, - ) -> Result<(), Reject> { - let snapshot = self.snapshot(); - let checker = OverlayCellChecker::new(&self.pool_map.entries, snapshot); - let mut seen_inputs = HashSet::new(); - rtx.check(&mut seen_inputs, &checker, snapshot) - .map_err(Reject::Resolve) - } - - pub(crate) fn check_rtx_from_proposed(&self, rtx: &ResolvedTransaction) -> Result<(), Reject> { - let snapshot = self.snapshot(); - let cell_checker = OverlayCellChecker::new(&self.proposed, snapshot); - let mut seen_inputs = HashSet::new(); - rtx.check(&mut seen_inputs, &cell_checker, snapshot) - .map_err(Reject::Resolve) - } - - pub(crate) fn check_rtx_from_proposed_v2( + pub(crate) fn resolve_tx_from_proposed( &self, - rtx: &ResolvedTransaction, - ) -> Result<(), Reject> { + tx: TransactionView, + ) -> Result, Reject> { let snapshot = self.snapshot(); - let cell_checker = OverlayCellChecker::new(&self.pool_map.entries, snapshot); + let proposed_provider = OverlayCellProvider::new(&self.pool_map, snapshot); let mut seen_inputs = HashSet::new(); - rtx.check(&mut seen_inputs, &cell_checker, snapshot) + resolve_transaction(tx, &mut seen_inputs, &proposed_provider, snapshot) + .map(Arc::new) .map_err(Reject::Resolve) } @@ -663,8 +337,9 @@ impl TxPool { let entry = TxEntry::new_with_timestamp(rtx, verified.cycles, verified.fee, size, timestamp); + let tx_hash = entry.transaction().hash(); - if self.add_gap(entry) { + if self.add_gap(entry).unwrap_or(false) { Ok(CacheEntry::Completed(verified)) } else { Err(Reject::Duplicated(tx_hash)) @@ -695,6 +370,11 @@ impl TxPool { let entry = TxEntry::new_with_timestamp(rtx, verified.cycles, verified.fee, size, timestamp); let tx_hash = entry.transaction().hash(); + debug!( + "proposed_rtx: {:?} => {:?}", + tx_hash, + entry.proposal_short_id() + ); if self.add_proposed(entry)? { Ok(CacheEntry::Completed(verified)) } else { @@ -703,76 +383,45 @@ impl TxPool { } /// Get to-be-proposal transactions that may be included in the next block. - pub fn get_proposals( - &self, - limit: usize, - exclusion: &HashSet, - ) -> HashSet { - let mut proposals = HashSet::with_capacity(limit); - self.pending - .fill_proposals(limit, exclusion, &mut proposals); - self.gap.fill_proposals(limit, exclusion, &mut proposals); - proposals - } - - /// Get to-be-proposal transactions that may be included in the next block. - pub fn get_proposals_v2( + /// TODO: do we need to consider the something like score, so that we can + /// provide best transactions to be proposed. + pub(crate) fn get_proposals( &self, limit: usize, exclusion: &HashSet, ) -> HashSet { let mut proposals = HashSet::with_capacity(limit); - self.pool_map.fill_proposals(limit, exclusion, &mut proposals, &Status::Pending); - self.pool_map.fill_proposals(limit, exclusion, &mut proposals, &Status::Gap); + self.pool_map + .fill_proposals(limit, exclusion, &mut proposals, &Status::Pending); + self.pool_map + .fill_proposals(limit, exclusion, &mut proposals, &Status::Gap); proposals } /// Returns tx from tx-pool or storage corresponding to the id. - pub fn get_tx_from_pool_or_store( + pub(crate) fn get_tx_from_pool_or_store( &self, proposal_id: &ProposalShortId, ) -> Option { - self.get_tx_from_proposed_and_others(proposal_id) - .cloned() - .or_else(|| { - self.committed_txs_hash_cache - .peek(proposal_id) - .and_then(|tx_hash| self.snapshot().get_transaction(tx_hash).map(|(tx, _)| tx)) - }) + self.get_tx_from_pool(proposal_id).cloned().or_else(|| { + self.committed_txs_hash_cache + .peek(proposal_id) + .and_then(|tx_hash| self.snapshot().get_transaction(tx_hash).map(|(tx, _)| tx)) + }) } pub(crate) fn get_ids(&self) -> TxPoolIds { - let pending = self - .pending - .iter() - .map(|(_, entry)| entry.transaction().hash()) - .chain(self.gap.iter().map(|(_, entry)| entry.transaction().hash())) - .collect(); - - let proposed = self - .proposed - .iter() - .map(|(_, entry)| entry.transaction().hash()) - .collect(); - - TxPoolIds { pending, proposed } - } - - // This is for RPC request, performance is not critical - pub(crate) fn get_ids_v2(&self) -> TxPoolIds { let pending: Vec = self - .pool_map - .entries .get_by_status(&Status::Pending) .iter() - .chain(self.pool_map.entries.get_by_status(&Status::Gap).iter()) + .chain(self.get_by_status(&Status::Gap).iter()) .map(|entry| entry.inner.transaction().hash()) .collect(); let proposed: Vec = self - .proposed + .get_by_status(&Status::Proposed) .iter() - .map(|(_, entry)| entry.transaction().hash()) + .map(|entry| entry.inner.transaction().hash()) .collect(); TxPoolIds { pending, proposed } @@ -780,38 +429,13 @@ impl TxPool { pub(crate) fn get_all_entry_info(&self) -> TxPoolEntryInfo { let pending = self - .pending - .iter() - .map(|(_, entry)| (entry.transaction().hash(), entry.to_info())) - .chain( - self.gap - .iter() - .map(|(_, entry)| (entry.transaction().hash(), entry.to_info())), - ) - .collect(); - - let proposed = self - .proposed - .iter() - .map(|(_, entry)| (entry.transaction().hash(), entry.to_info())) - .collect(); - - TxPoolEntryInfo { pending, proposed } - } - - pub(crate) fn get_all_entry_info_v2(&self) -> TxPoolEntryInfo { - let pending = self - .pool_map - .entries .get_by_status(&Status::Pending) .iter() - .chain(self.pool_map.entries.get_by_status(&Status::Gap).iter()) + .chain(self.get_by_status(&Status::Gap).iter()) .map(|entry| (entry.inner.transaction().hash(), entry.inner.to_info())) .collect(); let proposed = self - .pool_map - .entries .get_by_status(&Status::Proposed) .iter() .map(|entry| (entry.inner.transaction().hash(), entry.inner.to_info())) @@ -821,29 +445,12 @@ impl TxPool { } pub(crate) fn drain_all_transactions(&mut self) -> Vec { - let mut txs = CommitTxsScanner::new(&self.proposed, &self.pool_map.entries) - .txs_to_commit(self.total_tx_size, self.total_tx_cycles) - .0 - .into_iter() - .map(|tx_entry| tx_entry.into_transaction()) - .collect::>(); - self.proposed.clear(); - txs.append(&mut self.gap.drain()); - txs.append(&mut self.pending.drain()); - self.total_tx_size = 0; - self.total_tx_cycles = 0; - // self.touch_last_txs_updated_at(); - txs - } - - pub(crate) fn drain_all_transactions_v2(&mut self) -> Vec { - let mut txs = CommitTxsScanner::new(&self.proposed, &self.pool_map.entries) + let mut txs = CommitTxsScanner::new(&self.pool_map) .txs_to_commit(self.total_tx_size, self.total_tx_cycles) .0 .into_iter() .map(|tx_entry| tx_entry.into_transaction()) .collect::>(); - self.proposed.clear(); let mut pending = self .pool_map .entries @@ -868,9 +475,6 @@ impl TxPool { } pub(crate) fn clear(&mut self, snapshot: Arc) { - self.pending = PendingQueue::new(); - self.gap = PendingQueue::new(); - self.proposed = ProposedPool::new(self.config.max_ancestors_count); self.pool_map.clear(); self.snapshot = snapshot; self.committed_txs_hash_cache = LruCache::new(COMMITTED_HASH_CACHE_SIZE); @@ -896,8 +500,7 @@ impl TxPool { txs_size_limit: usize, ) -> (Vec, usize, Cycle) { let (entries, size, cycles) = - CommitTxsScanner::new(self.proposed(), &self.pool_map.entries) - .txs_to_commit(txs_size_limit, max_block_cycles); + CommitTxsScanner::new(&self.pool_map).txs_to_commit(txs_size_limit, max_block_cycles); if !entries.is_empty() { ckb_logger::info!( @@ -935,3 +538,49 @@ impl TxPool { } } } + +/// This is a hack right now, we use `CellProvider` to check if a transaction is in `Pending` or `Gap` status. +/// To make sure the behavior is same as before, we need to remove this if we have finished replace-by-fee strategy. +impl CellProvider for TxPool { + fn cell(&self, out_point: &OutPoint, _eager_load: bool) -> CellStatus { + let tx_hash = out_point.tx_hash(); + match self + .pool_map + .get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) + { + Some(pool_entry) if pool_entry.status != Status::Proposed => { + match pool_entry + .inner + .transaction() + .output_with_data(out_point.index().unpack()) + { + Some((output, data)) => { + let cell_meta = CellMetaBuilder::from_cell_output(output, data) + .out_point(out_point.to_owned()) + .build(); + CellStatus::live_cell(cell_meta) + } + None => CellStatus::Unknown, + } + } + _ => CellStatus::Unknown, + } + } +} + +impl CellChecker for TxPool { + fn is_live(&self, out_point: &OutPoint) -> Option { + let tx_hash = out_point.tx_hash(); + match self + .pool_map + .get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) + { + Some(pool_entry) if pool_entry.status != Status::Proposed => pool_entry + .inner + .transaction() + .output_with_data(out_point.index().unpack()) + .map(|_| true), + _ => None, + } + } +} diff --git a/tx-pool/src/process.rs b/tx-pool/src/process.rs index b5062b3b35..00dde6d264 100644 --- a/tx-pool/src/process.rs +++ b/tx-pool/src/process.rs @@ -1,6 +1,7 @@ use crate::callback::Callbacks; use crate::component::entry::TxEntry; use crate::component::orphan::Entry as OrphanEntry; +use crate::component::pool_map::Status; use crate::error::Reject; use crate::pool::TxPool; use crate::service::{BlockAssemblerMessage, TxPoolService, TxVerificationResult}; @@ -51,6 +52,7 @@ pub enum TxStatus { Proposed, } +#[derive(Debug, Clone, Copy, PartialEq, Eq)] pub(crate) enum ProcessResult { Suspended, Completed(Completed), @@ -122,7 +124,6 @@ impl TxPoolService { } _submit_entry(tx_pool, status, entry.clone(), &self.callbacks)?; - Ok(()) }) .await; @@ -276,7 +277,6 @@ impl TxPoolService { if let Some((ret, snapshot)) = self._process_tx(tx.clone(), remote.map(|r| r.0)).await { self.after_process(tx, remote, &snapshot, &ret).await; - ret } else { // currently, the returned cycles is not been used, mock 0 if delay @@ -360,7 +360,6 @@ impl TxPoolService { self.process_orphan_tx(&tx).await; } Err(reject) => { - debug!("after_process {} reject: {} ", tx_hash, reject); if is_missing_input(reject) && all_inputs_is_unknown(snapshot, &tx) { self.add_orphan(tx, peer, declared_cycle).await; } else { @@ -551,7 +550,6 @@ impl TxPoolService { let tx_hash = tx.hash(); let (ret, snapshot) = self.pre_check(&tx).await; - let (tip_hash, rtx, status, fee, tx_size) = try_or_return_with_snapshot!(ret, snapshot); if self.is_in_delay_window(&snapshot) { @@ -640,7 +638,6 @@ impl TxPoolService { try_or_return_with_snapshot!(ret, submit_snapshot); self.notify_block_assembler(status).await; - if cached.is_none() { // update cache let txs_verify_cache = Arc::clone(&self.txs_verify_cache); @@ -962,38 +959,36 @@ fn check_rtx( rtx: &ResolvedTransaction, ) -> Result { let short_id = rtx.transaction.proposal_short_id(); - if snapshot.proposals().contains_proposed(&short_id) { - tx_pool - .check_rtx_from_proposed(rtx) - .map(|_| TxStatus::Proposed) + let tx_status = if snapshot.proposals().contains_proposed(&short_id) { + TxStatus::Proposed + } else if snapshot.proposals().contains_gap(&short_id) { + TxStatus::Gap } else { - let tx_status = if snapshot.proposals().contains_gap(&short_id) { - TxStatus::Gap - } else { - TxStatus::Fresh - }; - tx_pool - .check_rtx_from_pending_and_proposed(rtx) - .map(|_| tx_status) + TxStatus::Fresh + }; + if tx_status == TxStatus::Proposed { + tx_pool.check_rtx_from_proposed(rtx) + } else { + tx_pool.check_rtx_from_pending_and_proposed(rtx) } + .map(|_| tx_status) } fn resolve_tx(tx_pool: &TxPool, snapshot: &Snapshot, tx: TransactionView) -> ResolveResult { let short_id = tx.proposal_short_id(); - if snapshot.proposals().contains_proposed(&short_id) { - tx_pool - .resolve_tx_from_proposed(tx) - .map(|rtx| (rtx, TxStatus::Proposed)) + let tx_status = if snapshot.proposals().contains_proposed(&short_id) { + TxStatus::Proposed + } else if snapshot.proposals().contains_gap(&short_id) { + TxStatus::Gap } else { - let tx_status = if snapshot.proposals().contains_gap(&short_id) { - TxStatus::Gap - } else { - TxStatus::Fresh - }; - tx_pool - .resolve_tx_from_pending_and_proposed(tx) - .map(|rtx| (rtx, tx_status)) + TxStatus::Fresh + }; + if tx_status == TxStatus::Proposed { + tx_pool.resolve_tx_from_proposed(tx) + } else { + tx_pool.resolve_tx_from_pending_and_proposed(tx) } + .map(|rtx| (rtx, tx_status)) } fn _submit_entry( @@ -1005,16 +1000,15 @@ fn _submit_entry( let tx_hash = entry.transaction().hash(); match status { TxStatus::Fresh => { - if tx_pool.add_pending(entry.clone()) { - debug!("submit_entry pending {}", tx_hash); + if tx_pool.add_pending(entry.clone()).unwrap_or(false) { callbacks.call_pending(tx_pool, &entry); } else { return Err(Reject::Duplicated(tx_hash)); } } + TxStatus::Gap => { - if tx_pool.add_gap(entry.clone()) { - debug!("submit_entry gap {}", tx_hash); + if tx_pool.add_gap(entry.clone()).unwrap_or(false) { callbacks.call_pending(tx_pool, &entry); } else { return Err(Reject::Duplicated(tx_hash)); @@ -1022,10 +1016,7 @@ fn _submit_entry( } TxStatus::Proposed => { if tx_pool.add_proposed(entry.clone())? { - debug!("submit_entry proposed {}", tx_hash); callbacks.call_proposed(tx_pool, &entry, true); - } else { - return Err(Reject::Duplicated(tx_hash)); } } } @@ -1055,38 +1046,39 @@ fn _update_tx_pool_for_reorg( // pending ---> gap ----> proposed // try move gap to proposed if mine_mode { - let mut entries = Vec::new(); + let mut proposals = Vec::new(); let mut gaps = Vec::new(); - tx_pool.gap.remove_entries_by_filter(|id, tx_entry| { - if snapshot.proposals().contains_proposed(id) { - entries.push(tx_entry.clone()); - true - } else { - false - } - }); - - tx_pool.pending.remove_entries_by_filter(|id, tx_entry| { - if snapshot.proposals().contains_proposed(id) { - entries.push(tx_entry.clone()); - true - } else if snapshot.proposals().contains_gap(id) { - gaps.push(tx_entry.clone()); - true - } else { - false - } - }); + tx_pool + .pool_map + .remove_entries_by_filter(&Status::Gap, |id, tx_entry| { + if snapshot.proposals().contains_proposed(id) { + proposals.push(tx_entry.clone()); + true + } else { + false + } + }); - for entry in entries { - debug!("tx move to proposed {}", entry.transaction().hash()); + tx_pool + .pool_map + .remove_entries_by_filter(&Status::Pending, |id, tx_entry| { + if snapshot.proposals().contains_proposed(id) { + proposals.push(tx_entry.clone()); + true + } else if snapshot.proposals().contains_gap(id) { + gaps.push(tx_entry.clone()); + true + } else { + false + } + }); + + for entry in proposals { let cached = CacheEntry::completed(entry.cycles, entry.fee); - let tx_hash = entry.transaction().hash(); if let Err(e) = tx_pool.proposed_rtx(cached, entry.size, entry.timestamp, Arc::clone(&entry.rtx)) { - debug!("Failed to add proposed tx {}, reason: {}", tx_hash, e); callbacks.call_reject(tx_pool, &entry, e.clone()); } else { callbacks.call_proposed(tx_pool, &entry, false); @@ -1094,7 +1086,6 @@ fn _update_tx_pool_for_reorg( } for entry in gaps { - debug!("tx move to gap {}", entry.transaction().hash()); let tx_hash = entry.transaction().hash(); let cached = CacheEntry::completed(entry.cycles, entry.fee); if let Err(e) = diff --git a/tx-pool/src/service.rs b/tx-pool/src/service.rs index 5dc54018b4..a008502c62 100644 --- a/tx-pool/src/service.rs +++ b/tx-pool/src/service.rs @@ -3,6 +3,7 @@ use crate::block_assembler::{self, BlockAssembler}; use crate::callback::{Callback, Callbacks, ProposedCallback, RejectCallback}; use crate::chunk_process::ChunkCommand; +use crate::component::pool_map::{PoolEntry, Status}; use crate::component::{chunk::ChunkQueue, orphan::OrphanPool}; use crate::error::{handle_recv_error, handle_send_cmd_error, handle_try_send_error}; use crate::pool::TxPool; @@ -735,10 +736,18 @@ async fn process(mut service: TxPoolService, message: Message) { }) => { let id = ProposalShortId::from_tx_hash(&hash); let tx_pool = service.tx_pool.read().await; - let ret = if let Some(entry) = tx_pool.proposed.get(&id) { - Ok((TxStatus::Proposed, Some(entry.cycles))) - } else if let Some(entry) = tx_pool.get_entry_from_pending_or_gap(&id) { - Ok((TxStatus::Pending, Some(entry.cycles))) + let ret = if let Some(PoolEntry { + status, + inner: entry, + .. + }) = tx_pool.pool_map.get_by_id(&id) + { + let status = if status == &Status::Proposed { + TxStatus::Proposed + } else { + TxStatus::Pending + }; + Ok((status, Some(entry.cycles))) } else if let Some(ref recent_reject_db) = tx_pool.recent_reject { let recent_reject_result = recent_reject_db.get(&hash); if let Ok(recent_reject) = recent_reject_result { @@ -764,14 +773,18 @@ async fn process(mut service: TxPoolService, message: Message) { }) => { let id = ProposalShortId::from_tx_hash(&hash); let tx_pool = service.tx_pool.read().await; - let ret = if let Some(entry) = tx_pool.proposed.get(&id) { - Ok(TransactionWithStatus::with_proposed( - Some(entry.transaction().clone()), - entry.cycles, - entry.timestamp, - )) - } else if let Some(entry) = tx_pool.get_entry_from_pending_or_gap(&id) { - Ok(TransactionWithStatus::with_pending( + let ret = if let Some(PoolEntry { + status, + inner: entry, + .. + }) = tx_pool.pool_map.get_by_id(&id) + { + let trans_status = if status == &Status::Proposed { + TransactionWithStatus::with_proposed + } else { + TransactionWithStatus::with_pending + }; + Ok(trans_status( Some(entry.transaction().clone()), entry.cycles, entry.timestamp, @@ -900,8 +913,8 @@ impl TxPoolService { TxPoolInfo { tip_hash: tip_header.hash(), tip_number: tip_header.number(), - pending_size: tx_pool.pending.size() + tx_pool.gap.size(), - proposed_size: tx_pool.proposed.size(), + pending_size: tx_pool.pool_map.pending_size(), + proposed_size: tx_pool.pool_map.proposed_size(), orphan_size: orphan.len(), total_tx_size: tx_pool.total_tx_size, total_tx_cycles: tx_pool.total_tx_cycles, @@ -968,7 +981,7 @@ impl TxPoolService { match target { PlugTarget::Pending => { for entry in entries { - tx_pool.add_pending(entry); + tx_pool.add_pending(entry).unwrap(); } } PlugTarget::Proposed => { From 5788226589aec10a471a0836b79e47c47fb2a60e Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 1 Jun 2023 18:12:17 +0800 Subject: [PATCH 04/58] move double spend checking and fix tests --- Cargo.lock | 32 +++--- chain/src/chain.rs | 1 - chain/src/tests/dep_cell.rs | 15 ++- rpc/src/tests/module/pool.rs | 2 +- test/src/main.rs | 8 -- test/src/node.rs | 12 ++ test/src/specs/relay/transaction_relay.rs | 14 ++- test/src/specs/tx_pool/collision.rs | 27 ++--- .../tx_pool/different_txs_with_same_input.rs | 38 +++--- test/src/specs/tx_pool/send_tx_chain.rs | 12 +- tx-pool/Cargo.toml | 3 +- tx-pool/src/component/edges.rs | 4 + tx-pool/src/component/pool_map.rs | 88 +++++++------- tx-pool/src/pool.rs | 108 +++--------------- tx-pool/src/process.rs | 55 ++++----- 15 files changed, 180 insertions(+), 239 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ddd51deff0..0072b84283 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1423,9 +1423,9 @@ dependencies = [ "ckb-types", "ckb-util", "ckb-verification", + "ckb_multi_index_map", "hyper", "lru", - "multi_index_map", "rand 0.8.5", "rustc-hash", "sentry", @@ -1550,6 +1550,21 @@ version = "0.24.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "83869c9d322de1ddbfde5b54b7376f9a1ac32273c50e21cdd5e8a1bd1a1cf632" +[[package]] +name = "ckb_multi_index_map" +version = "0.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2adba00c3dcb84fc4634c948cf3d24c05ce3193810bfa568effe13ad814f662a" +dependencies = [ + "convert_case 0.6.0", + "proc-macro-error", + "proc-macro2", + "quote", + "rustc-hash", + "slab", + "syn", +] + [[package]] name = "clang-sys" version = "1.3.1" @@ -3125,21 +3140,6 @@ dependencies = [ "faster-hex", ] -[[package]] -name = "multi_index_map" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a58eea8dbf91e7420e0e843535f585491046d6017e669d36cb8342cfa4861e2" -dependencies = [ - "convert_case 0.6.0", - "proc-macro-error", - "proc-macro2", - "quote", - "rustc-hash", - "slab", - "syn", -] - [[package]] name = "native-tls" version = "0.2.11" diff --git a/chain/src/chain.rs b/chain/src/chain.rs index 3323492032..bcef1bfc12 100644 --- a/chain/src/chain.rs +++ b/chain/src/chain.rs @@ -560,7 +560,6 @@ impl ChainService { self.proposal_table .insert(blk.header().number(), blk.union_proposal_ids()); } - self.reload_proposal_table(fork); } diff --git a/chain/src/tests/dep_cell.rs b/chain/src/tests/dep_cell.rs index 3f557236fd..3e7b24eed1 100644 --- a/chain/src/tests/dep_cell.rs +++ b/chain/src/tests/dep_cell.rs @@ -436,7 +436,6 @@ fn test_package_txs_with_deps2() { .internal_process_block(Arc::new(block), Switch::DISABLE_ALL) .unwrap(); } - // skip gap { while Into::::into(block_template.number) != 2 { @@ -461,7 +460,7 @@ fn test_package_txs_with_deps2() { let mut tx_pool_info = tx_pool.get_tx_pool_info().unwrap(); while tx_pool_info.proposed_size != txs.len() { - tx_pool_info = tx_pool.get_tx_pool_info().unwrap() + tx_pool_info = tx_pool.get_tx_pool_info().unwrap(); } // get block template with txs @@ -534,11 +533,11 @@ fn test_package_txs_with_deps_priority() { Capacity::shannons(10000), ); - let txs = vec![tx2.clone(), tx1]; - for tx in &txs { - let ret = tx_pool.submit_local_tx(tx.clone()).unwrap(); - assert!(ret.is_ok(), "submit {} {:?}", tx.proposal_short_id(), ret); - } + let ret = tx_pool.submit_local_tx(tx2.clone()).unwrap(); + assert!(ret.is_ok(), "submit {} {:?}", tx2.proposal_short_id(), ret); + + let ret = tx_pool.submit_local_tx(tx1.clone()).unwrap(); + assert!(ret.is_err(), "submit {} {:?}", tx1.proposal_short_id(), ret); let mut block_template = shared .get_block_template(None, None, None) @@ -548,7 +547,7 @@ fn test_package_txs_with_deps_priority() { // proposal txs { while !(Into::::into(block_template.number) == 1 - && block_template.proposals.len() == 2) + && block_template.proposals.len() == 1) { block_template = shared .get_block_template(None, None, None) diff --git a/rpc/src/tests/module/pool.rs b/rpc/src/tests/module/pool.rs index 7288b91f0f..5349465f4f 100644 --- a/rpc/src/tests/module/pool.rs +++ b/rpc/src/tests/module/pool.rs @@ -172,7 +172,7 @@ fn test_send_transaction_exceeded_maximum_ancestors_count() { parent_tx_hash = tx.hash(); } - suite.wait_block_template_array_ge("proposals", 130); + suite.wait_block_template_array_ge("proposals", 125); // 130 txs will be added to proposal list while store.get_tip_header().unwrap().number() != (tip.number() + 2) { diff --git a/test/src/main.rs b/test/src/main.rs index 2e8a8e8620..f321bfe0a7 100644 --- a/test/src/main.rs +++ b/test/src/main.rs @@ -360,12 +360,6 @@ fn canonicalize_path>(path: P) -> PathBuf { .unwrap_or_else(|_| path.as_ref().to_path_buf()) } -fn _all_specs() -> Vec> { - // This case is not stable right now - //vec![Box::new(PoolResolveConflictAfterReorg)] - vec![Box::new(RemoveConflictFromPending)] -} - fn all_specs() -> Vec> { let mut specs: Vec> = vec![ Box::new(BlockSyncFromOne), @@ -408,8 +402,6 @@ fn all_specs() -> Vec> { Box::new(GetRawTxPool), Box::new(PoolReconcile), Box::new(PoolResurrect), - //TODO: (yukang) - //Box::new(PoolResolveConflictAfterReorg), Box::new(InvalidHeaderDep), #[cfg(not(target_os = "windows"))] Box::new(PoolPersisted), diff --git a/test/src/node.rs b/test/src/node.rs index da5d7421fe..acf6c19daf 100644 --- a/test/src/node.rs +++ b/test/src/node.rs @@ -5,6 +5,7 @@ use crate::{SYSTEM_CELL_ALWAYS_FAILURE_INDEX, SYSTEM_CELL_ALWAYS_SUCCESS_INDEX}; use ckb_app_config::CKBAppConfig; use ckb_chain_spec::consensus::Consensus; use ckb_chain_spec::ChainSpec; +use ckb_error::AnyError; use ckb_jsonrpc_types::TxStatus; use ckb_jsonrpc_types::{BlockFilter, BlockTemplate, TxPoolInfo}; use ckb_logger::{debug, error}; @@ -353,6 +354,17 @@ impl Node { .send_transaction(transaction.data().into()) } + pub fn submit_transaction_with_result( + &self, + transaction: &TransactionView, + ) -> Result { + let res = self + .rpc_client() + .send_transaction_result(transaction.data().into())? + .pack(); + Ok(res) + } + pub fn get_transaction(&self, tx_hash: Byte32) -> TxStatus { self.rpc_client().get_transaction(tx_hash).tx_status } diff --git a/test/src/specs/relay/transaction_relay.rs b/test/src/specs/relay/transaction_relay.rs index 39cb05f3da..e5af42eb45 100644 --- a/test/src/specs/relay/transaction_relay.rs +++ b/test/src/specs/relay/transaction_relay.rs @@ -5,7 +5,6 @@ use crate::util::transaction::{always_success_transaction, always_success_transa use crate::utils::{build_relay_tx_hashes, build_relay_txs, sleep, wait_until}; use crate::{Net, Node, Spec}; use ckb_constant::sync::RETRY_ASK_TX_TIMEOUT_INCREASE; -use ckb_jsonrpc_types::Status; use ckb_logger::info; use ckb_network::SupportProtocols; use ckb_types::{ @@ -234,10 +233,15 @@ impl Spec for TransactionRelayConflict { .build(); node0.rpc_client().send_transaction(tx1.data().into()); sleep(6); - node0.rpc_client().send_transaction(tx2.data().into()); + + let res = node0 + .rpc_client() + .send_transaction_result(tx2.data().into()); + assert!(res.is_err()); + eprintln!("res: {:?}", res); let relayed = wait_until(20, || { - [tx1.hash(), tx2.hash()].iter().all(|hash| { + [tx1.hash()].iter().all(|hash| { node1 .rpc_client() .get_transaction(hash.clone()) @@ -247,13 +251,14 @@ impl Spec for TransactionRelayConflict { }); assert!(relayed, "all transactions should be relayed"); - let proposed = node1.mine_with_blocking(|template| template.proposals.len() != 3); + let proposed = node1.mine_with_blocking(|template| template.proposals.len() != 2); node1.mine_with_blocking(|template| template.number.value() != (proposed + 1)); waiting_for_sync(nodes); node0.wait_for_tx_pool(); node1.wait_for_tx_pool(); + /* let ret = node1 .rpc_client() .get_transaction_with_verbosity(tx2.hash(), 1); @@ -289,5 +294,6 @@ impl Spec for TransactionRelayConflict { .is_some() }); assert!(relayed, "Transaction should be relayed to node1"); + */ } } diff --git a/test/src/specs/tx_pool/collision.rs b/test/src/specs/tx_pool/collision.rs index c349ddbb34..7dc606d459 100644 --- a/test/src/specs/tx_pool/collision.rs +++ b/test/src/specs/tx_pool/collision.rs @@ -1,6 +1,4 @@ -use crate::util::check::{ - is_transaction_committed, is_transaction_pending, is_transaction_rejected, -}; +use crate::util::check::{is_transaction_committed, is_transaction_pending}; use crate::utils::{assert_send_transaction_fail, blank, commit, propose}; use crate::{Node, Spec}; use ckb_types::bytes::Bytes; @@ -67,7 +65,8 @@ impl Spec for ConflictInPending { let (txa, txb) = conflict_transactions(node); node.submit_transaction(&txa); - node.submit_transaction(&txb); + let res = node.submit_transaction_with_result(&txb); + assert!(res.is_err()); node.submit_block(&propose(node, &[&txa])); (0..window.closest()).for_each(|_| { @@ -89,13 +88,15 @@ impl Spec for ConflictInGap { let (txa, txb) = conflict_transactions(node); node.submit_transaction(&txa); - node.submit_transaction(&txb); + let res = node.submit_transaction_with_result(&txb); + assert!(res.is_err()); node.submit_block(&propose(node, &[&txa])); (0..window.closest() - 1).for_each(|_| { node.submit_block(&blank(node)); }); - node.submit_block(&propose(node, &[&txb])); + + //node.submit_block(&propose(node, &[&txb])); let block = node.new_block(None, None, None); assert_eq!(&[txa], &block.transactions()[1..]); @@ -114,7 +115,8 @@ impl Spec for ConflictInProposed { let (txa, txb) = conflict_transactions(node); node.submit_transaction(&txa); - node.submit_transaction(&txb); + let res = node.submit_transaction_with_result(&txb); + assert!(res.is_err()); node.submit_block(&propose(node, &[&txa, &txb])); node.mine(window.farthest()); @@ -153,12 +155,13 @@ impl Spec for RemoveConflictFromPending { conflict_transactions_with_capacity(node, Bytes::new(), capacity_bytes!(1000)); let txc = node.new_transaction_with_since_capacity(txb.hash(), 0, capacity_bytes!(100)); node.submit_transaction(&txa); - node.submit_transaction(&txb); - node.submit_transaction(&txc); + let res = node.submit_transaction_with_result(&txb); + assert!(res.is_err()); + + let res = node.submit_transaction_with_result(&txc); + assert!(res.is_err()); assert!(is_transaction_pending(node, &txa)); - assert!(is_transaction_pending(node, &txb)); - assert!(is_transaction_pending(node, &txc)); node.submit_block(&propose(node, &[&txa])); (0..window.closest()).for_each(|_| { @@ -168,8 +171,6 @@ impl Spec for RemoveConflictFromPending { node.wait_for_tx_pool(); assert!(is_transaction_committed(node, &txa)); - assert!(is_transaction_rejected(node, &txb)); - assert!(is_transaction_rejected(node, &txc)); } } diff --git a/test/src/specs/tx_pool/different_txs_with_same_input.rs b/test/src/specs/tx_pool/different_txs_with_same_input.rs index 443cea4cb5..8539c7c8c6 100644 --- a/test/src/specs/tx_pool/different_txs_with_same_input.rs +++ b/test/src/specs/tx_pool/different_txs_with_same_input.rs @@ -28,10 +28,14 @@ impl Spec for DifferentTxsWithSameInput { .as_advanced_builder() .set_outputs(vec![output]) .build(); + node0.rpc_client().send_transaction(tx1.data().into()); - node0.rpc_client().send_transaction(tx2.data().into()); + let res = node0 + .rpc_client() + .send_transaction_result(tx2.data().into()); + assert!(res.is_err(), "tx2 should be rejected"); - node0.mine_with_blocking(|template| template.proposals.len() != 3); + node0.mine_with_blocking(|template| template.proposals.len() != 2); node0.mine_with_blocking(|template| template.number.value() != 14); node0.mine_with_blocking(|template| template.transactions.len() != 2); @@ -47,11 +51,11 @@ impl Spec for DifferentTxsWithSameInput { assert!(!commit_txs_hash.contains(&tx2.hash())); // when tx1 was confirmed, tx2 should be rejected - let ret = node0.rpc_client().get_transaction(tx2.hash()); - assert!( - matches!(ret.tx_status.status, Status::Rejected), - "tx2 should be rejected" - ); + // let ret = node0.rpc_client().get_transaction(tx2.hash()); + // assert!( + // matches!(ret.tx_status.status, Status::Rejected), + // "tx2 should be rejected" + // ); // verbosity = 1 let ret = node0 @@ -60,11 +64,11 @@ impl Spec for DifferentTxsWithSameInput { assert!(ret.transaction.is_none()); assert!(matches!(ret.tx_status.status, Status::Committed)); - let ret = node0 - .rpc_client() - .get_transaction_with_verbosity(tx2.hash(), 1); - assert!(ret.transaction.is_none()); - assert!(matches!(ret.tx_status.status, Status::Rejected)); + // let ret = node0 + // .rpc_client() + // .get_transaction_with_verbosity(tx2.hash(), 1); + // assert!(ret.transaction.is_none()); + // assert!(matches!(ret.tx_status.status, Status::Rejected)); // verbosity = 2 let ret = node0 @@ -73,10 +77,10 @@ impl Spec for DifferentTxsWithSameInput { assert!(ret.transaction.is_some()); assert!(matches!(ret.tx_status.status, Status::Committed)); - let ret = node0 - .rpc_client() - .get_transaction_with_verbosity(tx2.hash(), 2); - assert!(ret.transaction.is_none()); - assert!(matches!(ret.tx_status.status, Status::Rejected)); + // let ret = node0 + // .rpc_client() + // .get_transaction_with_verbosity(tx2.hash(), 2); + // assert!(ret.transaction.is_none()); + // assert!(matches!(ret.tx_status.status, Status::Rejected)); } } diff --git a/test/src/specs/tx_pool/send_tx_chain.rs b/test/src/specs/tx_pool/send_tx_chain.rs index 3065b690be..f27030a5a1 100644 --- a/test/src/specs/tx_pool/send_tx_chain.rs +++ b/test/src/specs/tx_pool/send_tx_chain.rs @@ -33,10 +33,15 @@ impl Spec for SendTxChain { assert_eq!(txs.len(), MAX_ANCESTORS_COUNT + 1); // send tx chain info!("submit fresh txs chain to node0"); - for tx in txs[..=MAX_ANCESTORS_COUNT].iter() { + for tx in txs[..=MAX_ANCESTORS_COUNT - 1].iter() { let ret = node0.rpc_client().send_transaction_result(tx.data().into()); assert!(ret.is_ok()); } + // The last one will be rejected + let ret = node0 + .rpc_client() + .send_transaction_result(txs[MAX_ANCESTORS_COUNT].data().into()); + assert!(ret.is_err()); node0.mine(3); @@ -76,6 +81,11 @@ impl Spec for SendTxChain { .rpc_client() .send_transaction_result(txs.last().unwrap().data().into()); assert!(ret.is_err()); + assert!(ret + .err() + .unwrap() + .to_string() + .contains("Transaction exceeded maximum ancestors count limit")); } fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { diff --git a/tx-pool/Cargo.toml b/tx-pool/Cargo.toml index 0ce764ecb0..d8a16bff50 100644 --- a/tx-pool/Cargo.toml +++ b/tx-pool/Cargo.toml @@ -36,8 +36,7 @@ sentry = { version = "0.26.0", optional = true } serde_json = "1.0" rand = "0.8.4" hyper = { version = "0.14", features = ["http1", "client", "tcp"] } -#multi_index_map = { git = "https://github.com/wyjin/multi_index_map.git", branch = "master" } -multi_index_map = "0.5.0" +ckb_multi_index_map = "0.0.1" # ckb team fork crate slab = "0.4" rustc-hash = "1.1" tokio-util = "0.7.8" diff --git a/tx-pool/src/component/edges.rs b/tx-pool/src/component/edges.rs index decf98c1a5..129b33c0a2 100644 --- a/tx-pool/src/component/edges.rs +++ b/tx-pool/src/component/edges.rs @@ -91,6 +91,10 @@ impl Edges { self.outputs.get(out_point) } + pub(crate) fn remove_deps(&mut self, out_point: &OutPoint) -> Option> { + self.deps.remove(out_point) + } + pub(crate) fn insert_deps(&mut self, out_point: OutPoint, txid: ProposalShortId) { self.deps.entry(out_point).or_default().insert(txid); } diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index f9b00b75e3..50c5c393dc 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -9,6 +9,7 @@ use crate::error::Reject; use crate::TxEntry; use ckb_logger::trace; +use ckb_multi_index_map::MultiIndexMap; use ckb_types::core::error::OutPointError; use ckb_types::packed::OutPoint; use ckb_types::{ @@ -20,7 +21,6 @@ use ckb_types::{ core::cell::{CellMetaBuilder, CellProvider, CellStatus}, prelude::*, }; -use multi_index_map::MultiIndexMap; use std::borrow::Cow; use std::collections::HashSet; @@ -242,40 +242,33 @@ impl PoolMap { conflicts } - /// pending gap and proposed store the inputs and deps in edges, it's removed in `remove_entry` - /// here we use `input_pts_iter` and `related_dep_out_points` to find the conflict txs pub(crate) fn resolve_conflict(&mut self, tx: &TransactionView) -> Vec { - let mut to_be_removed = Vec::new(); + let inputs = tx.input_pts_iter(); let mut conflicts = Vec::new(); - for (_, entry) in self.entries.iter() { - let entry = &entry.inner; - let tx_id = entry.proposal_short_id(); - let tx_inputs = entry.transaction().input_pts_iter(); - let deps = entry.related_dep_out_points(); - - // tx input conflict - for i in tx_inputs { - if tx.input_pts_iter().any(|j| i == j) { - to_be_removed.push((tx_id.to_owned(), i.clone())); + for i in inputs { + if let Some(id) = self.edges.remove_input(&i) { + let entries = self.remove_entry_and_descendants(&id); + if !entries.is_empty() { + let reject = Reject::Resolve(OutPointError::Dead(i.clone())); + let rejects = std::iter::repeat(reject).take(entries.len()); + conflicts.extend(entries.into_iter().zip(rejects)); } } - // tx deps conflict - for i in deps { - if tx.input_pts_iter().any(|j| *i == j) { - to_be_removed.push((tx_id.to_owned(), i.clone())); + // deps consumed + if let Some(x) = self.edges.remove_deps(&i) { + for id in x { + let entries = self.remove_entry_and_descendants(&id); + if !entries.is_empty() { + let reject = Reject::Resolve(OutPointError::Dead(i.clone())); + let rejects = std::iter::repeat(reject).take(entries.len()); + conflicts.extend(entries.into_iter().zip(rejects)); + } } } } - for (tx_id, input) in to_be_removed.iter() { - let entries = self.remove_entry_and_descendants(tx_id); - let reject = Reject::Resolve(OutPointError::Dead(input.to_owned())); - let rejects = std::iter::repeat(reject).take(entries.len()); - conflicts.extend(entries.into_iter().zip(rejects)); - } - conflicts } @@ -297,6 +290,7 @@ impl PoolMap { } } + #[cfg(test)] pub(crate) fn remove_entries_by_filter bool>( &mut self, status: &Status, @@ -360,10 +354,11 @@ impl PoolMap { EntryOp::Add => child.add_entry_weight(parent), } let short_id = child.proposal_short_id(); - //TODO: optimize it - self.entries.remove_by_id(&short_id); - self.insert_entry(&child, entry.status) - .expect("pool consistent"); + self.entries.modify_by_id(&short_id, |e| { + e.score = child.as_score_key(); + e.evict_key = child.as_evict_key(); + e.inner = child; + }); } } @@ -437,7 +432,7 @@ impl PoolMap { } /// Record the links for entry - fn record_entry_links(&mut self, entry: &mut TxEntry, status: &Status) -> Result { + fn record_entry_links(&mut self, entry: &mut TxEntry) -> Result { // find in pool parents let mut parents: HashSet = HashSet::with_capacity( entry.transaction().inputs().len() + entry.transaction().cell_deps().len(), @@ -476,7 +471,8 @@ impl PoolMap { .expect("pool consistent"); entry.add_entry_weight(&ancestor.inner); } - if *status == Status::Proposed && entry.ancestors_count > self.max_ancestors_count { + if entry.ancestors_count > self.max_ancestors_count { + eprintln!("debug: exceeded maximum ancestors count"); return Err(Reject::ExceededMaximumAncestorsCount); } @@ -534,13 +530,25 @@ impl PoolMap { return Ok(false); } trace!("add_{:?} {}", status, entry.transaction().hash()); - self.record_entry_links(&mut entry, &status)?; + self.record_entry_links(&mut entry)?; self.insert_entry(&entry, status)?; self.record_entry_deps(&entry); self.record_entry_edges(&entry); Ok(true) } + /// Change the status of the entry, only used for `gap_rtx` and `proposed_rtx` + pub(crate) fn set_entry(&mut self, entry: &TxEntry, status: Status) { + let tx_short_id = entry.proposal_short_id(); + let _ = self + .entries + .get_by_id(&tx_short_id) + .expect("unconsistent pool"); + self.entries.modify_by_id(&tx_short_id, |e| { + e.status = status; + }); + } + fn insert_entry(&mut self, entry: &TxEntry, status: Status) -> Result { let tx_short_id = entry.proposal_short_id(); let score = entry.as_score_key(); @@ -558,10 +566,8 @@ impl PoolMap { impl CellProvider for PoolMap { fn cell(&self, out_point: &OutPoint, _eager_load: bool) -> CellStatus { - if let Some(id) = self.edges.get_input_ref(out_point) { - if self.has_proposed(id) { - return CellStatus::Dead; - } + if self.edges.get_input_ref(out_point).is_some() { + return CellStatus::Dead; } match self.edges.get_output_ref(out_point) { Some(OutPointStatus::UnConsumed) => { @@ -571,7 +577,7 @@ impl CellProvider for PoolMap { .build(); CellStatus::live_cell(cell_meta) } - Some(OutPointStatus::Consumed(id)) if self.has_proposed(id) => CellStatus::Dead, + Some(OutPointStatus::Consumed(_id)) => CellStatus::Dead, _ => CellStatus::Unknown, } } @@ -579,13 +585,11 @@ impl CellProvider for PoolMap { impl CellChecker for PoolMap { fn is_live(&self, out_point: &OutPoint) -> Option { - if let Some(id) = self.edges.get_input_ref(out_point) { - if self.has_proposed(id) { - return Some(false); - } + if self.edges.get_input_ref(out_point).is_some() { + return Some(false); } match self.edges.get_output_ref(out_point) { - Some(OutPointStatus::Consumed(id)) if self.has_proposed(id) => Some(false), + Some(OutPointStatus::Consumed(_id)) => Some(false), Some(OutPointStatus::UnConsumed) => Some(true), _ => None, } diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index c01ad22b64..41e766e630 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -14,12 +14,10 @@ use ckb_store::ChainStore; use ckb_types::{ core::{ cell::{resolve_transaction, OverlayCellChecker, OverlayCellProvider, ResolvedTransaction}, - cell::{CellChecker, CellMetaBuilder, CellProvider, CellStatus}, tx_pool::{TxPoolEntryInfo, TxPoolIds}, Cycle, TransactionView, UncleBlockView, }, - packed::{Byte32, OutPoint, ProposalShortId}, - prelude::*, + packed::{Byte32, ProposalShortId}, }; use ckb_verification::{cache::CacheEntry, TxVerifyEnv}; use lru::LruCache; @@ -130,6 +128,14 @@ impl TxPool { self.pool_map.get_by_id(id).is_some() } + pub(crate) fn set_entry_proposed(&mut self, entry: &TxEntry) { + self.pool_map.set_entry(entry, Status::Proposed) + } + + pub(crate) fn set_entry_gap(&mut self, entry: &TxEntry) { + self.pool_map.set_entry(entry, Status::Gap) + } + /// Returns tx with cycles corresponding to the id. pub(crate) fn get_tx_with_cycles( &self, @@ -269,51 +275,26 @@ impl TxPool { false } - pub(crate) fn check_rtx_from_pending_and_proposed( - &self, - rtx: &ResolvedTransaction, - ) -> Result<(), Reject> { + pub(crate) fn check_rtx_from_pool(&self, rtx: &ResolvedTransaction) -> Result<(), Reject> { let snapshot = self.snapshot(); - let proposal_checker = OverlayCellChecker::new(&self.pool_map, snapshot); - let checker = OverlayCellChecker::new(self, &proposal_checker); + let checker = OverlayCellChecker::new(&self.pool_map, snapshot); let mut seen_inputs = HashSet::new(); rtx.check(&mut seen_inputs, &checker, snapshot) .map_err(Reject::Resolve) } - pub(crate) fn check_rtx_from_proposed(&self, rtx: &ResolvedTransaction) -> Result<(), Reject> { - let snapshot = self.snapshot(); - let proposal_checker = OverlayCellChecker::new(&self.pool_map, snapshot); - let mut seen_inputs = HashSet::new(); - rtx.check(&mut seen_inputs, &proposal_checker, snapshot) - .map_err(Reject::Resolve) - } - - pub(crate) fn resolve_tx_from_pending_and_proposed( + pub(crate) fn resolve_tx_from_pool( &self, tx: TransactionView, ) -> Result, Reject> { let snapshot = self.snapshot(); - let proposed_provider = OverlayCellProvider::new(&self.pool_map, snapshot); - let provider = OverlayCellProvider::new(self, &proposed_provider); + let provider = OverlayCellProvider::new(&self.pool_map, snapshot); let mut seen_inputs = HashSet::new(); resolve_transaction(tx, &mut seen_inputs, &provider, snapshot) .map(Arc::new) .map_err(Reject::Resolve) } - pub(crate) fn resolve_tx_from_proposed( - &self, - tx: TransactionView, - ) -> Result, Reject> { - let snapshot = self.snapshot(); - let proposed_provider = OverlayCellProvider::new(&self.pool_map, snapshot); - let mut seen_inputs = HashSet::new(); - resolve_transaction(tx, &mut seen_inputs, &proposed_provider, snapshot) - .map(Arc::new) - .map_err(Reject::Resolve) - } - pub(crate) fn gap_rtx( &mut self, cache_entry: CacheEntry, @@ -324,7 +305,6 @@ impl TxPool { let snapshot = self.cloned_snapshot(); let tip_header = snapshot.tip_header(); let tx_env = Arc::new(TxVerifyEnv::new_proposed(tip_header, 0)); - self.check_rtx_from_pending_and_proposed(&rtx)?; let max_cycles = snapshot.consensus().max_block_cycles(); let verified = verify_rtx( @@ -338,12 +318,8 @@ impl TxPool { let entry = TxEntry::new_with_timestamp(rtx, verified.cycles, verified.fee, size, timestamp); - let tx_hash = entry.transaction().hash(); - if self.add_gap(entry).unwrap_or(false) { - Ok(CacheEntry::Completed(verified)) - } else { - Err(Reject::Duplicated(tx_hash)) - } + self.set_entry_gap(&entry); + Ok(CacheEntry::Completed(verified)) } pub(crate) fn proposed_rtx( @@ -356,7 +332,6 @@ impl TxPool { let snapshot = self.cloned_snapshot(); let tip_header = snapshot.tip_header(); let tx_env = Arc::new(TxVerifyEnv::new_proposed(tip_header, 1)); - self.check_rtx_from_proposed(&rtx)?; let max_cycles = snapshot.consensus().max_block_cycles(); let verified = verify_rtx( @@ -375,11 +350,8 @@ impl TxPool { tx_hash, entry.proposal_short_id() ); - if self.add_proposed(entry)? { - Ok(CacheEntry::Completed(verified)) - } else { - Err(Reject::Duplicated(tx_hash)) - } + self.set_entry_proposed(&entry); + Ok(CacheEntry::Completed(verified)) } /// Get to-be-proposal transactions that may be included in the next block. @@ -538,49 +510,3 @@ impl TxPool { } } } - -/// This is a hack right now, we use `CellProvider` to check if a transaction is in `Pending` or `Gap` status. -/// To make sure the behavior is same as before, we need to remove this if we have finished replace-by-fee strategy. -impl CellProvider for TxPool { - fn cell(&self, out_point: &OutPoint, _eager_load: bool) -> CellStatus { - let tx_hash = out_point.tx_hash(); - match self - .pool_map - .get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) - { - Some(pool_entry) if pool_entry.status != Status::Proposed => { - match pool_entry - .inner - .transaction() - .output_with_data(out_point.index().unpack()) - { - Some((output, data)) => { - let cell_meta = CellMetaBuilder::from_cell_output(output, data) - .out_point(out_point.to_owned()) - .build(); - CellStatus::live_cell(cell_meta) - } - None => CellStatus::Unknown, - } - } - _ => CellStatus::Unknown, - } - } -} - -impl CellChecker for TxPool { - fn is_live(&self, out_point: &OutPoint) -> Option { - let tx_hash = out_point.tx_hash(); - match self - .pool_map - .get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) - { - Some(pool_entry) if pool_entry.status != Status::Proposed => pool_entry - .inner - .transaction() - .output_with_data(out_point.index().unpack()) - .map(|_| true), - _ => None, - } - } -} diff --git a/tx-pool/src/process.rs b/tx-pool/src/process.rs index 00dde6d264..354e56cac0 100644 --- a/tx-pool/src/process.rs +++ b/tx-pool/src/process.rs @@ -966,12 +966,7 @@ fn check_rtx( } else { TxStatus::Fresh }; - if tx_status == TxStatus::Proposed { - tx_pool.check_rtx_from_proposed(rtx) - } else { - tx_pool.check_rtx_from_pending_and_proposed(rtx) - } - .map(|_| tx_status) + tx_pool.check_rtx_from_pool(rtx).map(|_| tx_status) } fn resolve_tx(tx_pool: &TxPool, snapshot: &Snapshot, tx: TransactionView) -> ResolveResult { @@ -983,12 +978,7 @@ fn resolve_tx(tx_pool: &TxPool, snapshot: &Snapshot, tx: TransactionView) -> Res } else { TxStatus::Fresh }; - if tx_status == TxStatus::Proposed { - tx_pool.resolve_tx_from_proposed(tx) - } else { - tx_pool.resolve_tx_from_pending_and_proposed(tx) - } - .map(|rtx| (rtx, tx_status)) + tx_pool.resolve_tx_from_pool(tx).map(|rtx| (rtx, tx_status)) } fn _submit_entry( @@ -1049,32 +1039,26 @@ fn _update_tx_pool_for_reorg( let mut proposals = Vec::new(); let mut gaps = Vec::new(); - tx_pool - .pool_map - .remove_entries_by_filter(&Status::Gap, |id, tx_entry| { - if snapshot.proposals().contains_proposed(id) { - proposals.push(tx_entry.clone()); - true - } else { - false - } - }); + for entry in tx_pool.pool_map.entries.get_by_status(&Status::Gap) { + let e = &entry.inner; + let short_id = e.proposal_short_id(); + if snapshot.proposals().contains_proposed(&short_id) { + proposals.push(e.clone()); + } + } - tx_pool - .pool_map - .remove_entries_by_filter(&Status::Pending, |id, tx_entry| { - if snapshot.proposals().contains_proposed(id) { - proposals.push(tx_entry.clone()); - true - } else if snapshot.proposals().contains_gap(id) { - gaps.push(tx_entry.clone()); - true - } else { - false - } - }); + for entry in tx_pool.pool_map.entries.get_by_status(&Status::Pending) { + let e = &entry.inner; + let short_id = e.proposal_short_id(); + if snapshot.proposals().contains_proposed(&short_id) { + proposals.push(e.clone()); + } else if snapshot.proposals().contains_gap(&short_id) { + gaps.push(e.clone()); + } + } for entry in proposals { + debug!("begin to proposed: {:x}", entry.transaction().hash()); let cached = CacheEntry::completed(entry.cycles, entry.fee); if let Err(e) = tx_pool.proposed_rtx(cached, entry.size, entry.timestamp, Arc::clone(&entry.rtx)) @@ -1086,6 +1070,7 @@ fn _update_tx_pool_for_reorg( } for entry in gaps { + debug!("begin to gap: {:x}", entry.transaction().hash()); let tx_hash = entry.transaction().hash(); let cached = CacheEntry::completed(entry.cycles, entry.fee); if let Err(e) = From 92bc7339264813439cab9de9d2db52738cd2de9a Mon Sep 17 00:00:00 2001 From: yukang Date: Mon, 5 Jun 2023 16:51:52 +0800 Subject: [PATCH 05/58] fix makefile so that we may run a specific integration test with environment name --- Makefile | 2 +- test/src/main.rs | 8 ++++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/Makefile b/Makefile index 7d7adb98fb..3d8a92ba2b 100644 --- a/Makefile +++ b/Makefile @@ -5,7 +5,7 @@ MOLC_VERSION := 0.7.5 VERBOSE := $(if ${CI},--verbose,) CLIPPY_OPTS := -D warnings -D clippy::clone_on_ref_ptr -D clippy::enum_glob_use -D clippy::fallible_impl_from \ -A clippy::mutable_key_type -A clippy::upper_case_acronyms -CKB_TEST_ARGS := ${CKB_TEST_ARGS} -c 4 +CKB_TEST_ARGS := -c 4 ${CKB_TEST_ARGS} CKB_FEATURES ?= deadlock_detection,with_sentry ALL_FEATURES := deadlock_detection,with_sentry,with_dns_seeding,profiling,march-native CKB_BENCH_FEATURES ?= ci diff --git a/test/src/main.rs b/test/src/main.rs index f321bfe0a7..7d095cbef0 100644 --- a/test/src/main.rs +++ b/test/src/main.rs @@ -294,8 +294,12 @@ fn clap_app() -> App<'static> { .value_name("SECONDS") .help("Exit when total running time exceeds this limit"), ) - .arg(Arg::with_name("list-specs").long("list-specs")) - .arg(Arg::with_name("specs").multiple(true)) + .arg( + Arg::with_name("list-specs") + .long("list-specs") + .help("list all specs"), + ) + .arg(Arg::with_name("specs").multiple(true).help("Specs to run")) .arg( Arg::with_name("concurrent") .short('c') From 180159e68331f92a0d904c27e3ff698e8f6033d3 Mon Sep 17 00:00:00 2001 From: yukang Date: Mon, 5 Jun 2023 18:22:15 +0800 Subject: [PATCH 06/58] fix TransactionRelayConflict --- test/src/specs/relay/transaction_relay.rs | 16 ++++++++++++---- tx-pool/src/component/pool_map.rs | 3 ++- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/test/src/specs/relay/transaction_relay.rs b/test/src/specs/relay/transaction_relay.rs index e5af42eb45..137d36964a 100644 --- a/test/src/specs/relay/transaction_relay.rs +++ b/test/src/specs/relay/transaction_relay.rs @@ -5,6 +5,7 @@ use crate::util::transaction::{always_success_transaction, always_success_transa use crate::utils::{build_relay_tx_hashes, build_relay_txs, sleep, wait_until}; use crate::{Net, Node, Spec}; use ckb_constant::sync::RETRY_ASK_TX_TIMEOUT_INCREASE; +use ckb_jsonrpc_types::Status; use ckb_logger::info; use ckb_network::SupportProtocols; use ckb_types::{ @@ -234,11 +235,18 @@ impl Spec for TransactionRelayConflict { node0.rpc_client().send_transaction(tx1.data().into()); sleep(6); + let res = node0.rpc_client().get_transaction(tx1.hash()); + assert!(matches!(res.tx_status.status, Status::Pending)); + let res = node0 .rpc_client() .send_transaction_result(tx2.data().into()); assert!(res.is_err()); - eprintln!("res: {:?}", res); + assert!(res + .err() + .unwrap() + .to_string() + .contains("TransactionFailedToResolve: Resolve failed Dead")); let relayed = wait_until(20, || { [tx1.hash()].iter().all(|hash| { @@ -258,11 +266,10 @@ impl Spec for TransactionRelayConflict { node0.wait_for_tx_pool(); node1.wait_for_tx_pool(); - /* let ret = node1 .rpc_client() .get_transaction_with_verbosity(tx2.hash(), 1); - assert!(matches!(ret.tx_status.status, Status::Rejected)); + assert!(matches!(ret.tx_status.status, Status::Unknown)); node0.remove_transaction(tx1.hash()); node0.remove_transaction(tx2.hash()); @@ -284,9 +291,11 @@ impl Spec for TransactionRelayConflict { let relayed = wait_until(10, || { // re-broadcast + // TODO: (yukang) double comfirm this behavior let _ = node1 .rpc_client() .send_transaction_result(tx2.data().into()); + node0 .rpc_client() .get_transaction(tx2.hash()) @@ -294,6 +303,5 @@ impl Spec for TransactionRelayConflict { .is_some() }); assert!(relayed, "Transaction should be relayed to node1"); - */ } } diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index 50c5c393dc..89e4d0980f 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -60,6 +60,7 @@ impl MultiIndexPoolEntryMap { pub fn score_sorted_iter(&self) -> impl Iterator { // Note: multi_index don't support reverse order iteration now // so we need to collect and reverse + // TODO: @wyjin will add reverse order iteration support for multi_index let entries = self .iter_by_score() .filter(|entry| entry.status == Status::Proposed) @@ -202,7 +203,7 @@ impl PoolMap { let mut removed = vec![]; removed_ids.extend(self.calc_descendants(id)); - // update links state for remove + // update links state for remove, so that we won't update_descendants_index_key in remove_entry for id in &removed_ids { self.remove_entry_links(id); } From ea00a950321e7ddfc60bf9a55f317390674dadfc Mon Sep 17 00:00:00 2001 From: yukang Date: Tue, 6 Jun 2023 10:32:57 +0800 Subject: [PATCH 07/58] upgrade multi_index_map for iterator rev --- Cargo.lock | 4 +- tx-pool/Cargo.toml | 2 +- tx-pool/src/component/pool_map.rs | 70 +++++++++++++------------------ tx-pool/src/pool.rs | 2 +- 4 files changed, 34 insertions(+), 44 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 0072b84283..70723dc6bc 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1552,9 +1552,9 @@ checksum = "83869c9d322de1ddbfde5b54b7376f9a1ac32273c50e21cdd5e8a1bd1a1cf632" [[package]] name = "ckb_multi_index_map" -version = "0.0.1" +version = "0.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2adba00c3dcb84fc4634c948cf3d24c05ce3193810bfa568effe13ad814f662a" +checksum = "53c20823dfd9f9a8e30faa3b0bdcab4801fb2544957586fada3884c78dcdf38b" dependencies = [ "convert_case 0.6.0", "proc-macro-error", diff --git a/tx-pool/Cargo.toml b/tx-pool/Cargo.toml index d8a16bff50..04e465561a 100644 --- a/tx-pool/Cargo.toml +++ b/tx-pool/Cargo.toml @@ -36,7 +36,7 @@ sentry = { version = "0.26.0", optional = true } serde_json = "1.0" rand = "0.8.4" hyper = { version = "0.14", features = ["http1", "client", "tcp"] } -ckb_multi_index_map = "0.0.1" # ckb team fork crate +ckb_multi_index_map = "0.0.2" # ckb team fork crate slab = "0.4" rustc-hash = "1.1" tokio-util = "0.7.8" diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index 89e4d0980f..dd6b471e7e 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -55,20 +55,6 @@ pub struct PoolEntry { pub inner: TxEntry, } -impl MultiIndexPoolEntryMap { - /// sorted by ancestor score from higher to lower - pub fn score_sorted_iter(&self) -> impl Iterator { - // Note: multi_index don't support reverse order iteration now - // so we need to collect and reverse - // TODO: @wyjin will add reverse order iteration support for multi_index - let entries = self - .iter_by_score() - .filter(|entry| entry.status == Status::Proposed) - .collect::>(); - entries.into_iter().rev().map(move |entry| &entry.inner) - } -} - pub struct PoolMap { /// The pool entries with different kinds of sort strategies pub(crate) entries: MultiIndexPoolEntryMap, @@ -150,7 +136,11 @@ impl PoolMap { } pub(crate) fn score_sorted_iter(&self) -> impl Iterator { - self.entries.score_sorted_iter() + self.entries + .iter_by_score() + .rev() + .filter(|entry| entry.status == Status::Proposed) + .map(|entry| &entry.inner) } pub(crate) fn get(&self, id: &ProposalShortId) -> Option<&TxEntry> { @@ -187,6 +177,31 @@ impl PoolMap { }) } + pub(crate) fn add_entry(&mut self, mut entry: TxEntry, status: Status) -> Result { + let tx_short_id = entry.proposal_short_id(); + if self.entries.get_by_id(&tx_short_id).is_some() { + return Ok(false); + } + trace!("add_{:?} {}", status, entry.transaction().hash()); + self.record_entry_links(&mut entry)?; + self.insert_entry(&entry, status)?; + self.record_entry_deps(&entry); + self.record_entry_edges(&entry); + Ok(true) + } + + /// Change the status of the entry, only used for `gap_rtx` and `proposed_rtx` + pub(crate) fn set_entry(&mut self, entry: &TxEntry, status: Status) { + let tx_short_id = entry.proposal_short_id(); + let _ = self + .entries + .get_by_id(&tx_short_id) + .expect("unconsistent pool"); + self.entries.modify_by_id(&tx_short_id, |e| { + e.status = status; + }); + } + pub(crate) fn remove_entry(&mut self, id: &ProposalShortId) -> Option { if let Some(entry) = self.entries.remove_by_id(id) { self.update_descendants_index_key(&entry.inner, EntryOp::Remove); @@ -525,31 +540,6 @@ impl PoolMap { self.edges.header_deps.remove(&id); } - pub(crate) fn add_entry(&mut self, mut entry: TxEntry, status: Status) -> Result { - let tx_short_id = entry.proposal_short_id(); - if self.entries.get_by_id(&tx_short_id).is_some() { - return Ok(false); - } - trace!("add_{:?} {}", status, entry.transaction().hash()); - self.record_entry_links(&mut entry)?; - self.insert_entry(&entry, status)?; - self.record_entry_deps(&entry); - self.record_entry_edges(&entry); - Ok(true) - } - - /// Change the status of the entry, only used for `gap_rtx` and `proposed_rtx` - pub(crate) fn set_entry(&mut self, entry: &TxEntry, status: Status) { - let tx_short_id = entry.proposal_short_id(); - let _ = self - .entries - .get_by_id(&tx_short_id) - .expect("unconsistent pool"); - self.entries.modify_by_id(&tx_short_id, |e| { - e.status = status; - }); - } - fn insert_entry(&mut self, entry: &TxEntry, status: Status) -> Result { let tx_short_id = entry.proposal_short_id(); let score = entry.as_score_key(); diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 41e766e630..fe58a3abe7 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -356,7 +356,7 @@ impl TxPool { /// Get to-be-proposal transactions that may be included in the next block. /// TODO: do we need to consider the something like score, so that we can - /// provide best transactions to be proposed. + /// provide best transactions to be proposed. pub(crate) fn get_proposals( &self, limit: usize, From 9d375c8693f09055ee8c86bbce29d16065e095c1 Mon Sep 17 00:00:00 2001 From: yukang Date: Tue, 6 Jun 2023 10:53:29 +0800 Subject: [PATCH 08/58] enable PoolResolveConflictAfterReorg --- test/src/main.rs | 1 + test/src/specs/tx_pool/collision.rs | 12 ++++++-- .../tx_pool/different_txs_with_same_input.rs | 30 +++++++++---------- 3 files changed, 25 insertions(+), 18 deletions(-) diff --git a/test/src/main.rs b/test/src/main.rs index 7d095cbef0..00cc5a45fd 100644 --- a/test/src/main.rs +++ b/test/src/main.rs @@ -406,6 +406,7 @@ fn all_specs() -> Vec> { Box::new(GetRawTxPool), Box::new(PoolReconcile), Box::new(PoolResurrect), + Box::new(PoolResolveConflictAfterReorg), Box::new(InvalidHeaderDep), #[cfg(not(target_os = "windows"))] Box::new(PoolPersisted), diff --git a/test/src/specs/tx_pool/collision.rs b/test/src/specs/tx_pool/collision.rs index 7dc606d459..0d93697c03 100644 --- a/test/src/specs/tx_pool/collision.rs +++ b/test/src/specs/tx_pool/collision.rs @@ -1,4 +1,6 @@ -use crate::util::check::{is_transaction_committed, is_transaction_pending}; +use crate::util::check::{ + is_transaction_committed, is_transaction_pending, is_transaction_rejected, +}; use crate::utils::{assert_send_transaction_fail, blank, commit, propose}; use crate::{Node, Spec}; use ckb_types::bytes::Bytes; @@ -6,7 +8,7 @@ use ckb_types::core::{capacity_bytes, Capacity, TransactionView}; use ckb_types::prelude::*; // Convention: -// * `tx1` and `tx2` are cousin transactions, with the same transaction content, expect the +// * `tx1` and `tx2` are cousin transactions, with the same transaction content, except the // witnesses. Hence `tx1` and `tx2` have the same tx_hash/proposal-id but different witness_hash. pub struct TransactionHashCollisionDifferentWitnessHashes; @@ -95,8 +97,8 @@ impl Spec for ConflictInGap { (0..window.closest() - 1).for_each(|_| { node.submit_block(&blank(node)); }); + node.submit_block(&propose(node, &[&txb])); - //node.submit_block(&propose(node, &[&txb])); let block = node.new_block(None, None, None); assert_eq!(&[txa], &block.transactions()[1..]); @@ -162,6 +164,8 @@ impl Spec for RemoveConflictFromPending { assert!(res.is_err()); assert!(is_transaction_pending(node, &txa)); + assert!(is_transaction_rejected(node, &txb)); + assert!(is_transaction_rejected(node, &txc)); node.submit_block(&propose(node, &[&txa])); (0..window.closest()).for_each(|_| { @@ -171,6 +175,8 @@ impl Spec for RemoveConflictFromPending { node.wait_for_tx_pool(); assert!(is_transaction_committed(node, &txa)); + assert!(is_transaction_rejected(node, &txb)); + assert!(is_transaction_rejected(node, &txc)); } } diff --git a/test/src/specs/tx_pool/different_txs_with_same_input.rs b/test/src/specs/tx_pool/different_txs_with_same_input.rs index 8539c7c8c6..a816bd2eb9 100644 --- a/test/src/specs/tx_pool/different_txs_with_same_input.rs +++ b/test/src/specs/tx_pool/different_txs_with_same_input.rs @@ -51,11 +51,11 @@ impl Spec for DifferentTxsWithSameInput { assert!(!commit_txs_hash.contains(&tx2.hash())); // when tx1 was confirmed, tx2 should be rejected - // let ret = node0.rpc_client().get_transaction(tx2.hash()); - // assert!( - // matches!(ret.tx_status.status, Status::Rejected), - // "tx2 should be rejected" - // ); + let ret = node0.rpc_client().get_transaction(tx2.hash()); + assert!( + matches!(ret.tx_status.status, Status::Rejected), + "tx2 should be rejected" + ); // verbosity = 1 let ret = node0 @@ -64,11 +64,11 @@ impl Spec for DifferentTxsWithSameInput { assert!(ret.transaction.is_none()); assert!(matches!(ret.tx_status.status, Status::Committed)); - // let ret = node0 - // .rpc_client() - // .get_transaction_with_verbosity(tx2.hash(), 1); - // assert!(ret.transaction.is_none()); - // assert!(matches!(ret.tx_status.status, Status::Rejected)); + let ret = node0 + .rpc_client() + .get_transaction_with_verbosity(tx2.hash(), 1); + assert!(ret.transaction.is_none()); + assert!(matches!(ret.tx_status.status, Status::Rejected)); // verbosity = 2 let ret = node0 @@ -77,10 +77,10 @@ impl Spec for DifferentTxsWithSameInput { assert!(ret.transaction.is_some()); assert!(matches!(ret.tx_status.status, Status::Committed)); - // let ret = node0 - // .rpc_client() - // .get_transaction_with_verbosity(tx2.hash(), 2); - // assert!(ret.transaction.is_none()); - // assert!(matches!(ret.tx_status.status, Status::Rejected)); + let ret = node0 + .rpc_client() + .get_transaction_with_verbosity(tx2.hash(), 2); + assert!(ret.transaction.is_none()); + assert!(matches!(ret.tx_status.status, Status::Rejected)); } } From d46bd54a2df9602e2b5fc9bdf7740c34e2a1548c Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 8 Jun 2023 00:17:16 +0800 Subject: [PATCH 09/58] comments feedback and code cleanup --- tx-pool/src/component/pool_map.rs | 2 +- tx-pool/src/pool.rs | 2 +- tx-pool/src/process.rs | 10 ++-------- tx-pool/src/service.rs | 6 ++++-- 4 files changed, 8 insertions(+), 12 deletions(-) diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index dd6b471e7e..b0d591850b 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -182,7 +182,7 @@ impl PoolMap { if self.entries.get_by_id(&tx_short_id).is_some() { return Ok(false); } - trace!("add_{:?} {}", status, entry.transaction().hash()); + trace!("pool_map.add_{:?} {}", status, entry.transaction().hash()); self.record_entry_links(&mut entry)?; self.insert_entry(&entry, status)?; self.record_entry_deps(&entry); diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index fe58a3abe7..f271115cea 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -211,7 +211,7 @@ impl TxPool { } } - // Remove transactions from the pool until total size < size_limit. + // Remove transactions from the pool until total size <= size_limit. pub(crate) fn limit_size(&mut self, callbacks: &Callbacks) { while self.total_tx_size > self.config.max_tx_pool_size { if let Some(id) = self.pool_map.next_evict_entry() { diff --git a/tx-pool/src/process.rs b/tx-pool/src/process.rs index 354e56cac0..2ee84e2f6e 100644 --- a/tx-pool/src/process.rs +++ b/tx-pool/src/process.rs @@ -987,21 +987,15 @@ fn _submit_entry( entry: TxEntry, callbacks: &Callbacks, ) -> Result<(), Reject> { - let tx_hash = entry.transaction().hash(); match status { TxStatus::Fresh => { - if tx_pool.add_pending(entry.clone()).unwrap_or(false) { + if tx_pool.add_pending(entry.clone())? { callbacks.call_pending(tx_pool, &entry); - } else { - return Err(Reject::Duplicated(tx_hash)); } } - TxStatus::Gap => { - if tx_pool.add_gap(entry.clone()).unwrap_or(false) { + if tx_pool.add_gap(entry.clone())? { callbacks.call_pending(tx_pool, &entry); - } else { - return Err(Reject::Duplicated(tx_hash)); } } TxStatus::Proposed => { diff --git a/tx-pool/src/service.rs b/tx-pool/src/service.rs index a008502c62..1a187615c8 100644 --- a/tx-pool/src/service.rs +++ b/tx-pool/src/service.rs @@ -981,13 +981,15 @@ impl TxPoolService { match target { PlugTarget::Pending => { for entry in entries { - tx_pool.add_pending(entry).unwrap(); + if let Err(err) = tx_pool.add_pending(entry) { + error!("plug entry add_pending error {}", err); + } } } PlugTarget::Proposed => { for entry in entries { if let Err(err) = tx_pool.add_proposed(entry) { - error!("plug entry error {}", err); + error!("plug entry add_proposed error {}", err); } } } From 466c9712cf4f3a8d4e5bbfef1e22676319e058dc Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 8 Jun 2023 10:03:13 +0800 Subject: [PATCH 10/58] add index_map shrink --- tx-pool/src/component/pool_map.rs | 6 +----- tx-pool/src/pool.rs | 1 + 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index b0d591850b..df829c0875 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -328,12 +328,8 @@ impl PoolMap { self.entries.iter().map(|(_, entry)| entry) } - pub(crate) fn iter_by_evict_key(&self) -> impl Iterator { - self.entries.iter_by_evict_key() - } - pub(crate) fn next_evict_entry(&self) -> Option { - self.iter_by_evict_key() + self.entries.iter_by_evict_key() .next() .map(|entry| entry.id.clone()) } diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index f271115cea..dfe8b5a425 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -230,6 +230,7 @@ impl TxPool { } } } + self.pool_map.entries.shrink_to_fit(); } // remove transaction with detached proposal from gap and proposed From 12b211686472d440d24fdd0c5d122d9071345295 Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 8 Jun 2023 15:27:52 +0800 Subject: [PATCH 11/58] confirmed TransactionRelayConflict is ok --- test/src/specs/relay/transaction_relay.rs | 12 +++++++++++- tx-pool/src/component/pool_map.rs | 3 ++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/test/src/specs/relay/transaction_relay.rs b/test/src/specs/relay/transaction_relay.rs index 137d36964a..75a34e0df6 100644 --- a/test/src/specs/relay/transaction_relay.rs +++ b/test/src/specs/relay/transaction_relay.rs @@ -266,6 +266,11 @@ impl Spec for TransactionRelayConflict { node0.wait_for_tx_pool(); node1.wait_for_tx_pool(); + let ret = node1 + .rpc_client() + .get_transaction_with_verbosity(tx1.hash(), 1); + assert!(matches!(ret.tx_status.status, Status::Proposed)); + let ret = node1 .rpc_client() .get_transaction_with_verbosity(tx2.hash(), 1); @@ -278,6 +283,12 @@ impl Spec for TransactionRelayConflict { node0.wait_for_tx_pool(); node1.wait_for_tx_pool(); + // make sure tx1 is removed from tx-pool + let ret = node1 + .rpc_client() + .get_transaction_with_verbosity(tx1.hash(), 1); + assert!(matches!(ret.tx_status.status, Status::Unknown)); + let result = wait_until(5, || { let tx_pool_info = node0.get_tip_tx_pool_info(); tx_pool_info.orphan.value() == 0 && tx_pool_info.pending.value() == 0 @@ -291,7 +302,6 @@ impl Spec for TransactionRelayConflict { let relayed = wait_until(10, || { // re-broadcast - // TODO: (yukang) double comfirm this behavior let _ = node1 .rpc_client() .send_transaction_result(tx2.data().into()); diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index df829c0875..cc1c20d684 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -329,7 +329,8 @@ impl PoolMap { } pub(crate) fn next_evict_entry(&self) -> Option { - self.entries.iter_by_evict_key() + self.entries + .iter_by_evict_key() .next() .map(|entry| entry.id.clone()) } From abbcd23f3396c16dac4a16df64d3ddd963950221 Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 8 Jun 2023 18:39:42 +0800 Subject: [PATCH 12/58] clean up proposed_rtx and gap_rtx --- tx-pool/src/component/pool_map.rs | 14 ++--- tx-pool/src/pool.rs | 99 ++++++++++++------------------- tx-pool/src/process.rs | 42 ++++++------- 3 files changed, 63 insertions(+), 92 deletions(-) diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index cc1c20d684..dadb3b3bc2 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -126,6 +126,10 @@ impl PoolMap { self.entries.get_by_id(id) } + pub(crate) fn get_by_status(&self, status: &Status) -> Vec<&PoolEntry> { + self.entries.get_by_status(status) + } + pub(crate) fn pending_size(&self) -> usize { self.entries.get_by_status(&Status::Pending).len() + self.entries.get_by_status(&Status::Gap).len() @@ -191,13 +195,9 @@ impl PoolMap { } /// Change the status of the entry, only used for `gap_rtx` and `proposed_rtx` - pub(crate) fn set_entry(&mut self, entry: &TxEntry, status: Status) { - let tx_short_id = entry.proposal_short_id(); - let _ = self - .entries - .get_by_id(&tx_short_id) - .expect("unconsistent pool"); - self.entries.modify_by_id(&tx_short_id, |e| { + pub(crate) fn set_entry(&mut self, short_id: &ProposalShortId, status: Status) { + let _ = self.entries.get_by_id(short_id).expect("unconsistent pool"); + self.entries.modify_by_id(short_id, |e| { e.status = status; }); } diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index dfe8b5a425..88564c2058 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -6,7 +6,6 @@ use crate::callback::Callbacks; use crate::component::pool_map::{PoolEntry, PoolMap, Status}; use crate::component::recent_reject::RecentReject; use crate::error::Reject; -use crate::util::verify_rtx; use ckb_app_config::TxPoolConfig; use ckb_logger::{debug, error, warn}; use ckb_snapshot::Snapshot; @@ -19,7 +18,6 @@ use ckb_types::{ }, packed::{Byte32, ProposalShortId}, }; -use ckb_verification::{cache::CacheEntry, TxVerifyEnv}; use lru::LruCache; use std::collections::HashSet; use std::sync::Arc; @@ -72,7 +70,7 @@ impl TxPool { } fn get_by_status(&self, status: &Status) -> Vec<&PoolEntry> { - self.pool_map.entries.get_by_status(status) + self.pool_map.get_by_status(status) } /// Get tx-pool size @@ -128,12 +126,12 @@ impl TxPool { self.pool_map.get_by_id(id).is_some() } - pub(crate) fn set_entry_proposed(&mut self, entry: &TxEntry) { - self.pool_map.set_entry(entry, Status::Proposed) + pub(crate) fn set_entry_proposed(&mut self, short_id: &ProposalShortId) { + self.pool_map.set_entry(short_id, Status::Proposed) } - pub(crate) fn set_entry_gap(&mut self, entry: &TxEntry) { - self.pool_map.set_entry(entry, Status::Gap) + pub(crate) fn set_entry_gap(&mut self, short_id: &ProposalShortId) { + self.pool_map.set_entry(short_id, Status::Gap) } /// Returns tx with cycles corresponding to the id. @@ -146,6 +144,10 @@ impl TxPool { .map(|entry| (entry.inner.transaction().clone(), entry.inner.cycles)) } + pub(crate) fn get_pool_entry(&self, id: &ProposalShortId) -> Option<&PoolEntry> { + self.pool_map.get_by_id(id) + } + pub(crate) fn get_tx_from_pool(&self, id: &ProposalShortId) -> Option<&TransactionView> { self.pool_map .get_by_id(id) @@ -296,63 +298,36 @@ impl TxPool { .map_err(Reject::Resolve) } - pub(crate) fn gap_rtx( - &mut self, - cache_entry: CacheEntry, - size: usize, - timestamp: u64, - rtx: Arc, - ) -> Result { - let snapshot = self.cloned_snapshot(); - let tip_header = snapshot.tip_header(); - let tx_env = Arc::new(TxVerifyEnv::new_proposed(tip_header, 0)); - - let max_cycles = snapshot.consensus().max_block_cycles(); - let verified = verify_rtx( - snapshot, - Arc::clone(&rtx), - tx_env, - &Some(cache_entry), - max_cycles, - )?; - - let entry = - TxEntry::new_with_timestamp(rtx, verified.cycles, verified.fee, size, timestamp); - - self.set_entry_gap(&entry); - Ok(CacheEntry::Completed(verified)) + pub(crate) fn gap_rtx(&mut self, short_id: &ProposalShortId) -> Result<(), Reject> { + match self.get_pool_entry(short_id) { + Some(entry) => { + let tx_hash = entry.inner.transaction().hash(); + if entry.status == Status::Gap { + Err(Reject::Duplicated(tx_hash)) + } else { + debug!("gap_rtx: {:?} => {:?}", tx_hash, short_id); + self.set_entry_gap(short_id); + Ok(()) + } + } + None => Err(Reject::Malformed(String::from("invalid short_id"))), + } } - pub(crate) fn proposed_rtx( - &mut self, - cache_entry: CacheEntry, - size: usize, - timestamp: u64, - rtx: Arc, - ) -> Result { - let snapshot = self.cloned_snapshot(); - let tip_header = snapshot.tip_header(); - let tx_env = Arc::new(TxVerifyEnv::new_proposed(tip_header, 1)); - - let max_cycles = snapshot.consensus().max_block_cycles(); - let verified = verify_rtx( - snapshot, - Arc::clone(&rtx), - tx_env, - &Some(cache_entry), - max_cycles, - )?; - - let entry = - TxEntry::new_with_timestamp(rtx, verified.cycles, verified.fee, size, timestamp); - let tx_hash = entry.transaction().hash(); - debug!( - "proposed_rtx: {:?} => {:?}", - tx_hash, - entry.proposal_short_id() - ); - self.set_entry_proposed(&entry); - Ok(CacheEntry::Completed(verified)) + pub(crate) fn proposed_rtx(&mut self, short_id: &ProposalShortId) -> Result<(), Reject> { + match self.get_pool_entry(short_id) { + Some(entry) => { + let tx_hash = entry.inner.transaction().hash(); + if entry.status == Status::Proposed { + Err(Reject::Duplicated(tx_hash)) + } else { + debug!("proposed_rtx: {:?} => {:?}", tx_hash, short_id); + self.set_entry_proposed(short_id); + Ok(()) + } + } + None => Err(Reject::Malformed(String::from("invalid short_id"))), + } } /// Get to-be-proposal transactions that may be included in the next block. diff --git a/tx-pool/src/process.rs b/tx-pool/src/process.rs index 2ee84e2f6e..e921203aff 100644 --- a/tx-pool/src/process.rs +++ b/tx-pool/src/process.rs @@ -1034,43 +1034,39 @@ fn _update_tx_pool_for_reorg( let mut gaps = Vec::new(); for entry in tx_pool.pool_map.entries.get_by_status(&Status::Gap) { - let e = &entry.inner; - let short_id = e.proposal_short_id(); + let short_id = entry.inner.proposal_short_id(); if snapshot.proposals().contains_proposed(&short_id) { - proposals.push(e.clone()); + proposals.push((short_id, entry.inner.clone())); } } for entry in tx_pool.pool_map.entries.get_by_status(&Status::Pending) { - let e = &entry.inner; - let short_id = e.proposal_short_id(); + let short_id = entry.inner.proposal_short_id(); + let elem = (short_id.clone(), entry.inner.clone()); if snapshot.proposals().contains_proposed(&short_id) { - proposals.push(e.clone()); + proposals.push(elem); } else if snapshot.proposals().contains_gap(&short_id) { - gaps.push(e.clone()); + gaps.push(elem); } } - for entry in proposals { - debug!("begin to proposed: {:x}", entry.transaction().hash()); - let cached = CacheEntry::completed(entry.cycles, entry.fee); - if let Err(e) = - tx_pool.proposed_rtx(cached, entry.size, entry.timestamp, Arc::clone(&entry.rtx)) - { - callbacks.call_reject(tx_pool, &entry, e.clone()); + for (id, entry) in proposals { + debug!("begin to proposed: {:x}", id); + if let Err(e) = tx_pool.proposed_rtx(&id) { + callbacks.call_reject(tx_pool, &entry, e); } else { - callbacks.call_proposed(tx_pool, &entry, false); + callbacks.call_proposed(tx_pool, &entry, false) } } - for entry in gaps { - debug!("begin to gap: {:x}", entry.transaction().hash()); - let tx_hash = entry.transaction().hash(); - let cached = CacheEntry::completed(entry.cycles, entry.fee); - if let Err(e) = - tx_pool.gap_rtx(cached, entry.size, entry.timestamp, Arc::clone(&entry.rtx)) - { - debug!("Failed to add tx to gap {}, reason: {}", tx_hash, e); + for (id, entry) in gaps { + debug!("begin to gap: {:x}", id); + if let Err(e) = tx_pool.gap_rtx(&id) { + debug!( + "Failed to add tx to gap {}, reason: {}", + entry.transaction().hash(), + e + ); callbacks.call_reject(tx_pool, &entry, e.clone()); } } From 508e281fc4019c852c87885495e6a028ffcfb364 Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 8 Jun 2023 22:10:48 +0800 Subject: [PATCH 13/58] check keep_rejected_tx_hashes_days --- tx-pool/src/pool.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 88564c2058..7e1db62c57 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -465,7 +465,8 @@ impl TxPool { fn build_recent_reject(config: &TxPoolConfig) -> Option { if !config.recent_reject.as_os_str().is_empty() { - let recent_reject_ttl = config.keep_rejected_tx_hashes_days as i32 * 24 * 60 * 60; + let recent_reject_ttl = + u8::max(1, config.keep_rejected_tx_hashes_days) as i32 * 24 * 60 * 60; match RecentReject::new( &config.recent_reject, config.keep_rejected_tx_hashes_count, From f2b3b86b53e50e98b887bdc92afde11b828813bc Mon Sep 17 00:00:00 2001 From: yukang Date: Tue, 13 Jun 2023 16:25:05 +0800 Subject: [PATCH 14/58] add more test for pool --- tx-pool/src/component/tests/proposed.rs | 46 +++++++++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/tx-pool/src/component/tests/proposed.rs b/tx-pool/src/component/tests/proposed.rs index ff5fa67866..153b9324d1 100644 --- a/tx-pool/src/component/tests/proposed.rs +++ b/tx-pool/src/component/tests/proposed.rs @@ -1,7 +1,9 @@ +use crate::component::pool_map::Status; use crate::component::tests::util::{ build_tx, build_tx_with_dep, build_tx_with_header_dep, DEFAULT_MAX_ANCESTORS_COUNT, MOCK_CYCLES, MOCK_FEE, MOCK_SIZE, }; + use crate::component::{entry::TxEntry, pool_map::PoolMap}; use ckb_types::{ bytes::Bytes, @@ -695,3 +697,47 @@ fn test_max_ancestors_with_dep() { assert_eq!(pool.edges.inputs_len(), 1); assert_eq!(pool.edges.outputs_len(), 1); } + +#[test] +fn test_container_bench_add_limits() { + use rand::Rng; + let mut rng = rand::thread_rng(); + + let mut pool = PoolMap::new(1000000); + let tx1 = TxEntry::dummy_resolve( + TransactionBuilder::default().build(), + 100, + Capacity::shannons(100), + 100, + ); + pool.add_entry(tx1.clone(), Status::Proposed).unwrap(); + let mut prev_tx = tx1; + + for _i in 0..1000 { + let next_tx = TxEntry::dummy_resolve( + TransactionBuilder::default() + .input( + CellInput::new_builder() + .previous_output( + OutPoint::new_builder() + .tx_hash(prev_tx.transaction().hash()) + .index(0u32.pack()) + .build(), + ) + .build(), + ) + .witness(Bytes::new().pack()) + .build(), + rng.gen_range(0..1000), + Capacity::shannons(200), + rng.gen_range(0..1000), + ); + pool.add_entry(next_tx.clone(), Status::Proposed).unwrap(); + prev_tx = next_tx; + } + assert_eq!(pool.size(), 1001); + assert_eq!(pool.proposed_size(), 1001); + assert_eq!(pool.pending_size(), 0); + pool.clear(); + assert_eq!(pool.size(), 0); +} From bdf557d01f739e3ddcba5ee3b40cde0060750ef1 Mon Sep 17 00:00:00 2001 From: yukang Date: Tue, 13 Jun 2023 16:58:57 +0800 Subject: [PATCH 15/58] change name (add/sub)_ancestor_weight, prepare for (add/sub)_descendant_weight --- tx-pool/src/component/commit_txs_scanner.rs | 2 +- tx-pool/src/component/entry.rs | 4 ++-- tx-pool/src/component/pool_map.rs | 6 +++--- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tx-pool/src/component/commit_txs_scanner.rs b/tx-pool/src/component/commit_txs_scanner.rs index 4e5d487cdf..f90ac94094 100644 --- a/tx-pool/src/component/commit_txs_scanner.rs +++ b/tx-pool/src/component/commit_txs_scanner.rs @@ -209,7 +209,7 @@ impl<'a> CommitTxsScanner<'a> { .remove(desc_id) .or_else(|| self.pool_map.get(desc_id).cloned()) { - desc.sub_entry_weight(entry); + desc.sub_ancestor_weight(entry); self.modified_entries.insert(desc); } } diff --git a/tx-pool/src/component/entry.rs b/tx-pool/src/component/entry.rs index bcce6a2e16..544d7e2817 100644 --- a/tx-pool/src/component/entry.rs +++ b/tx-pool/src/component/entry.rs @@ -106,7 +106,7 @@ impl TxEntry { } /// Update ancestor state for add an entry - pub fn add_entry_weight(&mut self, entry: &TxEntry) { + pub fn add_ancestor_weight(&mut self, entry: &TxEntry) { self.ancestors_count = self.ancestors_count.saturating_add(1); self.ancestors_size = self.ancestors_size.saturating_add(entry.size); self.ancestors_cycles = self.ancestors_cycles.saturating_add(entry.cycles); @@ -118,7 +118,7 @@ impl TxEntry { } /// Update ancestor state for remove an entry - pub fn sub_entry_weight(&mut self, entry: &TxEntry) { + pub fn sub_ancestor_weight(&mut self, entry: &TxEntry) { self.ancestors_count = self.ancestors_count.saturating_sub(1); self.ancestors_size = self.ancestors_size.saturating_sub(entry.size); self.ancestors_cycles = self.ancestors_cycles.saturating_sub(entry.cycles); diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index dadb3b3bc2..af2c5979e2 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -363,8 +363,8 @@ impl PoolMap { let entry = self.entries.get_by_id(desc_id).unwrap().clone(); let mut child = entry.inner.clone(); match op { - EntryOp::Remove => child.sub_entry_weight(parent), - EntryOp::Add => child.add_entry_weight(parent), + EntryOp::Remove => child.sub_ancestor_weight(parent), + EntryOp::Add => child.add_ancestor_weight(parent), } let short_id = child.proposal_short_id(); self.entries.modify_by_id(&short_id, |e| { @@ -482,7 +482,7 @@ impl PoolMap { .entries .get_by_id(ancestor_id) .expect("pool consistent"); - entry.add_entry_weight(&ancestor.inner); + entry.add_ancestor_weight(&ancestor.inner); } if entry.ancestors_count > self.max_ancestors_count { eprintln!("debug: exceeded maximum ancestors count"); From 8a4172f543bf931628d8b1657f774e79c9bea66c Mon Sep 17 00:00:00 2001 From: yukang Date: Tue, 13 Jun 2023 17:16:46 +0800 Subject: [PATCH 16/58] remove ancestors_size from AncestorsScoreSortKey since it is useless --- tx-pool/src/component/entry.rs | 1 - tx-pool/src/component/score_key.rs | 1 - tx-pool/src/component/tests/proposed.rs | 19 +++++++++++-------- tx-pool/src/component/tests/score_key.rs | 2 -- 4 files changed, 11 insertions(+), 12 deletions(-) diff --git a/tx-pool/src/component/entry.rs b/tx-pool/src/component/entry.rs index 544d7e2817..2b8bfa6f56 100644 --- a/tx-pool/src/component/entry.rs +++ b/tx-pool/src/component/entry.rs @@ -160,7 +160,6 @@ impl From<&TxEntry> for AncestorsScoreSortKey { weight, id: entry.proposal_short_id(), ancestors_fee: entry.ancestors_fee, - ancestors_size: entry.ancestors_size, ancestors_weight, } } diff --git a/tx-pool/src/component/score_key.rs b/tx-pool/src/component/score_key.rs index 011fb4000b..18dd48fcb2 100644 --- a/tx-pool/src/component/score_key.rs +++ b/tx-pool/src/component/score_key.rs @@ -9,7 +9,6 @@ pub struct AncestorsScoreSortKey { pub id: ProposalShortId, pub ancestors_fee: Capacity, pub ancestors_weight: u64, - pub ancestors_size: usize, } impl AncestorsScoreSortKey { diff --git a/tx-pool/src/component/tests/proposed.rs b/tx-pool/src/component/tests/proposed.rs index 153b9324d1..2bfa5d2449 100644 --- a/tx-pool/src/component/tests/proposed.rs +++ b/tx-pool/src/component/tests/proposed.rs @@ -104,11 +104,11 @@ fn test_add_entry_from_detached() { assert_eq!(pool.size(), 3); - let expected = vec![(id1.clone(), 1), (id2.clone(), 2), (id3.clone(), 3)]; + let expected = vec![id1.clone(), id2.clone(), id3.clone()]; let got = pool .entries .iter() - .map(|(_, key)| (key.id.clone(), key.score.ancestors_size)) + .map(|(_, key)| key.id.clone()) .collect::>(); assert_eq!(expected, got); @@ -144,11 +144,11 @@ fn test_add_entry_from_detached() { assert_eq!(pool.edges.inputs_len(), 2); assert_eq!(pool.entries.len(), 2); - let left = vec![(id2.clone(), 1), (id3.clone(), 2)]; + let left = vec![id2.clone(), id3.clone()]; let got = pool .entries .iter() - .map(|(_, key)| (key.id.clone(), key.score.ancestors_size)) + .map(|(_, key)| key.id.clone()) .collect::>(); assert_eq!(left, got); @@ -160,10 +160,13 @@ fn test_add_entry_from_detached() { assert!(pool.add_proposed(entry1).unwrap()); - for (idx, (_, entry)) in pool.entries.iter().enumerate() { - assert_eq!(entry.id, expected[idx].0); - assert_eq!(entry.score.ancestors_size, expected[idx].1); - } + let ids = pool + .entries + .iter() + .map(|(_, entry)| entry.inner.proposal_short_id()) + .collect::>(); + assert_eq!(ids, expected); + { assert!(pool.links.get_parents(&id1).unwrap().is_empty()); assert_eq!( diff --git a/tx-pool/src/component/tests/score_key.rs b/tx-pool/src/component/tests/score_key.rs index 22da657f84..09475f3d19 100644 --- a/tx-pool/src/component/tests/score_key.rs +++ b/tx-pool/src/component/tests/score_key.rs @@ -30,7 +30,6 @@ fn test_min_fee_and_weight() { id: ProposalShortId::new([0u8; 10]), ancestors_fee: Capacity::shannons(ancestors_fee), ancestors_weight, - ancestors_size: 0, }; key.min_fee_and_weight() }) @@ -75,7 +74,6 @@ fn test_ancestors_sorted_key_order() { id: ProposalShortId::new(id), ancestors_fee: Capacity::shannons(ancestors_fee), ancestors_weight, - ancestors_size: 0, } }) .collect::>(); From 702791550e2b5f76752cdb7be1bdc6de63892bc0 Mon Sep 17 00:00:00 2001 From: yukang Date: Tue, 13 Jun 2023 22:05:45 +0800 Subject: [PATCH 17/58] trivial refactor for PoolTransactionEntry --- util/launcher/src/shared_builder.rs | 31 ++++++++++------------------- 1 file changed, 10 insertions(+), 21 deletions(-) diff --git a/util/launcher/src/shared_builder.rs b/util/launcher/src/shared_builder.rs index 09f9fd862c..96d750a198 100644 --- a/util/launcher/src/shared_builder.rs +++ b/util/launcher/src/shared_builder.rs @@ -421,18 +421,19 @@ fn register_tx_pool_callback(tx_pool_builder: &mut TxPoolServiceBuilder, notify: let notify_pending = notify.clone(); let tx_relay_sender = tx_pool_builder.tx_relay_sender(); + let create_notify_entry = |entry: &TxEntry| PoolTransactionEntry { + transaction: entry.rtx.transaction.clone(), + cycles: entry.cycles, + size: entry.size, + fee: entry.fee, + timestamp: entry.timestamp, + }; tx_pool_builder.register_pending(Box::new(move |tx_pool: &mut TxPool, entry: &TxEntry| { // update statics tx_pool.update_statics_for_add_tx(entry.size, entry.cycles); // notify - let notify_tx_entry = PoolTransactionEntry { - transaction: entry.rtx.transaction.clone(), - cycles: entry.cycles, - size: entry.size, - fee: entry.fee, - timestamp: entry.timestamp, - }; + let notify_tx_entry = create_notify_entry(entry); notify_pending.notify_new_transaction(notify_tx_entry); })); @@ -445,13 +446,7 @@ fn register_tx_pool_callback(tx_pool_builder: &mut TxPoolServiceBuilder, notify: } // notify - let notify_tx_entry = PoolTransactionEntry { - transaction: entry.rtx.transaction.clone(), - cycles: entry.cycles, - size: entry.size, - fee: entry.fee, - timestamp: entry.timestamp, - }; + let notify_tx_entry = create_notify_entry(entry); notify_proposed.notify_proposed_transaction(notify_tx_entry); }, )); @@ -483,13 +478,7 @@ fn register_tx_pool_callback(tx_pool_builder: &mut TxPoolServiceBuilder, notify: } // notify - let notify_tx_entry = PoolTransactionEntry { - transaction: entry.rtx.transaction.clone(), - cycles: entry.cycles, - size: entry.size, - fee: entry.fee, - timestamp: entry.timestamp, - }; + let notify_tx_entry = create_notify_entry(entry); notify_reject.notify_reject_transaction(notify_tx_entry, reject); }, )); From 8e72e3ef0c8b146e0525520e6f1242b222c193ff Mon Sep 17 00:00:00 2001 From: yukang Date: Wed, 14 Jun 2023 05:20:59 +0800 Subject: [PATCH 18/58] high score tx_entry will not blocked at pending --- tx-pool/src/component/entry.rs | 1 + tx-pool/src/component/pool_map.rs | 21 +++++---- tx-pool/src/component/score_key.rs | 8 +++- tx-pool/src/component/tests/pending.rs | 55 ++++++++++++++++++++++++ tx-pool/src/component/tests/score_key.rs | 2 + tx-pool/src/pool.rs | 2 - 6 files changed, 78 insertions(+), 11 deletions(-) diff --git a/tx-pool/src/component/entry.rs b/tx-pool/src/component/entry.rs index 2b8bfa6f56..67cc2c8035 100644 --- a/tx-pool/src/component/entry.rs +++ b/tx-pool/src/component/entry.rs @@ -161,6 +161,7 @@ impl From<&TxEntry> for AncestorsScoreSortKey { id: entry.proposal_short_id(), ancestors_fee: entry.ancestors_fee, ancestors_weight, + timestamp: entry.timestamp, } } } diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index af2c5979e2..c6d232fcd3 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -140,11 +140,7 @@ impl PoolMap { } pub(crate) fn score_sorted_iter(&self) -> impl Iterator { - self.entries - .iter_by_score() - .rev() - .filter(|entry| entry.status == Status::Proposed) - .map(|entry| &entry.inner) + self.score_sorted_iter_by(Status::Proposed) } pub(crate) fn get(&self, id: &ProposalShortId) -> Option<&TxEntry> { @@ -296,12 +292,13 @@ impl PoolMap { proposals: &mut HashSet, status: &Status, ) { - for entry in self.entries.get_by_status(status) { + for entry in self.score_sorted_iter_by(*status) { if proposals.len() == limit { break; } - if !exclusion.contains(&entry.id) { - proposals.insert(entry.id.clone()); + let id = entry.proposal_short_id(); + if !exclusion.contains(&id) { + proposals.insert(id); } } } @@ -341,6 +338,14 @@ impl PoolMap { self.links.clear(); } + fn score_sorted_iter_by(&self, status: Status) -> impl Iterator { + self.entries + .iter_by_score() + .rev() + .filter(move |entry| entry.status == status) + .map(|entry| &entry.inner) + } + fn remove_entry_links(&mut self, id: &ProposalShortId) { if let Some(parents) = self.links.get_parents(id).cloned() { for parent in parents { diff --git a/tx-pool/src/component/score_key.rs b/tx-pool/src/component/score_key.rs index 18dd48fcb2..1a9843b7ad 100644 --- a/tx-pool/src/component/score_key.rs +++ b/tx-pool/src/component/score_key.rs @@ -9,6 +9,7 @@ pub struct AncestorsScoreSortKey { pub id: ProposalShortId, pub ancestors_fee: Capacity, pub ancestors_weight: u64, + pub timestamp: u64, } impl AncestorsScoreSortKey { @@ -42,7 +43,12 @@ impl Ord for AncestorsScoreSortKey { if self_weight == other_weight { // if fee rate weight is same, then compare with ancestor weight if self.ancestors_weight == other.ancestors_weight { - self.id.raw_data().cmp(&other.id.raw_data()) + if self.timestamp == other.timestamp { + self.id.raw_data().cmp(&other.id.raw_data()) + } else { + // NOTE: we use timestamp to compare, so the order is reversed + self.timestamp.cmp(&other.timestamp).reverse() + } } else { self.ancestors_weight.cmp(&other.ancestors_weight) } diff --git a/tx-pool/src/component/tests/pending.rs b/tx-pool/src/component/tests/pending.rs index b07e2e96e6..764f6d0026 100644 --- a/tx-pool/src/component/tests/pending.rs +++ b/tx-pool/src/component/tests/pending.rs @@ -6,9 +6,11 @@ use crate::component::{ entry::TxEntry, pool_map::{PoolMap, Status}, }; +use ckb_types::core::Capacity; use ckb_types::packed::OutPoint; use ckb_types::{h256, packed::Byte32, prelude::*}; use std::collections::HashSet; +use std::time::Duration; #[test] fn test_basic() { @@ -206,7 +208,11 @@ fn test_fill_proposals() { 3, ); let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + std::thread::sleep(Duration::from_millis(1)); + let entry2 = TxEntry::dummy_resolve(tx2.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + std::thread::sleep(Duration::from_millis(1)); + let entry3 = TxEntry::dummy_resolve(tx3.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); assert!(pool.add_entry(entry1, Status::Pending).unwrap()); assert!(pool.add_entry(entry2, Status::Pending).unwrap()); @@ -242,6 +248,55 @@ fn test_fill_proposals() { assert_eq!(ret, HashSet::from_iter(vec![id1, id3])); } +#[test] +fn test_fill_proposals_with_high_score() { + let mut pool = PoolMap::new(1000); + let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); + let tx2 = build_tx( + vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 1)], + 3, + ); + let tx3 = build_tx_with_dep( + vec![(&h256!("0x4").pack(), 1)], + vec![(&h256!("0x5").pack(), 1)], + 3, + ); + let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + std::thread::sleep(Duration::from_millis(1)); + let entry2 = TxEntry::dummy_resolve(tx2.clone(), 2, Capacity::shannons(50), 2); + std::thread::sleep(Duration::from_millis(1)); + let entry3 = TxEntry::dummy_resolve(tx3.clone(), 2, Capacity::shannons(100), 2); + + assert!(pool.add_entry(entry1, Status::Pending).unwrap()); + assert!(pool.add_entry(entry2, Status::Pending).unwrap()); + assert!(pool.add_entry(entry3, Status::Pending).unwrap()); + + let id1 = tx1.proposal_short_id(); + let id2 = tx2.proposal_short_id(); + let id3 = tx3.proposal_short_id(); + + let mut ret = HashSet::new(); + pool.fill_proposals(10, &HashSet::new(), &mut ret, &Status::Pending); + assert_eq!( + ret, + HashSet::from_iter(vec![id3.clone(), id2.clone(), id1.clone()]) + ); + + let mut ret = HashSet::new(); + pool.fill_proposals(1, &HashSet::new(), &mut ret, &Status::Pending); + assert_eq!(ret, HashSet::from_iter(vec![id3.clone()])); + + let mut ret = HashSet::new(); + pool.fill_proposals(2, &HashSet::new(), &mut ret, &Status::Pending); + assert_eq!(ret, HashSet::from_iter(vec![id3.clone(), id2.clone()])); + + let mut ret = HashSet::new(); + let mut exclusion = HashSet::new(); + exclusion.insert(id2); + pool.fill_proposals(2, &exclusion, &mut ret, &Status::Pending); + assert_eq!(ret, HashSet::from_iter(vec![id1, id3])); +} + #[test] fn test_edges() { let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); diff --git a/tx-pool/src/component/tests/score_key.rs b/tx-pool/src/component/tests/score_key.rs index 09475f3d19..7acc22a895 100644 --- a/tx-pool/src/component/tests/score_key.rs +++ b/tx-pool/src/component/tests/score_key.rs @@ -30,6 +30,7 @@ fn test_min_fee_and_weight() { id: ProposalShortId::new([0u8; 10]), ancestors_fee: Capacity::shannons(ancestors_fee), ancestors_weight, + timestamp: 0, }; key.min_fee_and_weight() }) @@ -74,6 +75,7 @@ fn test_ancestors_sorted_key_order() { id: ProposalShortId::new(id), ancestors_fee: Capacity::shannons(ancestors_fee), ancestors_weight, + timestamp: 0, } }) .collect::>(); diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 7e1db62c57..2fdbe23d45 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -331,8 +331,6 @@ impl TxPool { } /// Get to-be-proposal transactions that may be included in the next block. - /// TODO: do we need to consider the something like score, so that we can - /// provide best transactions to be proposed. pub(crate) fn get_proposals( &self, limit: usize, From 08f58beb42619abb95de0deca13e06ed0b0590bc Mon Sep 17 00:00:00 2001 From: yukang Date: Wed, 14 Jun 2023 19:13:43 +0800 Subject: [PATCH 19/58] add descendants related info for evict, has performance regression --- test/src/specs/tx_pool/limit.rs | 2 +- tx-pool/src/component/entry.rs | 69 ++++++++++-- tx-pool/src/component/pool_map.rs | 32 +++++- tx-pool/src/component/tests/entry.rs | 54 +++++++++ tx-pool/src/component/tests/mod.rs | 1 + tx-pool/src/component/tests/pending.rs | 145 +++++++++++++++++++++++++ tx-pool/src/pool.rs | 11 +- util/types/src/core/tx_pool.rs | 4 + 8 files changed, 301 insertions(+), 17 deletions(-) create mode 100644 tx-pool/src/component/tests/entry.rs diff --git a/test/src/specs/tx_pool/limit.rs b/test/src/specs/tx_pool/limit.rs index b4defc175b..ad84e36124 100644 --- a/test/src/specs/tx_pool/limit.rs +++ b/test/src/specs/tx_pool/limit.rs @@ -34,7 +34,7 @@ impl Spec for SizeLimit { let max_tx_num = (MAX_MEM_SIZE_FOR_SIZE_LIMIT as u64) / one_tx_size; - info!("Generate as much as possible txs on node"); + info!("Generate as much as possible txs on : {}", max_tx_num); (0..(max_tx_num - 1)).for_each(|_| { let tx = node.new_transaction(hash.clone()); hash = node.rpc_client().send_transaction(tx.data().into()); diff --git a/tx-pool/src/component/entry.rs b/tx-pool/src/component/entry.rs index 67cc2c8035..3638e21285 100644 --- a/tx-pool/src/component/entry.rs +++ b/tx-pool/src/component/entry.rs @@ -31,6 +31,14 @@ pub struct TxEntry { pub ancestors_cycles: Cycle, /// ancestors txs count pub ancestors_count: usize, + /// descendants txs fee + pub descendants_fee: Capacity, + /// descendants txs size + pub descendants_size: usize, + /// descendants txs cycles + pub descendants_cycles: Cycle, + /// descendants txs count + pub descendants_count: usize, /// The unix timestamp when entering the Txpool, unit: Millisecond pub timestamp: u64, } @@ -58,6 +66,10 @@ impl TxEntry { ancestors_size: size, ancestors_fee: fee, ancestors_cycles: cycles, + descendants_fee: fee, + descendants_size: size, + descendants_cycles: cycles, + descendants_count: 1, ancestors_count: 1, } } @@ -105,6 +117,30 @@ impl TxEntry { FeeRate::calculate(self.fee, weight) } + /// Update ancestor state for add an entry + pub fn add_descendant_weight(&mut self, entry: &TxEntry) { + self.descendants_count = self.descendants_count.saturating_add(1); + self.descendants_size = self.descendants_size.saturating_add(entry.size); + self.descendants_cycles = self.descendants_cycles.saturating_add(entry.cycles); + self.descendants_fee = Capacity::shannons( + self.descendants_fee + .as_u64() + .saturating_add(entry.fee.as_u64()), + ); + } + + /// Update ancestor state for remove an entry + pub fn sub_descendant_weight(&mut self, entry: &TxEntry) { + self.descendants_count = self.descendants_count.saturating_sub(1); + self.descendants_size = self.descendants_size.saturating_sub(entry.size); + self.descendants_cycles = self.descendants_cycles.saturating_sub(entry.cycles); + self.descendants_fee = Capacity::shannons( + self.descendants_fee + .as_u64() + .saturating_sub(entry.fee.as_u64()), + ); + } + /// Update ancestor state for add an entry pub fn add_ancestor_weight(&mut self, entry: &TxEntry) { self.ancestors_count = self.ancestors_count.saturating_add(1); @@ -130,11 +166,16 @@ impl TxEntry { } /// Reset ancestor state by remove - pub fn reset_ancestors_state(&mut self) { + pub fn reset_statistic_state(&mut self) { self.ancestors_count = 1; self.ancestors_size = self.size; self.ancestors_cycles = self.cycles; self.ancestors_fee = self.fee; + + self.descendants_count = 1; + self.descendants_size = self.size; + self.descendants_cycles = self.cycles; + self.descendants_fee = self.fee; } /// Converts entry to a `TxEntryInfo`. @@ -145,6 +186,8 @@ impl TxEntry { fee: self.fee, ancestors_size: self.ancestors_size as u64, ancestors_cycles: self.ancestors_cycles, + descendants_size: self.descendants_size as u64, + descendants_cycles: self.descendants_cycles, ancestors_count: self.ancestors_count as u64, timestamp: self.timestamp, } @@ -190,22 +233,28 @@ impl Ord for TxEntry { } } -/// Currently we do not have trace descendants, -/// so first take the simplest strategy, -/// first compare fee_rate, select the smallest fee_rate, +/// First compare fee_rate, select the smallest fee_rate, /// and then select the latest timestamp, for eviction, /// the latest timestamp which also means that the fewer descendants may exist. #[derive(Eq, PartialEq, Clone, Debug)] pub struct EvictKey { - fee_rate: FeeRate, - timestamp: u64, + pub fee_rate: FeeRate, + pub timestamp: u64, + pub descendants_count: usize, } impl From<&TxEntry> for EvictKey { fn from(entry: &TxEntry) -> Self { + let weight = get_transaction_weight(entry.size, entry.cycles); + let descendants_weight = + get_transaction_weight(entry.descendants_size, entry.descendants_cycles); + + let descendants_feerate = FeeRate::calculate(entry.descendants_fee, descendants_weight); + let feerate = FeeRate::calculate(entry.fee, weight); EvictKey { - fee_rate: entry.fee_rate(), + fee_rate: descendants_feerate.max(feerate), timestamp: entry.timestamp, + descendants_count: entry.descendants_count, } } } @@ -219,7 +268,11 @@ impl PartialOrd for EvictKey { impl Ord for EvictKey { fn cmp(&self, other: &Self) -> Ordering { if self.fee_rate == other.fee_rate { - self.timestamp.cmp(&other.timestamp).reverse() + if self.descendants_count == other.descendants_count { + self.timestamp.cmp(&other.timestamp) + } else { + self.descendants_count.cmp(&other.descendants_count) + } } else { self.fee_rate.cmp(&other.fee_rate) } diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index c6d232fcd3..cb0393d4ec 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -184,7 +184,7 @@ impl PoolMap { } trace!("pool_map.add_{:?} {}", status, entry.transaction().hash()); self.record_entry_links(&mut entry)?; - self.insert_entry(&entry, status)?; + self.insert_entry(&entry, status); self.record_entry_deps(&entry); self.record_entry_edges(&entry); Ok(true) @@ -200,6 +200,7 @@ impl PoolMap { pub(crate) fn remove_entry(&mut self, id: &ProposalShortId) -> Option { if let Some(entry) = self.entries.remove_by_id(id) { + self.update_ancestors_index_key(&entry.inner, EntryOp::Remove); self.update_descendants_index_key(&entry.inner, EntryOp::Remove); self.remove_entry_deps(&entry.inner); self.remove_entry_edges(&entry.inner); @@ -325,10 +326,10 @@ impl PoolMap { self.entries.iter().map(|(_, entry)| entry) } - pub(crate) fn next_evict_entry(&self) -> Option { + pub(crate) fn next_evict_entry(&self, status: Status) -> Option { self.entries .iter_by_evict_key() - .next() + .find(move |entry| entry.status == status) .map(|entry| entry.id.clone()) } @@ -360,6 +361,25 @@ impl PoolMap { self.links.remove(id); } + fn update_ancestors_index_key(&mut self, child: &TxEntry, op: EntryOp) { + let ancestors: HashSet = + self.links.calc_ancestors(&child.proposal_short_id()); + for anc_id in &ancestors { + // update parent score + let entry = self.entries.get_by_id(anc_id).unwrap().clone(); + let mut parent = entry.inner.clone(); + match op { + EntryOp::Remove => parent.sub_descendant_weight(child), + EntryOp::Add => parent.add_descendant_weight(child), + } + let short_id = parent.proposal_short_id(); + self.entries.modify_by_id(&short_id, |e| { + e.evict_key = parent.as_evict_key(); + e.inner = parent; + }); + } + } + fn update_descendants_index_key(&mut self, parent: &TxEntry, op: EntryOp) { let descendants: HashSet = self.links.calc_descendants(&parent.proposal_short_id()); @@ -374,7 +394,6 @@ impl PoolMap { let short_id = child.proposal_short_id(); self.entries.modify_by_id(&short_id, |e| { e.score = child.as_score_key(); - e.evict_key = child.as_evict_key(); e.inner = child; }); } @@ -426,6 +445,8 @@ impl PoolMap { if !children.is_empty() { self.update_descendants_from_detached(&tx_short_id, children); } + // update ancestors + self.update_ancestors_index_key(entry, EntryOp::Add); } // update_descendants_from_detached is used to update @@ -542,7 +563,7 @@ impl PoolMap { self.edges.header_deps.remove(&id); } - fn insert_entry(&mut self, entry: &TxEntry, status: Status) -> Result { + fn insert_entry(&mut self, entry: &TxEntry, status: Status) { let tx_short_id = entry.proposal_short_id(); let score = entry.as_score_key(); let evict_key = entry.as_evict_key(); @@ -553,7 +574,6 @@ impl PoolMap { inner: entry.clone(), evict_key, }); - Ok(true) } } diff --git a/tx-pool/src/component/tests/entry.rs b/tx-pool/src/component/tests/entry.rs new file mode 100644 index 0000000000..8aa7edf3ff --- /dev/null +++ b/tx-pool/src/component/tests/entry.rs @@ -0,0 +1,54 @@ +use ckb_types::core::{Capacity, FeeRate}; + +use crate::component::entry::EvictKey; + +#[test] +fn test_min_fee_and_weight_evict() { + let mut result = vec![(500, 10, 30), (10, 10, 31), (100, 10, 32)] + .into_iter() + .map(|(fee, weight, timestamp)| EvictKey { + fee_rate: FeeRate::calculate(Capacity::shannons(fee), weight), + timestamp, + descendants_count: 0, + }) + .collect::>(); + result.sort(); + assert_eq!( + result.iter().map(|key| key.timestamp).collect::>(), + vec![31, 32, 30] + ); +} + +#[test] +fn test_min_timestamp_evict() { + let mut result = vec![(500, 10, 30), (500, 10, 31), (500, 10, 32)] + .into_iter() + .map(|(fee, weight, timestamp)| EvictKey { + fee_rate: FeeRate::calculate(Capacity::shannons(fee), weight), + timestamp, + descendants_count: 0, + }) + .collect::>(); + result.sort(); + assert_eq!( + result.iter().map(|key| key.timestamp).collect::>(), + vec![30, 31, 32] + ); +} + +#[test] +fn test_min_weight_evict() { + let mut result = vec![(500, 10, 30), (500, 12, 31), (500, 13, 32)] + .into_iter() + .map(|(fee, weight, timestamp)| EvictKey { + fee_rate: FeeRate::calculate(Capacity::shannons(fee), weight), + timestamp, + descendants_count: 0, + }) + .collect::>(); + result.sort(); + assert_eq!( + result.iter().map(|key| key.timestamp).collect::>(), + vec![32, 31, 30] + ); +} diff --git a/tx-pool/src/component/tests/mod.rs b/tx-pool/src/component/tests/mod.rs index d9a3529707..fb851e4855 100644 --- a/tx-pool/src/component/tests/mod.rs +++ b/tx-pool/src/component/tests/mod.rs @@ -1,4 +1,5 @@ mod chunk; +mod entry; mod pending; mod proposed; mod recent_reject; diff --git a/tx-pool/src/component/tests/pending.rs b/tx-pool/src/component/tests/pending.rs index 764f6d0026..f2c3b6b19d 100644 --- a/tx-pool/src/component/tests/pending.rs +++ b/tx-pool/src/component/tests/pending.rs @@ -314,3 +314,148 @@ fn test_edges() { edges.delete_txid_by_dep(outpoint, &short_id2); assert!(edges.deps.is_empty()); } + +#[test] +fn test_pool_evict() { + let mut pool = PoolMap::new(1000); + let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); + let tx2 = build_tx( + vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 1)], + 3, + ); + let tx3 = build_tx_with_dep( + vec![(&h256!("0x4").pack(), 1)], + vec![(&h256!("0x5").pack(), 1)], + 3, + ); + let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + std::thread::sleep(Duration::from_millis(1)); + let entry2 = TxEntry::dummy_resolve(tx2.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + std::thread::sleep(Duration::from_millis(1)); + let entry3 = TxEntry::dummy_resolve(tx3.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); + + assert!(pool.add_entry(entry1, Status::Pending).unwrap()); + assert!(pool.add_entry(entry2, Status::Pending).unwrap()); + assert!(pool.add_entry(entry3, Status::Pending).unwrap()); + + let e1 = pool.next_evict_entry(Status::Pending).unwrap(); + assert_eq!(e1, tx1.proposal_short_id()); + pool.remove_entry(&e1); + + let e2 = pool.next_evict_entry(Status::Pending).unwrap(); + assert_eq!(e2, tx2.proposal_short_id()); + pool.remove_entry(&e2); + + let e3 = pool.next_evict_entry(Status::Pending).unwrap(); + assert_eq!(e3, tx3.proposal_short_id()); + pool.remove_entry(&e3); + + assert!(pool.next_evict_entry(Status::Pending).is_none()); +} + +#[test] +fn test_pool_min_weight_evict() { + let mut pool = PoolMap::new(1000); + let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); + let tx2 = build_tx( + vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 1)], + 3, + ); + let tx3 = build_tx_with_dep( + vec![(&h256!("0x4").pack(), 1)], + vec![(&h256!("0x5").pack(), 1)], + 3, + ); + let entry1 = TxEntry::dummy_resolve(tx1.clone(), 2, Capacity::shannons(100), 2); + std::thread::sleep(Duration::from_millis(1)); + let entry2 = TxEntry::dummy_resolve(tx2.clone(), 2, Capacity::shannons(50), 2); + std::thread::sleep(Duration::from_millis(1)); + let entry3 = TxEntry::dummy_resolve(tx3.clone(), 2, Capacity::shannons(10), 2); + + assert!(pool.add_entry(entry1, Status::Pending).unwrap()); + assert!(pool.add_entry(entry2, Status::Pending).unwrap()); + assert!(pool.add_entry(entry3, Status::Pending).unwrap()); + + let e1 = pool.next_evict_entry(Status::Pending).unwrap(); + assert_eq!(e1, tx3.proposal_short_id()); + pool.remove_entry(&e1); + + let e2 = pool.next_evict_entry(Status::Pending).unwrap(); + assert_eq!(e2, tx2.proposal_short_id()); + pool.remove_entry(&e2); + + let e3 = pool.next_evict_entry(Status::Pending).unwrap(); + assert_eq!(e3, tx1.proposal_short_id()); + pool.remove_entry(&e3); + + assert!(pool.next_evict_entry(Status::Pending).is_none()); +} + +#[test] +fn test_pool_max_size_evict() { + let mut pool = PoolMap::new(1000); + let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); + let tx2 = build_tx( + vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 1)], + 3, + ); + let tx3 = build_tx_with_dep( + vec![(&h256!("0x4").pack(), 1)], + vec![(&h256!("0x5").pack(), 1)], + 3, + ); + let entry1 = TxEntry::dummy_resolve(tx1.clone(), 2, Capacity::shannons(100), 3); + std::thread::sleep(Duration::from_millis(1)); + let entry2 = TxEntry::dummy_resolve(tx2.clone(), 2, Capacity::shannons(100), 2); + std::thread::sleep(Duration::from_millis(1)); + let entry3 = TxEntry::dummy_resolve(tx3.clone(), 2, Capacity::shannons(100), 1); + + assert!(pool.add_entry(entry1, Status::Pending).unwrap()); + assert!(pool.add_entry(entry2, Status::Pending).unwrap()); + assert!(pool.add_entry(entry3, Status::Pending).unwrap()); + + let e1 = pool.next_evict_entry(Status::Pending).unwrap(); + assert_eq!(e1, tx1.proposal_short_id()); + pool.remove_entry(&e1); + + let e2 = pool.next_evict_entry(Status::Pending).unwrap(); + assert_eq!(e2, tx2.proposal_short_id()); + pool.remove_entry(&e2); + + let e3 = pool.next_evict_entry(Status::Pending).unwrap(); + assert_eq!(e3, tx3.proposal_short_id()); + pool.remove_entry(&e3); + + assert!(pool.next_evict_entry(Status::Pending).is_none()); +} + +#[test] +fn test_pool_min_descendants_evict() { + let mut pool = PoolMap::new(1000); + let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); + let tx2 = build_tx(vec![(&tx1.hash(), 1), (&h256!("0x3").pack(), 1)], 3); + let tx3 = build_tx_with_dep(vec![(&tx2.hash(), 1)], vec![(&h256!("0x5").pack(), 1)], 3); + let entry1 = TxEntry::dummy_resolve(tx1.clone(), 2, Capacity::shannons(100), 1); + std::thread::sleep(Duration::from_millis(1)); + let entry2 = TxEntry::dummy_resolve(tx2.clone(), 2, Capacity::shannons(100), 1); + std::thread::sleep(Duration::from_millis(1)); + let entry3 = TxEntry::dummy_resolve(tx3.clone(), 2, Capacity::shannons(100), 1); + + assert!(pool.add_entry(entry1, Status::Pending).unwrap()); + assert!(pool.add_entry(entry2, Status::Pending).unwrap()); + assert!(pool.add_entry(entry3, Status::Pending).unwrap()); + + let e1 = pool.next_evict_entry(Status::Pending).unwrap(); + assert_eq!(e1, tx3.proposal_short_id()); + pool.remove_entry(&e1); + + let e2 = pool.next_evict_entry(Status::Pending).unwrap(); + assert_eq!(e2, tx2.proposal_short_id()); + pool.remove_entry(&e2); + + let e3 = pool.next_evict_entry(Status::Pending).unwrap(); + assert_eq!(e3, tx1.proposal_short_id()); + pool.remove_entry(&e3); + + assert!(pool.next_evict_entry(Status::Pending).is_none()); +} diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 2fdbe23d45..afcb56aad2 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -216,7 +216,14 @@ impl TxPool { // Remove transactions from the pool until total size <= size_limit. pub(crate) fn limit_size(&mut self, callbacks: &Callbacks) { while self.total_tx_size > self.config.max_tx_pool_size { - if let Some(id) = self.pool_map.next_evict_entry() { + let next_evict_entry = || { + self.pool_map + .next_evict_entry(Status::Pending) + .or_else(|| self.pool_map.next_evict_entry(Status::Gap)) + .or_else(|| self.pool_map.next_evict_entry(Status::Proposed)) + }; + + if let Some(id) = next_evict_entry() { let removed = self.pool_map.remove_entry_and_descendants(&id); for entry in removed { let tx_hash = entry.transaction().hash(); @@ -251,7 +258,7 @@ impl TxPool { entries.sort_unstable_by_key(|entry| entry.ancestors_count); for mut entry in entries { let tx_hash = entry.transaction().hash(); - entry.reset_ancestors_state(); + entry.reset_statistic_state(); let ret = self.add_pending(entry); debug!( "remove_by_detached_proposal from {:?} {} add_pending {:?}", diff --git a/util/types/src/core/tx_pool.rs b/util/types/src/core/tx_pool.rs index d5b41e1d4d..43fcd7e547 100644 --- a/util/types/src/core/tx_pool.rs +++ b/util/types/src/core/tx_pool.rs @@ -128,6 +128,10 @@ pub struct TxEntryInfo { pub ancestors_size: u64, /// Cycles of in-tx-pool ancestor transactions pub ancestors_cycles: u64, + /// Size of in-tx-pool descendants transactions + pub descendants_size: u64, + /// Cycles of in-tx-pool descendants transactions + pub descendants_cycles: u64, /// Number of in-tx-pool ancestor transactions pub ancestors_count: u64, /// The unix timestamp when entering the Txpool, unit: Millisecond From 0d9a1031ba863e04f4316266d19a41f92eb3b177 Mon Sep 17 00:00:00 2001 From: yukang Date: Wed, 14 Jun 2023 23:30:38 +0800 Subject: [PATCH 20/58] cleanup and remove unused function names and tests --- tx-pool/src/component/pool_map.rs | 59 +++++-------------------- tx-pool/src/component/tests/pending.rs | 29 ------------ tx-pool/src/component/tests/proposed.rs | 14 +++--- tx-pool/src/pool.rs | 4 +- 4 files changed, 19 insertions(+), 87 deletions(-) diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index cb0393d4ec..596a9dce8f 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -117,11 +117,6 @@ impl PoolMap { self.add_entry(entry, Status::Proposed) } - #[cfg(test)] - pub(crate) fn remove_committed_tx(&mut self, tx: &TransactionView) -> Option { - self.remove_entry(&tx.proposal_short_id()) - } - pub(crate) fn get_by_id(&self, id: &ProposalShortId) -> Option<&PoolEntry> { self.entries.get_by_id(id) } @@ -192,10 +187,11 @@ impl PoolMap { /// Change the status of the entry, only used for `gap_rtx` and `proposed_rtx` pub(crate) fn set_entry(&mut self, short_id: &ProposalShortId, status: Status) { - let _ = self.entries.get_by_id(short_id).expect("unconsistent pool"); - self.entries.modify_by_id(short_id, |e| { - e.status = status; - }); + self.entries + .modify_by_id(short_id, |e| { + e.status = status; + }) + .expect("unconsistent pool"); } pub(crate) fn remove_entry(&mut self, id: &ProposalShortId) -> Option { @@ -304,24 +300,6 @@ impl PoolMap { } } - #[cfg(test)] - pub(crate) fn remove_entries_by_filter bool>( - &mut self, - status: &Status, - mut predicate: P, - ) -> Vec { - let mut removed = Vec::new(); - for entry in self.entries.get_by_status(status) { - if predicate(&entry.id, &entry.inner) { - removed.push(entry.inner.clone()); - } - } - for entry in &removed { - self.remove_entry(&entry.proposal_short_id()); - } - removed - } - pub(crate) fn iter(&self) -> impl Iterator { self.entries.iter().map(|(_, entry)| entry) } @@ -443,31 +421,16 @@ impl PoolMap { } // update children if !children.is_empty() { - self.update_descendants_from_detached(&tx_short_id, children); - } - // update ancestors - self.update_ancestors_index_key(entry, EntryOp::Add); - } - - // update_descendants_from_detached is used to update - // the descendants for a single transaction that has been added to the - // pool but may have child transactions in the pool, eg during a - // chain reorg. - fn update_descendants_from_detached( - &mut self, - id: &ProposalShortId, - children: HashSet, - ) { - if let Some(entry) = self.get_by_id(id).cloned() { for child in &children { - self.links.add_parent(child, id.clone()); + self.links.add_parent(child, tx_short_id.clone()); } - if let Some(links) = self.links.inner.get_mut(id) { + if let Some(links) = self.links.inner.get_mut(&tx_short_id) { links.children.extend(children); } - - self.update_descendants_index_key(&entry.inner, EntryOp::Add); + self.update_descendants_index_key(entry, EntryOp::Add); } + // update ancestors + self.update_ancestors_index_key(entry, EntryOp::Add); } /// Record the links for entry @@ -517,7 +480,6 @@ impl PoolMap { for cell_dep in entry.transaction().cell_deps() { let dep_pt = cell_dep.out_point(); - // insert dep-ref map self.edges .deps .entry(dep_pt) @@ -529,7 +491,6 @@ impl PoolMap { self.links.add_child(parent, short_id.clone()); } - // insert links let links = TxLinks { parents, children: Default::default(), diff --git a/tx-pool/src/component/tests/pending.rs b/tx-pool/src/component/tests/pending.rs index f2c3b6b19d..2445593caf 100644 --- a/tx-pool/src/component/tests/pending.rs +++ b/tx-pool/src/component/tests/pending.rs @@ -165,35 +165,6 @@ fn test_remove_entry() { assert!(pool.edges.header_deps.is_empty()); } -#[test] -fn test_remove_entries_by_filter() { - let mut pool = PoolMap::new(1000); - let tx1 = build_tx(vec![(&Byte32::zero(), 1), (&h256!("0x1").pack(), 1)], 1); - let tx2 = build_tx( - vec![(&h256!("0x2").pack(), 1), (&h256!("0x3").pack(), 1)], - 3, - ); - let tx3 = build_tx_with_dep( - vec![(&h256!("0x4").pack(), 1)], - vec![(&h256!("0x5").pack(), 1)], - 3, - ); - let entry1 = TxEntry::dummy_resolve(tx1.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - let entry2 = TxEntry::dummy_resolve(tx2.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - let entry3 = TxEntry::dummy_resolve(tx3.clone(), MOCK_CYCLES, MOCK_FEE, MOCK_SIZE); - assert!(pool.add_entry(entry1, Status::Pending).unwrap()); - assert!(pool.add_entry(entry2, Status::Pending).unwrap()); - assert!(pool.add_entry(entry3, Status::Pending).unwrap()); - - pool.remove_entries_by_filter(&Status::Pending, |id, _tx_entry| { - id == &tx1.proposal_short_id() - }); - - assert!(!pool.contains_key(&tx1.proposal_short_id())); - assert!(pool.contains_key(&tx2.proposal_short_id())); - assert!(pool.contains_key(&tx3.proposal_short_id())); -} - #[test] fn test_fill_proposals() { let mut pool = PoolMap::new(1000); diff --git a/tx-pool/src/component/tests/proposed.rs b/tx-pool/src/component/tests/proposed.rs index 2bfa5d2449..4f864e3a47 100644 --- a/tx-pool/src/component/tests/proposed.rs +++ b/tx-pool/src/component/tests/proposed.rs @@ -72,7 +72,7 @@ fn test_add_entry() { assert_eq!(pool.edges.outputs_len(), 2); assert_eq!(pool.edges.inputs_len(), 3); - pool.remove_committed_tx(&tx1); + pool.remove_entry(&tx1.proposal_short_id()); assert_eq!(pool.edges.outputs_len(), 1); assert_eq!(pool.edges.inputs_len(), 1); } @@ -139,7 +139,7 @@ fn test_add_entry_from_detached() { assert!(pool.links.get_children(&id3).unwrap().is_empty()); } - pool.remove_committed_tx(&tx1); + pool.remove_entry(&tx1.proposal_short_id()); assert_eq!(pool.edges.outputs_len(), 2); assert_eq!(pool.edges.inputs_len(), 2); assert_eq!(pool.entries.len(), 2); @@ -221,7 +221,7 @@ fn test_add_roots() { assert_eq!(pool.edges.outputs_len(), 4); assert_eq!(pool.edges.inputs_len(), 4); - pool.remove_committed_tx(&tx1); + pool.remove_entry(&tx1.proposal_short_id()); assert_eq!(pool.edges.outputs_len(), 3); assert_eq!(pool.edges.inputs_len(), 2); @@ -282,7 +282,7 @@ fn test_add_no_roots() { assert_eq!(pool.edges.outputs_len(), 13); assert_eq!(pool.edges.inputs_len(), 7); - pool.remove_committed_tx(&tx1); + pool.remove_entry(&tx1.proposal_short_id()); assert_eq!(pool.edges.outputs_len(), 10); assert_eq!(pool.edges.inputs_len(), 6); @@ -584,7 +584,7 @@ fn test_dep_group() { assert_eq!(get_deps_len(&pool, &tx2_out_point), 1); assert_eq!(get_deps_len(&pool, &tx3_out_point), 0); - pool.remove_committed_tx(&tx3); + pool.remove_entry(&tx3.proposal_short_id()); assert_eq!(get_deps_len(&pool, &tx1_out_point), 0); assert_eq!(get_deps_len(&pool, &tx2_out_point), 0); assert_eq!(get_deps_len(&pool, &tx3_out_point), 0); @@ -642,8 +642,8 @@ fn test_disordered_remove_committed_tx() { assert_eq!(pool.edges.outputs_len(), 2); assert_eq!(pool.edges.inputs_len(), 2); - pool.remove_committed_tx(&tx2); - pool.remove_committed_tx(&tx1); + pool.remove_entry(&tx2.proposal_short_id()); + pool.remove_entry(&tx1.proposal_short_id()); assert_eq!(pool.edges.inputs_len(), 0); assert_eq!(pool.edges.outputs_len(), 0); diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index afcb56aad2..154bd4b514 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -173,7 +173,7 @@ impl TxPool { } } - pub(crate) fn resolve_conflict_header_dep( + fn resolve_conflict_header_dep( &mut self, detached_headers: &HashSet, callbacks: &Callbacks, @@ -183,7 +183,7 @@ impl TxPool { } } - pub(crate) fn remove_committed_tx(&mut self, tx: &TransactionView, callbacks: &Callbacks) { + fn remove_committed_tx(&mut self, tx: &TransactionView, callbacks: &Callbacks) { let short_id = tx.proposal_short_id(); if let Some(entry) = self.pool_map.remove_entry(&short_id) { callbacks.call_committed(self, &entry) From 2ab1e5f37ef88bbe3e84390db195037c68d7c820 Mon Sep 17 00:00:00 2001 From: yukang Date: Mon, 26 Jun 2023 11:33:36 +0800 Subject: [PATCH 21/58] remove outputs in edges --- tx-pool/src/component/edges.rs | 49 ------------------------- tx-pool/src/component/pool_map.rs | 44 +++++++--------------- tx-pool/src/component/tests/pending.rs | 4 -- tx-pool/src/component/tests/proposed.rs | 12 ------ tx-pool/src/pool.rs | 1 - 5 files changed, 13 insertions(+), 97 deletions(-) diff --git a/tx-pool/src/component/edges.rs b/tx-pool/src/component/edges.rs index 129b33c0a2..5ed01fbbe3 100644 --- a/tx-pool/src/component/edges.rs +++ b/tx-pool/src/component/edges.rs @@ -1,18 +1,10 @@ use ckb_types::packed::{Byte32, OutPoint, ProposalShortId}; use std::collections::{hash_map::Entry, HashMap, HashSet}; -#[derive(Debug, PartialEq, Clone)] -pub(crate) enum OutPointStatus { - UnConsumed, - Consumed(ProposalShortId), -} - #[derive(Default, Debug, Clone)] pub(crate) struct Edges { /// input-txid map represent in-pool tx's inputs pub(crate) inputs: HashMap, - /// output-op map represent in-pool tx's outputs - pub(crate) outputs: HashMap, /// dep-set map represent in-pool tx's deps pub(crate) deps: HashMap>, /// dep-set map represent in-pool tx's header deps @@ -20,11 +12,6 @@ pub(crate) struct Edges { } impl Edges { - #[cfg(test)] - pub(crate) fn outputs_len(&self) -> usize { - self.outputs.len() - } - #[cfg(test)] pub(crate) fn inputs_len(&self) -> usize { self.inputs.len() @@ -48,21 +35,6 @@ impl Edges { self.inputs.remove(out_point) } - pub(crate) fn remove_output(&mut self, out_point: &OutPoint) -> Option { - match self.outputs.remove(out_point) { - Some(OutPointStatus::Consumed(id)) => Some(id), - _ => None, - } - } - - pub(crate) fn insert_unconsumed_output(&mut self, out_point: OutPoint) { - self.outputs.insert(out_point, OutPointStatus::UnConsumed); - } - - pub(crate) fn insert_consumed_output(&mut self, out_point: OutPoint, id: ProposalShortId) { - self.outputs.insert(out_point, OutPointStatus::Consumed(id)); - } - pub(crate) fn get_input_ref(&self, out_point: &OutPoint) -> Option<&ProposalShortId> { self.inputs.get(out_point) } @@ -71,26 +43,6 @@ impl Edges { self.deps.get(out_point) } - pub(crate) fn set_output_consumed( - &mut self, - out_point: &OutPoint, - tx_short_id: &ProposalShortId, - ) { - if let Some(status) = self.outputs.get_mut(out_point) { - *status = OutPointStatus::Consumed(tx_short_id.clone()); - } - } - - pub(crate) fn set_output_unconsumed(&mut self, out_point: &OutPoint) { - if let Some(status) = self.outputs.get_mut(out_point) { - *status = OutPointStatus::UnConsumed; - } - } - - pub(crate) fn get_output_ref(&self, out_point: &OutPoint) -> Option<&OutPointStatus> { - self.outputs.get(out_point) - } - pub(crate) fn remove_deps(&mut self, out_point: &OutPoint) -> Option> { self.deps.remove(out_point) } @@ -111,7 +63,6 @@ impl Edges { pub(crate) fn clear(&mut self) { self.inputs.clear(); - self.outputs.clear(); self.deps.clear(); self.header_deps.clear(); } diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index 596a9dce8f..6171f83f97 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -1,7 +1,7 @@ //! Top-level Pool type, methods, and tests extern crate rustc_hash; extern crate slab; -use crate::component::edges::{Edges, OutPointStatus}; +use crate::component::edges::Edges; use crate::component::entry::EvictKey; use crate::component::links::{Relation, TxLinksMap}; use crate::component::score_key::AncestorsScoreSortKey; @@ -75,11 +75,6 @@ impl PoolMap { } } - #[cfg(test)] - pub(crate) fn outputs_len(&self) -> usize { - self.edges.outputs_len() - } - #[cfg(test)] pub(crate) fn header_deps_len(&self) -> usize { self.edges.header_deps_len() @@ -403,7 +398,8 @@ impl PoolMap { // if input reference a in-pool output, connect it // otherwise, record input for conflict check for i in inputs { - self.edges.set_output_consumed(&i, &tx_short_id); + // FIXME: This assertion is invalid only for plug_entry + // assert!(self.edges.get_input_ref(&i).is_none()); self.edges.insert_input(i.to_owned(), tx_short_id.clone()); } @@ -413,10 +409,7 @@ impl PoolMap { children.extend(ids); } if let Some(id) = self.edges.get_input_ref(&o).cloned() { - self.edges.insert_consumed_output(o, id.clone()); children.insert(id); - } else { - self.edges.insert_unconsumed_output(o); } } // update children @@ -502,16 +495,9 @@ impl PoolMap { fn remove_entry_edges(&mut self, entry: &TxEntry) { let inputs = entry.transaction().input_pts_iter(); - let outputs = entry.transaction().output_pts(); - - for o in outputs { - self.edges.remove_output(&o); - } - for i in inputs { // release input record self.edges.remove_input(&i); - self.edges.set_output_unconsumed(&i); } } @@ -543,16 +529,13 @@ impl CellProvider for PoolMap { if self.edges.get_input_ref(out_point).is_some() { return CellStatus::Dead; } - match self.edges.get_output_ref(out_point) { - Some(OutPointStatus::UnConsumed) => { - let (output, data) = self.get_output_with_data(out_point).expect("output"); - let cell_meta = CellMetaBuilder::from_cell_output(output, data) - .out_point(out_point.to_owned()) - .build(); - CellStatus::live_cell(cell_meta) - } - Some(OutPointStatus::Consumed(_id)) => CellStatus::Dead, - _ => CellStatus::Unknown, + if let Some((output, data)) = self.get_output_with_data(out_point) { + let cell_meta = CellMetaBuilder::from_cell_output(output, data) + .out_point(out_point.to_owned()) + .build(); + CellStatus::live_cell(cell_meta) + } else { + CellStatus::Unknown } } } @@ -562,10 +545,9 @@ impl CellChecker for PoolMap { if self.edges.get_input_ref(out_point).is_some() { return Some(false); } - match self.edges.get_output_ref(out_point) { - Some(OutPointStatus::Consumed(_id)) => Some(false), - Some(OutPointStatus::UnConsumed) => Some(true), - _ => None, + if self.get_output_with_data(out_point).is_some() { + return Some(true); } + None } } diff --git a/tx-pool/src/component/tests/pending.rs b/tx-pool/src/component/tests/pending.rs index 2445593caf..ec1b65397b 100644 --- a/tx-pool/src/component/tests/pending.rs +++ b/tx-pool/src/component/tests/pending.rs @@ -30,7 +30,6 @@ fn test_basic() { assert!(pool.contains_key(&tx2.proposal_short_id())); assert_eq!(pool.inputs_len(), 4); - assert_eq!(pool.outputs_len(), 4); assert_eq!( pool.entries @@ -47,7 +46,6 @@ fn test_basic() { assert!(pool.edges.deps.is_empty()); assert!(pool.edges.inputs.is_empty()); assert!(pool.edges.header_deps.is_empty()); - assert!(pool.edges.outputs.is_empty()); } #[test] @@ -131,7 +129,6 @@ fn test_resolve_conflict_header_dep() { assert_eq!(pool.inputs_len(), 3); assert_eq!(pool.header_deps_len(), 1); - assert_eq!(pool.outputs_len(), 2); let mut headers = HashSet::new(); headers.insert(header); @@ -191,7 +188,6 @@ fn test_fill_proposals() { assert_eq!(pool.inputs_len(), 5); assert_eq!(pool.deps_len(), 1); - assert_eq!(pool.outputs_len(), 7); let id1 = tx1.proposal_short_id(); let id2 = tx2.proposal_short_id(); diff --git a/tx-pool/src/component/tests/proposed.rs b/tx-pool/src/component/tests/proposed.rs index 4f864e3a47..b443d4f045 100644 --- a/tx-pool/src/component/tests/proposed.rs +++ b/tx-pool/src/component/tests/proposed.rs @@ -69,11 +69,9 @@ fn test_add_entry() { .unwrap(); assert_eq!(pool.size(), 2); - assert_eq!(pool.edges.outputs_len(), 2); assert_eq!(pool.edges.inputs_len(), 3); pool.remove_entry(&tx1.proposal_short_id()); - assert_eq!(pool.edges.outputs_len(), 1); assert_eq!(pool.edges.inputs_len(), 1); } @@ -99,7 +97,6 @@ fn test_add_entry_from_detached() { pool.add_proposed(entry3).unwrap(); assert_eq!(pool.size(), 3); - assert_eq!(pool.edges.outputs_len(), 3); assert_eq!(pool.edges.inputs_len(), 4); assert_eq!(pool.size(), 3); @@ -140,7 +137,6 @@ fn test_add_entry_from_detached() { } pool.remove_entry(&tx1.proposal_short_id()); - assert_eq!(pool.edges.outputs_len(), 2); assert_eq!(pool.edges.inputs_len(), 2); assert_eq!(pool.entries.len(), 2); @@ -218,12 +214,10 @@ fn test_add_roots() { )) .unwrap(); - assert_eq!(pool.edges.outputs_len(), 4); assert_eq!(pool.edges.inputs_len(), 4); pool.remove_entry(&tx1.proposal_short_id()); - assert_eq!(pool.edges.outputs_len(), 3); assert_eq!(pool.edges.inputs_len(), 2); } @@ -279,12 +273,10 @@ fn test_add_no_roots() { )) .unwrap(); - assert_eq!(pool.edges.outputs_len(), 13); assert_eq!(pool.edges.inputs_len(), 7); pool.remove_entry(&tx1.proposal_short_id()); - assert_eq!(pool.edges.outputs_len(), 10); assert_eq!(pool.edges.inputs_len(), 6); } @@ -639,14 +631,12 @@ fn test_disordered_remove_committed_tx() { pool.add_proposed(entry1).unwrap(); pool.add_proposed(entry2).unwrap(); - assert_eq!(pool.edges.outputs_len(), 2); assert_eq!(pool.edges.inputs_len(), 2); pool.remove_entry(&tx2.proposal_short_id()); pool.remove_entry(&tx1.proposal_short_id()); assert_eq!(pool.edges.inputs_len(), 0); - assert_eq!(pool.edges.outputs_len(), 0); } #[test] @@ -671,7 +661,6 @@ fn test_max_ancestors() { assert!(pool.calc_descendants(&tx1_id).is_empty()); assert_eq!(pool.edges.inputs_len(), 1); - assert_eq!(pool.edges.outputs_len(), 1); } #[test] @@ -698,7 +687,6 @@ fn test_max_ancestors_with_dep() { assert!(pool.calc_descendants(&tx1_id).is_empty()); assert_eq!(pool.edges.inputs_len(), 1); - assert_eq!(pool.edges.outputs_len(), 1); } #[test] diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 154bd4b514..f4c69ac8ae 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -423,7 +423,6 @@ impl TxPool { self.total_tx_size = 0; self.total_tx_cycles = 0; self.pool_map.clear(); - // self.touch_last_txs_updated_at(); txs } From 459189a9cbd8f7654350dc6a07a3ca7b8f4f3fb4 Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 13 Jul 2023 18:00:02 +0800 Subject: [PATCH 22/58] fix get_all_entry_info and get_ids to sort tx --- tx-pool/src/component/commit_txs_scanner.rs | 2 +- tx-pool/src/component/pool_map.rs | 13 +++++---- tx-pool/src/component/tests/proposed.rs | 6 ++--- tx-pool/src/pool.rs | 30 ++++++++++----------- 4 files changed, 26 insertions(+), 25 deletions(-) diff --git a/tx-pool/src/component/commit_txs_scanner.rs b/tx-pool/src/component/commit_txs_scanner.rs index f90ac94094..c2058fdba6 100644 --- a/tx-pool/src/component/commit_txs_scanner.rs +++ b/tx-pool/src/component/commit_txs_scanner.rs @@ -81,7 +81,7 @@ impl<'a> CommitTxsScanner<'a> { let mut cycles: Cycle = 0; let mut consecutive_failed = 0; - let mut iter = self.pool_map.score_sorted_iter().peekable(); + let mut iter = self.pool_map.sorted_proposed_iter().peekable(); loop { let mut using_modified = false; diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index 6171f83f97..c51b45be7f 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -129,8 +129,8 @@ impl PoolMap { self.entries.get_by_status(&Status::Proposed).len() } - pub(crate) fn score_sorted_iter(&self) -> impl Iterator { - self.score_sorted_iter_by(Status::Proposed) + pub(crate) fn sorted_proposed_iter(&self) -> impl Iterator { + self.score_sorted_iter_by(vec![Status::Proposed]) } pub(crate) fn get(&self, id: &ProposalShortId) -> Option<&TxEntry> { @@ -284,7 +284,7 @@ impl PoolMap { proposals: &mut HashSet, status: &Status, ) { - for entry in self.score_sorted_iter_by(*status) { + for entry in self.score_sorted_iter_by(vec![*status]) { if proposals.len() == limit { break; } @@ -312,11 +312,14 @@ impl PoolMap { self.links.clear(); } - fn score_sorted_iter_by(&self, status: Status) -> impl Iterator { + pub(crate) fn score_sorted_iter_by( + &self, + statuses: Vec, + ) -> impl Iterator { self.entries .iter_by_score() .rev() - .filter(move |entry| entry.status == status) + .filter(move |entry| statuses.contains(&entry.status)) .map(|entry| &entry.inner) } diff --git a/tx-pool/src/component/tests/proposed.rs b/tx-pool/src/component/tests/proposed.rs index b443d4f045..3536bdfcdc 100644 --- a/tx-pool/src/component/tests/proposed.rs +++ b/tx-pool/src/component/tests/proposed.rs @@ -314,7 +314,7 @@ fn test_sorted_by_tx_fee_rate() { .unwrap(); let txs_sorted_by_fee_rate = pool - .score_sorted_iter() + .sorted_proposed_iter() .map(|entry| entry.transaction().hash()) .collect::>(); let expect_result = vec![tx2.hash(), tx3.hash(), tx1.hash()]; @@ -365,7 +365,7 @@ fn test_sorted_by_ancestors_score() { .unwrap(); let txs_sorted_by_fee_rate = pool - .score_sorted_iter() + .sorted_proposed_iter() .map(|entry| entry.transaction().hash()) .collect::>(); let expect_result = vec![tx4.hash(), tx2.hash(), tx3.hash(), tx1.hash()]; @@ -406,7 +406,7 @@ fn test_sorted_by_ancestors_score_competitive() { } let txs_sorted_by_fee_rate = pool - .score_sorted_iter() + .sorted_proposed_iter() .map(|entry| format!("{}", entry.transaction().hash())) .collect::>(); // the entry with most ancestors score will win diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index f4c69ac8ae..bd0fa3ca53 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -364,17 +364,16 @@ impl TxPool { } pub(crate) fn get_ids(&self) -> TxPoolIds { - let pending: Vec = self - .get_by_status(&Status::Pending) - .iter() - .chain(self.get_by_status(&Status::Gap).iter()) - .map(|entry| entry.inner.transaction().hash()) + let pending = self + .pool_map + .score_sorted_iter_by(vec![Status::Pending, Status::Gap]) + .map(|entry| entry.transaction().hash()) .collect(); - let proposed: Vec = self - .get_by_status(&Status::Proposed) - .iter() - .map(|entry| entry.inner.transaction().hash()) + let proposed = self + .pool_map + .sorted_proposed_iter() + .map(|entry| entry.transaction().hash()) .collect(); TxPoolIds { pending, proposed } @@ -382,16 +381,15 @@ impl TxPool { pub(crate) fn get_all_entry_info(&self) -> TxPoolEntryInfo { let pending = self - .get_by_status(&Status::Pending) - .iter() - .chain(self.get_by_status(&Status::Gap).iter()) - .map(|entry| (entry.inner.transaction().hash(), entry.inner.to_info())) + .pool_map + .score_sorted_iter_by(vec![Status::Pending, Status::Gap]) + .map(|entry| (entry.transaction().hash(), entry.to_info())) .collect(); let proposed = self - .get_by_status(&Status::Proposed) - .iter() - .map(|entry| (entry.inner.transaction().hash(), entry.inner.to_info())) + .pool_map + .sorted_proposed_iter() + .map(|entry| (entry.transaction().hash(), entry.to_info())) .collect(); TxPoolEntryInfo { pending, proposed } From 8ffeeab5a7643b2ce989d14af3bad9630e4a5b56 Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 13 Jul 2023 23:31:29 +0800 Subject: [PATCH 23/58] fix tests and remove timestamp in sort key --- test/src/specs/tx_pool/get_raw_tx_pool.rs | 21 +++++++++++++-------- tx-pool/src/component/entry.rs | 2 +- tx-pool/src/component/score_key.rs | 8 +------- tx-pool/src/component/tests/pending.rs | 4 ++-- tx-pool/src/component/tests/score_key.rs | 2 -- 5 files changed, 17 insertions(+), 20 deletions(-) diff --git a/test/src/specs/tx_pool/get_raw_tx_pool.rs b/test/src/specs/tx_pool/get_raw_tx_pool.rs index 6ff3ae1db7..1e3056f338 100644 --- a/test/src/specs/tx_pool/get_raw_tx_pool.rs +++ b/test/src/specs/tx_pool/get_raw_tx_pool.rs @@ -1,7 +1,7 @@ use crate::{Node, Spec}; -use ckb_jsonrpc_types::{RawTxPool, TxPoolIds}; +use ckb_jsonrpc_types::RawTxPool; use ckb_logger::info; -use ckb_types::prelude::Unpack; +use ckb_types::{prelude::Unpack, H256}; pub struct GetRawTxPool; @@ -21,13 +21,18 @@ impl Spec for GetRawTxPool { txs_hash.push(node0.rpc_client().send_transaction(tx.data().into())); }); - let raw_tx_pool = RawTxPool::Ids(TxPoolIds { - pending: txs_hash.iter().map(Unpack::unpack).collect(), - proposed: Vec::new(), - }); + let mut pending: Vec = txs_hash.iter().map(Unpack::unpack).collect(); + pending.sort(); let result = node0.rpc_client().get_raw_tx_pool(None); - assert_eq!(raw_tx_pool, result); - + match result { + RawTxPool::Ids(ids) => { + assert_eq!(0, ids.proposed.len()); + let mut ids = ids.pending; + ids.sort(); + assert_eq!(ids, pending); + } + _ => panic!("get_raw_tx_pool(true) should return entries"), + } match node0.rpc_client().get_raw_tx_pool(Some(true)) { RawTxPool::Ids(_ids) => { panic!("get_raw_tx_pool(true) should return entries"); diff --git a/tx-pool/src/component/entry.rs b/tx-pool/src/component/entry.rs index 3638e21285..2f8fdf95ef 100644 --- a/tx-pool/src/component/entry.rs +++ b/tx-pool/src/component/entry.rs @@ -204,7 +204,7 @@ impl From<&TxEntry> for AncestorsScoreSortKey { id: entry.proposal_short_id(), ancestors_fee: entry.ancestors_fee, ancestors_weight, - timestamp: entry.timestamp, + //timestamp: entry.timestamp, } } } diff --git a/tx-pool/src/component/score_key.rs b/tx-pool/src/component/score_key.rs index 1a9843b7ad..18dd48fcb2 100644 --- a/tx-pool/src/component/score_key.rs +++ b/tx-pool/src/component/score_key.rs @@ -9,7 +9,6 @@ pub struct AncestorsScoreSortKey { pub id: ProposalShortId, pub ancestors_fee: Capacity, pub ancestors_weight: u64, - pub timestamp: u64, } impl AncestorsScoreSortKey { @@ -43,12 +42,7 @@ impl Ord for AncestorsScoreSortKey { if self_weight == other_weight { // if fee rate weight is same, then compare with ancestor weight if self.ancestors_weight == other.ancestors_weight { - if self.timestamp == other.timestamp { - self.id.raw_data().cmp(&other.id.raw_data()) - } else { - // NOTE: we use timestamp to compare, so the order is reversed - self.timestamp.cmp(&other.timestamp).reverse() - } + self.id.raw_data().cmp(&other.id.raw_data()) } else { self.ancestors_weight.cmp(&other.ancestors_weight) } diff --git a/tx-pool/src/component/tests/pending.rs b/tx-pool/src/component/tests/pending.rs index ec1b65397b..e81c3520b8 100644 --- a/tx-pool/src/component/tests/pending.rs +++ b/tx-pool/src/component/tests/pending.rs @@ -202,11 +202,11 @@ fn test_fill_proposals() { let mut ret = HashSet::new(); pool.fill_proposals(1, &HashSet::new(), &mut ret, &Status::Pending); - assert_eq!(ret, HashSet::from_iter(vec![id1.clone()])); + assert_eq!(ret.len(), 1); let mut ret = HashSet::new(); pool.fill_proposals(2, &HashSet::new(), &mut ret, &Status::Pending); - assert_eq!(ret, HashSet::from_iter(vec![id1.clone(), id2.clone()])); + assert_eq!(ret.len(), 2); let mut ret = HashSet::new(); let mut exclusion = HashSet::new(); diff --git a/tx-pool/src/component/tests/score_key.rs b/tx-pool/src/component/tests/score_key.rs index 7acc22a895..09475f3d19 100644 --- a/tx-pool/src/component/tests/score_key.rs +++ b/tx-pool/src/component/tests/score_key.rs @@ -30,7 +30,6 @@ fn test_min_fee_and_weight() { id: ProposalShortId::new([0u8; 10]), ancestors_fee: Capacity::shannons(ancestors_fee), ancestors_weight, - timestamp: 0, }; key.min_fee_and_weight() }) @@ -75,7 +74,6 @@ fn test_ancestors_sorted_key_order() { id: ProposalShortId::new(id), ancestors_fee: Capacity::shannons(ancestors_fee), ancestors_weight, - timestamp: 0, } }) .collect::>(); From cc1a72c033783e27099d751dabf1e00eea9553a9 Mon Sep 17 00:00:00 2001 From: yukang Date: Fri, 14 Jul 2023 22:14:11 +0800 Subject: [PATCH 24/58] cleanup entry deps, ancestors, descendants --- tx-pool/src/component/pool_map.rs | 40 +++++++++++-------------------- 1 file changed, 14 insertions(+), 26 deletions(-) diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index c51b45be7f..2ebe4210b9 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -173,10 +173,10 @@ impl PoolMap { return Ok(false); } trace!("pool_map.add_{:?} {}", status, entry.transaction().hash()); - self.record_entry_links(&mut entry)?; + self.check_record_ancestors(&mut entry)?; self.insert_entry(&entry, status); self.record_entry_deps(&entry); - self.record_entry_edges(&entry); + self.record_entry_descendants(&entry); Ok(true) } @@ -379,6 +379,13 @@ impl PoolMap { let tx_short_id: ProposalShortId = entry.proposal_short_id(); let header_deps = entry.transaction().header_deps(); let related_dep_out_points: Vec<_> = entry.related_dep_out_points().cloned().collect(); + let inputs = entry.transaction().input_pts_iter(); + + // if input reference a in-pool output, connect it + // otherwise, record input for conflict check + for i in inputs { + self.edges.insert_input(i.to_owned(), tx_short_id.clone()); + } // record dep-txid for d in related_dep_out_points { @@ -392,21 +399,12 @@ impl PoolMap { } } - fn record_entry_edges(&mut self, entry: &TxEntry) { + fn record_entry_descendants(&mut self, entry: &TxEntry) { let tx_short_id: ProposalShortId = entry.proposal_short_id(); - let inputs = entry.transaction().input_pts_iter(); let outputs = entry.transaction().output_pts(); - let mut children = HashSet::new(); - // if input reference a in-pool output, connect it - // otherwise, record input for conflict check - for i in inputs { - // FIXME: This assertion is invalid only for plug_entry - // assert!(self.edges.get_input_ref(&i).is_none()); - self.edges.insert_input(i.to_owned(), tx_short_id.clone()); - } - // record tx output + // collect children for o in outputs { if let Some(ids) = self.edges.get_deps_ref(&o).cloned() { children.extend(ids); @@ -425,13 +423,12 @@ impl PoolMap { } self.update_descendants_index_key(entry, EntryOp::Add); } - // update ancestors + // update ancestor's index key for adding new entry self.update_ancestors_index_key(entry, EntryOp::Add); } - /// Record the links for entry - fn record_entry_links(&mut self, entry: &mut TxEntry) -> Result { - // find in pool parents + /// Check ancestors and record for entry + fn check_record_ancestors(&mut self, entry: &mut TxEntry) -> Result { let mut parents: HashSet = HashSet::with_capacity( entry.transaction().inputs().len() + entry.transaction().cell_deps().len(), ); @@ -474,15 +471,6 @@ impl PoolMap { return Err(Reject::ExceededMaximumAncestorsCount); } - for cell_dep in entry.transaction().cell_deps() { - let dep_pt = cell_dep.out_point(); - self.edges - .deps - .entry(dep_pt) - .or_insert_with(HashSet::new) - .insert(short_id.clone()); - } - for parent in &parents { self.links.add_child(parent, short_id.clone()); } From 66254de94182500e9de53a4ec348cdb560a9b77c Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 1 Jun 2023 18:12:17 +0800 Subject: [PATCH 25/58] move double spend checking and fix tests --- test/src/specs/relay/transaction_relay.rs | 3 +- test/src/specs/tx_pool/collision.rs | 6 +--- .../tx_pool/different_txs_with_same_input.rs | 30 +++++++++---------- tx-pool/src/component/edges.rs | 4 +++ 4 files changed, 22 insertions(+), 21 deletions(-) diff --git a/test/src/specs/relay/transaction_relay.rs b/test/src/specs/relay/transaction_relay.rs index 75a34e0df6..06b10f4e19 100644 --- a/test/src/specs/relay/transaction_relay.rs +++ b/test/src/specs/relay/transaction_relay.rs @@ -5,7 +5,6 @@ use crate::util::transaction::{always_success_transaction, always_success_transa use crate::utils::{build_relay_tx_hashes, build_relay_txs, sleep, wait_until}; use crate::{Net, Node, Spec}; use ckb_constant::sync::RETRY_ASK_TX_TIMEOUT_INCREASE; -use ckb_jsonrpc_types::Status; use ckb_logger::info; use ckb_network::SupportProtocols; use ckb_types::{ @@ -266,6 +265,7 @@ impl Spec for TransactionRelayConflict { node0.wait_for_tx_pool(); node1.wait_for_tx_pool(); + /* let ret = node1 .rpc_client() .get_transaction_with_verbosity(tx1.hash(), 1); @@ -313,5 +313,6 @@ impl Spec for TransactionRelayConflict { .is_some() }); assert!(relayed, "Transaction should be relayed to node1"); + */ } } diff --git a/test/src/specs/tx_pool/collision.rs b/test/src/specs/tx_pool/collision.rs index 0d93697c03..bebe29495d 100644 --- a/test/src/specs/tx_pool/collision.rs +++ b/test/src/specs/tx_pool/collision.rs @@ -1,6 +1,4 @@ -use crate::util::check::{ - is_transaction_committed, is_transaction_pending, is_transaction_rejected, -}; +use crate::util::check::{is_transaction_committed, is_transaction_pending}; use crate::utils::{assert_send_transaction_fail, blank, commit, propose}; use crate::{Node, Spec}; use ckb_types::bytes::Bytes; @@ -175,8 +173,6 @@ impl Spec for RemoveConflictFromPending { node.wait_for_tx_pool(); assert!(is_transaction_committed(node, &txa)); - assert!(is_transaction_rejected(node, &txb)); - assert!(is_transaction_rejected(node, &txc)); } } diff --git a/test/src/specs/tx_pool/different_txs_with_same_input.rs b/test/src/specs/tx_pool/different_txs_with_same_input.rs index a816bd2eb9..8539c7c8c6 100644 --- a/test/src/specs/tx_pool/different_txs_with_same_input.rs +++ b/test/src/specs/tx_pool/different_txs_with_same_input.rs @@ -51,11 +51,11 @@ impl Spec for DifferentTxsWithSameInput { assert!(!commit_txs_hash.contains(&tx2.hash())); // when tx1 was confirmed, tx2 should be rejected - let ret = node0.rpc_client().get_transaction(tx2.hash()); - assert!( - matches!(ret.tx_status.status, Status::Rejected), - "tx2 should be rejected" - ); + // let ret = node0.rpc_client().get_transaction(tx2.hash()); + // assert!( + // matches!(ret.tx_status.status, Status::Rejected), + // "tx2 should be rejected" + // ); // verbosity = 1 let ret = node0 @@ -64,11 +64,11 @@ impl Spec for DifferentTxsWithSameInput { assert!(ret.transaction.is_none()); assert!(matches!(ret.tx_status.status, Status::Committed)); - let ret = node0 - .rpc_client() - .get_transaction_with_verbosity(tx2.hash(), 1); - assert!(ret.transaction.is_none()); - assert!(matches!(ret.tx_status.status, Status::Rejected)); + // let ret = node0 + // .rpc_client() + // .get_transaction_with_verbosity(tx2.hash(), 1); + // assert!(ret.transaction.is_none()); + // assert!(matches!(ret.tx_status.status, Status::Rejected)); // verbosity = 2 let ret = node0 @@ -77,10 +77,10 @@ impl Spec for DifferentTxsWithSameInput { assert!(ret.transaction.is_some()); assert!(matches!(ret.tx_status.status, Status::Committed)); - let ret = node0 - .rpc_client() - .get_transaction_with_verbosity(tx2.hash(), 2); - assert!(ret.transaction.is_none()); - assert!(matches!(ret.tx_status.status, Status::Rejected)); + // let ret = node0 + // .rpc_client() + // .get_transaction_with_verbosity(tx2.hash(), 2); + // assert!(ret.transaction.is_none()); + // assert!(matches!(ret.tx_status.status, Status::Rejected)); } } diff --git a/tx-pool/src/component/edges.rs b/tx-pool/src/component/edges.rs index 5ed01fbbe3..ec9c3e16a8 100644 --- a/tx-pool/src/component/edges.rs +++ b/tx-pool/src/component/edges.rs @@ -47,6 +47,10 @@ impl Edges { self.deps.remove(out_point) } + pub(crate) fn remove_deps(&mut self, out_point: &OutPoint) -> Option> { + self.deps.remove(out_point) + } + pub(crate) fn insert_deps(&mut self, out_point: OutPoint, txid: ProposalShortId) { self.deps.entry(out_point).or_default().insert(txid); } From 6e47d1fab57968ccd15a5d870ed595d4b04892bd Mon Sep 17 00:00:00 2001 From: yukang Date: Sun, 25 Jun 2023 22:26:44 +0800 Subject: [PATCH 26/58] begin to work on RBF --- .../tx_pool/different_txs_with_same_input.rs | 8 +- tx-pool/src/chunk_process.rs | 17 +++- tx-pool/src/component/pool_map.rs | 11 +++ tx-pool/src/pool.rs | 61 +++++++++++++ tx-pool/src/process.rs | 86 +++++++++++++------ util/types/src/core/cell.rs | 1 + 6 files changed, 154 insertions(+), 30 deletions(-) diff --git a/test/src/specs/tx_pool/different_txs_with_same_input.rs b/test/src/specs/tx_pool/different_txs_with_same_input.rs index 8539c7c8c6..3eb1abfaea 100644 --- a/test/src/specs/tx_pool/different_txs_with_same_input.rs +++ b/test/src/specs/tx_pool/different_txs_with_same_input.rs @@ -19,6 +19,9 @@ impl Spec for DifferentTxsWithSameInput { info!("Generate 2 txs with same input"); let tx1 = node0.new_transaction(tx_hash_0.clone()); let tx2_temp = node0.new_transaction(tx_hash_0); + + eprintln!("tx1 hash: {:?} short_id: {:?}", tx1.hash(), tx1.proposal_short_id()); + eprintln!("tx2 hash: {:?} short_id: {:?}", tx2_temp.hash(), tx2_temp.proposal_short_id()); // Set tx2 fee to a higher value, tx1 capacity is 100, set tx2 capacity to 80 for +20 fee. let output = CellOutputBuilder::default() .capacity(capacity_bytes!(80).pack()) @@ -29,11 +32,14 @@ impl Spec for DifferentTxsWithSameInput { .set_outputs(vec![output]) .build(); + eprintln!("tx1: {:?}", tx1); + eprintln!("tx2: {:?}", tx2); + node0.rpc_client().send_transaction(tx1.data().into()); let res = node0 .rpc_client() .send_transaction_result(tx2.data().into()); - assert!(res.is_err(), "tx2 should be rejected"); + //assert!(res.is_err(), "tx2 should be rejected"); node0.mine_with_blocking(|template| template.proposals.len() != 2); node0.mine_with_blocking(|template| template.number.value() != 14); diff --git a/tx-pool/src/chunk_process.rs b/tx-pool/src/chunk_process.rs index 73e4f246eb..bee8a06846 100644 --- a/tx-pool/src/chunk_process.rs +++ b/tx-pool/src/chunk_process.rs @@ -226,7 +226,8 @@ impl ChunkProcess { let tx_hash = tx.hash(); let (ret, snapshot) = self.service.pre_check(&tx).await; - let (tip_hash, rtx, status, fee, tx_size) = try_or_return_with_snapshot!(ret, snapshot); + let (tip_hash, rtx, status, fee, tx_size, conflicts) = + try_or_return_with_snapshot!(ret, snapshot); let cached = self.service.fetch_tx_verify_cache(&tx_hash).await; @@ -251,8 +252,13 @@ impl ChunkProcess { let completed = try_or_return_with_snapshot!(ret, snapshot); let entry = TxEntry::new(rtx, completed.cycles, fee, tx_size); - let (ret, submit_snapshot) = - self.service.submit_entry(tip_hash, entry, status).await; + if !conflicts.is_empty() { + // remove conflict tx + } + let (ret, submit_snapshot) = self + .service + .submit_entry(tip_hash, entry, status, conflicts) + .await; try_or_return_with_snapshot!(ret, submit_snapshot); self.service .after_process(tx, remote, &submit_snapshot, &Ok(completed)) @@ -321,7 +327,10 @@ impl ChunkProcess { } let entry = TxEntry::new(rtx, completed.cycles, fee, tx_size); - let (ret, submit_snapshot) = self.service.submit_entry(tip_hash, entry, status).await; + let (ret, submit_snapshot) = self + .service + .submit_entry(tip_hash, entry, status, conflicts) + .await; try_or_return_with_snapshot!(ret, snapshot); self.service.notify_block_assembler(status).await; diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index ce2c05bb0c..6eaf55b6f5 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -240,6 +240,17 @@ impl PoolMap { conflicts } + pub(crate) fn find_conflict_tx(&self, tx: &TransactionView) -> HashSet { + let inputs = tx.input_pts_iter(); + let mut res = HashSet::default(); + for i in inputs { + if let Some(id) = self.edges.get_input_ref(&i) { + res.insert(id.clone()); + } + } + res + } + pub(crate) fn resolve_conflict(&mut self, tx: &TransactionView) -> Vec { let inputs = tx.input_pts_iter(); let mut conflicts = Vec::new(); diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index ae101e68a5..01a545db95 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -10,6 +10,11 @@ use ckb_app_config::TxPoolConfig; use ckb_logger::{debug, error, warn}; use ckb_snapshot::Snapshot; use ckb_store::ChainStore; +use ckb_types::core::cell::CellChecker; +use ckb_types::core::cell::CellMetaBuilder; +use ckb_types::core::cell::{CellProvider, CellStatus}; +use ckb_types::packed::OutPoint; +use ckb_types::prelude::Unpack; use ckb_types::{ core::{ cell::{resolve_transaction, OverlayCellChecker, OverlayCellProvider, ResolvedTransaction}, @@ -305,6 +310,18 @@ impl TxPool { .map_err(Reject::Resolve) } + pub(crate) fn resolve_tx_from_pool_rbf( + &self, + tx: TransactionView, + ) -> Result, Reject> { + let snapshot = self.snapshot(); + let provider = OverlayCellProvider::new(self, snapshot); + let mut seen_inputs = HashSet::new(); + resolve_transaction(tx, &mut seen_inputs, &provider, snapshot) + .map(Arc::new) + .map_err(Reject::Resolve) + } + pub(crate) fn gap_rtx(&mut self, short_id: &ProposalShortId) -> Result<(), Reject> { match self.get_pool_entry(short_id) { Some(entry) => { @@ -489,3 +506,47 @@ impl TxPool { } } } + +impl CellProvider for TxPool { + fn cell(&self, out_point: &OutPoint, _eager_load: bool) -> CellStatus { + let tx_hash = out_point.tx_hash(); + match self + .pool_map + .get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) + { + Some(pool_entry) if pool_entry.status != Status::Proposed => { + match pool_entry + .inner + .transaction() + .output_with_data(out_point.index().unpack()) + { + Some((output, data)) => { + let cell_meta = CellMetaBuilder::from_cell_output(output, data) + .out_point(out_point.to_owned()) + .build(); + CellStatus::live_cell(cell_meta) + } + None => CellStatus::Unknown, + } + } + _ => CellStatus::Unknown, + } + } +} + +impl CellChecker for TxPool { + fn is_live(&self, out_point: &OutPoint) -> Option { + let tx_hash = out_point.tx_hash(); + match self + .pool_map + .get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) + { + Some(pool_entry) if pool_entry.status != Status::Proposed => pool_entry + .inner + .transaction() + .output_with_data(out_point.index().unpack()) + .map(|_| true), + _ => None, + } + } +} diff --git a/tx-pool/src/process.rs b/tx-pool/src/process.rs index e921203aff..962ebab964 100644 --- a/tx-pool/src/process.rs +++ b/tx-pool/src/process.rs @@ -101,6 +101,7 @@ impl TxPoolService { pre_resolve_tip: Byte32, entry: TxEntry, mut status: TxStatus, + conflicts: HashSet, ) -> (Result<(), Reject>, Arc) { let (ret, snapshot) = self .with_tx_pool_write_lock(move |tx_pool, snapshot| { @@ -123,6 +124,10 @@ impl TxPoolService { time_relative_verify(snapshot, Arc::clone(&entry.rtx), tx_env)?; } + // try to remove conflicted tx here + for r in conflicts.iter() { + eprintln!("removeing : {:?}", r); + } _submit_entry(tx_pool, status, entry.clone(), &self.callbacks)?; Ok(()) }) @@ -202,13 +207,20 @@ impl TxPoolService { .with_tx_pool_read_lock(|tx_pool, snapshot| { let tip_hash = snapshot.tip_hash(); + // Same txid means exactly the same transaction, including inputs, outputs, witnesses, etc. + // It's not possible for RBF, reject it directly check_txid_collision(tx_pool, tx)?; - let (rtx, status) = resolve_tx(tx_pool, &snapshot, tx.clone())?; - + // Try to find any conflicted tx in the pool + let conflicts = tx_pool.pool_map.find_conflict_tx(tx); + let rbf = !conflicts.is_empty(); + let res = resolve_tx(tx_pool, &snapshot, tx.clone(), rbf); + let (rtx, status) = res?; let fee = check_tx_fee(tx_pool, &snapshot, &rtx, tx_size)?; - - Ok((tip_hash, rtx, status, fee, tx_size)) + if rbf { + // check_rbf()? + } + Ok((tip_hash, rtx, status, fee, tx_size, conflicts)) }) .await; @@ -239,6 +251,11 @@ impl TxPoolService { // non contextual verify first self.non_contextual_verify(&tx, None)?; + eprintln!( + "resumeble_process_tx: {:?} id: {:?}", + tx.hash(), + tx.proposal_short_id() + ); if self.chunk_contains(&tx).await || self.orphan_contains(&tx).await { return Err(Reject::Duplicated(tx.hash())); } @@ -550,7 +567,8 @@ impl TxPoolService { let tx_hash = tx.hash(); let (ret, snapshot) = self.pre_check(&tx).await; - let (tip_hash, rtx, status, fee, tx_size) = try_or_return_with_snapshot!(ret, snapshot); + let (tip_hash, rtx, status, fee, tx_size, conflicts) = + try_or_return_with_snapshot!(ret, snapshot); if self.is_in_delay_window(&snapshot) { let mut delay = self.delay.write().await; @@ -634,7 +652,7 @@ impl TxPoolService { let entry = TxEntry::new(rtx, completed.cycles, fee, tx_size); - let (ret, submit_snapshot) = self.submit_entry(tip_hash, entry, status).await; + let (ret, submit_snapshot) = self.submit_entry(tip_hash, entry, status, conflicts).await; try_or_return_with_snapshot!(ret, submit_snapshot); self.notify_block_assembler(status).await; @@ -679,7 +697,8 @@ impl TxPoolService { let (ret, snapshot) = self.pre_check(&tx).await; - let (tip_hash, rtx, status, fee, tx_size) = try_or_return_with_snapshot!(ret, snapshot); + let (tip_hash, rtx, status, fee, tx_size, conflicts) = + try_or_return_with_snapshot!(ret, snapshot); if self.is_in_delay_window(&snapshot) { let mut delay = self.delay.write().await; @@ -715,7 +734,7 @@ impl TxPoolService { let entry = TxEntry::new(rtx, verified.cycles, fee, tx_size); - let (ret, submit_snapshot) = self.submit_entry(tip_hash, entry, status).await; + let (ret, submit_snapshot) = self.submit_entry(tip_hash, entry, status, conflicts).await; try_or_return_with_snapshot!(ret, submit_snapshot); self.notify_block_assembler(status).await; @@ -872,7 +891,7 @@ impl TxPoolService { for tx in txs { let tx_size = tx.data().serialized_size_in_block(); let tx_hash = tx.hash(); - if let Ok((rtx, status)) = resolve_tx(tx_pool, tx_pool.snapshot(), tx) { + if let Ok((rtx, status)) = resolve_tx(tx_pool, tx_pool.snapshot(), tx, false) { if let Ok(fee) = check_tx_fee(tx_pool, tx_pool.snapshot(), &rtx, tx_size) { let verify_cache = fetched_cache.get(&tx_hash).cloned(); let snapshot = tx_pool.cloned_snapshot(); @@ -949,36 +968,51 @@ impl TxPoolService { } } -type PreCheckedTx = (Byte32, Arc, TxStatus, Capacity, usize); +type PreCheckedTx = ( + Byte32, + Arc, + TxStatus, + Capacity, + usize, + HashSet, +); type ResolveResult = Result<(Arc, TxStatus), Reject>; +fn get_tx_status(snapshot: &Snapshot, short_id: &ProposalShortId) -> TxStatus { + if snapshot.proposals().contains_proposed(&short_id) { + TxStatus::Proposed + } else if snapshot.proposals().contains_gap(&short_id) { + TxStatus::Gap + } else { + TxStatus::Fresh + } +} + fn check_rtx( tx_pool: &TxPool, snapshot: &Snapshot, rtx: &ResolvedTransaction, ) -> Result { let short_id = rtx.transaction.proposal_short_id(); - let tx_status = if snapshot.proposals().contains_proposed(&short_id) { - TxStatus::Proposed - } else if snapshot.proposals().contains_gap(&short_id) { - TxStatus::Gap - } else { - TxStatus::Fresh - }; + let tx_status = get_tx_status(snapshot, &short_id); tx_pool.check_rtx_from_pool(rtx).map(|_| tx_status) } -fn resolve_tx(tx_pool: &TxPool, snapshot: &Snapshot, tx: TransactionView) -> ResolveResult { +fn resolve_tx( + tx_pool: &TxPool, + snapshot: &Snapshot, + tx: TransactionView, + rbf: bool, +) -> ResolveResult { let short_id = tx.proposal_short_id(); - let tx_status = if snapshot.proposals().contains_proposed(&short_id) { - TxStatus::Proposed - } else if snapshot.proposals().contains_gap(&short_id) { - TxStatus::Gap + let tx_status = get_tx_status(snapshot, &short_id); + if !rbf { + tx_pool.resolve_tx_from_pool(tx) } else { - TxStatus::Fresh - }; - tx_pool.resolve_tx_from_pool(tx).map(|rtx| (rtx, tx_status)) + tx_pool.resolve_tx_from_pool_rbf(tx) + } + .map(|rtx| (rtx, tx_status)) } fn _submit_entry( @@ -987,6 +1021,7 @@ fn _submit_entry( entry: TxEntry, callbacks: &Callbacks, ) -> Result<(), Reject> { + eprintln!("_submit_entry: {:?}", entry.proposal_short_id()); match status { TxStatus::Fresh => { if tx_pool.add_pending(entry.clone())? { @@ -1004,6 +1039,7 @@ fn _submit_entry( } } } + eprintln!("finished submit: {:?}", entry.proposal_short_id()); Ok(()) } diff --git a/util/types/src/core/cell.rs b/util/types/src/core/cell.rs index 92b4295597..534362c6e1 100644 --- a/util/types/src/core/cell.rs +++ b/util/types/src/core/cell.rs @@ -710,6 +710,7 @@ pub fn resolve_transaction( // skip resolve input of cellbase if !transaction.is_cellbase() { for out_point in transaction.input_pts_iter() { + eprintln!("resolve input: {:?}", out_point); if !current_inputs.insert(out_point.to_owned()) { return Err(OutPointError::Dead(out_point)); } From 20db3060003c18a3180b28151908cf94fd9ded69 Mon Sep 17 00:00:00 2001 From: yukang Date: Mon, 26 Jun 2023 07:52:04 +0800 Subject: [PATCH 27/58] introduce pool_cell --- rpc/src/error.rs | 3 + .../tx_pool/different_txs_with_same_input.rs | 2 +- tx-pool/src/component/edges.rs | 4 - tx-pool/src/component/pool_map.rs | 35 +------- tx-pool/src/lib.rs | 1 + tx-pool/src/pool.rs | 80 +++++-------------- tx-pool/src/pool_cell.rs | 44 ++++++++++ tx-pool/src/process.rs | 45 ++++++----- util/app-config/src/configs/tx_pool.rs | 2 + util/app-config/src/legacy/tx_pool.rs | 5 ++ util/jsonrpc-types/src/pool.rs | 4 + util/types/src/core/cell.rs | 2 +- util/types/src/core/tx_pool.rs | 4 + 13 files changed, 108 insertions(+), 123 deletions(-) create mode 100644 tx-pool/src/pool_cell.rs diff --git a/rpc/src/error.rs b/rpc/src/error.rs index 1f86db9b72..b067fbf9d0 100644 --- a/rpc/src/error.rs +++ b/rpc/src/error.rs @@ -112,6 +112,8 @@ pub enum RPCError { TransactionExpired = -1109, /// (-1110): The transaction exceeded maximum size limit. PoolRejectedTransactionBySizeLimit = -1110, + /// (-1111): The transaction is rejected for RBF checking. + PoolRejctedRBF = -1111, /// (-1200): The indexer error. Indexer = -1200, } @@ -173,6 +175,7 @@ impl RPCError { Reject::DeclaredWrongCycles(..) => RPCError::PoolRejectedMalformedTransaction, Reject::Resolve(_) => RPCError::TransactionFailedToResolve, Reject::Verification(_) => RPCError::TransactionFailedToVerify, + Reject::RBFRejected(_) => RPCError::PoolRejctedRBF, Reject::ExceededTransactionSizeLimit(_, _) => { RPCError::PoolRejectedTransactionBySizeLimit } diff --git a/test/src/specs/tx_pool/different_txs_with_same_input.rs b/test/src/specs/tx_pool/different_txs_with_same_input.rs index 3eb1abfaea..c6b0594bac 100644 --- a/test/src/specs/tx_pool/different_txs_with_same_input.rs +++ b/test/src/specs/tx_pool/different_txs_with_same_input.rs @@ -39,7 +39,7 @@ impl Spec for DifferentTxsWithSameInput { let res = node0 .rpc_client() .send_transaction_result(tx2.data().into()); - //assert!(res.is_err(), "tx2 should be rejected"); + assert!(res.is_err(), "tx2 should be rejected"); node0.mine_with_blocking(|template| template.proposals.len() != 2); node0.mine_with_blocking(|template| template.number.value() != 14); diff --git a/tx-pool/src/component/edges.rs b/tx-pool/src/component/edges.rs index ec9c3e16a8..5ed01fbbe3 100644 --- a/tx-pool/src/component/edges.rs +++ b/tx-pool/src/component/edges.rs @@ -47,10 +47,6 @@ impl Edges { self.deps.remove(out_point) } - pub(crate) fn remove_deps(&mut self, out_point: &OutPoint) -> Option> { - self.deps.remove(out_point) - } - pub(crate) fn insert_deps(&mut self, out_point: OutPoint, txid: ProposalShortId) { self.deps.entry(out_point).or_default().insert(txid); } diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index 6eaf55b6f5..84f90cd2e1 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -11,15 +11,12 @@ use ckb_logger::{debug, trace}; use ckb_multi_index_map::MultiIndexMap; use ckb_types::core::error::OutPointError; use ckb_types::packed::OutPoint; +use ckb_types::prelude::*; use ckb_types::{ bytes::Bytes, - core::{cell::CellChecker, TransactionView}, + core::TransactionView, packed::{Byte32, CellOutput, ProposalShortId}, }; -use ckb_types::{ - core::cell::{CellMetaBuilder, CellProvider, CellStatus}, - prelude::*, -}; use std::collections::HashSet; use super::links::TxLinks; @@ -511,31 +508,3 @@ impl PoolMap { }); } } - -impl CellProvider for PoolMap { - fn cell(&self, out_point: &OutPoint, _eager_load: bool) -> CellStatus { - if self.edges.get_input_ref(out_point).is_some() { - return CellStatus::Dead; - } - if let Some((output, data)) = self.get_output_with_data(out_point) { - let cell_meta = CellMetaBuilder::from_cell_output(output, data) - .out_point(out_point.to_owned()) - .build(); - CellStatus::live_cell(cell_meta) - } else { - CellStatus::Unknown - } - } -} - -impl CellChecker for PoolMap { - fn is_live(&self, out_point: &OutPoint) -> Option { - if self.edges.get_input_ref(out_point).is_some() { - return Some(false); - } - if self.get_output_with_data(out_point).is_some() { - return Some(true); - } - None - } -} diff --git a/tx-pool/src/lib.rs b/tx-pool/src/lib.rs index 48a2157679..cb2ed5ae90 100644 --- a/tx-pool/src/lib.rs +++ b/tx-pool/src/lib.rs @@ -5,6 +5,7 @@ pub mod block_assembler; mod callback; mod chunk_process; mod component; +mod pool_cell; pub mod error; mod persisted; pub mod pool; diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 01a545db95..5c9ae8be7e 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -6,15 +6,12 @@ use crate::callback::Callbacks; use crate::component::pool_map::{PoolEntry, PoolMap, Status}; use crate::component::recent_reject::RecentReject; use crate::error::Reject; +use crate::pool_cell::PoolCell; use ckb_app_config::TxPoolConfig; +use ckb_jsonrpc_types::Capacity; use ckb_logger::{debug, error, warn}; use ckb_snapshot::Snapshot; use ckb_store::ChainStore; -use ckb_types::core::cell::CellChecker; -use ckb_types::core::cell::CellMetaBuilder; -use ckb_types::core::cell::{CellProvider, CellStatus}; -use ckb_types::packed::OutPoint; -use ckb_types::prelude::Unpack; use ckb_types::{ core::{ cell::{resolve_transaction, OverlayCellChecker, OverlayCellProvider, ResolvedTransaction}, @@ -292,7 +289,8 @@ impl TxPool { pub(crate) fn check_rtx_from_pool(&self, rtx: &ResolvedTransaction) -> Result<(), Reject> { let snapshot = self.snapshot(); - let checker = OverlayCellChecker::new(&self.pool_map, snapshot); + let pool_cell = PoolCell::new(&self.pool_map, false); + let checker = OverlayCellChecker::new(&pool_cell, snapshot); let mut seen_inputs = HashSet::new(); rtx.check(&mut seen_inputs, &checker, snapshot) .map_err(Reject::Resolve) @@ -301,21 +299,11 @@ impl TxPool { pub(crate) fn resolve_tx_from_pool( &self, tx: TransactionView, + rbf: bool, ) -> Result, Reject> { let snapshot = self.snapshot(); - let provider = OverlayCellProvider::new(&self.pool_map, snapshot); - let mut seen_inputs = HashSet::new(); - resolve_transaction(tx, &mut seen_inputs, &provider, snapshot) - .map(Arc::new) - .map_err(Reject::Resolve) - } - - pub(crate) fn resolve_tx_from_pool_rbf( - &self, - tx: TransactionView, - ) -> Result, Reject> { - let snapshot = self.snapshot(); - let provider = OverlayCellProvider::new(self, snapshot); + let pool_cell = PoolCell::new(&self.pool_map, rbf); + let provider = OverlayCellProvider::new(&pool_cell, snapshot); let mut seen_inputs = HashSet::new(); resolve_transaction(tx, &mut seen_inputs, &provider, snapshot) .map(Arc::new) @@ -482,6 +470,17 @@ impl TxPool { (entries, size, cycles) } + pub(crate) fn check_rbf(&self, tx: &ResolvedTransaction, conflicts: &HashSet, fee: Capacity) -> Result<(), Reject> { + if !self.config.enable_rbf { + return Err(Reject::RBFRejected("node disabled RBF".to_string())); + } + if conflicts.len() == 0 { + return Err(Reject::RBFRejected("can not find conflict txs to replace".to_string())); + } + + Ok(()) + } + fn build_recent_reject(config: &TxPoolConfig) -> Option { if !config.recent_reject.as_os_str().is_empty() { let recent_reject_ttl = @@ -507,46 +506,3 @@ impl TxPool { } } -impl CellProvider for TxPool { - fn cell(&self, out_point: &OutPoint, _eager_load: bool) -> CellStatus { - let tx_hash = out_point.tx_hash(); - match self - .pool_map - .get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) - { - Some(pool_entry) if pool_entry.status != Status::Proposed => { - match pool_entry - .inner - .transaction() - .output_with_data(out_point.index().unpack()) - { - Some((output, data)) => { - let cell_meta = CellMetaBuilder::from_cell_output(output, data) - .out_point(out_point.to_owned()) - .build(); - CellStatus::live_cell(cell_meta) - } - None => CellStatus::Unknown, - } - } - _ => CellStatus::Unknown, - } - } -} - -impl CellChecker for TxPool { - fn is_live(&self, out_point: &OutPoint) -> Option { - let tx_hash = out_point.tx_hash(); - match self - .pool_map - .get_by_id(&ProposalShortId::from_tx_hash(&tx_hash)) - { - Some(pool_entry) if pool_entry.status != Status::Proposed => pool_entry - .inner - .transaction() - .output_with_data(out_point.index().unpack()) - .map(|_| true), - _ => None, - } - } -} diff --git a/tx-pool/src/pool_cell.rs b/tx-pool/src/pool_cell.rs new file mode 100644 index 0000000000..4e70d44c82 --- /dev/null +++ b/tx-pool/src/pool_cell.rs @@ -0,0 +1,44 @@ +extern crate rustc_hash; +extern crate slab; +use crate::component::pool_map::PoolMap; +use ckb_types::core::cell::{CellChecker, CellMetaBuilder, CellProvider, CellStatus}; +use ckb_types::packed::OutPoint; + +pub(crate) struct PoolCell<'a> { + pub pool_map: &'a PoolMap, + pub rbf: bool, +} + +impl<'a> PoolCell<'a> { + pub fn new(pool_map: &'a PoolMap, rbf: bool) -> Self { + PoolCell { pool_map, rbf } + } +} + +impl<'a> CellProvider for PoolCell<'a> { + fn cell(&self, out_point: &OutPoint, _eager_load: bool) -> CellStatus { + if !self.rbf && self.pool_map.edges.get_input_ref(out_point).is_some() { + return CellStatus::Dead; + } + if let Some((output, data)) = self.pool_map.get_output_with_data(out_point) { + let cell_meta = CellMetaBuilder::from_cell_output(output, data) + .out_point(out_point.to_owned()) + .build(); + CellStatus::live_cell(cell_meta) + } else { + CellStatus::Unknown + } + } +} + +impl<'a> CellChecker for PoolCell<'a> { + fn is_live(&self, out_point: &OutPoint) -> Option { + if !self.rbf && self.pool_map.edges.get_input_ref(out_point).is_some() { + return Some(false); + } + if self.pool_map.get_output_with_data(out_point).is_some() { + return Some(true); + } + None + } +} diff --git a/tx-pool/src/process.rs b/tx-pool/src/process.rs index 962ebab964..29f73619e2 100644 --- a/tx-pool/src/process.rs +++ b/tx-pool/src/process.rs @@ -211,16 +211,20 @@ impl TxPoolService { // It's not possible for RBF, reject it directly check_txid_collision(tx_pool, tx)?; - // Try to find any conflicted tx in the pool - let conflicts = tx_pool.pool_map.find_conflict_tx(tx); - let rbf = !conflicts.is_empty(); - let res = resolve_tx(tx_pool, &snapshot, tx.clone(), rbf); - let (rtx, status) = res?; - let fee = check_tx_fee(tx_pool, &snapshot, &rtx, tx_size)?; - if rbf { - // check_rbf()? + // Try normal path first, if double-spending check success we don't need RBF check + // this make sure RBF won't introduce extra performance cost for hot path + let res = resolve_tx(tx_pool, &snapshot, tx.clone(), false); + if let Ok((rtx, status)) = res { + let fee = check_tx_fee(tx_pool, &snapshot, &rtx, tx_size)?; + return Ok((tip_hash, rtx, status, fee, tx_size, HashSet::new())); + } else { + // Try RBF check + let conflicts = tx_pool.pool_map.find_conflict_tx(tx); + let (rtx, status) = resolve_tx(tx_pool, &snapshot, tx.clone(), false)?; + let fee = check_tx_fee(tx_pool, &snapshot, &rtx, tx_size)?; + tx_pool.check_rbf(&rtx, &conflicts, fee.into())?; + return Ok((tip_hash, rtx, status, fee, tx_size, conflicts)); } - Ok((tip_hash, rtx, status, fee, tx_size, conflicts)) }) .await; @@ -251,11 +255,11 @@ impl TxPoolService { // non contextual verify first self.non_contextual_verify(&tx, None)?; - eprintln!( - "resumeble_process_tx: {:?} id: {:?}", - tx.hash(), - tx.proposal_short_id() - ); + // eprintln!( + // "resumeble_process_tx: {:?} id: {:?}", + // tx.hash(), + // tx.proposal_short_id() + // ); if self.chunk_contains(&tx).await || self.orphan_contains(&tx).await { return Err(Reject::Duplicated(tx.hash())); } @@ -1007,12 +1011,9 @@ fn resolve_tx( ) -> ResolveResult { let short_id = tx.proposal_short_id(); let tx_status = get_tx_status(snapshot, &short_id); - if !rbf { - tx_pool.resolve_tx_from_pool(tx) - } else { - tx_pool.resolve_tx_from_pool_rbf(tx) - } - .map(|rtx| (rtx, tx_status)) + tx_pool + .resolve_tx_from_pool(tx, rbf) + .map(|rtx| (rtx, tx_status)) } fn _submit_entry( @@ -1021,7 +1022,7 @@ fn _submit_entry( entry: TxEntry, callbacks: &Callbacks, ) -> Result<(), Reject> { - eprintln!("_submit_entry: {:?}", entry.proposal_short_id()); + //eprintln!("_submit_entry: {:?}", entry.proposal_short_id()); match status { TxStatus::Fresh => { if tx_pool.add_pending(entry.clone())? { @@ -1039,7 +1040,7 @@ fn _submit_entry( } } } - eprintln!("finished submit: {:?}", entry.proposal_short_id()); + //eprintln!("finished submit: {:?}", entry.proposal_short_id()); Ok(()) } diff --git a/util/app-config/src/configs/tx_pool.rs b/util/app-config/src/configs/tx_pool.rs index b71223ef7e..0b14987f94 100644 --- a/util/app-config/src/configs/tx_pool.rs +++ b/util/app-config/src/configs/tx_pool.rs @@ -34,6 +34,8 @@ pub struct TxPoolConfig { pub recent_reject: PathBuf, /// The expiration time for pool transactions in hours pub expiry_hours: u8, + /// Enable RBF + pub enable_rbf: bool, } /// Block assembler config options. diff --git a/util/app-config/src/legacy/tx_pool.rs b/util/app-config/src/legacy/tx_pool.rs index bf82ecc1af..ff9c64e437 100644 --- a/util/app-config/src/legacy/tx_pool.rs +++ b/util/app-config/src/legacy/tx_pool.rs @@ -41,6 +41,8 @@ pub(crate) struct TxPoolConfig { recent_reject: PathBuf, #[serde(default = "default_expiry_hours")] expiry_hours: u8, + #[serde(default)] + enable_rbf: bool, } fn default_keep_rejected_tx_hashes_days() -> u8 { @@ -82,6 +84,7 @@ impl Default for TxPoolConfig { persisted_data: Default::default(), recent_reject: Default::default(), expiry_hours: DEFAULT_EXPIRY_HOURS, + enable_rbf: false } } } @@ -103,6 +106,7 @@ impl From for crate::TxPoolConfig { persisted_data, recent_reject, expiry_hours, + enable_rbf, } = input; Self { @@ -115,6 +119,7 @@ impl From for crate::TxPoolConfig { persisted_data, recent_reject, expiry_hours, + enable_rbf, } } } diff --git a/util/jsonrpc-types/src/pool.rs b/util/jsonrpc-types/src/pool.rs index 4a18c73e7c..1aae977b28 100644 --- a/util/jsonrpc-types/src/pool.rs +++ b/util/jsonrpc-types/src/pool.rs @@ -241,6 +241,9 @@ pub enum PoolTransactionReject { /// Transaction expired Expiry(String), + + /// RBF rejected + RBFRejected(String), } impl From for PoolTransactionReject { @@ -260,6 +263,7 @@ impl From for PoolTransactionReject { Reject::Resolve(_) => Self::Resolve(format!("{reject}")), Reject::Verification(_) => Self::Verification(format!("{reject}")), Reject::Expiry(_) => Self::Expiry(format!("{reject}")), + Reject::RBFRejected(_) => Self::RBFRejected(format!("{reject}")), } } } diff --git a/util/types/src/core/cell.rs b/util/types/src/core/cell.rs index 534362c6e1..33a1e0a218 100644 --- a/util/types/src/core/cell.rs +++ b/util/types/src/core/cell.rs @@ -710,7 +710,7 @@ pub fn resolve_transaction( // skip resolve input of cellbase if !transaction.is_cellbase() { for out_point in transaction.input_pts_iter() { - eprintln!("resolve input: {:?}", out_point); + //eprintln!("resolve input: {:?}", out_point); if !current_inputs.insert(out_point.to_owned()) { return Err(OutPointError::Dead(out_point)); } diff --git a/util/types/src/core/tx_pool.rs b/util/types/src/core/tx_pool.rs index 43fcd7e547..2bed88ff7a 100644 --- a/util/types/src/core/tx_pool.rs +++ b/util/types/src/core/tx_pool.rs @@ -52,6 +52,10 @@ pub enum Reject { /// Expired #[error("Expiry transaction, timestamp {0}")] Expiry(u64), + + /// RBF rejected + #[error("RBF rejected: {0}")] + RBFRejected(String), } fn is_malformed_from_verification(error: &Error) -> bool { From 8bbac254282f6c1464b7024aa043f2fe38ecb213 Mon Sep 17 00:00:00 2001 From: yukang Date: Tue, 27 Jun 2023 19:08:09 +0800 Subject: [PATCH 28/58] more on RBF, make first RBF test pass --- test/src/main.rs | 4 +- .../tx_pool/different_txs_with_same_input.rs | 23 ++- test/src/specs/tx_pool/mod.rs | 2 + test/src/specs/tx_pool/replace.rs | 144 ++++++++++++++++++ tx-pool/src/chunk_process.rs | 3 - tx-pool/src/lib.rs | 2 +- tx-pool/src/pool.rs | 22 ++- tx-pool/src/process.rs | 71 ++++++--- tx-pool/src/service.rs | 13 +- util/app-config/src/legacy/tx_pool.rs | 2 +- util/jsonrpc-types/src/blockchain.rs | 18 ++- util/types/src/core/tx_pool.rs | 2 + 12 files changed, 261 insertions(+), 45 deletions(-) create mode 100644 test/src/specs/tx_pool/replace.rs diff --git a/test/src/main.rs b/test/src/main.rs index 02e09c809b..f16b212ae1 100644 --- a/test/src/main.rs +++ b/test/src/main.rs @@ -458,7 +458,9 @@ fn all_specs() -> Vec> { Box::new(RelayWithWrongTx::new()), Box::new(TxsRelayOrder), Box::new(SendTxChain), - Box::new(DifferentTxsWithSameInput), + Box::new(DifferentTxsWithSameInputWithOutRBF), + Box::new(RbfBasic), + Box::new(RbfSameInput), Box::new(CompactBlockEmpty), Box::new(CompactBlockEmptyParentUnknown), Box::new(CompactBlockPrefilled), diff --git a/test/src/specs/tx_pool/different_txs_with_same_input.rs b/test/src/specs/tx_pool/different_txs_with_same_input.rs index c6b0594bac..261ec9b2e8 100644 --- a/test/src/specs/tx_pool/different_txs_with_same_input.rs +++ b/test/src/specs/tx_pool/different_txs_with_same_input.rs @@ -7,9 +7,9 @@ use ckb_types::{ prelude::*, }; -pub struct DifferentTxsWithSameInput; +pub struct DifferentTxsWithSameInputWithOutRBF; -impl Spec for DifferentTxsWithSameInput { +impl Spec for DifferentTxsWithSameInputWithOutRBF { fn run(&self, nodes: &mut Vec) { let node0 = &nodes[0]; @@ -20,8 +20,16 @@ impl Spec for DifferentTxsWithSameInput { let tx1 = node0.new_transaction(tx_hash_0.clone()); let tx2_temp = node0.new_transaction(tx_hash_0); - eprintln!("tx1 hash: {:?} short_id: {:?}", tx1.hash(), tx1.proposal_short_id()); - eprintln!("tx2 hash: {:?} short_id: {:?}", tx2_temp.hash(), tx2_temp.proposal_short_id()); + eprintln!( + "tx1 hash: {:?} short_id: {:?}", + tx1.hash(), + tx1.proposal_short_id() + ); + eprintln!( + "tx2 hash: {:?} short_id: {:?}", + tx2_temp.hash(), + tx2_temp.proposal_short_id() + ); // Set tx2 fee to a higher value, tx1 capacity is 100, set tx2 capacity to 80 for +20 fee. let output = CellOutputBuilder::default() .capacity(capacity_bytes!(80).pack()) @@ -52,7 +60,7 @@ impl Spec for DifferentTxsWithSameInput { .map(TransactionView::hash) .collect(); - // RBF (Replace-By-Fees) is not implemented + // RBF (Replace-By-Fees) is not enabled assert!(commit_txs_hash.contains(&tx1.hash())); assert!(!commit_txs_hash.contains(&tx2.hash())); @@ -89,4 +97,9 @@ impl Spec for DifferentTxsWithSameInput { // assert!(ret.transaction.is_none()); // assert!(matches!(ret.tx_status.status, Status::Rejected)); } + + fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { + config.tx_pool.enable_rbf = false; + } } + diff --git a/test/src/specs/tx_pool/mod.rs b/test/src/specs/tx_pool/mod.rs index bd8ffd3a87..2b2d39230e 100644 --- a/test/src/specs/tx_pool/mod.rs +++ b/test/src/specs/tx_pool/mod.rs @@ -24,6 +24,7 @@ mod send_tx_chain; mod txs_relay_order; mod utils; mod valid_since; +mod replace; pub use cellbase_maturity::*; pub use collision::*; @@ -50,6 +51,7 @@ pub use send_secp_tx::*; pub use send_tx_chain::*; pub use txs_relay_order::*; pub use valid_since::*; +pub use replace::*; use ckb_app_config::BlockAssemblerConfig; use ckb_chain_spec::{build_genesis_type_id_script, OUTPUT_INDEX_SECP256K1_BLAKE160_SIGHASH_ALL}; diff --git a/test/src/specs/tx_pool/replace.rs b/test/src/specs/tx_pool/replace.rs new file mode 100644 index 0000000000..9b6bae9f81 --- /dev/null +++ b/test/src/specs/tx_pool/replace.rs @@ -0,0 +1,144 @@ +use crate::{Node, Spec}; +use ckb_jsonrpc_types::Status; +use ckb_logger::info; +use ckb_types::{ + core::{capacity_bytes, Capacity, TransactionView}, + packed::CellOutputBuilder, + prelude::*, +}; + +pub struct RbfBasic; +pub struct RbfSameInput; + +impl Spec for RbfBasic { + fn run(&self, nodes: &mut Vec) { + let node0 = &nodes[0]; + + node0.mine_until_out_bootstrap_period(); + node0.new_block_with_blocking(|template| template.number.value() != 13); + let tx_hash_0 = node0.generate_transaction(); + info!("Generate 2 txs with same input"); + let tx1 = node0.new_transaction(tx_hash_0.clone()); + let tx2_temp = node0.new_transaction(tx_hash_0); + + eprintln!( + "tx1 hash: {:?} short_id: {:?}", + tx1.hash(), + tx1.proposal_short_id() + ); + eprintln!( + "tx2 hash: {:?} short_id: {:?}", + tx2_temp.hash(), + tx2_temp.proposal_short_id() + ); + // Set tx2 fee to a higher value, tx1 capacity is 100, set tx2 capacity to 80 for +20 fee. + let output = CellOutputBuilder::default() + .capacity(capacity_bytes!(80).pack()) + .build(); + + let tx2 = tx2_temp + .as_advanced_builder() + .set_outputs(vec![output]) + .build(); + + eprintln!("tx1: {:?}", tx1); + eprintln!("tx2: {:?}", tx2); + + node0.rpc_client().send_transaction(tx1.data().into()); + let res = node0 + .rpc_client() + .send_transaction_result(tx2.data().into()); + assert!(res.is_ok(), "tx2 should replace old tx"); + + node0.mine_with_blocking(|template| template.proposals.len() != 2); + node0.mine_with_blocking(|template| template.number.value() != 14); + node0.mine_with_blocking(|template| template.transactions.len() != 2); + + let tip_block = node0.get_tip_block(); + let commit_txs_hash: Vec<_> = tip_block + .transactions() + .iter() + .map(TransactionView::hash) + .collect(); + + // RBF (Replace-By-Fees) is enabled + assert!(!commit_txs_hash.contains(&tx1.hash())); + assert!(commit_txs_hash.contains(&tx2.hash())); + + // when tx1 was confirmed, tx2 should be rejected + let ret = node0.rpc_client().get_transaction(tx2.hash()); + assert!( + matches!(ret.tx_status.status, Status::Committed), + "tx2 should be committed" + ); + + // verbosity = 1 + let ret = node0 + .rpc_client() + .get_transaction_with_verbosity(tx1.hash(), 1); + eprintln!("ret: {:?}", ret); + assert!(ret.transaction.is_none()); + assert!(matches!(ret.tx_status.status, Status::Rejected)); + assert!(ret.tx_status.reason.unwrap().contains("RBFRejected")); + + // verbosity = 2 + let ret = node0 + .rpc_client() + .get_transaction_with_verbosity(tx2.hash(), 2); + assert!(ret.transaction.is_some()); + assert!(matches!(ret.tx_status.status, Status::Committed)); + + let ret = node0 + .rpc_client() + .get_transaction_with_verbosity(tx1.hash(), 2); + assert!(ret.transaction.is_none()); + assert!(matches!(ret.tx_status.status, Status::Rejected)); + assert!(ret.tx_status.reason.unwrap().contains("RBFRejected")); + } + + fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { + config.tx_pool.enable_rbf = true; + } +} + + +impl Spec for RbfSameInput { + fn run(&self, nodes: &mut Vec) { + let node0 = &nodes[0]; + + node0.mine_until_out_bootstrap_period(); + node0.new_block_with_blocking(|template| template.number.value() != 13); + let tx_hash_0 = node0.generate_transaction(); + info!("Generate 2 txs with same input"); + let tx1 = node0.new_transaction(tx_hash_0.clone()); + let tx2_temp = node0.new_transaction(tx_hash_0); + + eprintln!( + "tx1 hash: {:?} short_id: {:?}", + tx1.hash(), + tx1.proposal_short_id() + ); + eprintln!( + "tx2 hash: {:?} short_id: {:?}", + tx2_temp.hash(), + tx2_temp.proposal_short_id() + ); + let tx2 = tx2_temp + .as_advanced_builder() + .build(); + + eprintln!("tx1: {:?}", tx1); + eprintln!("tx2: {:?}", tx2); + + node0.rpc_client().send_transaction(tx1.data().into()); + let res = node0 + .rpc_client() + .send_transaction_result(tx2.data().into()); + assert!(res.is_err(), "tx2 should be rejected"); + } + + + fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { + config.tx_pool.enable_rbf = true; + } +} \ No newline at end of file diff --git a/tx-pool/src/chunk_process.rs b/tx-pool/src/chunk_process.rs index bee8a06846..51ebdec07d 100644 --- a/tx-pool/src/chunk_process.rs +++ b/tx-pool/src/chunk_process.rs @@ -252,9 +252,6 @@ impl ChunkProcess { let completed = try_or_return_with_snapshot!(ret, snapshot); let entry = TxEntry::new(rtx, completed.cycles, fee, tx_size); - if !conflicts.is_empty() { - // remove conflict tx - } let (ret, submit_snapshot) = self .service .submit_entry(tip_hash, entry, status, conflicts) diff --git a/tx-pool/src/lib.rs b/tx-pool/src/lib.rs index cb2ed5ae90..d122177c01 100644 --- a/tx-pool/src/lib.rs +++ b/tx-pool/src/lib.rs @@ -5,10 +5,10 @@ pub mod block_assembler; mod callback; mod chunk_process; mod component; -mod pool_cell; pub mod error; mod persisted; pub mod pool; +mod pool_cell; mod process; pub mod service; mod util; diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 5c9ae8be7e..4360ad12a4 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -156,6 +156,14 @@ impl TxPool { .map(|entry| entry.inner.transaction()) } + pub(crate) fn put_recent_reject(&mut self, tx_hash: &Byte32, reject: &Reject) { + if let Some(ref mut recent_reject) = self.recent_reject { + if let Err(e) = recent_reject.put(tx_hash, reject.clone()) { + error!("record recent_reject failed {} {} {}", tx_hash, reject, e); + } + } + } + pub(crate) fn remove_committed_txs<'a>( &mut self, txs: impl Iterator, @@ -470,12 +478,19 @@ impl TxPool { (entries, size, cycles) } - pub(crate) fn check_rbf(&self, tx: &ResolvedTransaction, conflicts: &HashSet, fee: Capacity) -> Result<(), Reject> { + pub(crate) fn check_rbf( + &self, + _tx: &ResolvedTransaction, + conflicts: &HashSet, + _fee: Capacity, + ) -> Result<(), Reject> { if !self.config.enable_rbf { return Err(Reject::RBFRejected("node disabled RBF".to_string())); } - if conflicts.len() == 0 { - return Err(Reject::RBFRejected("can not find conflict txs to replace".to_string())); + if conflicts.is_empty() { + return Err(Reject::RBFRejected( + "can not find conflict txs to replace".to_string(), + )); } Ok(()) @@ -505,4 +520,3 @@ impl TxPool { } } } - diff --git a/tx-pool/src/process.rs b/tx-pool/src/process.rs index 29f73619e2..a7f3c17311 100644 --- a/tx-pool/src/process.rs +++ b/tx-pool/src/process.rs @@ -125,8 +125,31 @@ impl TxPoolService { } // try to remove conflicted tx here - for r in conflicts.iter() { - eprintln!("removeing : {:?}", r); + for id in conflicts.iter() { + let removed = tx_pool.pool_map.remove_entry_and_descendants(id); + if removed.is_empty() { + return Err(Reject::RBFRejected( + "RBF remove old entries error".to_string(), + )); + } + eprintln!("removed: {:?}", id); + for old in removed { + let reject = Reject::RBFRejected(format!( + "replaced by {}", + entry.proposal_short_id() + )); + eprintln!( + "add recent_reject: id: {:?} reject: {:?}", + &old.proposal_short_id(), + reject + ); + // remove old tx from tx_pool, not happened in service so we didn't call reject callbacks + // here we call them manually + // TODO: how to call reject notify like service? + tx_pool.put_recent_reject(&old.transaction().hash(), &reject); + tx_pool.update_statics_for_remove_tx(old.size, old.cycles); + self.callbacks.call_reject(tx_pool, &old, reject) + } } _submit_entry(tx_pool, status, entry.clone(), &self.callbacks)?; Ok(()) @@ -205,7 +228,7 @@ impl TxPoolService { let (ret, snapshot) = self .with_tx_pool_read_lock(|tx_pool, snapshot| { - let tip_hash = snapshot.tip_hash(); + let tip_hash: Byte32 = snapshot.tip_hash(); // Same txid means exactly the same transaction, including inputs, outputs, witnesses, etc. // It's not possible for RBF, reject it directly @@ -214,16 +237,28 @@ impl TxPoolService { // Try normal path first, if double-spending check success we don't need RBF check // this make sure RBF won't introduce extra performance cost for hot path let res = resolve_tx(tx_pool, &snapshot, tx.clone(), false); - if let Ok((rtx, status)) = res { - let fee = check_tx_fee(tx_pool, &snapshot, &rtx, tx_size)?; - return Ok((tip_hash, rtx, status, fee, tx_size, HashSet::new())); - } else { - // Try RBF check - let conflicts = tx_pool.pool_map.find_conflict_tx(tx); - let (rtx, status) = resolve_tx(tx_pool, &snapshot, tx.clone(), false)?; - let fee = check_tx_fee(tx_pool, &snapshot, &rtx, tx_size)?; - tx_pool.check_rbf(&rtx, &conflicts, fee.into())?; - return Ok((tip_hash, rtx, status, fee, tx_size, conflicts)); + match res { + Ok((rtx, status)) => { + let fee = check_tx_fee(tx_pool, &snapshot, &rtx, tx_size)?; + Ok((tip_hash, rtx, status, fee, tx_size, HashSet::new())) + } + Err(err) => { + eprintln!( + "resolve_tx error: {:?}, try RBF check", + tx_pool.config.enable_rbf + ); + if tx_pool.config.enable_rbf { + // Try RBF check + eprintln!("begin RBF check ...."); + let conflicts = tx_pool.pool_map.find_conflict_tx(tx); + let (rtx, status) = resolve_tx(tx_pool, &snapshot, tx.clone(), true)?; + let fee = check_tx_fee(tx_pool, &snapshot, &rtx, tx_size)?; + tx_pool.check_rbf(&rtx, &conflicts, fee.into())?; + Ok((tip_hash, rtx, status, fee, tx_size, conflicts)) + } else { + Err(err) + } + } } }) .await; @@ -310,11 +345,7 @@ impl TxPoolService { pub(crate) async fn put_recent_reject(&self, tx_hash: &Byte32, reject: &Reject) { let mut tx_pool = self.tx_pool.write().await; - if let Some(ref mut recent_reject) = tx_pool.recent_reject { - if let Err(e) = recent_reject.put(tx_hash, reject.clone()) { - error!("record recent_reject failed {} {} {}", tx_hash, reject, e); - } - } + tx_pool.put_recent_reject(tx_hash, reject); } pub(crate) async fn remove_tx(&self, tx_hash: Byte32) -> bool { @@ -984,9 +1015,9 @@ type PreCheckedTx = ( type ResolveResult = Result<(Arc, TxStatus), Reject>; fn get_tx_status(snapshot: &Snapshot, short_id: &ProposalShortId) -> TxStatus { - if snapshot.proposals().contains_proposed(&short_id) { + if snapshot.proposals().contains_proposed(short_id) { TxStatus::Proposed - } else if snapshot.proposals().contains_gap(&short_id) { + } else if snapshot.proposals().contains_gap(short_id) { TxStatus::Gap } else { TxStatus::Fresh diff --git a/tx-pool/src/service.rs b/tx-pool/src/service.rs index 1a187615c8..abd945de75 100644 --- a/tx-pool/src/service.rs +++ b/tx-pool/src/service.rs @@ -790,15 +790,10 @@ async fn process(mut service: TxPoolService, message: Message) { entry.timestamp, )) } else if let Some(ref recent_reject_db) = tx_pool.recent_reject { - let recent_reject_result = recent_reject_db.get(&hash); - if let Ok(recent_reject) = recent_reject_result { - if let Some(record) = recent_reject { - Ok(TransactionWithStatus::with_rejected(record)) - } else { - Ok(TransactionWithStatus::with_unknown()) - } - } else { - Err(recent_reject_result.unwrap_err()) + match recent_reject_db.get(&hash) { + Ok(Some(record)) => Ok(TransactionWithStatus::with_rejected(record)), + Ok(_) => Ok(TransactionWithStatus::with_unknown()), + Err(err) => Err(err), } } else { Ok(TransactionWithStatus::with_unknown()) diff --git a/util/app-config/src/legacy/tx_pool.rs b/util/app-config/src/legacy/tx_pool.rs index ff9c64e437..76b41d0c47 100644 --- a/util/app-config/src/legacy/tx_pool.rs +++ b/util/app-config/src/legacy/tx_pool.rs @@ -84,7 +84,7 @@ impl Default for TxPoolConfig { persisted_data: Default::default(), recent_reject: Default::default(), expiry_hours: DEFAULT_EXPIRY_HOURS, - enable_rbf: false + enable_rbf: false, } } } diff --git a/util/jsonrpc-types/src/blockchain.rs b/util/jsonrpc-types/src/blockchain.rs index 8acaf95ce7..d785ead6c3 100644 --- a/util/jsonrpc-types/src/blockchain.rs +++ b/util/jsonrpc-types/src/blockchain.rs @@ -577,6 +577,8 @@ pub enum Status { /// Status "rejected". The transaction has been recently removed from the pool. /// Due to storage limitations, the node can only hold the most recently removed transactions. Rejected, + /// Status "replaced". The transaction has been recently replace from the pool. + Replaced, } /// Transaction status and the block hash if it is committed. @@ -596,8 +598,9 @@ impl From for TxStatus { tx_pool::TxStatus::Pending => TxStatus::pending(), tx_pool::TxStatus::Proposed => TxStatus::proposed(), tx_pool::TxStatus::Committed(hash) => TxStatus::committed(hash), - tx_pool::TxStatus::Unknown => TxStatus::unknown(), tx_pool::TxStatus::Rejected(reason) => TxStatus::rejected(reason), + tx_pool::TxStatus::Replaced(reason) => TxStatus::replaced(reason), + tx_pool::TxStatus::Unknown => TxStatus::unknown(), } } } @@ -647,6 +650,19 @@ impl TxStatus { } } + /// Transaction which has already been replaced recently. + /// + /// ## Params + /// + /// * `reason` - the reason why the transaction is replaced. + pub fn replaced(reason: String) -> Self { + Self { + status: Status::Replaced, + block_hash: None, + reason: Some(reason), + } + } + /// The node has not seen the transaction, pub fn unknown() -> Self { Self { diff --git a/util/types/src/core/tx_pool.rs b/util/types/src/core/tx_pool.rs index 2bed88ff7a..a061aca8db 100644 --- a/util/types/src/core/tx_pool.rs +++ b/util/types/src/core/tx_pool.rs @@ -117,6 +117,8 @@ pub enum TxStatus { /// Status "rejected". The transaction has been recently removed from the pool. /// Due to storage limitations, the node can only hold the most recently removed transactions. Rejected(String), + /// Status "replaced", The transaction has been recently replaced for RBF. + Replaced(String), } /// Tx-pool entry info From bcb79a6331da51cfc48a7607464674ca6daa7f9a Mon Sep 17 00:00:00 2001 From: yukang Date: Wed, 28 Jun 2023 17:05:28 +0800 Subject: [PATCH 29/58] add checking rules for RBF --- resource/ckb.toml | 1 + rpc/src/module/pool.rs | 1 + .../tx_pool/different_txs_with_same_input.rs | 1 - test/src/specs/tx_pool/mod.rs | 4 +- test/src/specs/tx_pool/replace.rs | 8 +- test/template/ckb.toml | 1 + tx-pool/src/pool.rs | 80 +++++++++++++++++-- tx-pool/src/process.rs | 18 +---- tx-pool/src/service.rs | 1 + util/app-config/src/configs/tx_pool.rs | 3 + util/app-config/src/legacy/tx_pool.rs | 7 ++ util/jsonrpc-types/src/pool.rs | 5 ++ util/types/src/core/tx_pool.rs | 6 ++ 13 files changed, 102 insertions(+), 34 deletions(-) diff --git a/resource/ckb.toml b/resource/ckb.toml index 89ec89f6fb..d59d5d9edd 100644 --- a/resource/ckb.toml +++ b/resource/ckb.toml @@ -134,6 +134,7 @@ enable_deprecated_rpc = false # {{ [tx_pool] max_tx_pool_size = 180_000_000 # 180mb min_fee_rate = 1_000 # Here fee_rate are calculated directly using size in units of shannons/KB +min_rbf_rate = 1_000 # Here fee_rate are calculated directly using size in units of shannons/KB max_tx_verify_cycles = 70_000_000 max_ancestors_count = 25 diff --git a/rpc/src/module/pool.rs b/rpc/src/module/pool.rs index dfd4d9531d..1fe3e45de6 100644 --- a/rpc/src/module/pool.rs +++ b/rpc/src/module/pool.rs @@ -166,6 +166,7 @@ pub trait PoolRpc { /// "result": { /// "last_txs_updated_at": "0x0", /// "min_fee_rate": "0x3e8", + /// "min_rbf_rate": "0x5dc", /// "max_tx_pool_size": "0xaba9500", /// "orphan": "0x0", /// "pending": "0x1", diff --git a/test/src/specs/tx_pool/different_txs_with_same_input.rs b/test/src/specs/tx_pool/different_txs_with_same_input.rs index 261ec9b2e8..87fd7d3c56 100644 --- a/test/src/specs/tx_pool/different_txs_with_same_input.rs +++ b/test/src/specs/tx_pool/different_txs_with_same_input.rs @@ -102,4 +102,3 @@ impl Spec for DifferentTxsWithSameInputWithOutRBF { config.tx_pool.enable_rbf = false; } } - diff --git a/test/src/specs/tx_pool/mod.rs b/test/src/specs/tx_pool/mod.rs index 2b2d39230e..925b5618fc 100644 --- a/test/src/specs/tx_pool/mod.rs +++ b/test/src/specs/tx_pool/mod.rs @@ -15,6 +15,7 @@ mod pool_resurrect; mod proposal_expire_rule; mod remove_tx; mod reorg_proposals; +mod replace; mod send_defected_binary; mod send_large_cycles_tx; mod send_low_fee_rate_tx; @@ -24,7 +25,6 @@ mod send_tx_chain; mod txs_relay_order; mod utils; mod valid_since; -mod replace; pub use cellbase_maturity::*; pub use collision::*; @@ -43,6 +43,7 @@ pub use pool_resurrect::*; pub use proposal_expire_rule::*; pub use remove_tx::*; pub use reorg_proposals::*; +pub use replace::*; pub use send_defected_binary::*; pub use send_large_cycles_tx::*; pub use send_low_fee_rate_tx::*; @@ -51,7 +52,6 @@ pub use send_secp_tx::*; pub use send_tx_chain::*; pub use txs_relay_order::*; pub use valid_since::*; -pub use replace::*; use ckb_app_config::BlockAssemblerConfig; use ckb_chain_spec::{build_genesis_type_id_script, OUTPUT_INDEX_SECP256K1_BLAKE160_SIGHASH_ALL}; diff --git a/test/src/specs/tx_pool/replace.rs b/test/src/specs/tx_pool/replace.rs index 9b6bae9f81..a5e742fbbc 100644 --- a/test/src/specs/tx_pool/replace.rs +++ b/test/src/specs/tx_pool/replace.rs @@ -101,7 +101,6 @@ impl Spec for RbfBasic { } } - impl Spec for RbfSameInput { fn run(&self, nodes: &mut Vec) { let node0 = &nodes[0]; @@ -123,9 +122,7 @@ impl Spec for RbfSameInput { tx2_temp.hash(), tx2_temp.proposal_short_id() ); - let tx2 = tx2_temp - .as_advanced_builder() - .build(); + let tx2 = tx2_temp.as_advanced_builder().build(); eprintln!("tx1: {:?}", tx1); eprintln!("tx2: {:?}", tx2); @@ -137,8 +134,7 @@ impl Spec for RbfSameInput { assert!(res.is_err(), "tx2 should be rejected"); } - fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { config.tx_pool.enable_rbf = true; } -} \ No newline at end of file +} diff --git a/test/template/ckb.toml b/test/template/ckb.toml index c722fa1e55..0eea3eb7b1 100644 --- a/test/template/ckb.toml +++ b/test/template/ckb.toml @@ -79,6 +79,7 @@ enable_deprecated_rpc = true [tx_pool] max_tx_pool_size = 180_000_000 # 180mb min_fee_rate = 0 # Here fee_rate are calculated directly using size in units of shannons/KB +min_rbf_rate = 0 # Here rbf_rate are calculated directly using size in units of shannons/KB max_tx_verify_cycles = 70_000_000 max_ancestors_count = 25 diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 4360ad12a4..c62c175141 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -8,7 +8,6 @@ use crate::component::recent_reject::RecentReject; use crate::error::Reject; use crate::pool_cell::PoolCell; use ckb_app_config::TxPoolConfig; -use ckb_jsonrpc_types::Capacity; use ckb_logger::{debug, error, warn}; use ckb_snapshot::Snapshot; use ckb_store::ChainStore; @@ -16,7 +15,7 @@ use ckb_types::{ core::{ cell::{resolve_transaction, OverlayCellChecker, OverlayCellProvider, ResolvedTransaction}, tx_pool::{TxPoolEntryInfo, TxPoolIds}, - Cycle, TransactionView, UncleBlockView, + Capacity, Cycle, TransactionView, UncleBlockView, }, packed::{Byte32, ProposalShortId}, }; @@ -25,7 +24,7 @@ use std::collections::HashSet; use std::sync::Arc; const COMMITTED_HASH_CACHE_SIZE: usize = 100_000; - +const MAX_REPLACEMENT_CANDIDATES: usize = 100; /// Tx-pool implementation pub struct TxPool { pub(crate) config: TxPoolConfig, @@ -480,19 +479,84 @@ impl TxPool { pub(crate) fn check_rbf( &self, - _tx: &ResolvedTransaction, + snapshot: &Snapshot, + rtx: &ResolvedTransaction, conflicts: &HashSet, - _fee: Capacity, + fee: Capacity, + tx_size: usize, ) -> Result<(), Reject> { - if !self.config.enable_rbf { - return Err(Reject::RBFRejected("node disabled RBF".to_string())); - } + assert!(self.config.enable_rbf); if conflicts.is_empty() { return Err(Reject::RBFRejected( "can not find conflict txs to replace".to_string(), )); } + let conflicts = conflicts + .iter() + .map(|id| { + &self + .get_pool_entry(id) + .expect("conflict tx should be in pool or store") + .inner + }) + .collect::>(); + + // TODO: Rule #1, the conflicted tx need to confirmed as `can_be_replaced` + + // Rule #2, new tx don't contain any new unconfirmed inputs + // TODO: confirm whether this could be used in ckb + // https://github.com/bitcoin/bitcoin/blob/d9c7c2fd3ec7b0fcae7e0c9423bff6c6799dd67c/src/policy/rbf.cpp#L107 + let mut inputs = HashSet::new(); + for c in conflicts.iter() { + inputs.extend(c.transaction().input_pts_iter()); + } + if rtx + .transaction + .input_pts_iter() + .any(|pt| !inputs.contains(&pt) && !snapshot.transaction_exists(&pt.tx_hash())) + { + return Err(Reject::RBFRejected( + "new tx contains unconfirmed inputs".to_string(), + )); + } + + // Rule #4, new tx' fee need to higher than min_rbf_fee computed from the tx_pool configuration + let min_rbf_fee = self.config.min_rbf_rate.fee(tx_size as u64); + if fee <= min_rbf_fee { + return Err(Reject::RBFRejected(format!( + "tx fee lower than min_rbf_fee, min_rbf_fee: {}, tx fee: {}", + min_rbf_fee, fee, + ))); + } + + // Rule #3, new tx's fee need to higher than conflicts + for conflict in conflicts.iter() { + eprintln!("old fee: {:?} new_fee: {:?}", conflict.fee, fee); + if conflict.fee >= fee { + return Err(Reject::RBFRejected(format!( + "tx fee lower than conflict tx fee, conflict id: {}, conflict fee: {}, tx fee: {}", + conflict.proposal_short_id(), + conflict.fee, + fee, + ))); + } + } + + // Rule #5, new replaced tx's descendants can not more than 100 + let mut replace_count: usize = 0; + for conflict in conflicts.iter() { + let id = conflict.proposal_short_id(); + let descendants = self.pool_map.calc_descendants(&id); + replace_count += descendants.len() + 1; + if replace_count > MAX_REPLACEMENT_CANDIDATES { + return Err(Reject::RBFRejected(format!( + "tx conflict too many txs, conflict txs count: {}", + replace_count, + ))); + } + } + Ok(()) } diff --git a/tx-pool/src/process.rs b/tx-pool/src/process.rs index a7f3c17311..86e25e5a21 100644 --- a/tx-pool/src/process.rs +++ b/tx-pool/src/process.rs @@ -132,17 +132,11 @@ impl TxPoolService { "RBF remove old entries error".to_string(), )); } - eprintln!("removed: {:?}", id); for old in removed { let reject = Reject::RBFRejected(format!( "replaced by {}", entry.proposal_short_id() )); - eprintln!( - "add recent_reject: id: {:?} reject: {:?}", - &old.proposal_short_id(), - reject - ); // remove old tx from tx_pool, not happened in service so we didn't call reject callbacks // here we call them manually // TODO: how to call reject notify like service? @@ -243,17 +237,12 @@ impl TxPoolService { Ok((tip_hash, rtx, status, fee, tx_size, HashSet::new())) } Err(err) => { - eprintln!( - "resolve_tx error: {:?}, try RBF check", - tx_pool.config.enable_rbf - ); if tx_pool.config.enable_rbf { // Try RBF check - eprintln!("begin RBF check ...."); let conflicts = tx_pool.pool_map.find_conflict_tx(tx); let (rtx, status) = resolve_tx(tx_pool, &snapshot, tx.clone(), true)?; let fee = check_tx_fee(tx_pool, &snapshot, &rtx, tx_size)?; - tx_pool.check_rbf(&rtx, &conflicts, fee.into())?; + tx_pool.check_rbf(&snapshot, &rtx, &conflicts, fee, tx_size)?; Ok((tip_hash, rtx, status, fee, tx_size, conflicts)) } else { Err(err) @@ -290,11 +279,6 @@ impl TxPoolService { // non contextual verify first self.non_contextual_verify(&tx, None)?; - // eprintln!( - // "resumeble_process_tx: {:?} id: {:?}", - // tx.hash(), - // tx.proposal_short_id() - // ); if self.chunk_contains(&tx).await || self.orphan_contains(&tx).await { return Err(Reject::Duplicated(tx.hash())); } diff --git a/tx-pool/src/service.rs b/tx-pool/src/service.rs index abd945de75..6414f3161f 100644 --- a/tx-pool/src/service.rs +++ b/tx-pool/src/service.rs @@ -914,6 +914,7 @@ impl TxPoolService { total_tx_size: tx_pool.total_tx_size, total_tx_cycles: tx_pool.total_tx_cycles, min_fee_rate: self.tx_pool_config.min_fee_rate, + min_rbf_rate: self.tx_pool_config.min_rbf_rate, last_txs_updated_at: 0, tx_size_limit: TRANSACTION_SIZE_LIMIT, max_tx_pool_size: self.tx_pool_config.max_tx_pool_size as u64, diff --git a/util/app-config/src/configs/tx_pool.rs b/util/app-config/src/configs/tx_pool.rs index 0b14987f94..eb514c3d8a 100644 --- a/util/app-config/src/configs/tx_pool.rs +++ b/util/app-config/src/configs/tx_pool.rs @@ -14,6 +14,9 @@ pub struct TxPoolConfig { /// txs with lower fee rate than this will not be relayed or be mined #[serde(with = "FeeRateDef")] pub min_fee_rate: FeeRate, + /// txs need to pay more than this for RBF + #[serde(with = "FeeRateDef")] + pub min_rbf_rate: FeeRate, /// tx pool rejects txs that cycles greater than max_tx_verify_cycles pub max_tx_verify_cycles: Cycle, /// max ancestors size limit for a single tx diff --git a/util/app-config/src/legacy/tx_pool.rs b/util/app-config/src/legacy/tx_pool.rs index 76b41d0c47..1927bbda7e 100644 --- a/util/app-config/src/legacy/tx_pool.rs +++ b/util/app-config/src/legacy/tx_pool.rs @@ -7,6 +7,8 @@ use std::path::PathBuf; // default min fee rate, 1000 shannons per kilobyte const DEFAULT_MIN_FEE_RATE: FeeRate = FeeRate::from_u64(1000); +// default min rbf rate, 1500 shannons per kilobyte +const DEFAULT_MIN_RBF_RATE: FeeRate = FeeRate::from_u64(1500); // default max tx verify cycles const DEFAULT_MAX_TX_VERIFY_CYCLES: Cycle = TWO_IN_TWO_OUT_CYCLES * 20; // default max ancestors count @@ -33,6 +35,8 @@ pub(crate) struct TxPoolConfig { keep_rejected_tx_hashes_count: u64, #[serde(with = "FeeRateDef")] min_fee_rate: FeeRate, + #[serde(with = "FeeRateDef")] + min_rbf_rate: FeeRate, max_tx_verify_cycles: Cycle, max_ancestors_count: usize, #[serde(default)] @@ -79,6 +83,7 @@ impl Default for TxPoolConfig { keep_rejected_tx_hashes_days: default_keep_rejected_tx_hashes_days(), keep_rejected_tx_hashes_count: default_keep_rejected_tx_hashes_count(), min_fee_rate: DEFAULT_MIN_FEE_RATE, + min_rbf_rate: DEFAULT_MIN_RBF_RATE, max_tx_verify_cycles: DEFAULT_MAX_TX_VERIFY_CYCLES, max_ancestors_count: DEFAULT_MAX_ANCESTORS_COUNT, persisted_data: Default::default(), @@ -101,6 +106,7 @@ impl From for crate::TxPoolConfig { keep_rejected_tx_hashes_days, keep_rejected_tx_hashes_count, min_fee_rate, + min_rbf_rate, max_tx_verify_cycles, max_ancestors_count, persisted_data, @@ -112,6 +118,7 @@ impl From for crate::TxPoolConfig { Self { max_tx_pool_size, min_fee_rate, + min_rbf_rate, max_tx_verify_cycles, max_ancestors_count: cmp::max(DEFAULT_MAX_ANCESTORS_COUNT, max_ancestors_count), keep_rejected_tx_hashes_days, diff --git a/util/jsonrpc-types/src/pool.rs b/util/jsonrpc-types/src/pool.rs index 1aae977b28..3e773c1ca2 100644 --- a/util/jsonrpc-types/src/pool.rs +++ b/util/jsonrpc-types/src/pool.rs @@ -40,6 +40,10 @@ pub struct TxPoolInfo { /// /// The unit is Shannons per 1000 bytes transaction serialization size in the block. pub min_fee_rate: Uint64, + /// RBF rate threshold. The pool reject to resort for transactions which fee rate is below this threshold. + /// + /// The unit is Shannons per 1000 bytes transaction serialization size in the block. + pub min_rbf_rate: Uint64, /// Last updated time. This is the Unix timestamp in milliseconds. pub last_txs_updated_at: Timestamp, /// Limiting transactions to tx_size_limit @@ -63,6 +67,7 @@ impl From for TxPoolInfo { total_tx_size: (tx_pool_info.total_tx_size as u64).into(), total_tx_cycles: tx_pool_info.total_tx_cycles.into(), min_fee_rate: tx_pool_info.min_fee_rate.as_u64().into(), + min_rbf_rate: tx_pool_info.min_rbf_rate.as_u64().into(), last_txs_updated_at: tx_pool_info.last_txs_updated_at.into(), tx_size_limit: tx_pool_info.tx_size_limit.into(), max_tx_pool_size: tx_pool_info.max_tx_pool_size.into(), diff --git a/util/types/src/core/tx_pool.rs b/util/types/src/core/tx_pool.rs index a061aca8db..6c19213502 100644 --- a/util/types/src/core/tx_pool.rs +++ b/util/types/src/core/tx_pool.rs @@ -321,6 +321,12 @@ pub struct TxPoolInfo { /// /// The unit is Shannons per 1000 bytes transaction serialization size in the block. pub min_fee_rate: FeeRate, + + /// Min RBF rate threshold. The pool reject RBF transactions which fee rate is below this threshold. + /// + /// The unit is Shannons per 1000 bytes transaction serialization size in the block. + pub min_rbf_rate: FeeRate, + /// Last updated time. This is the Unix timestamp in milliseconds. pub last_txs_updated_at: u64, /// Limiting transactions to tx_size_limit From 2f6b13b57813626e917029d92db4513a5c59f890 Mon Sep 17 00:00:00 2001 From: yukang Date: Tue, 4 Jul 2023 14:33:36 +0800 Subject: [PATCH 30/58] fix RBF callbacks issues --- tx-pool/src/process.rs | 26 +++++++++++++++++++------- util/launcher/src/shared_builder.rs | 2 +- 2 files changed, 20 insertions(+), 8 deletions(-) diff --git a/tx-pool/src/process.rs b/tx-pool/src/process.rs index 86e25e5a21..144321b4b6 100644 --- a/tx-pool/src/process.rs +++ b/tx-pool/src/process.rs @@ -140,8 +140,6 @@ impl TxPoolService { // remove old tx from tx_pool, not happened in service so we didn't call reject callbacks // here we call them manually // TODO: how to call reject notify like service? - tx_pool.put_recent_reject(&old.transaction().hash(), &reject); - tx_pool.update_statics_for_remove_tx(old.size, old.cycles); self.callbacks.call_reject(tx_pool, &old, reject) } } @@ -225,7 +223,7 @@ impl TxPoolService { let tip_hash: Byte32 = snapshot.tip_hash(); // Same txid means exactly the same transaction, including inputs, outputs, witnesses, etc. - // It's not possible for RBF, reject it directly + // It's also not possible for RBF, reject it directly check_txid_collision(tx_pool, tx)?; // Try normal path first, if double-spending check success we don't need RBF check @@ -408,7 +406,12 @@ impl TxPoolService { }); } - if matches!(reject, Reject::Resolve(..) | Reject::Verification(..)) { + if matches!( + reject, + Reject::Resolve(..) + | Reject::Verification(..) + | Reject::RBFRejected(..) + ) { self.put_recent_reject(&tx_hash, reject).await; } } @@ -433,7 +436,12 @@ impl TxPoolService { }); } Err(reject) => { - if matches!(reject, Reject::Resolve(..) | Reject::Verification(..)) { + if matches!( + reject, + Reject::Resolve(..) + | Reject::Verification(..) + | Reject::RBFRejected(..) + ) { self.put_recent_reject(&tx_hash, reject).await; } } @@ -533,8 +541,12 @@ impl TxPoolService { tx_hash: orphan.tx.hash(), }); } - if matches!(reject, Reject::Resolve(..) | Reject::Verification(..)) - { + if matches!( + reject, + Reject::Resolve(..) + | Reject::Verification(..) + | Reject::RBFRejected(..) + ) { self.put_recent_reject(&orphan.tx.hash(), &reject).await; } } diff --git a/util/launcher/src/shared_builder.rs b/util/launcher/src/shared_builder.rs index 96d750a198..f019b377ad 100644 --- a/util/launcher/src/shared_builder.rs +++ b/util/launcher/src/shared_builder.rs @@ -463,7 +463,7 @@ fn register_tx_pool_callback(tx_pool_builder: &mut TxPoolServiceBuilder, notify: let tx_hash = entry.transaction().hash(); // record recent reject - if matches!(reject, Reject::Resolve(..)) { + if matches!(reject, Reject::Resolve(..) | Reject::RBFRejected(..) ) { if let Some(ref mut recent_reject) = tx_pool.recent_reject { if let Err(e) = recent_reject.put(&tx_hash, reject.clone()) { error!("record recent_reject failed {} {} {}", tx_hash, reject, e); From f2ec36ae4b80c029619fff484d05db0f8e4d23fe Mon Sep 17 00:00:00 2001 From: yukang Date: Wed, 5 Jul 2023 16:56:37 +0800 Subject: [PATCH 31/58] add more test for RBF and cleanup --- resource/ckb.toml | 2 +- rpc/README.md | 12 +- test/src/main.rs | 4 + .../tx_pool/different_txs_with_same_input.rs | 13 - test/src/specs/tx_pool/replace.rs | 280 ++++++++++++++++-- tx-pool/src/component/pool_map.rs | 7 +- tx-pool/src/pool.rs | 54 ++-- tx-pool/src/process.rs | 11 +- util/jsonrpc-types/src/blockchain.rs | 16 - util/launcher/src/shared_builder.rs | 2 +- util/types/src/core/cell.rs | 1 - util/types/src/core/tx_pool.rs | 2 - 12 files changed, 307 insertions(+), 97 deletions(-) diff --git a/resource/ckb.toml b/resource/ckb.toml index d59d5d9edd..c34450a3e5 100644 --- a/resource/ckb.toml +++ b/resource/ckb.toml @@ -134,7 +134,7 @@ enable_deprecated_rpc = false # {{ [tx_pool] max_tx_pool_size = 180_000_000 # 180mb min_fee_rate = 1_000 # Here fee_rate are calculated directly using size in units of shannons/KB -min_rbf_rate = 1_000 # Here fee_rate are calculated directly using size in units of shannons/KB +min_rbf_rate = 1_500 # Here fee_rate are calculated directly using size in units of shannons/KB max_tx_verify_cycles = 70_000_000 max_ancestors_count = 25 diff --git a/rpc/README.md b/rpc/README.md index 6db0bf3d40..e7f6e24131 100644 --- a/rpc/README.md +++ b/rpc/README.md @@ -4510,6 +4510,7 @@ Response "result": { "last_txs_updated_at": "0x0", "min_fee_rate": "0x3e8", + "min_rbf_rate": "0x5dc", "max_tx_pool_size": "0xaba9500", "orphan": "0x0", "pending": "0x1", @@ -5072,6 +5073,10 @@ For example, a cellbase transaction is not allowed in `send_transaction` RPC. (-1110): The transaction exceeded maximum size limit. +### Error `PoolRejctedRBF` + +(-1111): The transaction is rejected for RBF checking. + ### Error `Indexer` (-1200): The indexer error. @@ -6426,7 +6431,7 @@ TX reject message `PoolTransactionReject` is a JSON object with following fields. -* `type`: `"LowFeeRate" | "ExceededMaximumAncestorsCount" | "ExceededTransactionSizeLimit" | "Full" | "Duplicated" | "Malformed" | "DeclaredWrongCycles" | "Resolve" | "Verification" | "Expiry"` - Reject type. +* `type`: `"LowFeeRate" | "ExceededMaximumAncestorsCount" | "ExceededTransactionSizeLimit" | "Full" | "Duplicated" | "Malformed" | "DeclaredWrongCycles" | "Resolve" | "Verification" | "Expiry" | "RBFRejected"` - Reject type. * `description`: `string` - Detailed description about why the transaction is rejected. Different reject types: @@ -6441,6 +6446,7 @@ Different reject types: * `Resolve`: Resolve failed * `Verification`: Verification failed * `Expiry`: Transaction expired +* `RBFRejected`: RBF rejected ### Type `ProposalShortId` @@ -7033,6 +7039,10 @@ Transaction pool information. The unit is Shannons per 1000 bytes transaction serialization size in the block. +* `min_rbf_rate`: [`Uint64`](#type-uint64) - RBF rate threshold. The pool reject to resort for transactions which fee rate is below this threshold. + + The unit is Shannons per 1000 bytes transaction serialization size in the block. + * `last_txs_updated_at`: [`Timestamp`](#type-timestamp) - Last updated time. This is the Unix timestamp in milliseconds. * `tx_size_limit`: [`Uint64`](#type-uint64) - Limiting transactions to tx_size_limit diff --git a/test/src/main.rs b/test/src/main.rs index f16b212ae1..6757d18307 100644 --- a/test/src/main.rs +++ b/test/src/main.rs @@ -461,6 +461,10 @@ fn all_specs() -> Vec> { Box::new(DifferentTxsWithSameInputWithOutRBF), Box::new(RbfBasic), Box::new(RbfSameInput), + Box::new(RbfSameInputwithLessFee), + Box::new(RbfTooManyDescendants), + Box::new(RbfContainNewTx), + Box::new(RbfContainInvalidInput), Box::new(CompactBlockEmpty), Box::new(CompactBlockEmptyParentUnknown), Box::new(CompactBlockPrefilled), diff --git a/test/src/specs/tx_pool/different_txs_with_same_input.rs b/test/src/specs/tx_pool/different_txs_with_same_input.rs index 87fd7d3c56..db86a2de71 100644 --- a/test/src/specs/tx_pool/different_txs_with_same_input.rs +++ b/test/src/specs/tx_pool/different_txs_with_same_input.rs @@ -20,16 +20,6 @@ impl Spec for DifferentTxsWithSameInputWithOutRBF { let tx1 = node0.new_transaction(tx_hash_0.clone()); let tx2_temp = node0.new_transaction(tx_hash_0); - eprintln!( - "tx1 hash: {:?} short_id: {:?}", - tx1.hash(), - tx1.proposal_short_id() - ); - eprintln!( - "tx2 hash: {:?} short_id: {:?}", - tx2_temp.hash(), - tx2_temp.proposal_short_id() - ); // Set tx2 fee to a higher value, tx1 capacity is 100, set tx2 capacity to 80 for +20 fee. let output = CellOutputBuilder::default() .capacity(capacity_bytes!(80).pack()) @@ -40,9 +30,6 @@ impl Spec for DifferentTxsWithSameInputWithOutRBF { .set_outputs(vec![output]) .build(); - eprintln!("tx1: {:?}", tx1); - eprintln!("tx2: {:?}", tx2); - node0.rpc_client().send_transaction(tx1.data().into()); let res = node0 .rpc_client() diff --git a/test/src/specs/tx_pool/replace.rs b/test/src/specs/tx_pool/replace.rs index a5e742fbbc..35af8a8a98 100644 --- a/test/src/specs/tx_pool/replace.rs +++ b/test/src/specs/tx_pool/replace.rs @@ -4,12 +4,11 @@ use ckb_logger::info; use ckb_types::{ core::{capacity_bytes, Capacity, TransactionView}, packed::CellOutputBuilder, + packed::{CellInput, OutPoint}, prelude::*, }; pub struct RbfBasic; -pub struct RbfSameInput; - impl Spec for RbfBasic { fn run(&self, nodes: &mut Vec) { let node0 = &nodes[0]; @@ -21,16 +20,6 @@ impl Spec for RbfBasic { let tx1 = node0.new_transaction(tx_hash_0.clone()); let tx2_temp = node0.new_transaction(tx_hash_0); - eprintln!( - "tx1 hash: {:?} short_id: {:?}", - tx1.hash(), - tx1.proposal_short_id() - ); - eprintln!( - "tx2 hash: {:?} short_id: {:?}", - tx2_temp.hash(), - tx2_temp.proposal_short_id() - ); // Set tx2 fee to a higher value, tx1 capacity is 100, set tx2 capacity to 80 for +20 fee. let output = CellOutputBuilder::default() .capacity(capacity_bytes!(80).pack()) @@ -41,9 +30,6 @@ impl Spec for RbfBasic { .set_outputs(vec![output]) .build(); - eprintln!("tx1: {:?}", tx1); - eprintln!("tx2: {:?}", tx2); - node0.rpc_client().send_transaction(tx1.data().into()); let res = node0 .rpc_client() @@ -76,7 +62,6 @@ impl Spec for RbfBasic { let ret = node0 .rpc_client() .get_transaction_with_verbosity(tx1.hash(), 1); - eprintln!("ret: {:?}", ret); assert!(ret.transaction.is_none()); assert!(matches!(ret.tx_status.status, Status::Rejected)); assert!(ret.tx_status.reason.unwrap().contains("RBFRejected")); @@ -101,6 +86,7 @@ impl Spec for RbfBasic { } } +pub struct RbfSameInput; impl Spec for RbfSameInput { fn run(&self, nodes: &mut Vec) { let node0 = &nodes[0]; @@ -112,26 +98,264 @@ impl Spec for RbfSameInput { let tx1 = node0.new_transaction(tx_hash_0.clone()); let tx2_temp = node0.new_transaction(tx_hash_0); - eprintln!( - "tx1 hash: {:?} short_id: {:?}", - tx1.hash(), - tx1.proposal_short_id() - ); - eprintln!( - "tx2 hash: {:?} short_id: {:?}", - tx2_temp.hash(), - tx2_temp.proposal_short_id() - ); let tx2 = tx2_temp.as_advanced_builder().build(); - eprintln!("tx1: {:?}", tx1); - eprintln!("tx2: {:?}", tx2); + node0.rpc_client().send_transaction(tx1.data().into()); + let res = node0 + .rpc_client() + .send_transaction_result(tx2.data().into()); + assert!(res.is_err(), "tx2 should be rejected"); + } + + fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { + config.tx_pool.enable_rbf = true; + } +} + +pub struct RbfSameInputwithLessFee; + +// RBF Rule #3 +impl Spec for RbfSameInputwithLessFee { + fn run(&self, nodes: &mut Vec) { + let node0 = &nodes[0]; + + node0.mine_until_out_bootstrap_period(); + node0.new_block_with_blocking(|template| template.number.value() != 13); + let tx_hash_0 = node0.generate_transaction(); + info!("Generate 2 txs with same input"); + let tx1 = node0.new_transaction(tx_hash_0.clone()); + let tx2_temp = node0.new_transaction(tx_hash_0); + + let output1 = CellOutputBuilder::default() + .capacity(capacity_bytes!(80).pack()) + .build(); + + let tx1 = tx1.as_advanced_builder().set_outputs(vec![output1]).build(); + + // Set tx2 fee to a lower value + let output2 = CellOutputBuilder::default() + .capacity(capacity_bytes!(90).pack()) + .build(); + + let tx2 = tx2_temp + .as_advanced_builder() + .set_outputs(vec![output2]) + .build(); node0.rpc_client().send_transaction(tx1.data().into()); let res = node0 .rpc_client() .send_transaction_result(tx2.data().into()); assert!(res.is_err(), "tx2 should be rejected"); + assert!(res + .err() + .unwrap() + .to_string() + .contains("Tx fee lower than old conflict Tx fee")); + } + + fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { + config.tx_pool.enable_rbf = true; + } +} + +pub struct RbfTooManyDescendants; + +// RBF Rule #5 +impl Spec for RbfTooManyDescendants { + fn run(&self, nodes: &mut Vec) { + let node0 = &nodes[0]; + + node0.mine_until_out_bootstrap_period(); + + // build txs chain + let tx0 = node0.new_transaction_spend_tip_cellbase(); + let tx0_temp = tx0.clone(); + let mut txs = vec![tx0]; + let max_count = 101; + while txs.len() <= max_count { + let parent = txs.last().unwrap(); + let child = parent + .as_advanced_builder() + .set_inputs(vec![{ + CellInput::new_builder() + .previous_output(OutPoint::new(parent.hash(), 0)) + .build() + }]) + .set_outputs(vec![parent.output(0).unwrap()]) + .build(); + txs.push(child); + } + assert_eq!(txs.len(), max_count + 1); + // send tx chain + for tx in txs[..=max_count - 1].iter() { + let ret = node0.rpc_client().send_transaction_result(tx.data().into()); + assert!(ret.is_ok()); + } + + // Set tx2 fee to a higher value + let output2 = CellOutputBuilder::default() + .capacity(capacity_bytes!(70).pack()) + .build(); + + let tx2 = tx0_temp + .as_advanced_builder() + .set_outputs(vec![output2]) + .build(); + + let res = node0 + .rpc_client() + .send_transaction_result(tx2.data().into()); + assert!(res.is_err(), "tx2 should be rejected"); + assert!(res + .err() + .unwrap() + .to_string() + .contains("Tx conflict too many txs")); + } + + fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { + config.tx_pool.enable_rbf = true; + } +} + +pub struct RbfContainNewTx; + +// RBF Rule #2 +impl Spec for RbfContainNewTx { + fn run(&self, nodes: &mut Vec) { + let node0 = &nodes[0]; + + node0.mine_until_out_bootstrap_period(); + + // build txs chain + let tx0 = node0.new_transaction_spend_tip_cellbase(); + let mut txs = vec![tx0]; + let max_count = 5; + while txs.len() <= max_count { + let parent = txs.last().unwrap(); + let child = parent + .as_advanced_builder() + .set_inputs(vec![{ + CellInput::new_builder() + .previous_output(OutPoint::new(parent.hash(), 0)) + .build() + }]) + .set_outputs(vec![parent.output(0).unwrap()]) + .build(); + txs.push(child); + } + assert_eq!(txs.len(), max_count + 1); + // send tx chain + for tx in txs[..=max_count - 1].iter() { + let ret = node0.rpc_client().send_transaction_result(tx.data().into()); + assert!(ret.is_ok()); + } + + let clone_tx = txs[2].clone(); + // Set tx2 fee to a higher value + let output2 = CellOutputBuilder::default() + .capacity(capacity_bytes!(70).pack()) + .build(); + + let tx2 = clone_tx + .as_advanced_builder() + .set_inputs(vec![ + { + CellInput::new_builder() + .previous_output(OutPoint::new(txs[1].hash(), 0)) + .build() + }, + { + CellInput::new_builder() + .previous_output(OutPoint::new(txs[4].hash(), 0)) + .build() + }, + ]) + .set_outputs(vec![output2]) + .build(); + + let res = node0 + .rpc_client() + .send_transaction_result(tx2.data().into()); + assert!(res.is_err(), "tx2 should be rejected"); + assert!(res + .err() + .unwrap() + .to_string() + .contains("new Tx contains unconfirmed inputs")); + } + + fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { + config.tx_pool.enable_rbf = true; + } +} + +pub struct RbfContainInvalidInput; + +// RBF Rule #2 +impl Spec for RbfContainInvalidInput { + fn run(&self, nodes: &mut Vec) { + let node0 = &nodes[0]; + + node0.mine_until_out_bootstrap_period(); + + // build txs chain + let tx0 = node0.new_transaction_spend_tip_cellbase(); + let mut txs = vec![tx0]; + let max_count = 5; + while txs.len() <= max_count { + let parent = txs.last().unwrap(); + let child = parent + .as_advanced_builder() + .set_inputs(vec![{ + CellInput::new_builder() + .previous_output(OutPoint::new(parent.hash(), 0)) + .build() + }]) + .set_outputs(vec![parent.output(0).unwrap()]) + .build(); + txs.push(child); + } + assert_eq!(txs.len(), max_count + 1); + // send Tx chain + for tx in txs[..=max_count - 1].iter() { + let ret = node0.rpc_client().send_transaction_result(tx.data().into()); + assert!(ret.is_ok()); + } + + let clone_tx = txs[2].clone(); + // Set tx2 fee to a higher value + let output2 = CellOutputBuilder::default() + .capacity(capacity_bytes!(70).pack()) + .build(); + + let tx2 = clone_tx + .as_advanced_builder() + .set_inputs(vec![ + { + CellInput::new_builder() + .previous_output(OutPoint::new(txs[1].hash(), 0)) + .build() + }, + { + CellInput::new_builder() + .previous_output(OutPoint::new(txs[3].hash(), 0)) + .build() + }, + ]) + .set_outputs(vec![output2]) + .build(); + + let res = node0 + .rpc_client() + .send_transaction_result(tx2.data().into()); + assert!(res.is_err(), "tx2 should be rejected"); + assert!(res + .err() + .unwrap() + .to_string() + .contains("new Tx contains inputs in descendants of to be replaced Tx")); } fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index 84f90cd2e1..231b419bfa 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -238,9 +238,8 @@ impl PoolMap { } pub(crate) fn find_conflict_tx(&self, tx: &TransactionView) -> HashSet { - let inputs = tx.input_pts_iter(); let mut res = HashSet::default(); - for i in inputs { + for i in tx.input_pts_iter() { if let Some(id) = self.edges.get_input_ref(&i) { res.insert(id.clone()); } @@ -249,10 +248,9 @@ impl PoolMap { } pub(crate) fn resolve_conflict(&mut self, tx: &TransactionView) -> Vec { - let inputs = tx.input_pts_iter(); let mut conflicts = Vec::new(); - for i in inputs { + for i in tx.input_pts_iter() { if let Some(id) = self.edges.remove_input(&i) { let entries = self.remove_entry_and_descendants(&id); if !entries.is_empty() { @@ -461,7 +459,6 @@ impl PoolMap { entry.add_ancestor_weight(&ancestor.inner); } if entry.ancestors_count > self.max_ancestors_count { - debug!("debug: exceeded maximum ancestors count"); return Err(Reject::ExceededMaximumAncestorsCount); } diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index c62c175141..6cbe18fe51 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -486,27 +486,20 @@ impl TxPool { tx_size: usize, ) -> Result<(), Reject> { assert!(self.config.enable_rbf); - if conflicts.is_empty() { - return Err(Reject::RBFRejected( - "can not find conflict txs to replace".to_string(), - )); - } + assert!(!conflicts.is_empty()); + let short_id = rtx.transaction.proposal_short_id(); let conflicts = conflicts .iter() .map(|id| { &self .get_pool_entry(id) - .expect("conflict tx should be in pool or store") + .expect("conflict Tx should be in pool") .inner }) .collect::>(); - // TODO: Rule #1, the conflicted tx need to confirmed as `can_be_replaced` - // Rule #2, new tx don't contain any new unconfirmed inputs - // TODO: confirm whether this could be used in ckb - // https://github.com/bitcoin/bitcoin/blob/d9c7c2fd3ec7b0fcae7e0c9423bff6c6799dd67c/src/policy/rbf.cpp#L107 let mut inputs = HashSet::new(); for c in conflicts.iter() { inputs.extend(c.transaction().input_pts_iter()); @@ -517,7 +510,7 @@ impl TxPool { .any(|pt| !inputs.contains(&pt) && !snapshot.transaction_exists(&pt.tx_hash())) { return Err(Reject::RBFRejected( - "new tx contains unconfirmed inputs".to_string(), + "new Tx contains unconfirmed inputs".to_string(), )); } @@ -525,36 +518,57 @@ impl TxPool { let min_rbf_fee = self.config.min_rbf_rate.fee(tx_size as u64); if fee <= min_rbf_fee { return Err(Reject::RBFRejected(format!( - "tx fee lower than min_rbf_fee, min_rbf_fee: {}, tx fee: {}", + "Tx fee lower than min_rbf_fee, min_rbf_fee: {}, tx fee: {}", min_rbf_fee, fee, ))); } - // Rule #3, new tx's fee need to higher than conflicts + // Rule #3, new tx's fee need to higher than conflicts, here we only check the root tx for conflict in conflicts.iter() { - eprintln!("old fee: {:?} new_fee: {:?}", conflict.fee, fee); if conflict.fee >= fee { return Err(Reject::RBFRejected(format!( - "tx fee lower than conflict tx fee, conflict id: {}, conflict fee: {}, tx fee: {}", - conflict.proposal_short_id(), - conflict.fee, - fee, + "Tx fee lower than old conflict Tx fee, tx fee: {}, conflict fee: {}", + fee, conflict.fee, ))); } } - // Rule #5, new replaced tx's descendants can not more than 100 + // Rule #5, the replaced tx's descendants can not more than 100 + // and the ancestor of the new tx don't have common set with the replaced tx's descendants let mut replace_count: usize = 0; + let ancestors = self.pool_map.calc_ancestors(&short_id); for conflict in conflicts.iter() { let id = conflict.proposal_short_id(); let descendants = self.pool_map.calc_descendants(&id); replace_count += descendants.len() + 1; if replace_count > MAX_REPLACEMENT_CANDIDATES { return Err(Reject::RBFRejected(format!( - "tx conflict too many txs, conflict txs count: {}", + "Tx conflict too many txs, conflict txs count: {}", replace_count, ))); } + + if !descendants.is_disjoint(&ancestors) { + return Err(Reject::RBFRejected( + "Tx ancestors have common with conflict Tx descendants".to_string(), + )); + } + + for id in descendants.iter() { + if let Some(entry) = self.get_pool_entry(id) { + let hash = entry.inner.transaction().hash(); + if rtx + .transaction + .input_pts_iter() + .any(|pt| pt.tx_hash() == hash) + { + return Err(Reject::RBFRejected( + "new Tx contains inputs in descendants of to be replaced Tx" + .to_string(), + )); + } + } + } } Ok(()) diff --git a/tx-pool/src/process.rs b/tx-pool/src/process.rs index 144321b4b6..eecc12d7b0 100644 --- a/tx-pool/src/process.rs +++ b/tx-pool/src/process.rs @@ -127,11 +127,6 @@ impl TxPoolService { // try to remove conflicted tx here for id in conflicts.iter() { let removed = tx_pool.pool_map.remove_entry_and_descendants(id); - if removed.is_empty() { - return Err(Reject::RBFRejected( - "RBF remove old entries error".to_string(), - )); - } for old in removed { let reject = Reject::RBFRejected(format!( "replaced by {}", @@ -220,7 +215,7 @@ impl TxPoolService { let (ret, snapshot) = self .with_tx_pool_read_lock(|tx_pool, snapshot| { - let tip_hash: Byte32 = snapshot.tip_hash(); + let tip_hash = snapshot.tip_hash(); // Same txid means exactly the same transaction, including inputs, outputs, witnesses, etc. // It's also not possible for RBF, reject it directly @@ -235,7 +230,7 @@ impl TxPoolService { Ok((tip_hash, rtx, status, fee, tx_size, HashSet::new())) } Err(err) => { - if tx_pool.config.enable_rbf { + if tx_pool.config.enable_rbf && matches!(err, Reject::Resolve(_)) { // Try RBF check let conflicts = tx_pool.pool_map.find_conflict_tx(tx); let (rtx, status) = resolve_tx(tx_pool, &snapshot, tx.clone(), true)?; @@ -1049,7 +1044,6 @@ fn _submit_entry( entry: TxEntry, callbacks: &Callbacks, ) -> Result<(), Reject> { - //eprintln!("_submit_entry: {:?}", entry.proposal_short_id()); match status { TxStatus::Fresh => { if tx_pool.add_pending(entry.clone())? { @@ -1067,7 +1061,6 @@ fn _submit_entry( } } } - //eprintln!("finished submit: {:?}", entry.proposal_short_id()); Ok(()) } diff --git a/util/jsonrpc-types/src/blockchain.rs b/util/jsonrpc-types/src/blockchain.rs index d785ead6c3..aaa20ffccf 100644 --- a/util/jsonrpc-types/src/blockchain.rs +++ b/util/jsonrpc-types/src/blockchain.rs @@ -577,8 +577,6 @@ pub enum Status { /// Status "rejected". The transaction has been recently removed from the pool. /// Due to storage limitations, the node can only hold the most recently removed transactions. Rejected, - /// Status "replaced". The transaction has been recently replace from the pool. - Replaced, } /// Transaction status and the block hash if it is committed. @@ -599,7 +597,6 @@ impl From for TxStatus { tx_pool::TxStatus::Proposed => TxStatus::proposed(), tx_pool::TxStatus::Committed(hash) => TxStatus::committed(hash), tx_pool::TxStatus::Rejected(reason) => TxStatus::rejected(reason), - tx_pool::TxStatus::Replaced(reason) => TxStatus::replaced(reason), tx_pool::TxStatus::Unknown => TxStatus::unknown(), } } @@ -650,19 +647,6 @@ impl TxStatus { } } - /// Transaction which has already been replaced recently. - /// - /// ## Params - /// - /// * `reason` - the reason why the transaction is replaced. - pub fn replaced(reason: String) -> Self { - Self { - status: Status::Replaced, - block_hash: None, - reason: Some(reason), - } - } - /// The node has not seen the transaction, pub fn unknown() -> Self { Self { diff --git a/util/launcher/src/shared_builder.rs b/util/launcher/src/shared_builder.rs index f019b377ad..e1ad368a24 100644 --- a/util/launcher/src/shared_builder.rs +++ b/util/launcher/src/shared_builder.rs @@ -463,7 +463,7 @@ fn register_tx_pool_callback(tx_pool_builder: &mut TxPoolServiceBuilder, notify: let tx_hash = entry.transaction().hash(); // record recent reject - if matches!(reject, Reject::Resolve(..) | Reject::RBFRejected(..) ) { + if matches!(reject, Reject::Resolve(..) | Reject::RBFRejected(..)) { if let Some(ref mut recent_reject) = tx_pool.recent_reject { if let Err(e) = recent_reject.put(&tx_hash, reject.clone()) { error!("record recent_reject failed {} {} {}", tx_hash, reject, e); diff --git a/util/types/src/core/cell.rs b/util/types/src/core/cell.rs index 33a1e0a218..92b4295597 100644 --- a/util/types/src/core/cell.rs +++ b/util/types/src/core/cell.rs @@ -710,7 +710,6 @@ pub fn resolve_transaction( // skip resolve input of cellbase if !transaction.is_cellbase() { for out_point in transaction.input_pts_iter() { - //eprintln!("resolve input: {:?}", out_point); if !current_inputs.insert(out_point.to_owned()) { return Err(OutPointError::Dead(out_point)); } diff --git a/util/types/src/core/tx_pool.rs b/util/types/src/core/tx_pool.rs index 6c19213502..75eb6e2790 100644 --- a/util/types/src/core/tx_pool.rs +++ b/util/types/src/core/tx_pool.rs @@ -117,8 +117,6 @@ pub enum TxStatus { /// Status "rejected". The transaction has been recently removed from the pool. /// Due to storage limitations, the node can only hold the most recently removed transactions. Rejected(String), - /// Status "replaced", The transaction has been recently replaced for RBF. - Replaced(String), } /// Tx-pool entry info From d37254e5b5f3b8a6d0ba8f57302b0b25fd802036 Mon Sep 17 00:00:00 2001 From: yukang Date: Wed, 19 Jul 2023 16:20:37 +0800 Subject: [PATCH 32/58] add enbale_rbf config option --- pow/src/lib.rs | 4 +-- rpc/README.md | 2 +- sync/src/synchronizer/mod.rs | 4 +-- test/src/specs/relay/transaction_relay.rs | 3 +- test/src/specs/tx_pool/collision.rs | 6 +++- .../tx_pool/different_txs_with_same_input.rs | 4 --- test/src/specs/tx_pool/pool_reconcile.rs | 6 +--- test/src/specs/tx_pool/replace.rs | 20 ++++++------ tx-pool/src/pool.rs | 31 ++++++++++--------- tx-pool/src/process.rs | 5 ++- util/app-config/src/configs/tx_pool.rs | 4 +-- util/app-config/src/legacy/tx_pool.rs | 11 +++---- util/jsonrpc-types/src/pool.rs | 2 +- 13 files changed, 49 insertions(+), 53 deletions(-) diff --git a/pow/src/lib.rs b/pow/src/lib.rs index 4429f24b35..6ce8c1c163 100644 --- a/pow/src/lib.rs +++ b/pow/src/lib.rs @@ -27,10 +27,10 @@ pub enum Pow { /// Mocking dummy PoW engine Dummy, /// The Eaglesong PoW engine - /// Check details of Eaglesong from: https://github.com/nervosnetwork/rfcs/blob/master/rfcs/0010-eaglesong/0010-eaglesong.md + /// Check details of Eaglesong from: Eaglesong, /// The Eaglesong PoW engine, similar to `Eaglesong`, but using `blake2b` hash as the final output. - /// Check details of blake2b from: https://tools.ietf.org/html/rfc7693 and blake2b-rs from: https://github.com/nervosnetwork/blake2b-rs + /// Check details of blake2b from: and blake2b-rs from: EaglesongBlake2b, } diff --git a/rpc/README.md b/rpc/README.md index e7f6e24131..73109788be 100644 --- a/rpc/README.md +++ b/rpc/README.md @@ -7039,7 +7039,7 @@ Transaction pool information. The unit is Shannons per 1000 bytes transaction serialization size in the block. -* `min_rbf_rate`: [`Uint64`](#type-uint64) - RBF rate threshold. The pool reject to resort for transactions which fee rate is below this threshold. +* `min_rbf_rate`: [`Uint64`](#type-uint64) - RBF rate threshold. The pool reject to replace for transactions which fee rate is below this threshold. The unit is Shannons per 1000 bytes transaction serialization size in the block. diff --git a/sync/src/synchronizer/mod.rs b/sync/src/synchronizer/mod.rs index b348226560..7f0a78c757 100644 --- a/sync/src/synchronizer/mod.rs +++ b/sync/src/synchronizer/mod.rs @@ -1,10 +1,10 @@ //! CKB node has initial block download phase (IBD mode) like Bitcoin: -//! https://btcinformation.org/en/glossary/initial-block-download +//! //! //! When CKB node is in IBD mode, it will respond `packed::InIBD` to `GetHeaders` and `GetBlocks` requests //! //! And CKB has a headers-first synchronization style like Bitcoin: -//! https://btcinformation.org/en/glossary/headers-first-sync +//! //! mod block_fetcher; mod block_process; diff --git a/test/src/specs/relay/transaction_relay.rs b/test/src/specs/relay/transaction_relay.rs index 06b10f4e19..75a34e0df6 100644 --- a/test/src/specs/relay/transaction_relay.rs +++ b/test/src/specs/relay/transaction_relay.rs @@ -5,6 +5,7 @@ use crate::util::transaction::{always_success_transaction, always_success_transa use crate::utils::{build_relay_tx_hashes, build_relay_txs, sleep, wait_until}; use crate::{Net, Node, Spec}; use ckb_constant::sync::RETRY_ASK_TX_TIMEOUT_INCREASE; +use ckb_jsonrpc_types::Status; use ckb_logger::info; use ckb_network::SupportProtocols; use ckb_types::{ @@ -265,7 +266,6 @@ impl Spec for TransactionRelayConflict { node0.wait_for_tx_pool(); node1.wait_for_tx_pool(); - /* let ret = node1 .rpc_client() .get_transaction_with_verbosity(tx1.hash(), 1); @@ -313,6 +313,5 @@ impl Spec for TransactionRelayConflict { .is_some() }); assert!(relayed, "Transaction should be relayed to node1"); - */ } } diff --git a/test/src/specs/tx_pool/collision.rs b/test/src/specs/tx_pool/collision.rs index bebe29495d..0d93697c03 100644 --- a/test/src/specs/tx_pool/collision.rs +++ b/test/src/specs/tx_pool/collision.rs @@ -1,4 +1,6 @@ -use crate::util::check::{is_transaction_committed, is_transaction_pending}; +use crate::util::check::{ + is_transaction_committed, is_transaction_pending, is_transaction_rejected, +}; use crate::utils::{assert_send_transaction_fail, blank, commit, propose}; use crate::{Node, Spec}; use ckb_types::bytes::Bytes; @@ -173,6 +175,8 @@ impl Spec for RemoveConflictFromPending { node.wait_for_tx_pool(); assert!(is_transaction_committed(node, &txa)); + assert!(is_transaction_rejected(node, &txb)); + assert!(is_transaction_rejected(node, &txc)); } } diff --git a/test/src/specs/tx_pool/different_txs_with_same_input.rs b/test/src/specs/tx_pool/different_txs_with_same_input.rs index db86a2de71..7b50dbdbef 100644 --- a/test/src/specs/tx_pool/different_txs_with_same_input.rs +++ b/test/src/specs/tx_pool/different_txs_with_same_input.rs @@ -84,8 +84,4 @@ impl Spec for DifferentTxsWithSameInputWithOutRBF { // assert!(ret.transaction.is_none()); // assert!(matches!(ret.tx_status.status, Status::Rejected)); } - - fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { - config.tx_pool.enable_rbf = false; - } } diff --git a/test/src/specs/tx_pool/pool_reconcile.rs b/test/src/specs/tx_pool/pool_reconcile.rs index 280c506cc8..c84b32fbc2 100644 --- a/test/src/specs/tx_pool/pool_reconcile.rs +++ b/test/src/specs/tx_pool/pool_reconcile.rs @@ -5,7 +5,7 @@ use crate::util::mining::out_ibd_mode; use crate::{Node, Spec}; use ckb_jsonrpc_types::ProposalShortId; use ckb_logger::info; -use ckb_types::core::{capacity_bytes, Capacity, FeeRate}; +use ckb_types::core::{capacity_bytes, Capacity}; use ckb_types::packed::CellOutputBuilder; use ckb_types::{ packed::{self, CellInput, OutPoint}, @@ -172,8 +172,4 @@ impl Spec for PoolResolveConflictAfterReorg { let err_msg = ret.err().unwrap().to_string(); assert!(err_msg.contains("Resolve failed Dead")); } - - fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { - config.tx_pool.min_fee_rate = FeeRate::from_u64(0); - } } diff --git a/test/src/specs/tx_pool/replace.rs b/test/src/specs/tx_pool/replace.rs index 35af8a8a98..b1e1fe6322 100644 --- a/test/src/specs/tx_pool/replace.rs +++ b/test/src/specs/tx_pool/replace.rs @@ -82,7 +82,7 @@ impl Spec for RbfBasic { } fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { - config.tx_pool.enable_rbf = true; + config.tx_pool.min_rbf_rate = ckb_types::core::FeeRate(1500); } } @@ -108,7 +108,7 @@ impl Spec for RbfSameInput { } fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { - config.tx_pool.enable_rbf = true; + config.tx_pool.min_rbf_rate = ckb_types::core::FeeRate(1500); } } @@ -147,15 +147,13 @@ impl Spec for RbfSameInputwithLessFee { .rpc_client() .send_transaction_result(tx2.data().into()); assert!(res.is_err(), "tx2 should be rejected"); - assert!(res - .err() - .unwrap() - .to_string() - .contains("Tx fee lower than old conflict Tx fee")); + let message = res.err().unwrap().to_string(); + eprintln!("res: {:?}", message); + assert!(message.contains("Tx's current fee is 1000000000, expect it to be larger than")); } fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { - config.tx_pool.enable_rbf = true; + config.tx_pool.min_rbf_rate = ckb_types::core::FeeRate(1500); } } @@ -215,7 +213,7 @@ impl Spec for RbfTooManyDescendants { } fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { - config.tx_pool.enable_rbf = true; + config.tx_pool.min_rbf_rate = ckb_types::core::FeeRate(1500); } } @@ -287,7 +285,7 @@ impl Spec for RbfContainNewTx { } fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { - config.tx_pool.enable_rbf = true; + config.tx_pool.min_rbf_rate = ckb_types::core::FeeRate(1500); } } @@ -359,6 +357,6 @@ impl Spec for RbfContainInvalidInput { } fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { - config.tx_pool.enable_rbf = true; + config.tx_pool.min_rbf_rate = ckb_types::core::FeeRate(1500); } } diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 6cbe18fe51..0792f7fcb0 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -85,6 +85,11 @@ impl TxPool { self.total_tx_cycles += cycles; } + /// Check whether tx-pool enable RBF + pub fn enable_rbf(&self) -> bool { + self.config.min_rbf_rate > self.config.min_fee_rate + } + /// Update size and cycles statics for remove tx /// cycles overflow is possible, currently obtaining cycles is not accurate pub fn update_statics_for_remove_tx(&mut self, tx_size: usize, cycles: Cycle) { @@ -485,7 +490,7 @@ impl TxPool { fee: Capacity, tx_size: usize, ) -> Result<(), Reject> { - assert!(self.config.enable_rbf); + assert!(self.enable_rbf()); assert!(!conflicts.is_empty()); let short_id = rtx.transaction.proposal_short_id(); @@ -515,24 +520,22 @@ impl TxPool { } // Rule #4, new tx' fee need to higher than min_rbf_fee computed from the tx_pool configuration + // Rule #3, new tx's fee need to higher than conflicts, here we only check the root tx let min_rbf_fee = self.config.min_rbf_rate.fee(tx_size as u64); - if fee <= min_rbf_fee { + let max_fee = conflicts + .iter() + .map(|c| c.fee) + .max() + .unwrap_or(min_rbf_fee) + .max(min_rbf_fee); + + if fee <= max_fee { return Err(Reject::RBFRejected(format!( - "Tx fee lower than min_rbf_fee, min_rbf_fee: {}, tx fee: {}", - min_rbf_fee, fee, + "Tx's current fee is {}, expect it to be larger than: {} to replace old txs", + fee, max_fee, ))); } - // Rule #3, new tx's fee need to higher than conflicts, here we only check the root tx - for conflict in conflicts.iter() { - if conflict.fee >= fee { - return Err(Reject::RBFRejected(format!( - "Tx fee lower than old conflict Tx fee, tx fee: {}, conflict fee: {}", - fee, conflict.fee, - ))); - } - } - // Rule #5, the replaced tx's descendants can not more than 100 // and the ancestor of the new tx don't have common set with the replaced tx's descendants let mut replace_count: usize = 0; diff --git a/tx-pool/src/process.rs b/tx-pool/src/process.rs index eecc12d7b0..e6747b32cb 100644 --- a/tx-pool/src/process.rs +++ b/tx-pool/src/process.rs @@ -230,9 +230,12 @@ impl TxPoolService { Ok((tip_hash, rtx, status, fee, tx_size, HashSet::new())) } Err(err) => { - if tx_pool.config.enable_rbf && matches!(err, Reject::Resolve(_)) { + if tx_pool.enable_rbf() && matches!(err, Reject::Resolve(_)) { // Try RBF check let conflicts = tx_pool.pool_map.find_conflict_tx(tx); + if conflicts.is_empty() { + return Err(err); + } let (rtx, status) = resolve_tx(tx_pool, &snapshot, tx.clone(), true)?; let fee = check_tx_fee(tx_pool, &snapshot, &rtx, tx_size)?; tx_pool.check_rbf(&snapshot, &rtx, &conflicts, fee, tx_size)?; diff --git a/util/app-config/src/configs/tx_pool.rs b/util/app-config/src/configs/tx_pool.rs index eb514c3d8a..a24c7938de 100644 --- a/util/app-config/src/configs/tx_pool.rs +++ b/util/app-config/src/configs/tx_pool.rs @@ -14,7 +14,7 @@ pub struct TxPoolConfig { /// txs with lower fee rate than this will not be relayed or be mined #[serde(with = "FeeRateDef")] pub min_fee_rate: FeeRate, - /// txs need to pay more than this for RBF + /// txs need to pay larger fee rate than this for RBF #[serde(with = "FeeRateDef")] pub min_rbf_rate: FeeRate, /// tx pool rejects txs that cycles greater than max_tx_verify_cycles @@ -37,8 +37,6 @@ pub struct TxPoolConfig { pub recent_reject: PathBuf, /// The expiration time for pool transactions in hours pub expiry_hours: u8, - /// Enable RBF - pub enable_rbf: bool, } /// Block assembler config options. diff --git a/util/app-config/src/legacy/tx_pool.rs b/util/app-config/src/legacy/tx_pool.rs index 1927bbda7e..562f3ac6a6 100644 --- a/util/app-config/src/legacy/tx_pool.rs +++ b/util/app-config/src/legacy/tx_pool.rs @@ -35,7 +35,7 @@ pub(crate) struct TxPoolConfig { keep_rejected_tx_hashes_count: u64, #[serde(with = "FeeRateDef")] min_fee_rate: FeeRate, - #[serde(with = "FeeRateDef")] + #[serde(with = "FeeRateDef", default = "default_min_rbf_rate")] min_rbf_rate: FeeRate, max_tx_verify_cycles: Cycle, max_ancestors_count: usize, @@ -45,8 +45,6 @@ pub(crate) struct TxPoolConfig { recent_reject: PathBuf, #[serde(default = "default_expiry_hours")] expiry_hours: u8, - #[serde(default)] - enable_rbf: bool, } fn default_keep_rejected_tx_hashes_days() -> u8 { @@ -65,6 +63,10 @@ fn default_max_tx_pool_size() -> usize { DEFAULT_MAX_TX_POOL_SIZE } +fn default_min_rbf_rate() -> FeeRate { + DEFAULT_MIN_RBF_RATE +} + impl Default for crate::TxPoolConfig { fn default() -> Self { TxPoolConfig::default().into() @@ -89,7 +91,6 @@ impl Default for TxPoolConfig { persisted_data: Default::default(), recent_reject: Default::default(), expiry_hours: DEFAULT_EXPIRY_HOURS, - enable_rbf: false, } } } @@ -112,7 +113,6 @@ impl From for crate::TxPoolConfig { persisted_data, recent_reject, expiry_hours, - enable_rbf, } = input; Self { @@ -126,7 +126,6 @@ impl From for crate::TxPoolConfig { persisted_data, recent_reject, expiry_hours, - enable_rbf, } } } diff --git a/util/jsonrpc-types/src/pool.rs b/util/jsonrpc-types/src/pool.rs index 3e773c1ca2..a7c7bf4d6b 100644 --- a/util/jsonrpc-types/src/pool.rs +++ b/util/jsonrpc-types/src/pool.rs @@ -40,7 +40,7 @@ pub struct TxPoolInfo { /// /// The unit is Shannons per 1000 bytes transaction serialization size in the block. pub min_fee_rate: Uint64, - /// RBF rate threshold. The pool reject to resort for transactions which fee rate is below this threshold. + /// RBF rate threshold. The pool reject to replace for transactions which fee rate is below this threshold. /// /// The unit is Shannons per 1000 bytes transaction serialization size in the block. pub min_rbf_rate: Uint64, From f731e81e4a4994b77ad13c478f2b165397c601f9 Mon Sep 17 00:00:00 2001 From: yukang Date: Tue, 25 Jul 2023 15:09:36 +0800 Subject: [PATCH 33/58] add Rule 7 for RBF, any old Tx should be in Pending status --- test/src/main.rs | 1 + test/src/specs/tx_pool/replace.rs | 74 ++++++++++++++++++++++++++++++- tx-pool/src/pool.rs | 19 ++++++-- 3 files changed, 89 insertions(+), 5 deletions(-) diff --git a/test/src/main.rs b/test/src/main.rs index 6757d18307..4e9d0fbc5c 100644 --- a/test/src/main.rs +++ b/test/src/main.rs @@ -465,6 +465,7 @@ fn all_specs() -> Vec> { Box::new(RbfTooManyDescendants), Box::new(RbfContainNewTx), Box::new(RbfContainInvalidInput), + Box::new(RbfRejectReplaceProposed), Box::new(CompactBlockEmpty), Box::new(CompactBlockEmptyParentUnknown), Box::new(CompactBlockPrefilled), diff --git a/test/src/specs/tx_pool/replace.rs b/test/src/specs/tx_pool/replace.rs index b1e1fe6322..abbfca053c 100644 --- a/test/src/specs/tx_pool/replace.rs +++ b/test/src/specs/tx_pool/replace.rs @@ -148,7 +148,6 @@ impl Spec for RbfSameInputwithLessFee { .send_transaction_result(tx2.data().into()); assert!(res.is_err(), "tx2 should be rejected"); let message = res.err().unwrap().to_string(); - eprintln!("res: {:?}", message); assert!(message.contains("Tx's current fee is 1000000000, expect it to be larger than")); } @@ -360,3 +359,76 @@ impl Spec for RbfContainInvalidInput { config.tx_pool.min_rbf_rate = ckb_types::core::FeeRate(1500); } } + +pub struct RbfRejectReplaceProposed; + +// RBF Rule #6 +impl Spec for RbfRejectReplaceProposed { + fn run(&self, nodes: &mut Vec) { + let node0 = &nodes[0]; + + node0.mine_until_out_bootstrap_period(); + + // build txs chain + let tx0 = node0.new_transaction_spend_tip_cellbase(); + let mut txs = vec![tx0]; + let max_count = 5; + while txs.len() <= max_count { + let parent = txs.last().unwrap(); + let child = parent + .as_advanced_builder() + .set_inputs(vec![{ + CellInput::new_builder() + .previous_output(OutPoint::new(parent.hash(), 0)) + .build() + }]) + .set_outputs(vec![parent.output(0).unwrap()]) + .build(); + txs.push(child); + } + assert_eq!(txs.len(), max_count + 1); + // send Tx chain + for tx in txs[..=max_count - 1].iter() { + let ret = node0.rpc_client().send_transaction_result(tx.data().into()); + assert!(ret.is_ok()); + } + + node0.mine_with_blocking(|template| template.proposals.len() != max_count); + let ret = node0.rpc_client().get_transaction(txs[2].hash()); + assert!( + matches!(ret.tx_status.status, Status::Pending), + "tx1 should be pending" + ); + node0.mine(1); + let ret = node0.rpc_client().get_transaction(txs[2].hash()); + assert!( + matches!(ret.tx_status.status, Status::Proposed), + "tx1 should be proposed" + ); + + let clone_tx = txs[2].clone(); + // Set tx2 fee to a higher value + let output2 = CellOutputBuilder::default() + .capacity(capacity_bytes!(70).pack()) + .build(); + + let tx2 = clone_tx + .as_advanced_builder() + .set_outputs(vec![output2]) + .build(); + + let res = node0 + .rpc_client() + .send_transaction_result(tx2.data().into()); + assert!(res.is_err(), "tx2 should be rejected"); + assert!(res + .err() + .unwrap() + .to_string() + .contains("all conflict Txs should be in Pending status")); + } + + fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { + config.tx_pool.min_rbf_rate = ckb_types::core::FeeRate(1500); + } +} diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 0792f7fcb0..a0b7fb3ddf 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -494,16 +494,27 @@ impl TxPool { assert!(!conflicts.is_empty()); let short_id = rtx.transaction.proposal_short_id(); - let conflicts = conflicts + let entries = conflicts .iter() .map(|id| { - &self - .get_pool_entry(id) + self.get_pool_entry(id) .expect("conflict Tx should be in pool") - .inner }) .collect::>(); + // Rule #6, any old Tx should be in `Pending` or `Gap` status + if entries + .iter() + .any(|e| ![Status::Pending, Status::Gap].contains(&e.status)) + { + // Here we only refer to `Pending` status, since `Gap` is an internal status + return Err(Reject::RBFRejected( + "all conflict Txs should be in Pending status".to_string(), + )); + } + + let conflicts = entries.iter().map(|e| e.inner.clone()).collect::>(); + // Rule #2, new tx don't contain any new unconfirmed inputs let mut inputs = HashSet::new(); for c in conflicts.iter() { From 0464eab5ff6224474f45298cc15a6283af6f5e0b Mon Sep 17 00:00:00 2001 From: yukang Date: Wed, 26 Jul 2023 10:50:41 +0800 Subject: [PATCH 34/58] merge score_key and evict_key to sort_key --- tx-pool/src/component/commit_txs_scanner.rs | 2 +- tx-pool/src/component/entry.rs | 57 +++++-------------- tx-pool/src/component/mod.rs | 2 +- tx-pool/src/component/pool_map.rs | 3 +- .../component/{score_key.rs => sort_key.rs} | 35 +++++++++++- tx-pool/src/component/tests/entry.rs | 2 +- tx-pool/src/component/tests/score_key.rs | 2 +- 7 files changed, 52 insertions(+), 51 deletions(-) rename tx-pool/src/component/{score_key.rs => sort_key.rs} (65%) diff --git a/tx-pool/src/component/commit_txs_scanner.rs b/tx-pool/src/component/commit_txs_scanner.rs index c2058fdba6..3546aa0b7f 100644 --- a/tx-pool/src/component/commit_txs_scanner.rs +++ b/tx-pool/src/component/commit_txs_scanner.rs @@ -1,5 +1,5 @@ use crate::component::pool_map::PoolMap; -use crate::component::{entry::TxEntry, score_key::AncestorsScoreSortKey}; +use crate::component::{entry::TxEntry, sort_key::AncestorsScoreSortKey}; use ckb_types::{core::Cycle, packed::ProposalShortId}; use ckb_util::LinkedHashMap; use std::collections::{BTreeSet, HashMap, HashSet}; diff --git a/tx-pool/src/component/entry.rs b/tx-pool/src/component/entry.rs index 2f8fdf95ef..09f9f3c988 100644 --- a/tx-pool/src/component/entry.rs +++ b/tx-pool/src/component/entry.rs @@ -1,4 +1,4 @@ -use crate::component::score_key::AncestorsScoreSortKey; +use crate::component::sort_key::{AncestorsScoreSortKey, EvictKey}; use ckb_systemtime::unix_time_as_millis; use ckb_types::{ core::{ @@ -194,21 +194,6 @@ impl TxEntry { } } -impl From<&TxEntry> for AncestorsScoreSortKey { - fn from(entry: &TxEntry) -> Self { - let weight = get_transaction_weight(entry.size, entry.cycles); - let ancestors_weight = get_transaction_weight(entry.ancestors_size, entry.ancestors_cycles); - AncestorsScoreSortKey { - fee: entry.fee, - weight, - id: entry.proposal_short_id(), - ancestors_fee: entry.ancestors_fee, - ancestors_weight, - //timestamp: entry.timestamp, - } - } -} - impl Hash for TxEntry { fn hash(&self, state: &mut H) { Hash::hash(self.transaction(), state); @@ -233,14 +218,18 @@ impl Ord for TxEntry { } } -/// First compare fee_rate, select the smallest fee_rate, -/// and then select the latest timestamp, for eviction, -/// the latest timestamp which also means that the fewer descendants may exist. -#[derive(Eq, PartialEq, Clone, Debug)] -pub struct EvictKey { - pub fee_rate: FeeRate, - pub timestamp: u64, - pub descendants_count: usize, +impl From<&TxEntry> for AncestorsScoreSortKey { + fn from(entry: &TxEntry) -> Self { + let weight = get_transaction_weight(entry.size, entry.cycles); + let ancestors_weight = get_transaction_weight(entry.ancestors_size, entry.ancestors_cycles); + AncestorsScoreSortKey { + fee: entry.fee, + weight, + id: entry.proposal_short_id(), + ancestors_fee: entry.ancestors_fee, + ancestors_weight, + } + } } impl From<&TxEntry> for EvictKey { @@ -258,23 +247,3 @@ impl From<&TxEntry> for EvictKey { } } } - -impl PartialOrd for EvictKey { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} - -impl Ord for EvictKey { - fn cmp(&self, other: &Self) -> Ordering { - if self.fee_rate == other.fee_rate { - if self.descendants_count == other.descendants_count { - self.timestamp.cmp(&other.timestamp) - } else { - self.descendants_count.cmp(&other.descendants_count) - } - } else { - self.fee_rate.cmp(&other.fee_rate) - } - } -} diff --git a/tx-pool/src/component/mod.rs b/tx-pool/src/component/mod.rs index 7f325424ba..e5b8ab3cfc 100644 --- a/tx-pool/src/component/mod.rs +++ b/tx-pool/src/component/mod.rs @@ -7,7 +7,7 @@ pub(crate) mod links; pub(crate) mod orphan; pub(crate) mod pool_map; pub(crate) mod recent_reject; -pub(crate) mod score_key; +pub(crate) mod sort_key; #[cfg(test)] mod tests; diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index 231b419bfa..b7ccec0034 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -2,9 +2,8 @@ extern crate rustc_hash; extern crate slab; use crate::component::edges::Edges; -use crate::component::entry::EvictKey; use crate::component::links::{Relation, TxLinksMap}; -use crate::component::score_key::AncestorsScoreSortKey; +use crate::component::sort_key::{AncestorsScoreSortKey, EvictKey}; use crate::error::Reject; use crate::TxEntry; use ckb_logger::{debug, trace}; diff --git a/tx-pool/src/component/score_key.rs b/tx-pool/src/component/sort_key.rs similarity index 65% rename from tx-pool/src/component/score_key.rs rename to tx-pool/src/component/sort_key.rs index 18dd48fcb2..50f2363aff 100644 --- a/tx-pool/src/component/score_key.rs +++ b/tx-pool/src/component/sort_key.rs @@ -1,4 +1,7 @@ -use ckb_types::{core::Capacity, packed::ProposalShortId}; +use ckb_types::{ + core::{Capacity, FeeRate}, + packed::ProposalShortId, +}; use std::cmp::Ordering; /// A struct to use as a sorted key @@ -51,3 +54,33 @@ impl Ord for AncestorsScoreSortKey { } } } + +/// First compare fee_rate, select the smallest fee_rate, +/// and then select the latest timestamp, for eviction, +/// the latest timestamp which also means that the fewer descendants may exist. +#[derive(Eq, PartialEq, Clone, Debug)] +pub struct EvictKey { + pub fee_rate: FeeRate, + pub timestamp: u64, + pub descendants_count: usize, +} + +impl PartialOrd for EvictKey { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for EvictKey { + fn cmp(&self, other: &Self) -> Ordering { + if self.fee_rate == other.fee_rate { + if self.descendants_count == other.descendants_count { + self.timestamp.cmp(&other.timestamp) + } else { + self.descendants_count.cmp(&other.descendants_count) + } + } else { + self.fee_rate.cmp(&other.fee_rate) + } + } +} diff --git a/tx-pool/src/component/tests/entry.rs b/tx-pool/src/component/tests/entry.rs index 8aa7edf3ff..3bd1ea8ebd 100644 --- a/tx-pool/src/component/tests/entry.rs +++ b/tx-pool/src/component/tests/entry.rs @@ -1,6 +1,6 @@ use ckb_types::core::{Capacity, FeeRate}; -use crate::component::entry::EvictKey; +use crate::component::sort_key::EvictKey; #[test] fn test_min_fee_and_weight_evict() { diff --git a/tx-pool/src/component/tests/score_key.rs b/tx-pool/src/component/tests/score_key.rs index 09475f3d19..94aa51cf72 100644 --- a/tx-pool/src/component/tests/score_key.rs +++ b/tx-pool/src/component/tests/score_key.rs @@ -6,7 +6,7 @@ use ckb_types::{ }; use std::mem::size_of; -use crate::component::{entry::TxEntry, pool_map::PoolMap, score_key::AncestorsScoreSortKey}; +use crate::component::{entry::TxEntry, pool_map::PoolMap, sort_key::AncestorsScoreSortKey}; const DEFAULT_MAX_ANCESTORS_COUNT: usize = 125; From 0e0b7aec46729905f502980871ccb4f1e15ff5bc Mon Sep 17 00:00:00 2001 From: yukang Date: Wed, 26 Jul 2023 11:52:22 +0800 Subject: [PATCH 35/58] uncomment test --- .../tx_pool/different_txs_with_same_input.rs | 30 +++++++++---------- test/src/specs/tx_pool/pool_reconcile.rs | 6 +++- tx-pool/src/pool.rs | 8 ----- tx-pool/src/process.rs | 6 +++- 4 files changed, 25 insertions(+), 25 deletions(-) diff --git a/test/src/specs/tx_pool/different_txs_with_same_input.rs b/test/src/specs/tx_pool/different_txs_with_same_input.rs index 7b50dbdbef..f590f7ae81 100644 --- a/test/src/specs/tx_pool/different_txs_with_same_input.rs +++ b/test/src/specs/tx_pool/different_txs_with_same_input.rs @@ -52,11 +52,11 @@ impl Spec for DifferentTxsWithSameInputWithOutRBF { assert!(!commit_txs_hash.contains(&tx2.hash())); // when tx1 was confirmed, tx2 should be rejected - // let ret = node0.rpc_client().get_transaction(tx2.hash()); - // assert!( - // matches!(ret.tx_status.status, Status::Rejected), - // "tx2 should be rejected" - // ); + let ret = node0.rpc_client().get_transaction(tx2.hash()); + assert!( + matches!(ret.tx_status.status, Status::Rejected), + "tx2 should be rejected" + ); // verbosity = 1 let ret = node0 @@ -65,11 +65,11 @@ impl Spec for DifferentTxsWithSameInputWithOutRBF { assert!(ret.transaction.is_none()); assert!(matches!(ret.tx_status.status, Status::Committed)); - // let ret = node0 - // .rpc_client() - // .get_transaction_with_verbosity(tx2.hash(), 1); - // assert!(ret.transaction.is_none()); - // assert!(matches!(ret.tx_status.status, Status::Rejected)); + let ret = node0 + .rpc_client() + .get_transaction_with_verbosity(tx2.hash(), 1); + assert!(ret.transaction.is_none()); + assert!(matches!(ret.tx_status.status, Status::Rejected)); // verbosity = 2 let ret = node0 @@ -78,10 +78,10 @@ impl Spec for DifferentTxsWithSameInputWithOutRBF { assert!(ret.transaction.is_some()); assert!(matches!(ret.tx_status.status, Status::Committed)); - // let ret = node0 - // .rpc_client() - // .get_transaction_with_verbosity(tx2.hash(), 2); - // assert!(ret.transaction.is_none()); - // assert!(matches!(ret.tx_status.status, Status::Rejected)); + let ret = node0 + .rpc_client() + .get_transaction_with_verbosity(tx2.hash(), 2); + assert!(ret.transaction.is_none()); + assert!(matches!(ret.tx_status.status, Status::Rejected)); } } diff --git a/test/src/specs/tx_pool/pool_reconcile.rs b/test/src/specs/tx_pool/pool_reconcile.rs index c84b32fbc2..280c506cc8 100644 --- a/test/src/specs/tx_pool/pool_reconcile.rs +++ b/test/src/specs/tx_pool/pool_reconcile.rs @@ -5,7 +5,7 @@ use crate::util::mining::out_ibd_mode; use crate::{Node, Spec}; use ckb_jsonrpc_types::ProposalShortId; use ckb_logger::info; -use ckb_types::core::{capacity_bytes, Capacity}; +use ckb_types::core::{capacity_bytes, Capacity, FeeRate}; use ckb_types::packed::CellOutputBuilder; use ckb_types::{ packed::{self, CellInput, OutPoint}, @@ -172,4 +172,8 @@ impl Spec for PoolResolveConflictAfterReorg { let err_msg = ret.err().unwrap().to_string(); assert!(err_msg.contains("Resolve failed Dead")); } + + fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { + config.tx_pool.min_fee_rate = FeeRate::from_u64(0); + } } diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index a0b7fb3ddf..40f62212aa 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -160,14 +160,6 @@ impl TxPool { .map(|entry| entry.inner.transaction()) } - pub(crate) fn put_recent_reject(&mut self, tx_hash: &Byte32, reject: &Reject) { - if let Some(ref mut recent_reject) = self.recent_reject { - if let Err(e) = recent_reject.put(tx_hash, reject.clone()) { - error!("record recent_reject failed {} {} {}", tx_hash, reject, e); - } - } - } - pub(crate) fn remove_committed_txs<'a>( &mut self, txs: impl Iterator, diff --git a/tx-pool/src/process.rs b/tx-pool/src/process.rs index e6747b32cb..1ec8d1922b 100644 --- a/tx-pool/src/process.rs +++ b/tx-pool/src/process.rs @@ -325,7 +325,11 @@ impl TxPoolService { pub(crate) async fn put_recent_reject(&self, tx_hash: &Byte32, reject: &Reject) { let mut tx_pool = self.tx_pool.write().await; - tx_pool.put_recent_reject(tx_hash, reject); + if let Some(ref mut recent_reject) = tx_pool.recent_reject { + if let Err(e) = recent_reject.put(tx_hash, reject.clone()) { + error!("record recent_reject failed {} {} {}", tx_hash, reject, e); + } + } } pub(crate) async fn remove_tx(&self, tx_hash: Byte32) -> bool { From 193ccb12d850803bcb968c95ce83c01480d673ae Mon Sep 17 00:00:00 2001 From: yukang Date: Wed, 26 Jul 2023 12:10:48 +0800 Subject: [PATCH 36/58] add comments for min_rbf_rate --- resource/ckb.toml | 1 + rpc/README.md | 2 +- util/jsonrpc-types/src/pool.rs | 1 + util/types/src/core/tx_pool.rs | 1 + 4 files changed, 4 insertions(+), 1 deletion(-) diff --git a/resource/ckb.toml b/resource/ckb.toml index c34450a3e5..9f99d4bb98 100644 --- a/resource/ckb.toml +++ b/resource/ckb.toml @@ -134,6 +134,7 @@ enable_deprecated_rpc = false # {{ [tx_pool] max_tx_pool_size = 180_000_000 # 180mb min_fee_rate = 1_000 # Here fee_rate are calculated directly using size in units of shannons/KB +# min_rbf_rate > min_fee_rate means RBF is enabled min_rbf_rate = 1_500 # Here fee_rate are calculated directly using size in units of shannons/KB max_tx_verify_cycles = 70_000_000 max_ancestors_count = 25 diff --git a/rpc/README.md b/rpc/README.md index 73109788be..f0af4bc71d 100644 --- a/rpc/README.md +++ b/rpc/README.md @@ -7039,7 +7039,7 @@ Transaction pool information. The unit is Shannons per 1000 bytes transaction serialization size in the block. -* `min_rbf_rate`: [`Uint64`](#type-uint64) - RBF rate threshold. The pool reject to replace for transactions which fee rate is below this threshold. +* `min_rbf_rate`: [`Uint64`](#type-uint64) - RBF rate threshold. The pool reject to replace for transactions which fee rate is below this threshold. if min_rbf_rate > min_fee_rate then RBF is enabled on the node. The unit is Shannons per 1000 bytes transaction serialization size in the block. diff --git a/util/jsonrpc-types/src/pool.rs b/util/jsonrpc-types/src/pool.rs index a7c7bf4d6b..e13918e857 100644 --- a/util/jsonrpc-types/src/pool.rs +++ b/util/jsonrpc-types/src/pool.rs @@ -41,6 +41,7 @@ pub struct TxPoolInfo { /// The unit is Shannons per 1000 bytes transaction serialization size in the block. pub min_fee_rate: Uint64, /// RBF rate threshold. The pool reject to replace for transactions which fee rate is below this threshold. + /// if min_rbf_rate > min_fee_rate then RBF is enabled on the node. /// /// The unit is Shannons per 1000 bytes transaction serialization size in the block. pub min_rbf_rate: Uint64, diff --git a/util/types/src/core/tx_pool.rs b/util/types/src/core/tx_pool.rs index 75eb6e2790..25dfd70301 100644 --- a/util/types/src/core/tx_pool.rs +++ b/util/types/src/core/tx_pool.rs @@ -321,6 +321,7 @@ pub struct TxPoolInfo { pub min_fee_rate: FeeRate, /// Min RBF rate threshold. The pool reject RBF transactions which fee rate is below this threshold. + /// if min_rbf_rate > min_fee_rate then RBF is enabled on the node. /// /// The unit is Shannons per 1000 bytes transaction serialization size in the block. pub min_rbf_rate: FeeRate, From e1d9125b0a4a502492ea990d2dde147289568af8 Mon Sep 17 00:00:00 2001 From: yukang Date: Wed, 26 Jul 2023 16:30:01 +0800 Subject: [PATCH 37/58] refactor RBF and add fee related info to get_transaction --- rpc/src/module/chain.rs | 3 +- tx-pool/src/pool.rs | 92 +++++++++++++++++++++++----------- tx-pool/src/service.rs | 13 ++--- util/types/src/core/tx_pool.rs | 30 +++++------ 4 files changed, 85 insertions(+), 53 deletions(-) diff --git a/rpc/src/module/chain.rs b/rpc/src/module/chain.rs index 08574e13bb..666f8e0c3a 100644 --- a/rpc/src/module/chain.rs +++ b/rpc/src/module/chain.rs @@ -2130,11 +2130,11 @@ impl ChainRpcImpl { .and_then(|v| v.get(tx_info.index.saturating_sub(1)).copied()) }) }; - return Ok(TransactionWithStatus::with_committed( None, tx_info.block_hash.unpack(), cycles, + None, )); } @@ -2181,6 +2181,7 @@ impl ChainRpcImpl { Some(tx), tx_info.block_hash.unpack(), cycles, + None, )); } diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 40f62212aa..ffcc65c3fd 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -90,6 +90,38 @@ impl TxPool { self.config.min_rbf_rate > self.config.min_fee_rate } + /// The least required fee rate to allow tx to be replaced + pub fn min_replace_fee(&self, tx: &TxEntry) -> Option { + if !self.enable_rbf() { + return None; + } + let conflicts = self.pool_map.find_conflict_tx(tx.transaction()); + self.calculate_min_replace_fee(&conflicts, tx.size) + } + + fn calculate_min_replace_fee( + &self, + conflicts: &HashSet, + size: usize, + ) -> Option { + let entries = conflicts + .iter() + .map(|id| { + self.get_pool_entry(id) + .expect("conflict Tx should be in pool") + }) + .collect::>(); + let min_rbf_fee = self.config.min_rbf_rate.fee(size as u64); + Some( + entries + .iter() + .map(|c| c.inner.fee) + .max() + .unwrap_or(min_rbf_fee) + .max(min_rbf_fee), + ) + } + /// Update size and cycles statics for remove tx /// cycles overflow is possible, currently obtaining cycles is not accurate pub fn update_statics_for_remove_tx(&mut self, tx_size: usize, cycles: Cycle) { @@ -486,7 +518,20 @@ impl TxPool { assert!(!conflicts.is_empty()); let short_id = rtx.transaction.proposal_short_id(); - let entries = conflicts + // Rule #4, new tx' fee need to higher than min_rbf_fee computed from the tx_pool configuration + // Rule #3, new tx's fee need to higher than conflicts, here we only check the root tx + if let Some(min_replace_fee) = self.calculate_min_replace_fee(conflicts, tx_size) { + if fee < min_replace_fee { + return Err(Reject::RBFRejected(format!( + "Tx's current fee is {}, expect it to be larger than: {} to replace old txs", + fee, min_replace_fee, + ))); + } + } else { + panic!("calculate_min_replace_fee must success"); + } + + let pool_entries = conflicts .iter() .map(|id| { self.get_pool_entry(id) @@ -494,18 +539,12 @@ impl TxPool { }) .collect::>(); - // Rule #6, any old Tx should be in `Pending` or `Gap` status - if entries - .iter() - .any(|e| ![Status::Pending, Status::Gap].contains(&e.status)) - { - // Here we only refer to `Pending` status, since `Gap` is an internal status - return Err(Reject::RBFRejected( - "all conflict Txs should be in Pending status".to_string(), - )); - } + let mut all_statuses = pool_entries.iter().map(|e| e.status).collect::>(); - let conflicts = entries.iter().map(|e| e.inner.clone()).collect::>(); + let conflicts = pool_entries + .iter() + .map(|e| e.inner.clone()) + .collect::>(); // Rule #2, new tx don't contain any new unconfirmed inputs let mut inputs = HashSet::new(); @@ -522,23 +561,6 @@ impl TxPool { )); } - // Rule #4, new tx' fee need to higher than min_rbf_fee computed from the tx_pool configuration - // Rule #3, new tx's fee need to higher than conflicts, here we only check the root tx - let min_rbf_fee = self.config.min_rbf_rate.fee(tx_size as u64); - let max_fee = conflicts - .iter() - .map(|c| c.fee) - .max() - .unwrap_or(min_rbf_fee) - .max(min_rbf_fee); - - if fee <= max_fee { - return Err(Reject::RBFRejected(format!( - "Tx's current fee is {}, expect it to be larger than: {} to replace old txs", - fee, max_fee, - ))); - } - // Rule #5, the replaced tx's descendants can not more than 100 // and the ancestor of the new tx don't have common set with the replaced tx's descendants let mut replace_count: usize = 0; @@ -562,6 +584,7 @@ impl TxPool { for id in descendants.iter() { if let Some(entry) = self.get_pool_entry(id) { + all_statuses.push(entry.status); let hash = entry.inner.transaction().hash(); if rtx .transaction @@ -577,6 +600,17 @@ impl TxPool { } } + // Rule #6, any old Tx should be in `Pending` or `Gap` status + if all_statuses + .iter() + .any(|s| ![Status::Pending, Status::Gap].contains(s)) + { + // Here we only refer to `Pending` status, since `Gap` is an internal status + return Err(Reject::RBFRejected( + "all conflict Txs should be in Pending status".to_string(), + )); + } + Ok(()) } diff --git a/tx-pool/src/service.rs b/tx-pool/src/service.rs index 6414f3161f..668a2bb9a9 100644 --- a/tx-pool/src/service.rs +++ b/tx-pool/src/service.rs @@ -779,15 +779,16 @@ async fn process(mut service: TxPoolService, message: Message) { .. }) = tx_pool.pool_map.get_by_id(&id) { - let trans_status = if status == &Status::Proposed { - TransactionWithStatus::with_proposed - } else { - TransactionWithStatus::with_pending - }; - Ok(trans_status( + Ok(TransactionWithStatus::with_status( Some(entry.transaction().clone()), entry.cycles, entry.timestamp, + if status == &Status::Proposed { + TxStatus::Proposed + } else { + TxStatus::Pending + }, + Some(entry.fee), )) } else if let Some(ref recent_reject_db) = tx_pool.recent_reject { match recent_reject_db.get(&hash) { diff --git a/util/types/src/core/tx_pool.rs b/util/types/src/core/tx_pool.rs index 25dfd70301..30b6b00d2b 100644 --- a/util/types/src/core/tx_pool.rs +++ b/util/types/src/core/tx_pool.rs @@ -169,33 +169,24 @@ pub struct TransactionWithStatus { pub tx_status: TxStatus, /// The transaction verification consumed cycles pub cycles: Option, + /// The transaction fee of the transaction + pub fee: Option, /// If the transaction is in tx-pool, `time_added_to_pool` represent when it enter the tx-pool. unit: Millisecond pub time_added_to_pool: Option, } impl TransactionWithStatus { - /// Build with pending status - pub fn with_pending( + /// Build with tx status + pub fn with_status( tx: Option, cycles: core::Cycle, time_added_to_pool: u64, + tx_status: TxStatus, + fee: Option, ) -> Self { Self { - tx_status: TxStatus::Pending, - transaction: tx, - cycles: Some(cycles), - time_added_to_pool: Some(time_added_to_pool), - } - } - - /// Build with proposed status - pub fn with_proposed( - tx: Option, - cycles: core::Cycle, - time_added_to_pool: u64, - ) -> Self { - Self { - tx_status: TxStatus::Proposed, + tx_status, + fee, transaction: tx, cycles: Some(cycles), time_added_to_pool: Some(time_added_to_pool), @@ -207,11 +198,13 @@ impl TransactionWithStatus { tx: Option, hash: H256, cycles: Option, + fee: Option, ) -> Self { Self { tx_status: TxStatus::Committed(hash), transaction: tx, cycles, + fee, time_added_to_pool: None, } } @@ -222,6 +215,7 @@ impl TransactionWithStatus { tx_status: TxStatus::Rejected(reason), transaction: None, cycles: None, + fee: None, time_added_to_pool: None, } } @@ -232,6 +226,7 @@ impl TransactionWithStatus { tx_status: TxStatus::Unknown, transaction: None, cycles: None, + fee: None, time_added_to_pool: None, } } @@ -242,6 +237,7 @@ impl TransactionWithStatus { tx_status, transaction: None, cycles, + fee: None, time_added_to_pool: None, } } From d3a61b4cff8a648b20bd414ce61c5823cb3d77b6 Mon Sep 17 00:00:00 2001 From: yukang Date: Wed, 26 Jul 2023 16:47:38 +0800 Subject: [PATCH 38/58] add min_replace_fee to get_transaction --- rpc/README.md | 4 ++++ tx-pool/src/service.rs | 12 +++++++----- util/jsonrpc-types/src/blockchain.rs | 8 ++++++++ util/types/src/core/tx_pool.rs | 8 ++++++++ 4 files changed, 27 insertions(+), 5 deletions(-) diff --git a/rpc/README.md b/rpc/README.md index f0af4bc71d..48bd5cd76d 100644 --- a/rpc/README.md +++ b/rpc/README.md @@ -6955,6 +6955,10 @@ The JSON view of a transaction as well as its status. * `tx_status`: [`TxStatus`](#type-txstatus) - The Transaction status. +* `fee`: [`Capacity`](#type-capacity) `|` `null` - The transaction fee of the transaction + +* `min_replace_fee`: [`Capacity`](#type-capacity) `|` `null` - The minimal fee required to replace this transaction + ### Type `TxPoolEntries` diff --git a/tx-pool/src/service.rs b/tx-pool/src/service.rs index 668a2bb9a9..c02d68ab57 100644 --- a/tx-pool/src/service.rs +++ b/tx-pool/src/service.rs @@ -779,16 +779,18 @@ async fn process(mut service: TxPoolService, message: Message) { .. }) = tx_pool.pool_map.get_by_id(&id) { + let tx_status = if status == &Status::Proposed { + TxStatus::Proposed + } else { + TxStatus::Pending + }; Ok(TransactionWithStatus::with_status( Some(entry.transaction().clone()), entry.cycles, entry.timestamp, - if status == &Status::Proposed { - TxStatus::Proposed - } else { - TxStatus::Pending - }, + tx_status, Some(entry.fee), + tx_pool.min_replace_fee(entry), )) } else if let Some(ref recent_reject_db) = tx_pool.recent_reject { match recent_reject_db.get(&hash) { diff --git a/util/jsonrpc-types/src/blockchain.rs b/util/jsonrpc-types/src/blockchain.rs index aaa20ffccf..8d2824b445 100644 --- a/util/jsonrpc-types/src/blockchain.rs +++ b/util/jsonrpc-types/src/blockchain.rs @@ -534,6 +534,10 @@ pub struct TransactionWithStatusResponse { pub time_added_to_pool: Option, /// The Transaction status. pub tx_status: TxStatus, + /// The transaction fee of the transaction + pub fee: Option, + /// The minimal fee required to replace this transaction + pub min_replace_fee: Option, } impl TransactionWithStatusResponse { @@ -548,6 +552,8 @@ impl TransactionWithStatusResponse { tx_status: t.tx_status.into(), cycles: t.cycles.map(Into::into), time_added_to_pool: t.time_added_to_pool.map(Into::into), + fee: t.fee.map(Into::into), + min_replace_fee: t.min_replace_fee.map(Into::into), }, ResponseFormatInnerType::Json => TransactionWithStatusResponse { transaction: t @@ -556,6 +562,8 @@ impl TransactionWithStatusResponse { tx_status: t.tx_status.into(), cycles: t.cycles.map(Into::into), time_added_to_pool: t.time_added_to_pool.map(Into::into), + fee: t.fee.map(Into::into), + min_replace_fee: t.min_replace_fee.map(Into::into), }, } } diff --git a/util/types/src/core/tx_pool.rs b/util/types/src/core/tx_pool.rs index 30b6b00d2b..53baefbf72 100644 --- a/util/types/src/core/tx_pool.rs +++ b/util/types/src/core/tx_pool.rs @@ -171,6 +171,8 @@ pub struct TransactionWithStatus { pub cycles: Option, /// The transaction fee of the transaction pub fee: Option, + /// The minimal fee required to replace this transaction + pub min_replace_fee: Option, /// If the transaction is in tx-pool, `time_added_to_pool` represent when it enter the tx-pool. unit: Millisecond pub time_added_to_pool: Option, } @@ -183,10 +185,12 @@ impl TransactionWithStatus { time_added_to_pool: u64, tx_status: TxStatus, fee: Option, + min_replace_fee: Option, ) -> Self { Self { tx_status, fee, + min_replace_fee, transaction: tx, cycles: Some(cycles), time_added_to_pool: Some(time_added_to_pool), @@ -205,6 +209,7 @@ impl TransactionWithStatus { transaction: tx, cycles, fee, + min_replace_fee: None, time_added_to_pool: None, } } @@ -216,6 +221,7 @@ impl TransactionWithStatus { transaction: None, cycles: None, fee: None, + min_replace_fee: None, time_added_to_pool: None, } } @@ -227,6 +233,7 @@ impl TransactionWithStatus { transaction: None, cycles: None, fee: None, + min_replace_fee: None, time_added_to_pool: None, } } @@ -238,6 +245,7 @@ impl TransactionWithStatus { transaction: None, cycles, fee: None, + min_replace_fee: None, time_added_to_pool: None, } } From 339ebab20b4d7c5abb4dd3bff5becd54fd3a5d52 Mon Sep 17 00:00:00 2001 From: yukang Date: Wed, 26 Jul 2023 18:42:15 +0800 Subject: [PATCH 39/58] fix calculate_min_replace_fee, using sum of fee from replaced Tx --- rpc/README.md | 2 ++ rpc/src/module/chain.rs | 2 ++ test/src/specs/tx_pool/replace.rs | 6 +++-- tx-pool/src/pool.rs | 37 +++++++++++++++++++++++++------ 4 files changed, 38 insertions(+), 9 deletions(-) diff --git a/rpc/README.md b/rpc/README.md index 48bd5cd76d..57edcf01a3 100644 --- a/rpc/README.md +++ b/rpc/README.md @@ -882,6 +882,8 @@ Response }, "cycles": "0x219", "time_added_to_pool" : "0x187b3d137a1", + "fee": "0x16923f7dcf", + "min_replace_fee": "0x16923f7f6a", "tx_status": { "block_hash": null, "status": "pending", diff --git a/rpc/src/module/chain.rs b/rpc/src/module/chain.rs index 666f8e0c3a..36e0f4238f 100644 --- a/rpc/src/module/chain.rs +++ b/rpc/src/module/chain.rs @@ -617,6 +617,8 @@ pub trait ChainRpc { /// }, /// "cycles": "0x219", /// "time_added_to_pool" : "0x187b3d137a1", + /// "fee": "0x16923f7dcf", + /// "min_replace_fee": "0x16923f7f6a", /// "tx_status": { /// "block_hash": null, /// "status": "pending", diff --git a/test/src/specs/tx_pool/replace.rs b/test/src/specs/tx_pool/replace.rs index abbfca053c..937f63d32c 100644 --- a/test/src/specs/tx_pool/replace.rs +++ b/test/src/specs/tx_pool/replace.rs @@ -114,7 +114,7 @@ impl Spec for RbfSameInput { pub struct RbfSameInputwithLessFee; -// RBF Rule #3 +// RBF Rule #3, #4 impl Spec for RbfSameInputwithLessFee { fn run(&self, nodes: &mut Vec) { let node0 = &nodes[0]; @@ -148,7 +148,9 @@ impl Spec for RbfSameInputwithLessFee { .send_transaction_result(tx2.data().into()); assert!(res.is_err(), "tx2 should be rejected"); let message = res.err().unwrap().to_string(); - assert!(message.contains("Tx's current fee is 1000000000, expect it to be larger than")); + assert!(message.contains( + "Tx's current fee is 1000000000, expect it to >= 2000000363 to replace old txs" + )); } fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index ffcc65c3fd..4fb5a96dcf 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -96,6 +96,8 @@ impl TxPool { return None; } let conflicts = self.pool_map.find_conflict_tx(tx.transaction()); + // we don't allow conflicted Tx in pool now + assert!(conflicts.len() == 1); self.calculate_min_replace_fee(&conflicts, tx.size) } @@ -111,15 +113,36 @@ impl TxPool { .expect("conflict Tx should be in pool") }) .collect::>(); - let min_rbf_fee = self.config.min_rbf_rate.fee(size as u64); - Some( + + for x in entries.iter() { + eprintln!( + "old tx: {:?} fee: {:?}", + x.inner.transaction().hash(), + x.inner.fee + ); + } + + let extra_rbf_fee = self.config.min_rbf_rate.fee(size as u64); + let replaced_sum_fee: Capacity = entries .iter() .map(|c| c.inner.fee) - .max() - .unwrap_or(min_rbf_fee) - .max(min_rbf_fee), - ) + .fold(Capacity::zero(), |acc, x| { + acc.safe_add(x).unwrap_or_else(|_| { + error!("replaced_sum_fee {} overflow by add {}", acc, x); + Capacity::zero() + }) + }); + let res = replaced_sum_fee.safe_add(extra_rbf_fee); + if let Ok(res) = res { + Some(res) + } else { + error!( + "replaced_sum_fee {} overflow by add {}", + replaced_sum_fee, extra_rbf_fee + ); + None + } } /// Update size and cycles statics for remove tx @@ -523,7 +546,7 @@ impl TxPool { if let Some(min_replace_fee) = self.calculate_min_replace_fee(conflicts, tx_size) { if fee < min_replace_fee { return Err(Reject::RBFRejected(format!( - "Tx's current fee is {}, expect it to be larger than: {} to replace old txs", + "Tx's current fee is {}, expect it to >= {} to replace old txs", fee, min_replace_fee, ))); } From 9d364c67e875f1a239f32f0511e64cbb9e7d531d Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 27 Jul 2023 01:45:33 +0800 Subject: [PATCH 40/58] refactor TxModifiedEntries with MultiIndexMap --- tx-pool/src/component/commit_txs_scanner.rs | 49 +++++++++++---------- 1 file changed, 25 insertions(+), 24 deletions(-) diff --git a/tx-pool/src/component/commit_txs_scanner.rs b/tx-pool/src/component/commit_txs_scanner.rs index 3546aa0b7f..ecb5619e6d 100644 --- a/tx-pool/src/component/commit_txs_scanner.rs +++ b/tx-pool/src/component/commit_txs_scanner.rs @@ -1,44 +1,45 @@ +extern crate slab; use crate::component::pool_map::PoolMap; use crate::component::{entry::TxEntry, sort_key::AncestorsScoreSortKey}; +use ckb_multi_index_map::MultiIndexMap; use ckb_types::{core::Cycle, packed::ProposalShortId}; use ckb_util::LinkedHashMap; -use std::collections::{BTreeSet, HashMap, HashSet}; +use std::collections::HashSet; // A template data struct used to store modified entries when package txs -#[derive(Default)] -pub struct TxModifiedEntries { - entries: HashMap, - sorted_index: BTreeSet, +#[derive(MultiIndexMap, Clone)] +pub struct ModifiedTx { + #[multi_index(hashed_unique)] + pub id: ProposalShortId, + #[multi_index(ordered_non_unique)] + pub score: AncestorsScoreSortKey, + pub inner: TxEntry, } -impl TxModifiedEntries { +impl MultiIndexModifiedTxMap { pub fn next_best_entry(&self) -> Option<&TxEntry> { - self.sorted_index - .iter() - .max() - .map(|key| self.entries.get(&key.id).expect("consistent")) + self.iter_by_score().last().map(|x| &x.inner) } pub fn get(&self, id: &ProposalShortId) -> Option<&TxEntry> { - self.entries.get(id) + self.get_by_id(id).map(|x| &x.inner) } pub fn contains_key(&self, id: &ProposalShortId) -> bool { - self.entries.contains_key(id) + self.get_by_id(id).is_some() } - pub fn insert(&mut self, entry: TxEntry) { - let key = AncestorsScoreSortKey::from(&entry); - let short_id = entry.proposal_short_id(); - self.entries.insert(short_id, entry); - self.sorted_index.insert(key); + pub fn insert_entry(&mut self, entry: TxEntry) { + let score = AncestorsScoreSortKey::from(&entry); + self.insert(ModifiedTx { + id: entry.proposal_short_id(), + score, + inner: entry, + }); } pub fn remove(&mut self, id: &ProposalShortId) -> Option { - self.entries.remove(id).map(|entry| { - self.sorted_index.remove(&(&entry).into()); - entry - }) + self.remove_by_id(id).map(|x| x.inner) } } @@ -53,7 +54,7 @@ pub struct CommitTxsScanner<'a> { entries: Vec, // modified_entries will store sorted packages after they are modified // because some of their txs are already in the block - modified_entries: TxModifiedEntries, + modified_entries: MultiIndexModifiedTxMap, // txs that packaged in block fetched_txs: HashSet, // Keep track of entries that failed inclusion, to avoid duplicate work @@ -65,7 +66,7 @@ impl<'a> CommitTxsScanner<'a> { CommitTxsScanner { entries: Vec::new(), pool_map, - modified_entries: TxModifiedEntries::default(), + modified_entries: MultiIndexModifiedTxMap::default(), fetched_txs: HashSet::default(), failed_txs: HashSet::default(), } @@ -210,7 +211,7 @@ impl<'a> CommitTxsScanner<'a> { .or_else(|| self.pool_map.get(desc_id).cloned()) { desc.sub_ancestor_weight(entry); - self.modified_entries.insert(desc); + self.modified_entries.insert_entry(desc); } } } From 389c30af846034629d87e97a9bc5c08b4e69bd1f Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 27 Jul 2023 02:08:42 +0800 Subject: [PATCH 41/58] remove id from AncestorsScoreSortKey for perf --- tx-pool/src/component/entry.rs | 1 - tx-pool/src/component/sort_key.rs | 12 +---- tx-pool/src/component/tests/score_key.rs | 64 +++++++++++++----------- 3 files changed, 36 insertions(+), 41 deletions(-) diff --git a/tx-pool/src/component/entry.rs b/tx-pool/src/component/entry.rs index 09f9f3c988..f45d4feace 100644 --- a/tx-pool/src/component/entry.rs +++ b/tx-pool/src/component/entry.rs @@ -225,7 +225,6 @@ impl From<&TxEntry> for AncestorsScoreSortKey { AncestorsScoreSortKey { fee: entry.fee, weight, - id: entry.proposal_short_id(), ancestors_fee: entry.ancestors_fee, ancestors_weight, } diff --git a/tx-pool/src/component/sort_key.rs b/tx-pool/src/component/sort_key.rs index 50f2363aff..ceeab649bc 100644 --- a/tx-pool/src/component/sort_key.rs +++ b/tx-pool/src/component/sort_key.rs @@ -1,7 +1,4 @@ -use ckb_types::{ - core::{Capacity, FeeRate}, - packed::ProposalShortId, -}; +use ckb_types::core::{Capacity, FeeRate}; use std::cmp::Ordering; /// A struct to use as a sorted key @@ -9,7 +6,6 @@ use std::cmp::Ordering; pub struct AncestorsScoreSortKey { pub fee: Capacity, pub weight: u64, - pub id: ProposalShortId, pub ancestors_fee: Capacity, pub ancestors_weight: u64, } @@ -44,11 +40,7 @@ impl Ord for AncestorsScoreSortKey { let other_weight = u128::from(other_fee.as_u64()) * u128::from(weight); if self_weight == other_weight { // if fee rate weight is same, then compare with ancestor weight - if self.ancestors_weight == other.ancestors_weight { - self.id.raw_data().cmp(&other.id.raw_data()) - } else { - self.ancestors_weight.cmp(&other.ancestors_weight) - } + self.ancestors_weight.cmp(&other.ancestors_weight) } else { self_weight.cmp(&other_weight) } diff --git a/tx-pool/src/component/tests/score_key.rs b/tx-pool/src/component/tests/score_key.rs index 94aa51cf72..c12cc7426c 100644 --- a/tx-pool/src/component/tests/score_key.rs +++ b/tx-pool/src/component/tests/score_key.rs @@ -1,10 +1,9 @@ use ckb_types::{ bytes::Bytes, core::{Capacity, TransactionBuilder}, - packed::{CellInput, OutPoint, ProposalShortId}, + packed::{CellInput, OutPoint}, prelude::*, }; -use std::mem::size_of; use crate::component::{entry::TxEntry, pool_map::PoolMap, sort_key::AncestorsScoreSortKey}; @@ -27,7 +26,6 @@ fn test_min_fee_and_weight() { let key = AncestorsScoreSortKey { fee: Capacity::shannons(fee), weight, - id: ProposalShortId::new([0u8; 10]), ancestors_fee: Capacity::shannons(ancestors_fee), ancestors_weight, }; @@ -51,7 +49,7 @@ fn test_min_fee_and_weight() { #[test] fn test_ancestors_sorted_key_order() { - let mut keys = vec![ + let table = vec![ (0, 0, 0, 0), (1, 0, 1, 0), (500, 10, 1000, 30), @@ -62,33 +60,39 @@ fn test_ancestors_sorted_key_order() { (std::u64::MAX, 0, std::u64::MAX, 0), (std::u64::MAX, 100, std::u64::MAX, 2000), (std::u64::MAX, std::u64::MAX, std::u64::MAX, std::u64::MAX), - ] - .into_iter() - .enumerate() - .map(|(i, (fee, weight, ancestors_fee, ancestors_weight))| { - let mut id = [0u8; 10]; - id[..size_of::()].copy_from_slice(&(i as u32).to_be_bytes()); - AncestorsScoreSortKey { - fee: Capacity::shannons(fee), - weight, - id: ProposalShortId::new(id), - ancestors_fee: Capacity::shannons(ancestors_fee), - ancestors_weight, - } - }) - .collect::>(); + ]; + let mut keys = table + .clone() + .into_iter() + .enumerate() + .map( + |(_i, (fee, weight, ancestors_fee, ancestors_weight))| AncestorsScoreSortKey { + fee: Capacity::shannons(fee), + weight, + ancestors_fee: Capacity::shannons(ancestors_fee), + ancestors_weight, + }, + ) + .collect::>(); keys.sort(); - assert_eq!( - keys.into_iter().map(|k| k.id).collect::>(), - [0, 3, 5, 9, 2, 4, 6, 8, 1, 7] - .iter() - .map(|&i| { - let mut id = [0u8; 10]; - id[..size_of::()].copy_from_slice(&(i as u32).to_be_bytes()); - ProposalShortId::new(id) - }) - .collect::>() - ); + let now = keys + .into_iter() + .map(|k| (k.fee, k.weight, k.ancestors_fee, k.ancestors_weight)) + .collect::>(); + let expect = [0, 3, 5, 9, 2, 4, 6, 8, 1, 7] + .iter() + .map(|&i| { + let key = table[i as usize]; + ( + Capacity::shannons(key.0), + key.1, + Capacity::shannons(key.2), + key.3, + ) + }) + .collect::>(); + + assert_eq!(now, expect); } #[test] From 279c3b74b42b72dc74bb88c8761bf413da094745 Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 27 Jul 2023 14:57:13 +0800 Subject: [PATCH 42/58] Fix typo --- rpc/README.md | 2 +- rpc/src/error.rs | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/rpc/README.md b/rpc/README.md index 57edcf01a3..0f1b0020fd 100644 --- a/rpc/README.md +++ b/rpc/README.md @@ -5075,7 +5075,7 @@ For example, a cellbase transaction is not allowed in `send_transaction` RPC. (-1110): The transaction exceeded maximum size limit. -### Error `PoolRejctedRBF` +### Error `PoolRejectedRBF` (-1111): The transaction is rejected for RBF checking. diff --git a/rpc/src/error.rs b/rpc/src/error.rs index b067fbf9d0..e566e96eb2 100644 --- a/rpc/src/error.rs +++ b/rpc/src/error.rs @@ -113,7 +113,7 @@ pub enum RPCError { /// (-1110): The transaction exceeded maximum size limit. PoolRejectedTransactionBySizeLimit = -1110, /// (-1111): The transaction is rejected for RBF checking. - PoolRejctedRBF = -1111, + PoolRejectedRBF = -1111, /// (-1200): The indexer error. Indexer = -1200, } @@ -175,7 +175,7 @@ impl RPCError { Reject::DeclaredWrongCycles(..) => RPCError::PoolRejectedMalformedTransaction, Reject::Resolve(_) => RPCError::TransactionFailedToResolve, Reject::Verification(_) => RPCError::TransactionFailedToVerify, - Reject::RBFRejected(_) => RPCError::PoolRejctedRBF, + Reject::RBFRejected(_) => RPCError::PoolRejectedRBF, Reject::ExceededTransactionSizeLimit(_, _) => { RPCError::PoolRejectedTransactionBySizeLimit } From 14e5cbf8b4ca58fc2b0ed345df238f4f6020b0e4 Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 27 Jul 2023 17:51:46 +0800 Subject: [PATCH 43/58] Proposed tx will not get min_replace_fee --- tx-pool/src/service.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tx-pool/src/service.rs b/tx-pool/src/service.rs index c02d68ab57..9df681f071 100644 --- a/tx-pool/src/service.rs +++ b/tx-pool/src/service.rs @@ -779,10 +779,10 @@ async fn process(mut service: TxPoolService, message: Message) { .. }) = tx_pool.pool_map.get_by_id(&id) { - let tx_status = if status == &Status::Proposed { - TxStatus::Proposed + let (tx_status, min_replace_fee) = if status == &Status::Proposed { + (TxStatus::Proposed, None) } else { - TxStatus::Pending + (TxStatus::Pending, tx_pool.min_replace_fee(entry)) }; Ok(TransactionWithStatus::with_status( Some(entry.transaction().clone()), @@ -790,7 +790,7 @@ async fn process(mut service: TxPoolService, message: Message) { entry.timestamp, tx_status, Some(entry.fee), - tx_pool.min_replace_fee(entry), + min_replace_fee, )) } else if let Some(ref recent_reject_db) = tx_pool.recent_reject { match recent_reject_db.get(&hash) { From 446b0bdd098d49ffa04dd9d141e09f1e22529ee0 Mon Sep 17 00:00:00 2001 From: yukang Date: Fri, 28 Jul 2023 11:06:45 +0800 Subject: [PATCH 44/58] add changelog for tx-pool refactor and RBF --- CHANGELOG.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 1675622984..45cb473eaf 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,16 @@ +### Features +* #4079, **tx-pol:** Implement `Replace-by-Fee(RBF)` for tx-pool (@chenyukang) +This feature enables users to replace a transaction with a higher fee rate, which is useful when the transaction is stuck in the tx-pool: + * Add `min_rbf_rate` in `ckb.toml` with default value `1500`, which means the minimum extra fee rate for RBF, the unit is `shannons/KB` + * Add fields `fee` and `min_replace_fee` in `get_transaction`, which means the the minimal fee need to pay for RBF for a specific transaction + * The replaced transaction will be removed from `tx-pool` and with the status `Rejected`. + +### Improvements +* #3993, **tx-pool:** Almost reimplemented `tx-pool` with `multi_index_map`, with the following improvements (@chenyukang): + * Sort txs in pool by `score` in `Pending` stage, `txs` with higher `score` be processed first + * Evict `txs` from pool with `descendants_count` and `fee_rate` + * Eliminate redundant code for clean and consistent code + # [v0.110.0](https://github.com/nervosnetwork/ckb/compare/v0.109.0...v0.110.0) (2023-05-15) ### Features From 396584fccbd7eedb77623bbdc25e67be9c8194ed Mon Sep 17 00:00:00 2001 From: yukang Date: Tue, 1 Aug 2023 11:48:12 +0800 Subject: [PATCH 45/58] use proper name for tx-pool func --- tx-pool/src/component/pool_map.rs | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index b7ccec0034..842861e49b 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -167,9 +167,9 @@ impl PoolMap { return Ok(false); } trace!("pool_map.add_{:?} {}", status, entry.transaction().hash()); - self.check_record_ancestors(&mut entry)?; + self.check_and_record_ancestors(&mut entry)?; self.insert_entry(&entry, status); - self.record_entry_deps(&entry); + self.record_entry_edges(&entry); self.record_entry_descendants(&entry); Ok(true) } @@ -187,7 +187,6 @@ impl PoolMap { self.entries.remove_by_id(id).map(|entry| { self.update_ancestors_index_key(&entry.inner, EntryOp::Remove); self.update_descendants_index_key(&entry.inner, EntryOp::Remove); - self.remove_entry_deps(&entry.inner); self.remove_entry_edges(&entry.inner); self.remove_entry_links(id); entry.inner @@ -366,7 +365,7 @@ impl PoolMap { } } - fn record_entry_deps(&mut self, entry: &TxEntry) { + fn record_entry_edges(&mut self, entry: &TxEntry) { let tx_short_id: ProposalShortId = entry.proposal_short_id(); let header_deps = entry.transaction().header_deps(); let related_dep_out_points: Vec<_> = entry.related_dep_out_points().cloned().collect(); @@ -419,7 +418,7 @@ impl PoolMap { } /// Check ancestors and record for entry - fn check_record_ancestors(&mut self, entry: &mut TxEntry) -> Result { + fn check_and_record_ancestors(&mut self, entry: &mut TxEntry) -> Result { let mut parents: HashSet = HashSet::with_capacity( entry.transaction().inputs().len() + entry.transaction().cell_deps().len(), ); @@ -480,9 +479,7 @@ impl PoolMap { // release input record self.edges.remove_input(&i); } - } - fn remove_entry_deps(&mut self, entry: &TxEntry) { let id = entry.proposal_short_id(); for d in entry.related_dep_out_points().cloned() { self.edges.delete_txid_by_dep(d, &id); From 792ef8704ff7c15e844f18e119419d8f605d5d9e Mon Sep 17 00:00:00 2001 From: yukang Date: Tue, 1 Aug 2023 17:19:20 +0800 Subject: [PATCH 46/58] code cleanup to remove unwrap --- tx-pool/src/component/pool_map.rs | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index 842861e49b..ca2e224f69 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -110,6 +110,14 @@ impl PoolMap { self.entries.get_by_id(id) } + fn get_by_id_checked(&self, id: &ProposalShortId) -> &PoolEntry { + self.get_by_id(id).expect("unconsistent pool") + } + + fn get_by_id_checked(&self, id: &ProposalShortId) -> &PoolEntry { + self.get_by_id(id).expect("unconsistent pool") + } + pub(crate) fn get_by_status(&self, status: Status) -> Vec<&PoolEntry> { self.entries.get_by_status(&status) } @@ -450,10 +458,7 @@ impl PoolMap { // update parents references for ancestor_id in &ancestors { - let ancestor = self - .entries - .get_by_id(ancestor_id) - .expect("pool consistent"); + let ancestor = self.get_by_id_checked(ancestor_id); entry.add_ancestor_weight(&ancestor.inner); } if entry.ancestors_count > self.max_ancestors_count { From 96ae57b6c9d9530e884ac424de74258200c5027e Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 3 Aug 2023 11:28:12 +0800 Subject: [PATCH 47/58] add comments for PreCheckedTx --- tx-pool/src/pool.rs | 2 +- tx-pool/src/process.rs | 12 +++++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 4fb5a96dcf..f01c47a94d 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -541,7 +541,7 @@ impl TxPool { assert!(!conflicts.is_empty()); let short_id = rtx.transaction.proposal_short_id(); - // Rule #4, new tx' fee need to higher than min_rbf_fee computed from the tx_pool configuration + // Rule #4, new tx's fee need to higher than min_rbf_fee computed from the tx_pool configuration // Rule #3, new tx's fee need to higher than conflicts, here we only check the root tx if let Some(min_replace_fee) = self.calculate_min_replace_fee(conflicts, tx_size) { if fee < min_replace_fee { diff --git a/tx-pool/src/process.rs b/tx-pool/src/process.rs index 1ec8d1922b..ee0ed572aa 100644 --- a/tx-pool/src/process.rs +++ b/tx-pool/src/process.rs @@ -1002,11 +1002,13 @@ impl TxPoolService { } type PreCheckedTx = ( - Byte32, - Arc, - TxStatus, - Capacity, - usize, + Byte32, // tip_hash + Arc, // rtx + TxStatus, // status + Capacity, // tx fee + usize, // tx size + // the conflicted txs, used for latter `check_rbf` + // the root txs for removing from `tx-pool` when RBF is checked HashSet, ); From df06351e5d29bde4def8b7f650e6705634ba5097 Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 3 Aug 2023 12:09:10 +0800 Subject: [PATCH 48/58] return proper error when calculate_min_replace_fee failed --- tx-pool/src/pool.rs | 36 ++++++++++++++---------------------- tx-pool/src/process.rs | 1 - 2 files changed, 14 insertions(+), 23 deletions(-) diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index f01c47a94d..0b88cb4daf 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -11,6 +11,7 @@ use ckb_app_config::TxPoolConfig; use ckb_logger::{debug, error, warn}; use ckb_snapshot::Snapshot; use ckb_store::ChainStore; +use ckb_types::core::CapacityError; use ckb_types::{ core::{ cell::{resolve_transaction, OverlayCellChecker, OverlayCellProvider, ResolvedTransaction}, @@ -114,32 +115,21 @@ impl TxPool { }) .collect::>(); - for x in entries.iter() { - eprintln!( - "old tx: {:?} fee: {:?}", - x.inner.transaction().hash(), - x.inner.fee - ); - } - let extra_rbf_fee = self.config.min_rbf_rate.fee(size as u64); - let replaced_sum_fee: Capacity = - entries - .iter() - .map(|c| c.inner.fee) - .fold(Capacity::zero(), |acc, x| { - acc.safe_add(x).unwrap_or_else(|_| { - error!("replaced_sum_fee {} overflow by add {}", acc, x); - Capacity::zero() - }) - }); - let res = replaced_sum_fee.safe_add(extra_rbf_fee); + let replaced_sum_fee = entries + .iter() + .map(|c| c.inner.fee) + .try_fold(Capacity::zero(), |acc, x| acc.safe_add(x)); + let res = replaced_sum_fee.map_or(Err(CapacityError::Overflow), |sum| { + sum.safe_add(extra_rbf_fee) + }); if let Ok(res) = res { Some(res) } else { + let fees = entries.iter().map(|c| c.inner.fee).collect::>(); error!( - "replaced_sum_fee {} overflow by add {}", - replaced_sum_fee, extra_rbf_fee + "conflicts: {:?} replaced_sum_fee {:?} overflow by add {}", + conflicts, fees, extra_rbf_fee ); None } @@ -551,7 +541,9 @@ impl TxPool { ))); } } else { - panic!("calculate_min_replace_fee must success"); + return Err(Reject::RBFRejected( + "calculate_min_replace_fee failed".to_string(), + )); } let pool_entries = conflicts diff --git a/tx-pool/src/process.rs b/tx-pool/src/process.rs index ee0ed572aa..60f550e35d 100644 --- a/tx-pool/src/process.rs +++ b/tx-pool/src/process.rs @@ -134,7 +134,6 @@ impl TxPoolService { )); // remove old tx from tx_pool, not happened in service so we didn't call reject callbacks // here we call them manually - // TODO: how to call reject notify like service? self.callbacks.call_reject(tx_pool, &old, reject) } } From b4aec5aba3306b141bd9aaab5ca754c91a79a121 Mon Sep 17 00:00:00 2001 From: yukang Date: Fri, 4 Aug 2023 11:14:09 +0800 Subject: [PATCH 49/58] use upstream multi_index_map --- Cargo.lock | 41 +++++++++++++-------- tx-pool/Cargo.toml | 2 +- tx-pool/src/component/commit_txs_scanner.rs | 2 +- tx-pool/src/component/pool_map.rs | 9 ++--- 4 files changed, 30 insertions(+), 24 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8e42a97bf3..b83e58c5f8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1423,9 +1423,9 @@ dependencies = [ "ckb-types", "ckb-util", "ckb-verification", - "ckb_multi_index_map", "hyper", "lru", + "multi_index_map", "rand 0.8.5", "rustc-hash", "sentry", @@ -1553,21 +1553,6 @@ dependencies = [ "paste", ] -[[package]] -name = "ckb_multi_index_map" -version = "0.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53c20823dfd9f9a8e30faa3b0bdcab4801fb2544957586fada3884c78dcdf38b" -dependencies = [ - "convert_case 0.6.0", - "proc-macro-error", - "proc-macro2", - "quote", - "rustc-hash", - "slab", - "syn", -] - [[package]] name = "clang-sys" version = "1.3.1" @@ -3143,6 +3128,30 @@ dependencies = [ "faster-hex", ] +[[package]] +name = "multi_index_map" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "03f409d5f41e6b8a2faa0b363c4523742f0ef5e4015fd4e298a5c7dbb3a3e01c" +dependencies = [ + "multi_index_map_derive", + "rustc-hash", + "slab", +] + +[[package]] +name = "multi_index_map_derive" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "98e81cd436463efbaa95a2d2bac3028e6998a4bb2ef8a82a661de3567bb79d5a" +dependencies = [ + "convert_case 0.6.0", + "proc-macro-error", + "proc-macro2", + "quote", + "syn", +] + [[package]] name = "native-tls" version = "0.2.11" diff --git a/tx-pool/Cargo.toml b/tx-pool/Cargo.toml index 04e465561a..c2c6c2ec67 100644 --- a/tx-pool/Cargo.toml +++ b/tx-pool/Cargo.toml @@ -36,7 +36,7 @@ sentry = { version = "0.26.0", optional = true } serde_json = "1.0" rand = "0.8.4" hyper = { version = "0.14", features = ["http1", "client", "tcp"] } -ckb_multi_index_map = "0.0.2" # ckb team fork crate +multi_index_map = "0.6.0" slab = "0.4" rustc-hash = "1.1" tokio-util = "0.7.8" diff --git a/tx-pool/src/component/commit_txs_scanner.rs b/tx-pool/src/component/commit_txs_scanner.rs index ecb5619e6d..7795a711c4 100644 --- a/tx-pool/src/component/commit_txs_scanner.rs +++ b/tx-pool/src/component/commit_txs_scanner.rs @@ -1,9 +1,9 @@ extern crate slab; use crate::component::pool_map::PoolMap; use crate::component::{entry::TxEntry, sort_key::AncestorsScoreSortKey}; -use ckb_multi_index_map::MultiIndexMap; use ckb_types::{core::Cycle, packed::ProposalShortId}; use ckb_util::LinkedHashMap; +use multi_index_map::MultiIndexMap; use std::collections::HashSet; // A template data struct used to store modified entries when package txs diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index ca2e224f69..2480cf63ae 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -6,8 +6,8 @@ use crate::component::links::{Relation, TxLinksMap}; use crate::component::sort_key::{AncestorsScoreSortKey, EvictKey}; use crate::error::Reject; use crate::TxEntry; -use ckb_logger::{debug, trace}; -use ckb_multi_index_map::MultiIndexMap; + +use ckb_logger::trace; use ckb_types::core::error::OutPointError; use ckb_types::packed::OutPoint; use ckb_types::prelude::*; @@ -16,6 +16,7 @@ use ckb_types::{ core::TransactionView, packed::{Byte32, CellOutput, ProposalShortId}, }; +use multi_index_map::MultiIndexMap; use std::collections::HashSet; use super::links::TxLinks; @@ -114,10 +115,6 @@ impl PoolMap { self.get_by_id(id).expect("unconsistent pool") } - fn get_by_id_checked(&self, id: &ProposalShortId) -> &PoolEntry { - self.get_by_id(id).expect("unconsistent pool") - } - pub(crate) fn get_by_status(&self, status: Status) -> Vec<&PoolEntry> { self.entries.get_by_status(&status) } From c902bc2280599947cb2c709863f51e227abdffd1 Mon Sep 17 00:00:00 2001 From: yukang Date: Mon, 7 Aug 2023 20:58:04 +0800 Subject: [PATCH 50/58] fix fmt --- Cargo.lock | 17 +---------------- tx-pool/src/component/pool_map.rs | 1 - tx-pool/src/component/tests/entry.rs | 3 +-- 3 files changed, 2 insertions(+), 19 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8cc79b7eab..b83e58c5f8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1423,9 +1423,9 @@ dependencies = [ "ckb-types", "ckb-util", "ckb-verification", - "ckb_multi_index_map", "hyper", "lru", + "multi_index_map", "rand 0.8.5", "rustc-hash", "sentry", @@ -1553,21 +1553,6 @@ dependencies = [ "paste", ] -[[package]] -name = "ckb_multi_index_map" -version = "0.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53c20823dfd9f9a8e30faa3b0bdcab4801fb2544957586fada3884c78dcdf38b" -dependencies = [ - "convert_case 0.6.0", - "proc-macro-error", - "proc-macro2", - "quote", - "rustc-hash", - "slab", - "syn", -] - [[package]] name = "clang-sys" version = "1.3.1" diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index 06cd88bf60..edf1d2d0b3 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -502,4 +502,3 @@ impl PoolMap { }); } } - diff --git a/tx-pool/src/component/tests/entry.rs b/tx-pool/src/component/tests/entry.rs index 77a96bb8f9..6ae6708af2 100644 --- a/tx-pool/src/component/tests/entry.rs +++ b/tx-pool/src/component/tests/entry.rs @@ -1,6 +1,5 @@ -use ckb_types::core::{Capacity, FeeRate}; use crate::component::sort_key::EvictKey; - +use ckb_types::core::{Capacity, FeeRate}; #[test] fn test_min_fee_and_weight_evict() { From 21f51591c4ada88ba345924b7339aebd3f2c280c Mon Sep 17 00:00:00 2001 From: yukang Date: Thu, 10 Aug 2023 13:04:16 +0800 Subject: [PATCH 51/58] use hashed_non_unique for status in entry --- tx-pool/src/component/pool_map.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index ce2c05bb0c..238c63ad83 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -45,7 +45,7 @@ pub struct PoolEntry { pub id: ProposalShortId, #[multi_index(ordered_non_unique)] pub score: AncestorsScoreSortKey, - #[multi_index(ordered_non_unique)] + #[multi_index(hashed_non_unique)] pub status: Status, #[multi_index(ordered_non_unique)] pub evict_key: EvictKey, From 05e2f8a90bb1ea8015991f98ed501ca92817d713 Mon Sep 17 00:00:00 2001 From: yukang Date: Fri, 11 Aug 2023 21:36:12 +0800 Subject: [PATCH 52/58] only check_rbf for resolve dead outpoint --- test/src/main.rs | 1 + test/src/specs/tx_pool/replace.rs | 39 +++++++++++++++++++++++++++++-- tx-pool/src/process.rs | 5 +++- 3 files changed, 42 insertions(+), 3 deletions(-) diff --git a/test/src/main.rs b/test/src/main.rs index 4e9d0fbc5c..0658c18d01 100644 --- a/test/src/main.rs +++ b/test/src/main.rs @@ -461,6 +461,7 @@ fn all_specs() -> Vec> { Box::new(DifferentTxsWithSameInputWithOutRBF), Box::new(RbfBasic), Box::new(RbfSameInput), + Box::new(RbfOnlyForResolveDead), Box::new(RbfSameInputwithLessFee), Box::new(RbfTooManyDescendants), Box::new(RbfContainNewTx), diff --git a/test/src/specs/tx_pool/replace.rs b/test/src/specs/tx_pool/replace.rs index 937f63d32c..31ab21efd6 100644 --- a/test/src/specs/tx_pool/replace.rs +++ b/test/src/specs/tx_pool/replace.rs @@ -3,8 +3,7 @@ use ckb_jsonrpc_types::Status; use ckb_logger::info; use ckb_types::{ core::{capacity_bytes, Capacity, TransactionView}, - packed::CellOutputBuilder, - packed::{CellInput, OutPoint}, + packed::{Byte32, CellInput, CellOutputBuilder, OutPoint}, prelude::*, }; @@ -112,6 +111,42 @@ impl Spec for RbfSameInput { } } +pub struct RbfOnlyForResolveDead; +impl Spec for RbfOnlyForResolveDead { + fn run(&self, nodes: &mut Vec) { + let node0 = &nodes[0]; + + node0.mine_until_out_bootstrap_period(); + node0.new_block_with_blocking(|template| template.number.value() != 13); + + let tx_hash_0 = node0.generate_transaction(); + + let tx1 = node0.new_transaction(tx_hash_0.clone()); + let tx1_clone = tx1.clone(); + + // This is an unknown input + let tx_hash_1 = Byte32::zero(); + let tx2 = tx1_clone + .as_advanced_builder() + .set_inputs(vec![{ + CellInput::new_builder() + .previous_output(OutPoint::new(tx_hash_1, 0)) + .build() + }]) + .build(); + + let res = node0 + .rpc_client() + .send_transaction_result(tx2.data().into()); + let message = res.err().unwrap().to_string(); + assert!(message.contains("TransactionFailedToResolve: Resolve failed Unknown")); + } + + fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { + config.tx_pool.min_rbf_rate = ckb_types::core::FeeRate(1500); + } +} + pub struct RbfSameInputwithLessFee; // RBF Rule #3, #4 diff --git a/tx-pool/src/process.rs b/tx-pool/src/process.rs index 60f550e35d..c386bd0524 100644 --- a/tx-pool/src/process.rs +++ b/tx-pool/src/process.rs @@ -19,6 +19,7 @@ use ckb_network::PeerIndex; use ckb_snapshot::Snapshot; use ckb_store::data_loader_wrapper::AsDataLoader; use ckb_store::ChainStore; +use ckb_types::core::error::OutPointError; use ckb_types::{ core::{cell::ResolvedTransaction, BlockView, Capacity, Cycle, HeaderView, TransactionView}, packed::{Byte32, ProposalShortId}, @@ -229,7 +230,9 @@ impl TxPoolService { Ok((tip_hash, rtx, status, fee, tx_size, HashSet::new())) } Err(err) => { - if tx_pool.enable_rbf() && matches!(err, Reject::Resolve(_)) { + if tx_pool.enable_rbf() + && matches!(err, Reject::Resolve(OutPointError::Dead(_))) + { // Try RBF check let conflicts = tx_pool.pool_map.find_conflict_tx(tx); if conflicts.is_empty() { From bc7e613c4534eaee943e4e6d2f2a97b3780a6cd2 Mon Sep 17 00:00:00 2001 From: yukang Date: Fri, 11 Aug 2023 21:56:21 +0800 Subject: [PATCH 53/58] cleanup find_conflict_tx --- test/src/specs/tx_pool/replace.rs | 5 ++--- tx-pool/src/component/pool_map.rs | 10 +++------- tx-pool/src/pool.rs | 6 ++---- 3 files changed, 7 insertions(+), 14 deletions(-) diff --git a/test/src/specs/tx_pool/replace.rs b/test/src/specs/tx_pool/replace.rs index 31ab21efd6..0bb4182d0f 100644 --- a/test/src/specs/tx_pool/replace.rs +++ b/test/src/specs/tx_pool/replace.rs @@ -121,12 +121,11 @@ impl Spec for RbfOnlyForResolveDead { let tx_hash_0 = node0.generate_transaction(); - let tx1 = node0.new_transaction(tx_hash_0.clone()); - let tx1_clone = tx1.clone(); + let tx1 = node0.new_transaction(tx_hash_0); // This is an unknown input let tx_hash_1 = Byte32::zero(); - let tx2 = tx1_clone + let tx2 = tx1 .as_advanced_builder() .set_inputs(vec![{ CellInput::new_builder() diff --git a/tx-pool/src/component/pool_map.rs b/tx-pool/src/component/pool_map.rs index edf1d2d0b3..acbf53858d 100644 --- a/tx-pool/src/component/pool_map.rs +++ b/tx-pool/src/component/pool_map.rs @@ -241,13 +241,9 @@ impl PoolMap { } pub(crate) fn find_conflict_tx(&self, tx: &TransactionView) -> HashSet { - let mut res = HashSet::default(); - for i in tx.input_pts_iter() { - if let Some(id) = self.edges.get_input_ref(&i) { - res.insert(id.clone()); - } - } - res + tx.input_pts_iter() + .filter_map(|out_point| self.edges.get_input_ref(&out_point).cloned()) + .collect() } pub(crate) fn resolve_conflict(&mut self, tx: &TransactionView) -> Vec { diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 757b1c541b..2125d81460 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -97,10 +97,8 @@ impl TxPool { if !self.enable_rbf() { return None; } - let conflicts = self.pool_map.find_conflict_tx(tx.transaction()); - // we don't allow conflicted Tx in pool now - assert!(conflicts.len() == 1); - self.calculate_min_replace_fee(&conflicts, tx.size) + let ids = vec![tx.proposal_short_id()].iter().cloned().collect(); + self.calculate_min_replace_fee(&ids, tx.size) } fn calculate_min_replace_fee( From 9e8dc8ba56b045f13e593166f3b49058c985a7f1 Mon Sep 17 00:00:00 2001 From: yukang Date: Mon, 14 Aug 2023 11:22:32 +0800 Subject: [PATCH 54/58] fix comments --- tx-pool/src/pool.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 2125d81460..82bf11948f 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -614,7 +614,7 @@ impl TxPool { } } - // Rule #6, any old Tx should be in `Pending` or `Gap` status + // Rule #6, all conflict Txs should be in `Pending` or `Gap` status if all_statuses .iter() .any(|s| ![Status::Pending, Status::Gap].contains(s)) From 4512dc8b448478cfb1b57553140086d2dda4a63d Mon Sep 17 00:00:00 2001 From: yukang Date: Mon, 14 Aug 2023 11:37:59 +0800 Subject: [PATCH 55/58] remove unnecessary clone in rbf --- tx-pool/src/pool.rs | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 82bf11948f..59f3147cf9 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -555,16 +555,12 @@ impl TxPool { let mut all_statuses = pool_entries.iter().map(|e| e.status).collect::>(); - let conflicts = pool_entries - .iter() - .map(|e| e.inner.clone()) - .collect::>(); - // Rule #2, new tx don't contain any new unconfirmed inputs let mut inputs = HashSet::new(); - for c in conflicts.iter() { - inputs.extend(c.transaction().input_pts_iter()); + for c in pool_entries.iter() { + inputs.extend(c.inner.transaction().input_pts_iter()); } + if rtx .transaction .input_pts_iter() @@ -579,8 +575,8 @@ impl TxPool { // and the ancestor of the new tx don't have common set with the replaced tx's descendants let mut replace_count: usize = 0; let ancestors = self.pool_map.calc_ancestors(&short_id); - for conflict in conflicts.iter() { - let id = conflict.proposal_short_id(); + for conflict in pool_entries.iter() { + let id = conflict.inner.proposal_short_id(); let descendants = self.pool_map.calc_descendants(&id); replace_count += descendants.len() + 1; if replace_count > MAX_REPLACEMENT_CANDIDATES { From 95e3596c3943ec84116e6d2be9146e927eb89c7b Mon Sep 17 00:00:00 2001 From: yukang Date: Mon, 14 Aug 2023 16:18:33 +0800 Subject: [PATCH 56/58] refactor rbf check --- tx-pool/src/pool.rs | 107 ++++++++++++++++++++------------------------ 1 file changed, 48 insertions(+), 59 deletions(-) diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index 59f3147cf9..caa05c12d5 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -97,25 +97,14 @@ impl TxPool { if !self.enable_rbf() { return None; } - let ids = vec![tx.proposal_short_id()].iter().cloned().collect(); - self.calculate_min_replace_fee(&ids, tx.size) + let entry = vec![self.get_pool_entry(&tx.proposal_short_id()).unwrap()]; + self.calculate_min_replace_fee(&entry, tx.size) } - fn calculate_min_replace_fee( - &self, - conflicts: &HashSet, - size: usize, - ) -> Option { - let entries = conflicts - .iter() - .map(|id| { - self.get_pool_entry(id) - .expect("conflict Tx should be in pool") - }) - .collect::>(); - + /// min_replace_fee = sum(replaced_txs.fee) + extra_rbf_fee + fn calculate_min_replace_fee(&self, conflicts: &[&PoolEntry], size: usize) -> Option { let extra_rbf_fee = self.config.min_rbf_rate.fee(size as u64); - let replaced_sum_fee = entries + let replaced_sum_fee = conflicts .iter() .map(|c| c.inner.fee) .try_fold(Capacity::zero(), |acc, x| acc.safe_add(x)); @@ -125,10 +114,12 @@ impl TxPool { if let Ok(res) = res { Some(res) } else { - let fees = entries.iter().map(|c| c.inner.fee).collect::>(); + let fees = conflicts.iter().map(|c| c.inner.fee).collect::>(); error!( "conflicts: {:?} replaced_sum_fee {:?} overflow by add {}", - conflicts, fees, extra_rbf_fee + conflicts.iter().map(|e| e.id.clone()).collect::>(), + fees, + extra_rbf_fee ); None } @@ -522,17 +513,23 @@ impl TxPool { &self, snapshot: &Snapshot, rtx: &ResolvedTransaction, - conflicts: &HashSet, + conflict_ids: &HashSet, fee: Capacity, tx_size: usize, ) -> Result<(), Reject> { assert!(self.enable_rbf()); - assert!(!conflicts.is_empty()); + assert!(!conflict_ids.is_empty()); + + let conflicts = conflict_ids + .iter() + .filter_map(|id| self.get_pool_entry(id)) + .collect::>(); + assert!(conflicts.len() == conflict_ids.len()); let short_id = rtx.transaction.proposal_short_id(); // Rule #4, new tx's fee need to higher than min_rbf_fee computed from the tx_pool configuration // Rule #3, new tx's fee need to higher than conflicts, here we only check the root tx - if let Some(min_replace_fee) = self.calculate_min_replace_fee(conflicts, tx_size) { + if let Some(min_replace_fee) = self.calculate_min_replace_fee(&conflicts, tx_size) { if fee < min_replace_fee { return Err(Reject::RBFRejected(format!( "Tx's current fee is {}, expect it to >= {} to replace old txs", @@ -545,19 +542,9 @@ impl TxPool { )); } - let pool_entries = conflicts - .iter() - .map(|id| { - self.get_pool_entry(id) - .expect("conflict Tx should be in pool") - }) - .collect::>(); - - let mut all_statuses = pool_entries.iter().map(|e| e.status).collect::>(); - // Rule #2, new tx don't contain any new unconfirmed inputs let mut inputs = HashSet::new(); - for c in pool_entries.iter() { + for c in conflicts.iter() { inputs.extend(c.inner.transaction().input_pts_iter()); } @@ -575,9 +562,8 @@ impl TxPool { // and the ancestor of the new tx don't have common set with the replaced tx's descendants let mut replace_count: usize = 0; let ancestors = self.pool_map.calc_ancestors(&short_id); - for conflict in pool_entries.iter() { - let id = conflict.inner.proposal_short_id(); - let descendants = self.pool_map.calc_descendants(&id); + for conflict in conflicts.iter() { + let descendants = self.pool_map.calc_descendants(&conflict.id); replace_count += descendants.len() + 1; if replace_count > MAX_REPLACEMENT_CANDIDATES { return Err(Reject::RBFRejected(format!( @@ -592,33 +578,36 @@ impl TxPool { )); } - for id in descendants.iter() { - if let Some(entry) = self.get_pool_entry(id) { - all_statuses.push(entry.status); - let hash = entry.inner.transaction().hash(); - if rtx - .transaction - .input_pts_iter() - .any(|pt| pt.tx_hash() == hash) - { - return Err(Reject::RBFRejected( - "new Tx contains inputs in descendants of to be replaced Tx" - .to_string(), - )); - } + let entries = descendants + .iter() + .filter_map(|id| self.get_pool_entry(id)) + .collect::>(); + + for entry in entries.iter() { + let hash = entry.inner.transaction().hash(); + if rtx + .transaction + .input_pts_iter() + .any(|pt| pt.tx_hash() == hash) + { + return Err(Reject::RBFRejected( + "new Tx contains inputs in descendants of to be replaced Tx".to_string(), + )); } } - } - // Rule #6, all conflict Txs should be in `Pending` or `Gap` status - if all_statuses - .iter() - .any(|s| ![Status::Pending, Status::Gap].contains(s)) - { - // Here we only refer to `Pending` status, since `Gap` is an internal status - return Err(Reject::RBFRejected( - "all conflict Txs should be in Pending status".to_string(), - )); + let mut entries_status = entries.iter().map(|e| e.status).collect::>(); + entries_status.push(conflict.status); + // Rule #6, all conflict Txs should be in `Pending` or `Gap` status + if entries_status + .iter() + .any(|s| ![Status::Pending, Status::Gap].contains(s)) + { + // Here we only refer to `Pending` status, since `Gap` is an internal status + return Err(Reject::RBFRejected( + "all conflict Txs should be in Pending status".to_string(), + )); + } } Ok(()) From 0374910128710f5e42dc6a4d39ddae0cfbfe695e Mon Sep 17 00:00:00 2001 From: yukang Date: Mon, 14 Aug 2023 21:57:29 +0800 Subject: [PATCH 57/58] add test case for RBFEnable and get_transaction_with_verbosity --- test/src/main.rs | 1 + test/src/specs/tx_pool/replace.rs | 38 ++++++++++++++++++++++++++++++- 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/test/src/main.rs b/test/src/main.rs index 0658c18d01..81da1d8a27 100644 --- a/test/src/main.rs +++ b/test/src/main.rs @@ -459,6 +459,7 @@ fn all_specs() -> Vec> { Box::new(TxsRelayOrder), Box::new(SendTxChain), Box::new(DifferentTxsWithSameInputWithOutRBF), + Box::new(RbfEnable), Box::new(RbfBasic), Box::new(RbfSameInput), Box::new(RbfOnlyForResolveDead), diff --git a/test/src/specs/tx_pool/replace.rs b/test/src/specs/tx_pool/replace.rs index 0bb4182d0f..90b972a640 100644 --- a/test/src/specs/tx_pool/replace.rs +++ b/test/src/specs/tx_pool/replace.rs @@ -7,6 +7,36 @@ use ckb_types::{ prelude::*, }; +pub struct RbfEnable; +impl Spec for RbfEnable { + fn run(&self, nodes: &mut Vec) { + let node0 = &nodes[0]; + + node0.mine_until_out_bootstrap_period(); + node0.new_block_with_blocking(|template| template.number.value() != 13); + let tx_hash_0 = node0.generate_transaction(); + let tx1 = node0.new_transaction(tx_hash_0); + + let output = CellOutputBuilder::default() + .capacity(capacity_bytes!(70).pack()) + .build(); + + let tx1 = tx1.as_advanced_builder().set_outputs(vec![output]).build(); + + node0.rpc_client().send_transaction(tx1.data().into()); + let ret = node0 + .rpc_client() + .get_transaction_with_verbosity(tx1.hash(), 2); + + assert_eq!(ret.min_replace_fee, None); + } + + fn modify_app_config(&self, config: &mut ckb_app_config::CKBAppConfig) { + config.tx_pool.min_rbf_rate = ckb_types::core::FeeRate(100); + config.tx_pool.min_fee_rate = ckb_types::core::FeeRate(100); + } +} + pub struct RbfBasic; impl Spec for RbfBasic { fn run(&self, nodes: &mut Vec) { @@ -30,6 +60,12 @@ impl Spec for RbfBasic { .build(); node0.rpc_client().send_transaction(tx1.data().into()); + let ret = node0 + .rpc_client() + .get_transaction_with_verbosity(tx1.hash(), 2); + // min_replace_fee is 363 + assert_eq!(ret.min_replace_fee.unwrap().to_string(), "0x16b"); + let res = node0 .rpc_client() .send_transaction_result(tx2.data().into()); @@ -50,7 +86,7 @@ impl Spec for RbfBasic { assert!(!commit_txs_hash.contains(&tx1.hash())); assert!(commit_txs_hash.contains(&tx2.hash())); - // when tx1 was confirmed, tx2 should be rejected + // when tx2 should be committed let ret = node0.rpc_client().get_transaction(tx2.hash()); assert!( matches!(ret.tx_status.status, Status::Committed), From 3e4b947598f2de1e6716d7ed0d36a1dd2a58167b Mon Sep 17 00:00:00 2001 From: yukang Date: Tue, 15 Aug 2023 18:43:47 +0800 Subject: [PATCH 58/58] entries in Gap will not be fill proposals --- test/src/specs/tx_pool/descendant.rs | 23 +++++++++++++++++------ tx-pool/src/pool.rs | 2 -- 2 files changed, 17 insertions(+), 8 deletions(-) diff --git a/test/src/specs/tx_pool/descendant.rs b/test/src/specs/tx_pool/descendant.rs index 04b5a5bc4b..ed23bb7db3 100644 --- a/test/src/specs/tx_pool/descendant.rs +++ b/test/src/specs/tx_pool/descendant.rs @@ -1,3 +1,5 @@ +use ckb_jsonrpc_types::Status; + use crate::specs::tx_pool::utils::prepare_tx_family; use crate::utils::{blank, commit, propose}; use crate::{Node, Spec}; @@ -166,21 +168,30 @@ impl Spec for SubmitTransactionWhenItsParentInProposed { // 1. Propose `tx_family.a` into proposed-pool. let family = prepare_tx_family(node); - node.submit_transaction(family.a()); - node.submit_block(&propose(node, &[family.a()])); + let tx_a = family.a(); + node.submit_transaction(tx_a); + node.submit_block(&propose(node, &[tx_a])); (0..=window.closest()).for_each(|_| { node.submit_block(&blank(node)); }); + // tx_a should in Proposed status + let tx_a_status = node.get_transaction(tx_a.hash()); + assert_eq!(tx_a_status.status, Status::Proposed); + // 2. Submit `tx_family.b` into pending-pool. Then we expect that miner propose it. node.submit_transaction(family.b()); let block = node.new_block_with_blocking(|template| template.proposals.is_empty()); + let union_proposal_ids = block.union_proposal_ids(); assert!( - block - .union_proposal_ids() - .contains(&family.b().proposal_short_id()), + union_proposal_ids.contains(&family.b().proposal_short_id()), "Miner should propose tx_family.b since it has never been proposed, actual: {:?}", - block.union_proposal_ids(), + union_proposal_ids, + ); + assert!( + !union_proposal_ids.contains(&tx_a.proposal_short_id()), + "Miner should not propose tx_family.a since it has been proposed, actual: {:?}", + union_proposal_ids, ); node.submit_block(&block); } diff --git a/tx-pool/src/pool.rs b/tx-pool/src/pool.rs index caa05c12d5..b7952f4838 100644 --- a/tx-pool/src/pool.rs +++ b/tx-pool/src/pool.rs @@ -390,8 +390,6 @@ impl TxPool { let mut proposals = HashSet::with_capacity(limit); self.pool_map .fill_proposals(limit, exclusion, &mut proposals, Status::Pending); - self.pool_map - .fill_proposals(limit, exclusion, &mut proposals, Status::Gap); proposals }