diff --git a/prdoc/pr_6405.prdoc b/prdoc/pr_6405.prdoc new file mode 100644 index 000000000000..9e4e0b3c6c20 --- /dev/null +++ b/prdoc/pr_6405.prdoc @@ -0,0 +1,9 @@ +title: '`fatxpool`: handling limits and priorities improvements' +doc: +- audience: Node Dev + description: |- + This PR provides a number of improvements and fixes around handling limits and priorities in the fork-aware transaction pool. + +crates: +- name: sc-transaction-pool + bump: major diff --git a/substrate/client/transaction-pool/benches/basics.rs b/substrate/client/transaction-pool/benches/basics.rs index 0d8c1cbba9b4..5e40b0fb72d6 100644 --- a/substrate/client/transaction-pool/benches/basics.rs +++ b/substrate/client/transaction-pool/benches/basics.rs @@ -152,7 +152,7 @@ fn uxt(transfer: TransferData) -> Extrinsic { } fn bench_configured(pool: Pool, number: u64, api: Arc) { - let source = TransactionSource::External; + let source = TimedTransactionSource::new_external(false); let mut futures = Vec::new(); let mut tags = Vec::new(); let at = HashAndNumber { @@ -171,7 +171,7 @@ fn bench_configured(pool: Pool, number: u64, api: Arc) { tags.push(to_tag(nonce, AccountId::from_h256(H256::from_low_u64_be(1)))); - futures.push(pool.submit_one(&at, source, xt)); + futures.push(pool.submit_one(&at, source.clone(), xt)); } let res = block_on(futures::future::join_all(futures.into_iter())); diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/dropped_watcher.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/dropped_watcher.rs index ecae21395c91..63571347cc52 100644 --- a/substrate/client/transaction-pool/src/fork_aware_txpool/dropped_watcher.rs +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/dropped_watcher.rs @@ -24,7 +24,7 @@ use crate::{ common::log_xt::log_xt_trace, fork_aware_txpool::stream_map_util::next_event, - graph::{BlockHash, ChainApi, ExtrinsicHash}, + graph::{self, BlockHash, ExtrinsicHash}, LOG_TARGET, }; use futures::stream::StreamExt; @@ -33,12 +33,46 @@ use sc_transaction_pool_api::TransactionStatus; use sc_utils::mpsc; use sp_runtime::traits::Block as BlockT; use std::{ - collections::{hash_map::Entry, HashMap, HashSet}, + collections::{ + hash_map::{Entry, OccupiedEntry}, + HashMap, HashSet, + }, fmt::{self, Debug, Formatter}, pin::Pin, }; use tokio_stream::StreamMap; +/// Represents a transaction that was removed from the transaction pool, including the reason of its +/// removal. +#[derive(Debug, PartialEq)] +pub struct DroppedTransaction { + /// Hash of the dropped extrinsic. + pub tx_hash: Hash, + /// Reason of the transaction being dropped. + pub reason: DroppedReason, +} + +impl DroppedTransaction { + /// Creates an new instnance with reason set to `DroppedReason::Usurped(by)`. + pub fn new_usurped(tx_hash: Hash, by: Hash) -> Self { + Self { reason: DroppedReason::Usurped(by), tx_hash } + } + + /// Creates an new instnance with reason set to `DroppedReason::LimitsEnforced`. + pub fn new_enforced_by_limts(tx_hash: Hash) -> Self { + Self { reason: DroppedReason::LimitsEnforced, tx_hash } + } +} + +/// Provides reason of why transactions was dropped. +#[derive(Debug, PartialEq)] +pub enum DroppedReason { + /// Transaction was replaced by other transaction (e.g. because of higher priority). + Usurped(Hash), + /// Transaction was dropped because of internal pool limits being enforced. + LimitsEnforced, +} + /// Dropped-logic related event from the single view. pub type ViewStreamEvent = crate::graph::DroppedByLimitsEvent, BlockHash>; @@ -47,7 +81,8 @@ type ViewStream = Pin> + Se /// Stream of extrinsic hashes that were dropped by the views and have no references by existing /// views. -pub(crate) type StreamOfDropped = Pin> + Send>>; +pub(crate) type StreamOfDropped = + Pin>> + Send>>; /// A type alias for a sender used as the controller of the [`MultiViewDropWatcherContext`]. /// Used to send control commands from the [`MultiViewDroppedWatcherController`] to @@ -59,24 +94,24 @@ type Controller = mpsc::TracingUnboundedSender; type CommandReceiver = mpsc::TracingUnboundedReceiver; /// Commands to control the instance of dropped transactions stream [`StreamOfDropped`]. -enum Command +enum Command where - C: ChainApi, + ChainApi: graph::ChainApi, { /// Adds a new stream of dropped-related events originating in a view with a specific block /// hash - AddView(BlockHash, ViewStream), + AddView(BlockHash, ViewStream), /// Removes an existing view's stream associated with a specific block hash. - RemoveView(BlockHash), - /// Removes internal states for given extrinsic hashes. + RemoveView(BlockHash), + /// Removes referencing views for given extrinsic hashes. /// /// Intended to ba called on finalization. - RemoveFinalizedTxs(Vec>), + RemoveFinalizedTxs(Vec>), } -impl Debug for Command +impl Debug for Command where - C: ChainApi, + ChainApi: graph::ChainApi, { fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { match self { @@ -92,30 +127,60 @@ where /// /// This struct maintains a mapping of active views and their corresponding streams, as well as the /// state of each transaction with respect to these views. -struct MultiViewDropWatcherContext +struct MultiViewDropWatcherContext where - C: ChainApi, + ChainApi: graph::ChainApi, { /// A map that associates the views identified by corresponding block hashes with their streams /// of dropped-related events. This map is used to keep track of active views and their event /// streams. - stream_map: StreamMap, ViewStream>, + /// todo: rename: view_stream map + stream_map: StreamMap, ViewStream>, /// A receiver for commands to control the state of the stream, allowing the addition and /// removal of views. This is used to dynamically update which views are being tracked. - command_receiver: CommandReceiver>, - + command_receiver: CommandReceiver>, /// For each transaction hash we keep the set of hashes representing the views that see this - /// transaction as ready or future. + /// transaction as ready or in_block. + /// + /// Even if all views referencing a ready transactions are removed, we still want to keep + /// transaction, there can be a fork which sees the transaction as ready. /// /// Once transaction is dropped, dropping view is removed from the set. - transaction_states: HashMap, HashSet>>, + ready_transaction_views: HashMap, HashSet>>, + /// For each transaction hash we keep the set of hashes representing the views that see this + /// transaction as future. + /// + /// Once all views referencing a future transactions are removed, the future can be dropped. + /// + /// Once transaction is dropped, dropping view is removed from the set. + future_transaction_views: HashMap, HashSet>>, + + /// Transactions that need to be notified as dropped. + pending_dropped_transactions: Vec>, } impl MultiViewDropWatcherContext where - C: ChainApi + 'static, - <::Block as BlockT>::Hash: Unpin, + C: graph::ChainApi + 'static, + <::Block as BlockT>::Hash: Unpin, { + /// Provides the ready or future `HashSet` containing views referencing given transaction. + fn get_transaction_views( + &mut self, + tx_hash: ExtrinsicHash, + ) -> Option, HashSet>>> { + if let Entry::Occupied(views_keeping_tx_valid) = self.ready_transaction_views.entry(tx_hash) + { + return Some(views_keeping_tx_valid) + } + if let Entry::Occupied(views_keeping_tx_valid) = + self.future_transaction_views.entry(tx_hash) + { + return Some(views_keeping_tx_valid) + } + None + } + /// Processes a `ViewStreamEvent` from a specific view and updates the internal state /// accordingly. /// @@ -125,41 +190,64 @@ where &mut self, block_hash: BlockHash, event: ViewStreamEvent, - ) -> Option> { + ) -> Option>> { trace!( target: LOG_TARGET, - "dropped_watcher: handle_event: event:{:?} views:{:?}, ", - event, + "dropped_watcher: handle_event: event:{event:?} from:{block_hash:?} future_views:{:?} ready_views:{:?} stream_map views:{:?}, ", + self.future_transaction_views.get(&event.0), + self.ready_transaction_views.get(&event.0), self.stream_map.keys().collect::>(), ); let (tx_hash, status) = event; match status { - TransactionStatus::Ready | TransactionStatus::Future => { - self.transaction_states.entry(tx_hash).or_default().insert(block_hash); + TransactionStatus::Future => { + self.future_transaction_views.entry(tx_hash).or_default().insert(block_hash); + }, + TransactionStatus::Ready | TransactionStatus::InBlock(..) => { + if let Some(mut views) = self.future_transaction_views.remove(&tx_hash) { + views.insert(block_hash); + self.ready_transaction_views.insert(tx_hash, views); + } else { + self.ready_transaction_views.entry(tx_hash).or_default().insert(block_hash); + } }, - TransactionStatus::Dropped | TransactionStatus::Usurped(_) => { - if let Entry::Occupied(mut views_keeping_tx_valid) = - self.transaction_states.entry(tx_hash) - { + TransactionStatus::Dropped => { + if let Some(mut views_keeping_tx_valid) = self.get_transaction_views(tx_hash) { views_keeping_tx_valid.get_mut().remove(&block_hash); - if views_keeping_tx_valid.get().is_empty() || - views_keeping_tx_valid - .get() - .iter() - .all(|h| !self.stream_map.contains_key(h)) - { - return Some(tx_hash) + if views_keeping_tx_valid.get().is_empty() { + return Some(DroppedTransaction::new_enforced_by_limts(tx_hash)) } } else { debug!("[{:?}] dropped_watcher: removing (non-tracked) tx", tx_hash); - return Some(tx_hash) + return Some(DroppedTransaction::new_enforced_by_limts(tx_hash)) } }, + TransactionStatus::Usurped(by) => + return Some(DroppedTransaction::new_usurped(tx_hash, by)), _ => {}, }; None } + /// Gets pending dropped transactions if any. + fn get_pending_dropped_transaction(&mut self) -> Option>> { + while let Some(tx_hash) = self.pending_dropped_transactions.pop() { + // never drop transaction that was seens as ready. It may not have a referencing + // view now, but such fork can appear. + if let Some(_) = self.ready_transaction_views.get(&tx_hash) { + continue + } + + if let Some(views) = self.future_transaction_views.get(&tx_hash) { + if views.is_empty() { + self.future_transaction_views.remove(&tx_hash); + return Some(DroppedTransaction::new_enforced_by_limts(tx_hash)) + } + } + } + return None + } + /// Creates a new `StreamOfDropped` and its associated event stream controller. /// /// This method initializes the internal structures and unfolds the stream of dropped @@ -176,13 +264,25 @@ where let ctx = Self { stream_map: StreamMap::new(), command_receiver, - transaction_states: Default::default(), + ready_transaction_views: Default::default(), + future_transaction_views: Default::default(), + pending_dropped_transactions: Default::default(), }; let stream_map = futures::stream::unfold(ctx, |mut ctx| async move { loop { + if let Some(dropped) = ctx.get_pending_dropped_transaction() { + debug!("dropped_watcher: sending out (pending): {dropped:?}"); + return Some((dropped, ctx)); + } tokio::select! { biased; + Some(event) = next_event(&mut ctx.stream_map) => { + if let Some(dropped) = ctx.handle_event(event.0, event.1) { + debug!("dropped_watcher: sending out: {dropped:?}"); + return Some((dropped, ctx)); + } + }, cmd = ctx.command_receiver.next() => { match cmd? { Command::AddView(key,stream) => { @@ -192,26 +292,34 @@ where Command::RemoveView(key) => { trace!(target: LOG_TARGET,"dropped_watcher: Command::RemoveView {key:?} views:{:?}",ctx.stream_map.keys().collect::>()); ctx.stream_map.remove(&key); - ctx.transaction_states.iter_mut().for_each(|(_,state)| { - state.remove(&key); + ctx.ready_transaction_views.iter_mut().for_each(|(tx_hash,views)| { + trace!(target: LOG_TARGET,"[{:?}] dropped_watcher: Command::RemoveView ready views: {:?}",tx_hash, views); + views.remove(&key); + //todo: merge heads up warning! + if views.is_empty() { + ctx.pending_dropped_transactions.push(*tx_hash); + } + }); + + ctx.future_transaction_views.iter_mut().for_each(|(tx_hash,views)| { + trace!(target: LOG_TARGET,"[{:?}] dropped_watcher: Command::RemoveView future views: {:?}",tx_hash, views); + views.remove(&key); + if views.is_empty() { + ctx.pending_dropped_transactions.push(*tx_hash); + } }); }, Command::RemoveFinalizedTxs(xts) => { log_xt_trace!(target: LOG_TARGET, xts.clone(), "[{:?}] dropped_watcher: finalized xt removed"); xts.iter().for_each(|xt| { - ctx.transaction_states.remove(xt); + ctx.ready_transaction_views.remove(xt); + ctx.future_transaction_views.remove(xt); }); }, } - }, - - Some(event) = next_event(&mut ctx.stream_map) => { - if let Some(dropped) = ctx.handle_event(event.0, event.1) { - debug!("dropped_watcher: sending out: {dropped:?}"); - return Some((dropped, ctx)); - } } + } } }) @@ -225,30 +333,30 @@ where /// /// This struct provides methods to add and remove streams associated with views to and from the /// stream. -pub struct MultiViewDroppedWatcherController { +pub struct MultiViewDroppedWatcherController { /// A controller allowing to update the state of the associated [`StreamOfDropped`]. - controller: Controller>, + controller: Controller>, } -impl Clone for MultiViewDroppedWatcherController { +impl Clone for MultiViewDroppedWatcherController { fn clone(&self) -> Self { Self { controller: self.controller.clone() } } } -impl MultiViewDroppedWatcherController +impl MultiViewDroppedWatcherController where - C: ChainApi + 'static, - <::Block as BlockT>::Hash: Unpin, + ChainApi: graph::ChainApi + 'static, + <::Block as BlockT>::Hash: Unpin, { /// Creates new [`StreamOfDropped`] and its controller. - pub fn new() -> (MultiViewDroppedWatcherController, StreamOfDropped) { - let (stream_map, ctrl) = MultiViewDropWatcherContext::::event_stream(); + pub fn new() -> (MultiViewDroppedWatcherController, StreamOfDropped) { + let (stream_map, ctrl) = MultiViewDropWatcherContext::::event_stream(); (Self { controller: ctrl }, stream_map.boxed()) } /// Notifies the [`StreamOfDropped`] that new view was created. - pub fn add_view(&self, key: BlockHash, view: ViewStream) { + pub fn add_view(&self, key: BlockHash, view: ViewStream) { let _ = self.controller.unbounded_send(Command::AddView(key, view)).map_err(|e| { trace!(target: LOG_TARGET, "dropped_watcher: add_view {key:?} send message failed: {e}"); }); @@ -256,14 +364,17 @@ where /// Notifies the [`StreamOfDropped`] that the view was destroyed and shall be removed the /// stream map. - pub fn remove_view(&self, key: BlockHash) { + pub fn remove_view(&self, key: BlockHash) { let _ = self.controller.unbounded_send(Command::RemoveView(key)).map_err(|e| { trace!(target: LOG_TARGET, "dropped_watcher: remove_view {key:?} send message failed: {e}"); }); } /// Removes status info for finalized transactions. - pub fn remove_finalized_txs(&self, xts: impl IntoIterator> + Clone) { + pub fn remove_finalized_txs( + &self, + xts: impl IntoIterator> + Clone, + ) { let _ = self .controller .unbounded_send(Command::RemoveFinalizedTxs(xts.into_iter().collect())) @@ -298,7 +409,7 @@ mod dropped_watcher_tests { watcher.add_view(block_hash, view_stream); let handle = tokio::spawn(async move { output_stream.take(1).collect::>().await }); - assert_eq!(handle.await.unwrap(), vec![tx_hash]); + assert_eq!(handle.await.unwrap(), vec![DroppedTransaction::new_enforced_by_limts(tx_hash)]); } #[tokio::test] @@ -348,7 +459,10 @@ mod dropped_watcher_tests { watcher.add_view(block_hash0, view_stream0); watcher.add_view(block_hash1, view_stream1); let handle = tokio::spawn(async move { output_stream.take(1).collect::>().await }); - assert_eq!(handle.await.unwrap(), vec![tx_hash1]); + assert_eq!( + handle.await.unwrap(), + vec![DroppedTransaction::new_enforced_by_limts(tx_hash1)] + ); } #[tokio::test] @@ -373,10 +487,11 @@ mod dropped_watcher_tests { watcher.add_view(block_hash0, view_stream0); assert!(output_stream.next().now_or_never().is_none()); + watcher.remove_view(block_hash0); watcher.add_view(block_hash1, view_stream1); let handle = tokio::spawn(async move { output_stream.take(1).collect::>().await }); - assert_eq!(handle.await.unwrap(), vec![tx_hash]); + assert_eq!(handle.await.unwrap(), vec![DroppedTransaction::new_enforced_by_limts(tx_hash)]); } #[tokio::test] @@ -419,6 +534,6 @@ mod dropped_watcher_tests { let block_hash2 = H256::repeat_byte(0x03); watcher.add_view(block_hash2, view_stream2); let handle = tokio::spawn(async move { output_stream.take(1).collect::>().await }); - assert_eq!(handle.await.unwrap(), vec![tx_hash]); + assert_eq!(handle.await.unwrap(), vec![DroppedTransaction::new_enforced_by_limts(tx_hash)]); } } diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/fork_aware_txpool.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/fork_aware_txpool.rs index 065d0cb3a274..3f8d66110432 100644 --- a/substrate/client/transaction-pool/src/fork_aware_txpool/fork_aware_txpool.rs +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/fork_aware_txpool.rs @@ -23,7 +23,7 @@ use super::{ import_notification_sink::MultiViewImportNotificationSink, metrics::MetricsLink as PrometheusMetrics, multi_view_listener::MultiViewListener, - tx_mem_pool::{TxInMemPool, TxMemPool, TXMEMPOOL_TRANSACTION_LIMIT_MULTIPLIER}, + tx_mem_pool::{InsertionInfo, TxInMemPool, TxMemPool, TXMEMPOOL_TRANSACTION_LIMIT_MULTIPLIER}, view::View, view_store::ViewStore, }; @@ -31,8 +31,15 @@ use crate::{ api::FullChainApi, common::log_xt::log_xt_trace, enactment_state::{EnactmentAction, EnactmentState}, - fork_aware_txpool::revalidation_worker, - graph::{self, base_pool::Transaction, ExtrinsicFor, ExtrinsicHash, IsValidator, Options}, + fork_aware_txpool::{ + dropped_watcher::{DroppedReason, DroppedTransaction}, + revalidation_worker, + }, + graph::{ + self, + base_pool::{TimedTransactionSource, Transaction}, + ExtrinsicFor, ExtrinsicHash, IsValidator, Options, + }, ReadyIteratorFor, LOG_TARGET, }; use async_trait::async_trait; @@ -45,8 +52,9 @@ use futures::{ use parking_lot::Mutex; use prometheus_endpoint::Registry as PrometheusRegistry; use sc_transaction_pool_api::{ - ChainEvent, ImportNotificationStream, MaintainedTransactionPool, PoolStatus, TransactionFor, - TransactionPool, TransactionSource, TransactionStatusStreamFor, TxHash, + error::Error as TxPoolApiError, ChainEvent, ImportNotificationStream, + MaintainedTransactionPool, PoolStatus, TransactionFor, TransactionPool, TransactionSource, + TransactionStatusStreamFor, TxHash, }; use sp_blockchain::{HashAndNumber, TreeRoute}; use sp_core::traits::SpawnEssentialNamed; @@ -197,9 +205,14 @@ where let (dropped_stream_controller, dropped_stream) = MultiViewDroppedWatcherController::::new(); + + let view_store = + Arc::new(ViewStore::new(pool_api.clone(), listener, dropped_stream_controller)); + let dropped_monitor_task = Self::dropped_monitor_task( dropped_stream, mempool.clone(), + view_store.clone(), import_notification_sink.clone(), ); @@ -216,8 +229,8 @@ where ( Self { mempool, - api: pool_api.clone(), - view_store: Arc::new(ViewStore::new(pool_api, listener, dropped_stream_controller)), + api: pool_api, + view_store, ready_poll: Arc::from(Mutex::from(ReadyPoll::new())), enactment_state: Arc::new(Mutex::new(EnactmentState::new( best_block_hash, @@ -233,14 +246,17 @@ where ) } - /// Monitors the stream of dropped transactions and removes them from the mempool. + /// Monitors the stream of dropped transactions and removes them from the mempool and + /// view_store. /// /// This asynchronous task continuously listens for dropped transaction notifications provided /// within `dropped_stream` and ensures that these transactions are removed from the `mempool` - /// and `import_notification_sink` instances. + /// and `import_notification_sink` instances. For Usurped events, the transaction is also + /// removed from the view_store. async fn dropped_monitor_task( mut dropped_stream: StreamOfDropped, mempool: Arc>, + view_store: Arc>, import_notification_sink: MultiViewImportNotificationSink< Block::Hash, ExtrinsicHash, @@ -251,9 +267,33 @@ where log::debug!(target: LOG_TARGET, "fatp::dropped_monitor_task: terminated..."); break; }; - log::trace!(target: LOG_TARGET, "[{:?}] fatp::dropped notification, removing", dropped); - mempool.remove_dropped_transactions(&[dropped]).await; - import_notification_sink.clean_notified_items(&[dropped]); + let dropped_tx_hash = dropped.tx_hash; + log::trace!(target: LOG_TARGET, "[{:?}] fatp::dropped notification {:?}, removing", dropped_tx_hash,dropped.reason); + match dropped.reason { + DroppedReason::Usurped(new_tx_hash) => { + if let Some(new_tx) = mempool.get_by_hash(new_tx_hash) { + view_store + .replace_transaction( + new_tx.source(), + new_tx.tx(), + dropped_tx_hash, + new_tx.is_watched(), + ) + .await; + } else { + log::trace!( + target:LOG_TARGET, + "error: dropped_monitor_task: no entry in mempool for new transaction {:?}", + new_tx_hash, + ); + } + }, + DroppedReason::LimitsEnforced => {}, + }; + + mempool.remove_transaction(&dropped_tx_hash); + view_store.listener.transaction_dropped(dropped); + import_notification_sink.clean_notified_items(&[dropped_tx_hash]); } } @@ -288,9 +328,13 @@ where let (dropped_stream_controller, dropped_stream) = MultiViewDroppedWatcherController::::new(); + + let view_store = + Arc::new(ViewStore::new(pool_api.clone(), listener, dropped_stream_controller)); let dropped_monitor_task = Self::dropped_monitor_task( dropped_stream, mempool.clone(), + view_store.clone(), import_notification_sink.clone(), ); @@ -306,8 +350,8 @@ where Self { mempool, - api: pool_api.clone(), - view_store: Arc::new(ViewStore::new(pool_api, listener, dropped_stream_controller)), + api: pool_api, + view_store, ready_poll: Arc::from(Mutex::from(ReadyPoll::new())), enactment_state: Arc::new(Mutex::new(EnactmentState::new( best_block_hash, @@ -366,6 +410,16 @@ where self.mempool.unwatched_and_watched_count() } + /// Returns a set of future transactions for given block hash. + /// + /// Intended for logging / tests. + pub fn futures_at( + &self, + at: Block::Hash, + ) -> Option, ExtrinsicFor>>> { + self.view_store.futures_at(at) + } + /// Returns a best-effort set of ready transactions for a given block, without executing full /// maintain process. /// @@ -548,7 +602,7 @@ where /// out: /// [ Ok(xth0), Ok(xth1), Err ] /// ``` -fn reduce_multiview_result(input: HashMap>>) -> Vec> { +fn reduce_multiview_result(input: HashMap>>) -> Vec> { let mut values = input.values(); let Some(first) = values.next() else { return Default::default(); @@ -589,7 +643,7 @@ where /// are reduced to single result. Refer to `reduce_multiview_result` for more details. async fn submit_at( &self, - _: ::Hash, + at: ::Hash, source: TransactionSource, xts: Vec>, ) -> Result, Self::Error>>, Self::Error> { @@ -600,34 +654,60 @@ where let mempool_results = self.mempool.extend_unwatched(source, &xts); if view_store.is_empty() { - return Ok(mempool_results) + return Ok(mempool_results + .into_iter() + .map(|r| r.map(|r| r.hash).map_err(Into::into)) + .collect::>()) } + //todo: review + test maybe? + let retries = mempool_results + .into_iter() + .zip(xts.clone()) + .map(|(result, xt)| async move { + match result { + Err(TxPoolApiError::ImmediatelyDropped) => + self.attempt_transaction_replacement(at, source, false, xt).await, + result @ _ => result, + } + }) + .collect::>(); + + let mempool_results = futures::future::join_all(retries).await; + let to_be_submitted = mempool_results .iter() .zip(xts) - .filter_map(|(result, xt)| result.as_ref().ok().map(|_| xt)) + .filter_map(|(result, xt)| { + result.as_ref().ok().map(|insertion| (insertion.source.clone(), xt)) + }) .collect::>(); self.metrics .report(|metrics| metrics.submitted_transactions.inc_by(to_be_submitted.len() as _)); let mempool = self.mempool.clone(); - let results_map = view_store.submit(source, to_be_submitted.into_iter()).await; + let results_map = view_store.submit(to_be_submitted.into_iter()).await; let mut submission_results = reduce_multiview_result(results_map).into_iter(); Ok(mempool_results .into_iter() .map(|result| { - result.and_then(|xt_hash| { + result + .map_err(Into::into) + .and_then(|insertion| { submission_results .next() .expect("The number of Ok results in mempool is exactly the same as the size of to-views-submission result. qed.") - .inspect_err(|_| - mempool.remove(xt_hash) - ) + .inspect_err(|_|{ + mempool.remove_transaction(&insertion.hash); + }) }) }) + .map(|r| r.map(|r| { + mempool.update_transaction(&r); + r.hash() + })) .collect::>()) } @@ -660,19 +740,27 @@ where ) -> Result>>, Self::Error> { log::trace!(target: LOG_TARGET, "[{:?}] fatp::submit_and_watch views:{}", self.tx_hash(&xt), self.active_views_count()); let xt = Arc::from(xt); - let xt_hash = match self.mempool.push_watched(source, xt.clone()) { - Ok(xt_hash) => xt_hash, - Err(e) => return Err(e), - }; + + let InsertionInfo { hash: xt_hash, source: timed_source, .. } = + match self.mempool.push_watched(source, xt.clone()) { + Ok(result) => result, + Err(TxPoolApiError::ImmediatelyDropped) => + self.attempt_transaction_replacement(at, source, true, xt.clone()).await?, + Err(e) => return Err(e.into()), + }; self.metrics.report(|metrics| metrics.submitted_transactions.inc()); - let view_store = self.view_store.clone(); - let mempool = self.mempool.clone(); - view_store - .submit_and_watch(at, source, xt) + self.view_store + .submit_and_watch(at, timed_source, xt) .await - .inspect_err(|_| mempool.remove(xt_hash)) + .inspect_err(|_| { + self.mempool.remove_transaction(&xt_hash); + }) + .map(|mut outcome| { + self.mempool.update_transaction(&outcome); + outcome.expect_watcher() + }) } /// Intended to remove transactions identified by the given hashes, and any dependent @@ -801,12 +889,18 @@ where ) -> Result { log::debug!(target: LOG_TARGET, "fatp::submit_local views:{}", self.active_views_count()); let xt = Arc::from(xt); - let result = self + let InsertionInfo { hash: xt_hash, .. } = self .mempool .extend_unwatched(TransactionSource::Local, &[xt.clone()]) .remove(0)?; - self.view_store.submit_local(xt).or_else(|_| Ok(result)) + self.view_store + .submit_local(xt) + .map(|outcome| { + self.mempool.update_transaction(&outcome); + outcome.hash() + }) + .or_else(|_| Ok(xt_hash)) } } @@ -914,6 +1008,9 @@ where let start = Instant::now(); let watched_xts = self.register_listeners(&mut view).await; let duration = start.elapsed(); + // sync the transactions statuses and referencing views in all the listeners with newly + // cloned view. + view.pool.validated_pool().retrigger_notifications(); log::debug!(target: LOG_TARGET, "register_listeners: at {at:?} took {duration:?}"); // 2. Handle transactions from the tree route. Pruning transactions from the view first @@ -1041,58 +1138,41 @@ where self.active_views_count() ); let included_xts = self.extrinsics_included_since_finalized(view.at.hash).await; - let xts = self.mempool.clone_unwatched(); - - let mut all_submitted_count = 0; - if !xts.is_empty() { - let unwatched_count = xts.len(); - let mut buckets = HashMap::>>::default(); - xts.into_iter() - .filter(|(hash, _)| !view.pool.validated_pool().pool.read().is_imported(hash)) - .filter(|(hash, _)| !included_xts.contains(&hash)) - .map(|(_, tx)| (tx.source(), tx.tx())) - .for_each(|(source, tx)| buckets.entry(source).or_default().push(tx)); - - for (source, xts) in buckets { - all_submitted_count += xts.len(); - let _ = view.submit_many(source, xts).await; - } - log::debug!(target: LOG_TARGET, "update_view_with_mempool: at {:?} unwatched {}/{}", view.at.hash, all_submitted_count, unwatched_count); - } - let watched_submitted_count = watched_xts.len(); - - let mut buckets = HashMap::< - TransactionSource, - Vec<(ExtrinsicHash, ExtrinsicFor)>, - >::default(); - watched_xts + let (hashes, xts_filtered): (Vec<_>, Vec<_>) = watched_xts .into_iter() + .chain(self.mempool.clone_unwatched().into_iter()) + .filter(|(hash, _)| !view.is_imported(hash)) .filter(|(hash, _)| !included_xts.contains(&hash)) - .map(|(tx_hash, tx)| (tx.source(), tx_hash, tx.tx())) - .for_each(|(source, tx_hash, tx)| { - buckets.entry(source).or_default().push((tx_hash, tx)) - }); + .map(|(tx_hash, tx)| (tx_hash, (tx.source(), tx.tx()))) + .unzip(); - let mut watched_results = Vec::default(); - for (source, watched_xts) in buckets { - let hashes = watched_xts.iter().map(|i| i.0).collect::>(); - let results = view - .submit_many(source, watched_xts.into_iter().map(|i| i.1)) - .await - .into_iter() - .zip(hashes) - .map(|(result, tx_hash)| result.or_else(|_| Err(tx_hash))) - .collect::>(); - watched_results.extend(results); - } + let watched_results = view + .submit_many(xts_filtered) + .await + .into_iter() + .zip(hashes) + .map(|(result, tx_hash)| { + //todo: we may need a bool flag here indicating if we need to update priority + //(premature optimization) + result + .map(|outcome| self.mempool.update_transaction(&outcome.into())) + .or_else(|_| Err(tx_hash)) + }) + .collect::>(); - log::debug!(target: LOG_TARGET, "update_view_with_mempool: at {:?} watched {}/{}", view.at.hash, watched_submitted_count, self.mempool_len().1); + let submitted_count = watched_results.len(); - all_submitted_count += watched_submitted_count; - let _ = all_submitted_count - .try_into() - .map(|v| self.metrics.report(|metrics| metrics.submitted_from_mempool_txs.inc_by(v))); + log::debug!( + target: LOG_TARGET, + "update_view_with_mempool: at {:?} submitted {}/{}", + view.at.hash, + submitted_count, + self.mempool.len() + ); + + self.metrics + .report(|metrics| metrics.submitted_from_mempool_txs.inc_by(submitted_count as _)); // if there are no views yet, and a single newly created view is reporting error, just send // out the invalid event, and remove transaction. @@ -1100,7 +1180,7 @@ where for result in watched_results { if let Err(tx_hash) = result { self.view_store.listener.invalidate_transactions(&[tx_hash]); - self.mempool.remove(tx_hash); + self.mempool.remove_transaction(&tx_hash); } } } @@ -1176,7 +1256,14 @@ where }) .map(|(tx_hash, tx)| { //find arc if tx is known - self.mempool.get_by_hash(tx_hash).unwrap_or_else(|| Arc::from(tx)) + self.mempool + .get_by_hash(tx_hash) + .map(|tx| (tx.source(), tx.tx())) + .unwrap_or_else(|| { + // These transactions are coming from retracted blocks, we + // should simply consider them external. + (TimedTransactionSource::new_external(true), Arc::from(tx)) + }) }), ); @@ -1185,16 +1272,7 @@ where }); } - let _ = view - .pool - .resubmit_at( - &hash_and_number, - // These transactions are coming from retracted blocks, we should - // simply consider them external. - TransactionSource::External, - resubmit_transactions, - ) - .await; + let _ = view.pool.resubmit_at(&hash_and_number, resubmit_transactions).await; } } @@ -1234,6 +1312,78 @@ where fn tx_hash(&self, xt: &TransactionFor) -> TxHash { self.api.hash_and_length(xt).0 } + + /// Attempts to find and replace a lower-priority transaction in the transaction pool with a new + /// one. + /// + /// This asynchronous function verifies the new transaction against the most recent view. If a + /// transaction with a lower priority exists in the transaction pool, it is replaced with the + /// new transaction. + /// + /// If no lower-priority transaction is found, the function returns an error indicating the + /// transaction was dropped immediately. + async fn attempt_transaction_replacement( + &self, + _: Block::Hash, + source: TransactionSource, + watched: bool, + xt: ExtrinsicFor, + ) -> Result>, TxPoolApiError> { + // 1. we should validate at most_recent, and reuse result to submit there. + let at = self + .view_store + .most_recent_view + .read() + .ok_or(TxPoolApiError::ImmediatelyDropped)?; + + // 2. validate transaction at `at` + let (best_view, _) = self + .view_store + .get_view_at(at, false) + .ok_or(TxPoolApiError::ImmediatelyDropped)?; + + let (tx_hash, validated_tx) = best_view + .pool + .verify_one( + best_view.at.hash, + best_view.at.number, + TimedTransactionSource::from_transaction_source(source, false), + xt.clone(), + crate::graph::CheckBannedBeforeVerify::Yes, + ) + .await; + + if let Some(priority) = validated_tx.priority() { + // 3. check if mempool contains a transaction with lower priority (actually worse - do + // we want to check timestamp too - no: see #4609?) Would be perfect to choose + // transaction with lowest the number of dependant txs in its subtree. + // 4. if yes - remove worse transaction from mempool and add new one. + let insertion_info = + self.mempool.try_replace_transaction(xt, priority, source, watched)?; + + for worst_hash in &insertion_info.removed { + log::trace!(target: LOG_TARGET, "removed: {worst_hash:?} replaced by {tx_hash:?}"); + // 5. notify listner + self.view_store + .listener + .transaction_dropped(DroppedTransaction::new_enforced_by_limts(*worst_hash)); + + // 6. remove transaction from the view_store + self.view_store.remove_transaction_subtree( + *worst_hash, + |listener, removed_tx_hash| { + listener.limits_enforced(&removed_tx_hash); + }, + ); + } + + // 8. add to pending_replacements - make sure it will not sneak back via cloned view + // 9. subemit new one to the view, this will be done upon in the caller + return Ok(insertion_info) + } + + Err(TxPoolApiError::ImmediatelyDropped) + } } #[async_trait] @@ -1381,7 +1531,7 @@ mod reduce_multiview_result_tests { fn empty() { sp_tracing::try_init_simple(); let input = HashMap::default(); - let r = reduce_multiview_result::(input); + let r = reduce_multiview_result::(input); assert!(r.is_empty()); } diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/import_notification_sink.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/import_notification_sink.rs index 7fbdcade63b8..f9a41673bb8f 100644 --- a/substrate/client/transaction-pool/src/fork_aware_txpool/import_notification_sink.rs +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/import_notification_sink.rs @@ -326,6 +326,7 @@ mod tests { let j0 = tokio::spawn(runnable); let stream = ctrl.event_stream(); + let stream2 = ctrl.event_stream(); let mut v1 = View::new(vec![(10, 1), (10, 2), (10, 3)]); let mut v2 = View::new(vec![(20, 1), (20, 2), (20, 6)]); @@ -342,20 +343,16 @@ mod tests { ctrl.add_view(1000, o1); ctrl.add_view(2000, o2); - let j4 = { - let ctrl = ctrl.clone(); - tokio::spawn(async move { - tokio::time::sleep(Duration::from_millis(70)).await; - ctrl.clean_notified_items(&vec![1, 3]); - ctrl.add_view(3000, o3.boxed()); - }) - }; + let out = stream.take(4).collect::>().await; + assert_eq!(out, vec![1, 2, 3, 6]); - let out = stream.take(6).collect::>().await; + ctrl.clean_notified_items(&vec![1, 3]); + ctrl.add_view(3000, o3.boxed()); + let out = stream2.take(6).collect::>().await; assert_eq!(out, vec![1, 2, 3, 6, 1, 3]); - drop(ctrl); - futures::future::join_all(vec![j0, j1, j2, j3, j4]).await; + drop(ctrl); + futures::future::join_all(vec![j0, j1, j2, j3]).await; } #[tokio::test] diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/multi_view_listener.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/multi_view_listener.rs index 8d0e69db2e9a..a00234a99808 100644 --- a/substrate/client/transaction-pool/src/fork_aware_txpool/multi_view_listener.rs +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/multi_view_listener.rs @@ -36,6 +36,8 @@ use std::{ }; use tokio_stream::StreamMap; +use super::dropped_watcher::{DroppedReason, DroppedTransaction}; + /// A side channel allowing to control the external stream instance (one per transaction) with /// [`ControllerCommand`]. /// @@ -79,7 +81,7 @@ enum ControllerCommand { /// Notifies that a transaction was dropped from the pool. /// /// If all preconditions are met, an external dropped event will be sent out. - TransactionDropped, + TransactionDropped(DroppedReason>), } impl std::fmt::Debug for ControllerCommand @@ -99,8 +101,8 @@ where ControllerCommand::TransactionBroadcasted(_) => { write!(f, "ListenerAction::TransactionBroadcasted(...)") }, - ControllerCommand::TransactionDropped => { - write!(f, "ListenerAction::TransactionDropped") + ControllerCommand::TransactionDropped(r) => { + write!(f, "ListenerAction::TransactionDropped {r:?}") }, } } @@ -268,6 +270,7 @@ where /// stream map. fn remove_view(&mut self, block_hash: BlockHash) { self.status_stream_map.remove(&block_hash); + self.views_keeping_tx_valid.remove(&block_hash); trace!(target: LOG_TARGET, "[{:?}] RemoveView view: {:?} views:{:?}", self.tx_hash, block_hash, self.status_stream_map.keys().collect::>()); } } @@ -282,6 +285,11 @@ where Self { controllers: Default::default() } } + /// Returns `true` if the listener contains a stream controller for the specified hash. + pub fn contains_tx(&self, tx_hash: &ExtrinsicHash) -> bool { + self.controllers.read().contains_key(tx_hash) + } + /// Creates an external aggregated stream of events for given transaction. /// /// This method initializes an `ExternalWatcherContext` for the provided transaction hash, sets @@ -346,11 +354,16 @@ where log::trace!(target: LOG_TARGET, "[{:?}] mvl sending out: Broadcasted", ctx.tx_hash); return Some((TransactionStatus::Broadcast(peers), ctx)) }, - ControllerCommand::TransactionDropped => { + ControllerCommand::TransactionDropped(DroppedReason::LimitsEnforced) => { log::trace!(target: LOG_TARGET, "[{:?}] mvl sending out: Dropped", ctx.tx_hash); ctx.terminate = true; return Some((TransactionStatus::Dropped, ctx)) }, + ControllerCommand::TransactionDropped(DroppedReason::Usurped(by)) => { + log::trace!(target: LOG_TARGET, "[{:?}] mvl sending out: Usurped({:?})", ctx.tx_hash, by); + ctx.terminate = true; + return Some((TransactionStatus::Usurped(by), ctx)) + }, } }, }; @@ -445,16 +458,15 @@ where /// /// This method sends a `TransactionDropped` command to the controller of each requested /// transaction prompting and external `Broadcasted` event. - pub(crate) fn transactions_dropped(&self, dropped: &[ExtrinsicHash]) { + pub(crate) fn transaction_dropped(&self, dropped: DroppedTransaction>) { let mut controllers = self.controllers.write(); - debug!(target: LOG_TARGET, "mvl::transactions_dropped: {:?}", dropped); - for tx_hash in dropped { - if let Some(tx) = controllers.remove(&tx_hash) { - debug!(target: LOG_TARGET, "[{:?}] transaction_dropped", tx_hash); - if let Err(e) = tx.unbounded_send(ControllerCommand::TransactionDropped) { - trace!(target: LOG_TARGET, "[{:?}] transactions_dropped: send message failed: {:?}", tx_hash, e); - }; - } + debug!(target: LOG_TARGET, "mvl::transaction_dropped: {:?}", dropped); + if let Some(tx) = controllers.remove(&dropped.tx_hash) { + let DroppedTransaction { tx_hash, reason } = dropped; + debug!(target: LOG_TARGET, "[{:?}] transaction_dropped", tx_hash); + if let Err(e) = tx.unbounded_send(ControllerCommand::TransactionDropped(reason)) { + trace!(target: LOG_TARGET, "[{:?}] transaction_dropped: send message failed: {:?}", tx_hash, e); + }; } } diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/revalidation_worker.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/revalidation_worker.rs index 9464ab3f5766..eb898c35a134 100644 --- a/substrate/client/transaction-pool/src/fork_aware_txpool/revalidation_worker.rs +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/revalidation_worker.rs @@ -186,9 +186,9 @@ mod tests { use crate::{ common::tests::{uxt, TestApi}, fork_aware_txpool::view::FinishRevalidationLocalChannels, + TimedTransactionSource, }; use futures::executor::block_on; - use sc_transaction_pool_api::TransactionSource; use substrate_test_runtime::{AccountId, Transfer, H256}; use substrate_test_runtime_client::AccountKeyring::Alice; #[test] @@ -212,9 +212,10 @@ mod tests { nonce: 0, }); - let _ = block_on( - view.submit_many(TransactionSource::External, std::iter::once(uxt.clone().into())), - ); + let _ = block_on(view.submit_many(std::iter::once(( + TimedTransactionSource::new_external(false), + uxt.clone().into(), + )))); assert_eq!(api.validation_requests().len(), 1); let (finish_revalidation_request_tx, finish_revalidation_request_rx) = diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/tx_mem_pool.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/tx_mem_pool.rs index 86c07008c3f3..af274b18c925 100644 --- a/substrate/client/transaction-pool/src/fork_aware_txpool/tx_mem_pool.rs +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/tx_mem_pool.rs @@ -26,24 +26,31 @@ //! it), while on other forks tx can be valid. Depending on which view is chosen to be cloned, //! such transaction could not be present in the newly created view. -use super::{metrics::MetricsLink as PrometheusMetrics, multi_view_listener::MultiViewListener}; +use super::{ + metrics::MetricsLink as PrometheusMetrics, multi_view_listener::MultiViewListener, + view_store::ViewStoreSubmitOutcome, +}; use crate::{ common::log_xt::log_xt_trace, graph, - graph::{tracked_map::Size, ExtrinsicFor, ExtrinsicHash}, + graph::{base_pool::TimedTransactionSource, tracked_map::Size, ExtrinsicFor, ExtrinsicHash}, LOG_TARGET, }; use futures::FutureExt; use itertools::Itertools; -use sc_transaction_pool_api::TransactionSource; +use sc_transaction_pool_api::{TransactionPriority, TransactionSource}; use sp_blockchain::HashAndNumber; use sp_runtime::{ traits::Block as BlockT, transaction_validity::{InvalidTransaction, TransactionValidityError}, }; use std::{ + cmp::Ordering, collections::HashMap, - sync::{atomic, atomic::AtomicU64, Arc}, + sync::{ + atomic::{self, AtomicU64}, + Arc, + }, time::Instant, }; @@ -74,9 +81,12 @@ where /// Size of the extrinsics actual body. bytes: usize, /// Transaction source. - source: TransactionSource, + source: TimedTransactionSource, /// When the transaction was revalidated, used to periodically revalidate the mem pool buffer. validated_at: AtomicU64, + /// Priority of transaction at some block. It is assumed it will not be changed often. + //todo: Option is needed here. This means lock. So maybe +AtomicBool? + priority: AtomicU64, //todo: we need to add future / ready status at finalized block. //If future transactions are stuck in tx_mem_pool (due to limits being hit), we need a means // to replace them somehow with newly coming transactions. @@ -95,18 +105,46 @@ where /// Shall the progress of transaction be watched. /// /// Was transaction sent with `submit_and_watch`. - fn is_watched(&self) -> bool { + pub(crate) fn is_watched(&self) -> bool { self.watched } /// Creates a new instance of wrapper for unwatched transaction. fn new_unwatched(source: TransactionSource, tx: ExtrinsicFor, bytes: usize) -> Self { - Self { watched: false, tx, source, validated_at: AtomicU64::new(0), bytes } + Self::new(false, source, tx, bytes) } /// Creates a new instance of wrapper for watched transaction. fn new_watched(source: TransactionSource, tx: ExtrinsicFor, bytes: usize) -> Self { - Self { watched: true, tx, source, validated_at: AtomicU64::new(0), bytes } + Self::new(true, source, tx, bytes) + } + + /// Creates a new instance of wrapper for a transaction with no priority. + fn new( + watched: bool, + source: TransactionSource, + tx: ExtrinsicFor, + bytes: usize, + ) -> Self { + Self::new_with_priority(watched, source, tx, bytes, 0) + } + + /// Creates a new instance of wrapper for a transaction with given priority. + fn new_with_priority( + watched: bool, + source: TransactionSource, + tx: ExtrinsicFor, + bytes: usize, + priority: TransactionPriority, + ) -> Self { + Self { + watched, + tx, + source: TimedTransactionSource::from_transaction_source(source, true), + validated_at: AtomicU64::new(0), + bytes, + priority: AtomicU64::new(priority), + } } /// Provides a clone of actual transaction body. @@ -117,8 +155,13 @@ where } /// Returns the source of the transaction. - pub(crate) fn source(&self) -> TransactionSource { - self.source + pub(crate) fn source(&self) -> TimedTransactionSource { + self.source.clone() + } + + /// Returns the priority of the transaction. + pub(crate) fn priority(&self) -> TransactionPriority { + self.priority.load(atomic::Ordering::Relaxed) } } @@ -174,6 +217,23 @@ where max_transactions_total_bytes: usize, } +/// Helper structure to encapsulate a result of [`TxMemPool::try_insert`]. +#[derive(Debug)] +pub(super) struct InsertionInfo { + pub(super) hash: Hash, + pub(super) source: TimedTransactionSource, + pub(super) removed: Vec, +} + +impl InsertionInfo { + fn new(hash: Hash, source: TimedTransactionSource) -> Self { + Self::new_with_removed(hash, source, Default::default()) + } + fn new_with_removed(hash: Hash, source: TimedTransactionSource, removed: Vec) -> Self { + Self { hash, source, removed } + } +} + impl TxMemPool where Block: BlockT, @@ -220,8 +280,8 @@ where pub(super) fn get_by_hash( &self, hash: ExtrinsicHash, - ) -> Option> { - self.transactions.read().get(&hash).map(|t| t.tx()) + ) -> Option>> { + self.transactions.read().get(&hash).map(Clone::clone) } /// Returns a tuple with the count of unwatched and watched transactions in the memory pool. @@ -231,6 +291,11 @@ where (transactions.len() - watched_count, watched_count) } + /// Returns a total number of transactions kept within mempool. + pub fn len(&self) -> usize { + self.transactions.read().len() + } + /// Returns the number of bytes used by all extrinsics in the the pool. #[cfg(test)] pub fn bytes(&self) -> usize { @@ -249,26 +314,126 @@ where &self, hash: ExtrinsicHash, tx: TxInMemPool, - ) -> Result, ChainApi::Error> { - let bytes = self.transactions.bytes(); + ) -> Result>, sc_transaction_pool_api::error::Error> { let mut transactions = self.transactions.write(); + + let bytes = self.transactions.bytes(); + let result = match ( - !self.is_limit_exceeded(transactions.len() + 1, bytes + tx.bytes), + self.is_limit_exceeded(transactions.len() + 1, bytes + tx.bytes), transactions.contains_key(&hash), ) { - (true, false) => { + (false, false) => { + let source = tx.source(); transactions.insert(hash, Arc::from(tx)); - Ok(hash) + Ok(InsertionInfo::new(hash, source)) }, (_, true) => - Err(sc_transaction_pool_api::error::Error::AlreadyImported(Box::new(hash)).into()), - (false, _) => Err(sc_transaction_pool_api::error::Error::ImmediatelyDropped.into()), + Err(sc_transaction_pool_api::error::Error::AlreadyImported(Box::new(hash))), + (true, _) => Err(sc_transaction_pool_api::error::Error::ImmediatelyDropped), }; - log::trace!(target: LOG_TARGET, "[{:?}] mempool::try_insert: {:?}", hash, result); + log::trace!(target: LOG_TARGET, "[{:?}] mempool::try_insert: {:?}", hash, result.as_ref().map(|r| r.hash)); result } + /// Attempts to insert a new transaction in the memory pool and drop some worse existing + /// transactions. + /// + /// This operation will not overflow the limit of the mempool. It means that cumulative + /// size of removed transactions will be equal (or greated) then size of newly inserted + /// transaction. + /// + /// Returns a `Result` containing `InsertionInfo` if the new transaction is successfully + /// inserted; otherwise, returns an appropriate error indicating the failure. + pub(super) fn try_insert_with_dropping( + &self, + hash: ExtrinsicHash, + new_tx: TxInMemPool, + ) -> Result>, sc_transaction_pool_api::error::Error> { + let mut transactions = self.transactions.write(); + + if transactions.contains_key(&hash) { + return Err(sc_transaction_pool_api::error::Error::AlreadyImported(Box::new(hash))); + } + + let mut sorted = + transactions.iter().map(|(h, v)| (h.clone(), v.clone())).collect::>(); + + // When pushing higher prio transaction, we need to find a number of lower prio txs, such + // that the sum of their bytes is ge then size of new tx. Otherwise we could overflow size + // limits. Naive way to do it - rev-sort by priority and eat the tail. + + // reverse (lowest prio last) + sorted.sort_by(|(_, a), (_, b)| match b.priority().cmp(&a.priority()) { + Ordering::Equal => match (a.source.timestamp, b.source.timestamp) { + (Some(a), Some(b)) => b.cmp(&a), + _ => Ordering::Equal, + }, + ordering @ _ => ordering, + }); + + let required_size = new_tx.bytes; + let mut total_size = 0usize; + let mut to_be_removed = vec![]; + + while total_size < required_size { + let Some((worst_hash, worst_tx)) = sorted.pop() else { + return Err(sc_transaction_pool_api::error::Error::ImmediatelyDropped); + }; + + if worst_tx.priority() >= new_tx.priority() { + return Err(sc_transaction_pool_api::error::Error::ImmediatelyDropped); + } + + total_size += worst_tx.bytes; + to_be_removed.push(worst_hash); + } + + let source = new_tx.source(); + transactions.insert(hash, Arc::from(new_tx)); + for worst_hash in &to_be_removed { + transactions.remove(worst_hash); + } + debug_assert!(!self.is_limit_exceeded(transactions.len(), self.transactions.bytes())); + + Ok(InsertionInfo::new_with_removed(hash, source, to_be_removed)) + + // let result = match ( + // self.is_limit_exceeded(transactions.len() + 1, bytes + tx.bytes), + // transactions.contains_key(&hash), + // ) { + // (false, false) => { + // let source = tx.source(); + // transactions.insert(hash, Arc::from(tx)); + // Ok(InsertionInfo::new(hash, source)) + // }, + // (_, true) => + // Err(sc_transaction_pool_api::error::Error::AlreadyImported(Box::new(hash))), + // (true, _) => Err(sc_transaction_pool_api::error::Error::ImmediatelyDropped), + // }; + // log::trace!(target: LOG_TARGET, "[{:?}] mempool::try_insert: {:?}", hash, + // result.as_ref().map(|r| r.hash)); + } + + /// Attempts to replace an existing transaction in the memory pool with a new one. + /// + /// Returns a `Result` containing `InsertionInfo` if the new transaction is successfully + /// inserted; otherwise, returns an appropriate error indicating the failure. + pub(super) fn try_replace_transaction( + &self, + new_xt: ExtrinsicFor, + priority: TransactionPriority, + source: TransactionSource, + watched: bool, + ) -> Result>, sc_transaction_pool_api::error::Error> { + let (hash, length) = self.api.hash_and_length(&new_xt); + self.try_insert_with_dropping( + hash, + TxInMemPool::new_with_priority(watched, source, new_xt, length, priority), + ) + } + /// Adds a new unwatched transactions to the internal buffer not exceeding the limit. /// /// Returns the vector of results for each transaction, the order corresponds to the input @@ -277,7 +442,8 @@ where &self, source: TransactionSource, xts: &[ExtrinsicFor], - ) -> Vec, ChainApi::Error>> { + ) -> Vec>, sc_transaction_pool_api::error::Error>> + { let result = xts .iter() .map(|xt| { @@ -294,27 +460,11 @@ where &self, source: TransactionSource, xt: ExtrinsicFor, - ) -> Result, ChainApi::Error> { + ) -> Result>, sc_transaction_pool_api::error::Error> { let (hash, length) = self.api.hash_and_length(&xt); self.try_insert(hash, TxInMemPool::new_watched(source, xt.clone(), length)) } - /// Removes transactions from the memory pool which are specified by the given list of hashes - /// and send the `Dropped` event to the listeners of these transactions. - pub(super) async fn remove_dropped_transactions( - &self, - to_be_removed: &[ExtrinsicHash], - ) { - log::debug!(target: LOG_TARGET, "remove_dropped_transactions count:{:?}", to_be_removed.len()); - log_xt_trace!(target: LOG_TARGET, to_be_removed, "[{:?}] mempool::remove_dropped_transactions"); - let mut transactions = self.transactions.write(); - to_be_removed.iter().for_each(|t| { - transactions.remove(t); - }); - - self.listener.transactions_dropped(to_be_removed); - } - /// Clones and returns a `HashMap` of references to all unwatched transactions in the memory /// pool. pub(super) fn clone_unwatched( @@ -338,9 +488,13 @@ where .collect::>() } - /// Removes a transaction from the memory pool based on a given hash. - pub(super) fn remove(&self, hash: ExtrinsicHash) { - let _ = self.transactions.write().remove(&hash); + /// Removes a transaction with given hash from the memory pool. + pub(super) fn remove_transaction( + &self, + hash: &ExtrinsicHash, + ) -> Option>> { + log::debug!(target: LOG_TARGET, "[{hash:?}] mempool::remove_transaction"); + self.transactions.write().remove(hash) } /// Revalidates a batch of transactions against the provided finalized block. @@ -369,13 +523,13 @@ where }; let validations_futures = input.into_iter().map(|(xt_hash, xt)| { - self.api.validate_transaction(finalized_block.hash, xt.source, xt.tx()).map( - move |validation_result| { + self.api + .validate_transaction(finalized_block.hash, xt.source.source, xt.tx()) + .map(move |validation_result| { xt.validated_at .store(finalized_block.number.into().as_u64(), atomic::Ordering::Relaxed); (xt_hash, validation_result) - }, - ) + }) }); let validation_results = futures::future::join_all(validations_futures).await; let input_len = validation_results.len(); @@ -403,7 +557,7 @@ where log::debug!( target: LOG_TARGET, - "mempool::revalidate: at {finalized_block:?} count:{input_len}/{count} purged:{} took {duration:?}", invalid_hashes.len(), + "mempool::revalidate: at {finalized_block:?} count:{input_len}/{count} invalid_hashes:{} took {duration:?}", invalid_hashes.len(), ); invalid_hashes @@ -438,6 +592,14 @@ where }); self.listener.invalidate_transactions(&invalid_hashes); } + + pub(super) fn update_transaction(&self, outcome: &ViewStoreSubmitOutcome) { + if let Some(priority) = outcome.priority() { + if let Some(tx) = self.transactions.write().get_mut(&outcome.hash()) { + tx.priority.store(priority, atomic::Ordering::Relaxed); + } + } + } } #[cfg(test)] @@ -593,4 +755,6 @@ mod tx_mem_pool_tests { sc_transaction_pool_api::error::Error::ImmediatelyDropped )); } + + //add some test for try_insert_with_dropping } diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/view.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/view.rs index 99095d88cb0a..4f26b5bad84c 100644 --- a/substrate/client/transaction-pool/src/fork_aware_txpool/view.rs +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/view.rs @@ -27,13 +27,13 @@ use super::metrics::MetricsLink as PrometheusMetrics; use crate::{ common::log_xt::log_xt_trace, graph::{ - self, watcher::Watcher, ExtrinsicFor, ExtrinsicHash, IsValidator, ValidatedTransaction, - ValidatedTransactionFor, + self, base_pool::TimedTransactionSource, watcher::Watcher, ExtrinsicFor, ExtrinsicHash, + IsValidator, ValidatedPoolSubmitOutcome, ValidatedTransaction, ValidatedTransactionFor, }, LOG_TARGET, }; use parking_lot::Mutex; -use sc_transaction_pool_api::{error::Error as TxPoolError, PoolStatus, TransactionSource}; +use sc_transaction_pool_api::{error::Error as TxPoolError, PoolStatus}; use sp_blockchain::HashAndNumber; use sp_runtime::{ generic::BlockId, traits::Block as BlockT, transaction_validity::TransactionValidityError, @@ -157,24 +157,23 @@ where /// Imports many unvalidated extrinsics into the view. pub(super) async fn submit_many( &self, - source: TransactionSource, - xts: impl IntoIterator>, - ) -> Vec, ChainApi::Error>> { + xts: impl IntoIterator)>, + ) -> Vec, ChainApi::Error>> { if log::log_enabled!(target: LOG_TARGET, log::Level::Trace) { let xts = xts.into_iter().collect::>(); - log_xt_trace!(target: LOG_TARGET, xts.iter().map(|xt| self.pool.validated_pool().api().hash_and_length(xt).0), "[{:?}] view::submit_many at:{}", self.at.hash); - self.pool.submit_at(&self.at, source, xts).await + log_xt_trace!(target: LOG_TARGET, xts.iter().map(|(_,xt)| self.pool.validated_pool().api().hash_and_length(xt).0), "[{:?}] view::submit_many at:{}", self.at.hash); + self.pool.submit_at(&self.at, xts).await } else { - self.pool.submit_at(&self.at, source, xts).await + self.pool.submit_at(&self.at, xts).await } } /// Import a single extrinsic and starts to watch its progress in the view. pub(super) async fn submit_and_watch( &self, - source: TransactionSource, + source: TimedTransactionSource, xt: ExtrinsicFor, - ) -> Result, ExtrinsicHash>, ChainApi::Error> { + ) -> Result, ChainApi::Error> { log::trace!(target: LOG_TARGET, "[{:?}] view::submit_and_watch at:{}", self.pool.validated_pool().api().hash_and_length(&xt).0, self.at.hash); self.pool.submit_and_watch(&self.at, source, xt).await } @@ -183,7 +182,7 @@ where pub(super) fn submit_local( &self, xt: ExtrinsicFor, - ) -> Result, ChainApi::Error> { + ) -> Result, ChainApi::Error> { let (hash, length) = self.pool.validated_pool().api().hash_and_length(&xt); log::trace!(target: LOG_TARGET, "[{:?}] view::submit_local at:{}", hash, self.at.hash); @@ -193,7 +192,7 @@ where .api() .validate_transaction_blocking( self.at.hash, - TransactionSource::Local, + sc_transaction_pool_api::TransactionSource::Local, Arc::from(xt.clone()), )? .map_err(|e| { @@ -214,7 +213,7 @@ where let validated = ValidatedTransaction::valid_at( block_number.saturated_into::(), hash, - TransactionSource::Local, + TimedTransactionSource::new_local(true), Arc::from(xt), length, validity, @@ -285,7 +284,7 @@ where } _ = async { if let Some(tx) = batch_iter.next() { - let validation_result = (api.validate_transaction(self.at.hash, tx.source, tx.data.clone()).await, tx.hash, tx); + let validation_result = (api.validate_transaction(self.at.hash, tx.source.source, tx.data.clone()).await, tx.hash, tx); validation_results.push(validation_result); } else { self.revalidation_worker_channels.lock().as_mut().map(|ch| ch.remove_sender()); @@ -324,7 +323,7 @@ where ValidatedTransaction::valid_at( self.at.number.saturated_into::(), tx_hash, - tx.source, + tx.source.clone(), tx.data.clone(), api.hash_and_length(&tx.data).1, validity, @@ -455,4 +454,25 @@ where ); } } + + /// Returns true if the transaction with given hash is already imported into the view. + pub(super) fn is_imported(&self, tx_hash: &ExtrinsicHash) -> bool { + const IGNORE_BANNED: bool = false; + self.pool.validated_pool().check_is_known(tx_hash, IGNORE_BANNED).is_err() + } + + /// Removes the whole transaction subtree from the inner pool. + /// + /// Intended to be called when removal is a result of replacement. Provided `replaced_with` + /// transaction hash is used in emitted _usurped_ event. + pub fn remove_subtree( + &self, + tx_hash: ExtrinsicHash, + listener_action: F, + ) -> Vec> + where + F: Fn(&mut crate::graph::Listener, ExtrinsicHash), + { + self.pool.validated_pool().remove_subtree(tx_hash, listener_action) + } } diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/view_store.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/view_store.rs index f23dcedd5bfd..89bf47cc3895 100644 --- a/substrate/client/transaction-pool/src/fork_aware_txpool/view_store.rs +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/view_store.rs @@ -24,17 +24,120 @@ use super::{ }; use crate::{ fork_aware_txpool::dropped_watcher::MultiViewDroppedWatcherController, - graph, - graph::{base_pool::Transaction, ExtrinsicFor, ExtrinsicHash, TransactionFor}, + graph::{ + self, + base_pool::{TimedTransactionSource, Transaction}, + BaseSubmitOutcome, ExtrinsicFor, ExtrinsicHash, TransactionFor, ValidatedPoolSubmitOutcome, + }, ReadyIteratorFor, LOG_TARGET, }; use futures::prelude::*; use itertools::Itertools; use parking_lot::RwLock; -use sc_transaction_pool_api::{error::Error as PoolError, PoolStatus, TransactionSource}; +use sc_transaction_pool_api::{error::Error as PoolError, PoolStatus}; use sp_blockchain::TreeRoute; use sp_runtime::{generic::BlockId, traits::Block as BlockT}; -use std::{collections::HashMap, sync::Arc, time::Instant}; +use std::{ + collections::{hash_map::Entry, HashMap, HashSet}, + sync::Arc, + time::Instant, +}; + +/// Helper struct to maintain the context for pending transaction submission, executed for +/// newly inserted views. +#[derive(Clone)] +struct PendingTxSubmission +where + ChainApi: graph::ChainApi, +{ + /// New transaction replacing the old one. + xt: ExtrinsicFor, + /// Source of the transaction. + source: TimedTransactionSource, + /// Inidicates if transaction is watched. + watched: bool, +} + +/// Helper type representing the callback allowing to trigger per-transaction events on +/// `ValidatedPool`'s listener. +type RemovalListener = + Arc, ExtrinsicHash) + Send + Sync>; + +/// Helper struct to maintain the context for pending transaction removal, executed for +/// newly inserted views. +struct PendingTxRemoval +where + ChainApi: graph::ChainApi, +{ + /// Hash of the transaction that will be removed, + xt_hash: ExtrinsicHash, + /// Action that shall be executed on underlying `ValidatedPool`'s listener. + listener_action: RemovalListener, +} + +/// This enum represents an action that should be executed on the newly built +/// view before this view is inserted into the view store. +enum PreInsertAction +where + ChainApi: graph::ChainApi, +{ + /// Represents the action of submitting a new transaction. Intended to use to handle usurped + /// transactions. + SubmitTx(PendingTxSubmission), + + /// Represents the action of removing a subtree of transactions. + RemoveSubtree(PendingTxRemoval), +} + +/// Represents a task awaiting execution, to be performed immediately prior to the view insertion +/// into the view store. +struct PendingPreInsertTask +where + ChainApi: graph::ChainApi, +{ + /// The action to be applied when inserting a new view. + action: PreInsertAction, + /// Indicates if the action was already applied to all the views in the view_store. + /// If true, it can be removed after inserting any new view. + processed: bool, +} + +impl PendingPreInsertTask +where + ChainApi: graph::ChainApi, +{ + /// Creates new unprocessed instance of pending transaction submission. + fn new_submission_action( + xt: ExtrinsicFor, + source: TimedTransactionSource, + watched: bool, + ) -> Self { + Self { + processed: false, + action: PreInsertAction::SubmitTx(PendingTxSubmission { xt, source, watched }), + } + } + + /// Creates new processed instance of pending transaction removal. + fn new_removal_action( + xt_hash: ExtrinsicHash, + listener: RemovalListener, + ) -> Self { + Self { + processed: false, + action: PreInsertAction::RemoveSubtree(PendingTxRemoval { + xt_hash, + listener_action: listener, + }), + } + } + + /// Marks a task as done for every view present in view store. Basically means that can be + /// removed on new view insertion. + fn mark_processed(&mut self) { + self.processed = true; + } +} /// The helper structure encapsulates all the views. pub(super) struct ViewStore @@ -62,6 +165,24 @@ where pub(super) most_recent_view: RwLock>, /// The controller of multi view dropped stream. pub(super) dropped_stream_controller: MultiViewDroppedWatcherController, + /// The map used to synchronize replacement of transactions between maintain and dropped + /// notifcication threads. It is meant to assure that replaced transaction is also removed from + /// newly built views in maintain process. + /// + /// The map's key is hash of actionable extrinsic (to avoid duplicated entries). + pending_txs_tasks: RwLock, PendingPreInsertTask>>, +} + +/// Type alias to outcome of submission to `ViewStore`. +pub(super) type ViewStoreSubmitOutcome = + BaseSubmitOutcome>; + +impl From> + for ViewStoreSubmitOutcome +{ + fn from(value: ValidatedPoolSubmitOutcome) -> Self { + Self::new(value.hash(), value.priority()) + } } impl ViewStore @@ -83,15 +204,15 @@ where listener, most_recent_view: RwLock::from(None), dropped_stream_controller, + pending_txs_tasks: Default::default(), } } /// Imports a bunch of unverified extrinsics to every active view. pub(super) async fn submit( &self, - source: TransactionSource, - xts: impl IntoIterator> + Clone, - ) -> HashMap, ChainApi::Error>>> { + xts: impl IntoIterator)> + Clone, + ) -> HashMap, ChainApi::Error>>> { let submit_futures = { let active_views = self.active_views.read(); active_views @@ -99,7 +220,16 @@ where .map(|(_, view)| { let view = view.clone(); let xts = xts.clone(); - async move { (view.at.hash, view.submit_many(source, xts).await) } + async move { + ( + view.at.hash, + view.submit_many(xts) + .await + .into_iter() + .map(|r| r.map(Into::into)) + .collect::>(), + ) + } }) .collect::>() }; @@ -112,7 +242,7 @@ where pub(super) fn submit_local( &self, xt: ExtrinsicFor, - ) -> Result, ChainApi::Error> { + ) -> Result, ChainApi::Error> { let active_views = self .active_views .read() @@ -127,12 +257,14 @@ where .map(|view| view.submit_local(xt.clone())) .find_or_first(Result::is_ok); - if let Some(Err(err)) = result { - log::trace!(target: LOG_TARGET, "[{:?}] submit_local: err: {}", tx_hash, err); - return Err(err) - }; - - Ok(tx_hash) + match result { + Some(Err(err)) => { + log::trace!(target: LOG_TARGET, "[{:?}] submit_local: err: {}", tx_hash, err); + Err(err) + }, + None => Ok(ViewStoreSubmitOutcome::new(tx_hash, None)), + Some(Ok(r)) => Ok(r.into()), + } } /// Import a single extrinsic and starts to watch its progress in the pool. @@ -145,9 +277,9 @@ where pub(super) async fn submit_and_watch( &self, _at: Block::Hash, - source: TransactionSource, + source: TimedTransactionSource, xt: ExtrinsicFor, - ) -> Result, ChainApi::Error> { + ) -> Result, ChainApi::Error> { let tx_hash = self.api.hash_and_length(&xt).0; let Some(external_watcher) = self.listener.create_external_watcher_for_tx(tx_hash) else { return Err(PoolError::AlreadyImported(Box::new(tx_hash)).into()) @@ -159,15 +291,16 @@ where .map(|(_, view)| { let view = view.clone(); let xt = xt.clone(); + let source = source.clone(); async move { match view.submit_and_watch(source, xt).await { - Ok(watcher) => { + Ok(mut result) => { self.listener.add_view_watcher_for_tx( tx_hash, view.at.hash, - watcher.into_stream().boxed(), + result.expect_watcher().into_stream().boxed(), ); - Ok(()) + Ok(result) }, Err(e) => Err(e), } @@ -175,17 +308,20 @@ where }) .collect::>() }; - let maybe_error = futures::future::join_all(submit_and_watch_futures) + let result = futures::future::join_all(submit_and_watch_futures) .await .into_iter() .find_or_first(Result::is_ok); - if let Some(Err(err)) = maybe_error { - log::trace!(target: LOG_TARGET, "[{:?}] submit_and_watch: err: {}", tx_hash, err); - return Err(err); - }; - - Ok(external_watcher) + match result { + Some(Err(err)) => { + log::trace!(target: LOG_TARGET, "[{:?}] submit_and_watch: err: {}", tx_hash, err); + return Err(err); + }, + Some(Ok(result)) => + Ok(ViewStoreSubmitOutcome::from(result).with_watcher(external_watcher)), + None => Ok(ViewStoreSubmitOutcome::new(tx_hash, None).with_watcher(external_watcher)), + } } /// Returns the pool status for every active view. @@ -261,12 +397,20 @@ where ) -> Vec, ExtrinsicFor>> { self.most_recent_view .read() - .map(|at| self.get_view_at(at, true)) + .map(|at| self.futures_at(at)) .flatten() - .map(|(v, _)| v.pool.validated_pool().pool.read().futures().cloned().collect()) .unwrap_or_default() } + /// Returns a list of future transactions in the view at given block hash. + pub(super) fn futures_at( + &self, + at: Block::Hash, + ) -> Option, ExtrinsicFor>>> { + self.get_view_at(at, true) + .map(|(v, _)| v.pool.validated_pool().pool.read().futures().cloned().collect()) + } + /// Collects all the transactions included in the blocks on the provided `tree_route` and /// triggers finalization event for them. /// @@ -329,12 +473,16 @@ where /// - moved to the inactive views set (`inactive_views`), /// - removed from the multi view listeners. /// - /// The `most_recent_view` is update with the reference to the newly inserted view. + /// The `most_recent_view` is updated with the reference to the newly inserted view. + /// + /// If there are any pending tx replacments, they are applied to the new view. pub(super) async fn insert_new_view( &self, view: Arc>, tree_route: &TreeRoute, ) { + self.apply_pending_tx_replacements(view.clone()).await; + //note: most_recent_view must be synced with changes in in/active_views. { let mut most_recent_view_lock = self.most_recent_view.write(); @@ -386,8 +534,10 @@ where let mut removed_views = vec![]; { - self.active_views - .read() + let active_views = self.active_views.read(); + let inactive_views = self.inactive_views.read(); + + active_views .iter() .filter(|(hash, v)| !match finalized_number { Err(_) | Ok(None) => **hash == finalized_hash, @@ -396,11 +546,8 @@ where }) .map(|(_, v)| removed_views.push(v.at.hash)) .for_each(drop); - } - { - self.inactive_views - .read() + inactive_views .iter() .filter(|(_, v)| !match finalized_number { Err(_) | Ok(None) => false, @@ -438,30 +585,48 @@ where let finalized_xts = self.finalize_route(finalized_hash, tree_route).await; let finalized_number = self.api.block_id_to_number(&BlockId::Hash(finalized_hash)); + let mut dropped_views = vec![]; //clean up older then finalized { let mut active_views = self.active_views.write(); - active_views.retain(|hash, v| match finalized_number { - Err(_) | Ok(None) => *hash == finalized_hash, - Ok(Some(n)) if v.at.number == n => *hash == finalized_hash, - Ok(Some(n)) => v.at.number > n, + let mut inactive_views = self.inactive_views.write(); + active_views.retain(|hash, v| { + let retain = match finalized_number { + Err(_) | Ok(None) => *hash == finalized_hash, + Ok(Some(n)) if v.at.number == n => *hash == finalized_hash, + Ok(Some(n)) => v.at.number > n, + }; + if !retain { + dropped_views.push(*hash); + } + retain }); - } - { - let mut inactive_views = self.inactive_views.write(); - inactive_views.retain(|_, v| match finalized_number { - Err(_) | Ok(None) => false, - Ok(Some(n)) => v.at.number >= n, + inactive_views.retain(|hash, v| { + let retain = match finalized_number { + Err(_) | Ok(None) => false, + Ok(Some(n)) => v.at.number >= n, + }; + if !retain { + dropped_views.push(*hash); + } + retain }); log::trace!(target:LOG_TARGET,"handle_finalized: inactive_views: {:?}", inactive_views.keys()); } - self.listener.remove_view(finalized_hash); + log::trace!(target:LOG_TARGET,"handle_finalized: dropped_views: {:?}", dropped_views); + self.listener.remove_stale_controllers(); self.dropped_stream_controller.remove_finalized_txs(finalized_xts.clone()); + self.listener.remove_view(finalized_hash); + for view in dropped_views { + self.listener.remove_view(view); + self.dropped_stream_controller.remove_view(view); + } + finalized_xts } @@ -484,4 +649,205 @@ where futures::future::join_all(finish_revalidation_futures).await; log::trace!(target:LOG_TARGET,"finish_background_revalidations took {:?}", start.elapsed()); } + + /// Replaces an existing transaction in the view_store with a new one. + /// + /// Attempts to replace a transaction identified by `replaced` with a new transaction `xt`. + /// + /// Before submitting a transaction to the views, the new *unprocessed* transaction replacement + /// record will be inserted into a pending replacement map. Once the submission to all the views + /// is accomplished, the record is marked as *processed*. + /// + /// This map is later applied in `insert_new_view` method executed from different thread. + /// + /// If the transaction is already being replaced, it will simply return without making + /// changes. + pub(super) async fn replace_transaction( + &self, + source: TimedTransactionSource, + xt: ExtrinsicFor, + replaced: ExtrinsicHash, + watched: bool, + ) { + if let Entry::Vacant(entry) = self.pending_txs_tasks.write().entry(replaced) { + entry.insert(PendingPreInsertTask::new_submission_action( + xt.clone(), + source.clone(), + watched, + )); + } else { + return + }; + + let xt_hash = self.api.hash_and_length(&xt).0; + log::trace!(target:LOG_TARGET,"[{replaced:?}] replace_transaction wtih {xt_hash:?}, w:{watched}"); + + self.replace_transaction_in_views(source, xt, xt_hash, replaced, watched).await; + + if let Some(replacement) = self.pending_txs_tasks.write().get_mut(&replaced) { + replacement.mark_processed(); + } + } + + /// Applies pending transaction replacements to the specified view. + /// + /// After application, all already processed replacements are removed. + async fn apply_pending_tx_replacements(&self, view: Arc>) { + let mut futures = vec![]; + for replacement in self.pending_txs_tasks.read().values() { + match replacement.action { + PreInsertAction::SubmitTx(ref submission) => { + let xt_hash = self.api.hash_and_length(&submission.xt).0; + futures.push(self.replace_transaction_in_view( + view.clone(), + submission.source.clone(), + submission.xt.clone(), + xt_hash, + submission.watched, + )); + }, + PreInsertAction::RemoveSubtree(ref removal) => { + view.remove_subtree(removal.xt_hash, &*removal.listener_action); + }, + } + } + let _results = futures::future::join_all(futures).await; + self.pending_txs_tasks.write().retain(|_, r| r.processed); + } + + /// Submits `xt` to the given view. + /// + /// For watched transaction stream is added to the listener. + async fn replace_transaction_in_view( + &self, + view: Arc>, + source: TimedTransactionSource, + xt: ExtrinsicFor, + xt_hash: ExtrinsicHash, + watched: bool, + ) { + if watched { + match view.submit_and_watch(source, xt).await { + Ok(mut result) => { + self.listener.add_view_watcher_for_tx( + xt_hash, + view.at.hash, + result.expect_watcher().into_stream().boxed(), + ); + }, + Err(e) => { + log::trace!( + target:LOG_TARGET, + "[{:?}] replace_transaction: submit_and_watch to {} failed {}", + xt_hash, view.at.hash, e + ); + }, + } + } else { + if let Some(Err(e)) = view.submit_many(std::iter::once((source, xt))).await.pop() { + log::trace!( + target:LOG_TARGET, + "[{:?}] replace_transaction: submit to {} failed {}", + xt_hash, view.at.hash, e + ); + } + } + } + + /// Sends `xt` to every view (both active and inactive) containing `replaced` extrinsics. + /// + /// It is assumed that transaction is already known by the pool. Intended to ba called when `xt` + /// is replacing `replaced` extrinsic. + async fn replace_transaction_in_views( + &self, + source: TimedTransactionSource, + xt: ExtrinsicFor, + xt_hash: ExtrinsicHash, + replaced: ExtrinsicHash, + watched: bool, + ) { + if watched && !self.listener.contains_tx(&xt_hash) { + log::trace!( + target:LOG_TARGET, + "error: replace_transaction_in_views: no listener for watched transaction {:?}", + xt_hash, + ); + return; + } + + let submit_futures = { + let active_views = self.active_views.read(); + let inactive_views = self.inactive_views.read(); + active_views + .iter() + .chain(inactive_views.iter()) + .filter(|(_, view)| view.is_imported(&replaced)) + .map(|(_, view)| { + self.replace_transaction_in_view( + view.clone(), + source.clone(), + xt.clone(), + xt_hash, + watched, + ) + }) + .collect::>() + }; + let _results = futures::future::join_all(submit_futures).await; + } + + /// Removes a transaction subtree from the every view in the view_store, starting from the given + /// transaction hash. + /// + /// This function traverses the dependency graph of transactions and removes the specified + /// transaction along with all its descendant transactions from every view. + /// + /// A `listener_action` callback function is invoked for every transaction that is removed, + /// providing a reference to the pool's listener and the hash of the removed transaction. This + /// allows to trigger the required events. Note listener may be called multiple times for the + /// same hash. + /// + /// Function will also schedule view pre-insertion actions to ensure that transactions will be + /// removed from newly created view. + /// + /// Returns a vector containing the hashes of all removed transactions, including the root + /// transaction specified by `tx_hash`. Vector contains only unique hashes. + pub(super) fn remove_transaction_subtree( + &self, + xt_hash: ExtrinsicHash, + listener_action: F, + ) -> Vec> + where + F: Fn(&mut crate::graph::Listener, ExtrinsicHash) + + Clone + + Send + + Sync + + 'static, + { + if let Entry::Vacant(entry) = self.pending_txs_tasks.write().entry(xt_hash) { + entry.insert(PendingPreInsertTask::new_removal_action( + xt_hash, + Arc::from(listener_action.clone()), + )); + }; + + let mut seen = HashSet::new(); + + let removed = self + .active_views + .read() + .iter() + .chain(self.inactive_views.read().iter()) + .filter(|(_, view)| view.is_imported(&xt_hash)) + .map(|(_, view)| view.remove_subtree(xt_hash, &listener_action)) + .flatten() + .filter(|xt_hash| seen.insert(*xt_hash)) + .collect(); + + if let Some(removal_action) = self.pending_txs_tasks.write().get_mut(&xt_hash) { + removal_action.mark_processed(); + } + + removed + } } diff --git a/substrate/client/transaction-pool/src/graph/base_pool.rs b/substrate/client/transaction-pool/src/graph/base_pool.rs index e4c3a6c425a9..b3f11852b138 100644 --- a/substrate/client/transaction-pool/src/graph/base_pool.rs +++ b/substrate/client/transaction-pool/src/graph/base_pool.rs @@ -20,7 +20,7 @@ //! //! For a more full-featured pool, have a look at the `pool` module. -use std::{cmp::Ordering, collections::HashSet, fmt, hash, sync::Arc}; +use std::{cmp::Ordering, collections::HashSet, fmt, hash, sync::Arc, time::Instant}; use crate::LOG_TARGET; use log::{trace, warn}; @@ -30,8 +30,8 @@ use sp_core::hexdisplay::HexDisplay; use sp_runtime::{ traits::Member, transaction_validity::{ - TransactionLongevity as Longevity, TransactionPriority as Priority, - TransactionSource as Source, TransactionTag as Tag, + TransactionLongevity as Longevity, TransactionPriority as Priority, TransactionSource, + TransactionTag as Tag, }, }; @@ -83,6 +83,38 @@ pub struct PruneStatus { pub pruned: Vec>>, } +/// A transaction source that includes a timestamp indicating when the transaction was submitted. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct TimedTransactionSource { + /// The original source of the transaction. + pub source: TransactionSource, + + /// The time at which the transaction was submitted. + pub timestamp: Option, +} + +impl TimedTransactionSource { + /// Creates a new instance with an internal `TransactionSource::InBlock` source and an optional + /// timestamp. + pub fn new_in_block(with_timestamp: bool) -> Self { + Self { source: TransactionSource::InBlock, timestamp: with_timestamp.then(Instant::now) } + } + /// Creates a new instance with an internal `TransactionSource::External` source and an optional + /// timestamp. + pub fn new_external(with_timestamp: bool) -> Self { + Self { source: TransactionSource::External, timestamp: with_timestamp.then(Instant::now) } + } + /// Creates a new instance with an internal `TransactionSource::Local` source and an optional + /// timestamp. + pub fn new_local(with_timestamp: bool) -> Self { + Self { source: TransactionSource::Local, timestamp: with_timestamp.then(Instant::now) } + } + /// Creates a new instance with an given source and an optional timestamp. + pub fn from_transaction_source(source: TransactionSource, with_timestamp: bool) -> Self { + Self { source, timestamp: with_timestamp.then(Instant::now) } + } +} + /// Immutable transaction #[derive(PartialEq, Eq, Clone)] pub struct Transaction { @@ -102,8 +134,8 @@ pub struct Transaction { pub provides: Vec, /// Should that transaction be propagated. pub propagate: bool, - /// Source of that transaction. - pub source: Source, + /// Timed source of that transaction. + pub source: TimedTransactionSource, } impl AsRef for Transaction { @@ -157,7 +189,7 @@ impl Transaction { bytes: self.bytes, hash: self.hash.clone(), priority: self.priority, - source: self.source, + source: self.source.clone(), valid_till: self.valid_till, requires: self.requires.clone(), provides: self.provides.clone(), @@ -322,22 +354,36 @@ impl BasePool { if !first { - promoted.push(current_hash); + promoted.push(current_hash.clone()); } + // If there were conflicting future transactions promoted, removed them from + // promoted set. + promoted.retain(|hash| replaced.iter().all(|tx| *hash != tx.hash)); // The transactions were removed from the ready pool. We might attempt to // re-import them. removed.append(&mut replaced); }, + Err(e @ error::Error::TooLowPriority { .. }) => + if first { + trace!(target: LOG_TARGET, "[{:?}] Error importing {first}: {:?}", current_tx.hash, e); + return Err(e) + } else { + trace!(target: LOG_TARGET, "[{:?}] Error importing {first}: {:?}", current_tx.hash, e); + removed.push(current_tx); + promoted.retain(|hash| *hash != current_hash); + }, // transaction failed to be imported. Err(e) => if first { - trace!(target: LOG_TARGET, "[{:?}] Error importing: {:?}", current_hash, e); + trace!(target: LOG_TARGET, "[{:?}] Error importing {first}: {:?}", current_tx.hash, e); return Err(e) } else { - failed.push(current_hash); + trace!(target: LOG_TARGET, "[{:?}] Error importing {first}: {:?}", current_tx.hash, e); + failed.push(current_tx.hash.clone()); }, } first = false; @@ -386,6 +432,16 @@ impl BasePool(&self, init: R, mut f: F) -> R + where + F: FnMut(R, &Arc>) -> R, + { + self.ready + .fold::(init, |acc, current| f(acc, ¤t.transaction.transaction)) + } + /// Makes sure that the transactions in the queues stay within provided limits. /// /// Removes and returns worst transactions from the queues and all transactions that depend on @@ -401,27 +457,29 @@ impl BasePool, _>(|worst, current| { - let transaction = ¤t.transaction; - worst - .map(|worst| { - // Here we don't use `TransactionRef`'s ordering implementation because - // while it prefers priority like need here, it also prefers older - // transactions for inclusion purposes and limit enforcement needs to prefer - // newer transactions instead and drop the older ones. - match worst.transaction.priority.cmp(&transaction.transaction.priority) { - Ordering::Less => worst, - Ordering::Equal => - if worst.insertion_id > transaction.insertion_id { - transaction.clone() - } else { - worst - }, - Ordering::Greater => transaction.clone(), - } - }) - .or_else(|| Some(transaction.clone())) - }); + let worst = + self.ready.fold::>, _>(None, |worst, current| { + let transaction = ¤t.transaction; + worst + .map(|worst| { + // Here we don't use `TransactionRef`'s ordering implementation because + // while it prefers priority like need here, it also prefers older + // transactions for inclusion purposes and limit enforcement needs to + // prefer newer transactions instead and drop the older ones. + match worst.transaction.priority.cmp(&transaction.transaction.priority) + { + Ordering::Less => worst, + Ordering::Equal => + if worst.insertion_id > transaction.insertion_id { + transaction.clone() + } else { + worst + }, + Ordering::Greater => transaction.clone(), + } + }) + .or_else(|| Some(transaction.clone())) + }); if let Some(worst) = worst { removed.append(&mut self.remove_subtree(&[worst.transaction.hash.clone()])) @@ -434,8 +492,24 @@ impl BasePool Some(current.clone()), - Some(ref tx) if tx.imported_at > current.imported_at => Some(current.clone()), - other => other, + Some(worst) => Some( + match (worst.transaction.source.timestamp, current.transaction.source.timestamp) + { + (Some(worst_timestamp), Some(current_timestamp)) => { + if worst_timestamp > current_timestamp { + current.clone() + } else { + worst + } + }, + _ => + if worst.imported_at > current.imported_at { + current.clone() + } else { + worst + }, + }, + ), }); if let Some(worst) = worst { @@ -562,7 +636,7 @@ mod tests { requires: vec![], provides: vec![], propagate: true, - source: Source::External, + source: TimedTransactionSource::new_external(false), } } @@ -760,6 +834,58 @@ mod tests { ); } + #[test] + fn should_remove_conflicting_future() { + let mut pool = pool(); + pool.import(Transaction { + data: vec![3u8].into(), + hash: 3, + requires: vec![vec![1]], + priority: 50u64, + provides: vec![vec![3]], + ..default_tx().clone() + }) + .unwrap(); + assert_eq!(pool.ready().count(), 0); + assert_eq!(pool.ready.len(), 0); + + let tx2 = Transaction { + data: vec![2u8].into(), + hash: 2, + requires: vec![vec![1]], + provides: vec![vec![3]], + ..default_tx().clone() + }; + pool.import(tx2.clone()).unwrap(); + assert_eq!(pool.future.len(), 2); + + let res = pool + .import(Transaction { + data: vec![1u8].into(), + hash: 1, + provides: vec![vec![1]], + ..default_tx().clone() + }) + .unwrap(); + + assert_eq!( + res, + Imported::Ready { + hash: 1, + promoted: vec![3], + failed: vec![], + removed: vec![tx2.into()] + } + ); + + let mut it = pool.ready().into_iter().map(|tx| tx.data[0]); + assert_eq!(it.next(), Some(1)); + assert_eq!(it.next(), Some(3)); + assert_eq!(it.next(), None); + + assert_eq!(pool.future.len(), 0); + } + #[test] fn should_handle_a_cycle() { // given @@ -783,14 +909,14 @@ mod tests { assert_eq!(pool.ready.len(), 0); // when - pool.import(Transaction { + let tx2 = Transaction { data: vec![2u8].into(), hash: 2, requires: vec![vec![2]], provides: vec![vec![0]], ..default_tx().clone() - }) - .unwrap(); + }; + pool.import(tx2.clone()).unwrap(); // then { @@ -817,7 +943,12 @@ mod tests { assert_eq!(it.next(), None); assert_eq!( res, - Imported::Ready { hash: 4, promoted: vec![1, 3], failed: vec![2], removed: vec![] } + Imported::Ready { + hash: 4, + promoted: vec![1, 3], + failed: vec![], + removed: vec![tx2.into()] + } ); assert_eq!(pool.future.len(), 0); } @@ -1024,7 +1155,7 @@ mod tests { ), "Transaction { \ hash: 4, priority: 1000, valid_till: 64, bytes: 1, propagate: true, \ -source: TransactionSource::External, requires: [03, 02], provides: [04], data: [4]}" +source: TimedTransactionSource { source: TransactionSource::External, timestamp: None }, requires: [03, 02], provides: [04], data: [4]}" .to_owned() ); } diff --git a/substrate/client/transaction-pool/src/graph/listener.rs b/substrate/client/transaction-pool/src/graph/listener.rs index a5593920eec4..7b09ee4c6409 100644 --- a/substrate/client/transaction-pool/src/graph/listener.rs +++ b/substrate/client/transaction-pool/src/graph/listener.rs @@ -36,6 +36,7 @@ pub type DroppedByLimitsStream = TracingUnboundedReceiver { + /// Map containing per-transaction sinks for emitting transaction status events. watchers: HashMap>>, finality_watchers: LinkedHashMap, Vec>, @@ -119,32 +120,44 @@ impl Listener, limits_enforced: bool) { + /// Transaction was dropped from the pool because of enforcing the limit. + pub fn limits_enforced(&mut self, tx: &H) { + trace!(target: LOG_TARGET, "[{:?}] Dropped (limits enforced)", tx); + self.fire(tx, |watcher| watcher.limit_enforced()); + + if let Some(ref sink) = self.dropped_by_limits_sink { + if let Err(e) = sink.unbounded_send((tx.clone(), TransactionStatus::Dropped)) { + trace!(target: LOG_TARGET, "[{:?}] dropped_sink: send message failed: {:?}", tx, e); + } + } + } + + /// Transaction was replaced with other extrinsic. + pub fn usurped(&mut self, tx: &H, by: &H) { trace!(target: LOG_TARGET, "[{:?}] Dropped (replaced with {:?})", tx, by); - self.fire(tx, |watcher| match by { - Some(t) => watcher.usurped(t.clone()), - None => watcher.dropped(), - }); - - //note: LimitEnforced could be introduced as new status to get rid of this flag. - if limits_enforced { - if let Some(ref sink) = self.dropped_by_limits_sink { - if let Err(e) = sink.unbounded_send((tx.clone(), TransactionStatus::Dropped)) { - trace!(target: LOG_TARGET, "[{:?}] dropped_sink/future: send message failed: {:?}", tx, e); - } + self.fire(tx, |watcher| watcher.usurped(by.clone())); + + if let Some(ref sink) = self.dropped_by_limits_sink { + if let Err(e) = + sink.unbounded_send((tx.clone(), TransactionStatus::Usurped(by.clone()))) + { + trace!(target: LOG_TARGET, "[{:?}] dropped_sink: send message failed: {:?}", tx, e); } } } + /// Transaction was dropped from the pool because of the failure during the resubmission of + /// revalidate transactions or failure during pruning tags. + pub fn dropped(&mut self, tx: &H) { + trace!(target: LOG_TARGET, "[{:?}] Dropped", tx); + self.fire(tx, |watcher| watcher.dropped()); + } + /// Transaction was removed as invalid. pub fn invalid(&mut self, tx: &H) { trace!(target: LOG_TARGET, "[{:?}] Extrinsic invalid", tx); diff --git a/substrate/client/transaction-pool/src/graph/mod.rs b/substrate/client/transaction-pool/src/graph/mod.rs index d93898b1b22a..008503fb4cdf 100644 --- a/substrate/client/transaction-pool/src/graph/mod.rs +++ b/substrate/client/transaction-pool/src/graph/mod.rs @@ -41,6 +41,9 @@ pub use self::pool::{ BlockHash, ChainApi, ExtrinsicFor, ExtrinsicHash, NumberFor, Options, Pool, RawExtrinsicFor, TransactionFor, ValidatedTransactionFor, }; -pub use validated_pool::{IsValidator, ValidatedTransaction}; +pub use validated_pool::{ + BaseSubmitOutcome, IsValidator, Listener, ValidatedPoolSubmitOutcome, ValidatedTransaction, +}; +pub(crate) use self::pool::CheckBannedBeforeVerify; pub(crate) use listener::DroppedByLimitsEvent; diff --git a/substrate/client/transaction-pool/src/graph/pool.rs b/substrate/client/transaction-pool/src/graph/pool.rs index 2dd8de352c6b..b7aa7276f5e9 100644 --- a/substrate/client/transaction-pool/src/graph/pool.rs +++ b/substrate/client/transaction-pool/src/graph/pool.rs @@ -37,7 +37,7 @@ use std::{ use super::{ base_pool as base, validated_pool::{IsValidator, ValidatedPool, ValidatedTransaction}, - watcher::Watcher, + ValidatedPoolSubmitOutcome, }; /// Modification notification event stream type; @@ -161,7 +161,7 @@ impl Default for Options { /// Should we check that the transaction is banned /// in the pool, before we verify it? #[derive(Copy, Clone)] -enum CheckBannedBeforeVerify { +pub(crate) enum CheckBannedBeforeVerify { Yes, No, } @@ -181,10 +181,8 @@ impl Pool { pub async fn submit_at( &self, at: &HashAndNumber, - source: TransactionSource, - xts: impl IntoIterator>, - ) -> Vec, B::Error>> { - let xts = xts.into_iter().map(|xt| (source, xt)); + xts: impl IntoIterator)>, + ) -> Vec, B::Error>> { let validated_transactions = self.verify(at, xts, CheckBannedBeforeVerify::Yes).await; self.validated_pool.submit(validated_transactions.into_values()) } @@ -195,10 +193,8 @@ impl Pool { pub async fn resubmit_at( &self, at: &HashAndNumber, - source: TransactionSource, - xts: impl IntoIterator>, - ) -> Vec, B::Error>> { - let xts = xts.into_iter().map(|xt| (source, xt)); + xts: impl IntoIterator)>, + ) -> Vec, B::Error>> { let validated_transactions = self.verify(at, xts, CheckBannedBeforeVerify::No).await; self.validated_pool.submit(validated_transactions.into_values()) } @@ -207,10 +203,10 @@ impl Pool { pub async fn submit_one( &self, at: &HashAndNumber, - source: TransactionSource, + source: base::TimedTransactionSource, xt: ExtrinsicFor, - ) -> Result, B::Error> { - let res = self.submit_at(at, source, std::iter::once(xt)).await.pop(); + ) -> Result, B::Error> { + let res = self.submit_at(at, std::iter::once((source, xt))).await.pop(); res.expect("One extrinsic passed; one result returned; qed") } @@ -218,9 +214,9 @@ impl Pool { pub async fn submit_and_watch( &self, at: &HashAndNumber, - source: TransactionSource, + source: base::TimedTransactionSource, xt: ExtrinsicFor, - ) -> Result, ExtrinsicHash>, B::Error> { + ) -> Result, B::Error> { let (_, tx) = self .verify_one(at.hash, at.number, source, xt, CheckBannedBeforeVerify::Yes) .await; @@ -368,7 +364,7 @@ impl Pool { // Try to re-validate pruned transactions since some of them might be still valid. // note that `known_imported_hashes` will be rejected here due to temporary ban. let pruned_transactions = - prune_status.pruned.into_iter().map(|tx| (tx.source, tx.data.clone())); + prune_status.pruned.into_iter().map(|tx| (tx.source.clone(), tx.data.clone())); let reverified_transactions = self.verify(at, pruned_transactions, CheckBannedBeforeVerify::Yes).await; @@ -396,7 +392,7 @@ impl Pool { async fn verify( &self, at: &HashAndNumber, - xts: impl IntoIterator)>, + xts: impl IntoIterator)>, check: CheckBannedBeforeVerify, ) -> IndexMap, ValidatedTransactionFor> { let HashAndNumber { number, hash } = *at; @@ -413,11 +409,11 @@ impl Pool { } /// Returns future that validates single transaction at given block. - async fn verify_one( + pub(crate) async fn verify_one( &self, block_hash: ::Hash, block_number: NumberFor, - source: TransactionSource, + source: base::TimedTransactionSource, xt: ExtrinsicFor, check: CheckBannedBeforeVerify, ) -> (ExtrinsicHash, ValidatedTransactionFor) { @@ -431,7 +427,7 @@ impl Pool { let validation_result = self .validated_pool .api() - .validate_transaction(block_hash, source, xt.clone()) + .validate_transaction(block_hash, source.source, xt.clone()) .await; let status = match validation_result { @@ -488,6 +484,7 @@ mod tests { use super::{super::base_pool::Limit, *}; use crate::common::tests::{pool, uxt, TestApi, INVALID_NONCE}; use assert_matches::assert_matches; + use base::TimedTransactionSource; use codec::Encode; use futures::executor::block_on; use parking_lot::Mutex; @@ -497,7 +494,8 @@ mod tests { use substrate_test_runtime::{AccountId, ExtrinsicBuilder, Transfer, H256}; use substrate_test_runtime_client::AccountKeyring::{Alice, Bob}; - const SOURCE: TransactionSource = TransactionSource::External; + const SOURCE: TimedTransactionSource = + TimedTransactionSource { source: TransactionSource::External, timestamp: None }; #[test] fn should_validate_and_import_transaction() { @@ -518,6 +516,7 @@ mod tests { .into(), ), ) + .map(|outcome| outcome.hash()) .unwrap(); // then @@ -545,8 +544,11 @@ mod tests { let initial_hashes = txs.iter().map(|t| api.hash_and_length(t).0).collect::>(); // when - let txs = txs.into_iter().map(|x| Arc::from(x)).collect::>(); - let hashes = block_on(pool.submit_at(&api.expect_hash_and_number(0), SOURCE, txs)); + let txs = txs.into_iter().map(|x| (SOURCE, Arc::from(x))).collect::>(); + let hashes = block_on(pool.submit_at(&api.expect_hash_and_number(0), txs)) + .into_iter() + .map(|r| r.map(|o| o.hash())) + .collect::>(); log::debug!("--> {hashes:#?}"); // then @@ -570,7 +572,8 @@ mod tests { // when pool.validated_pool.ban(&Instant::now(), vec![pool.hash_of(&uxt)]); - let res = block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.into())); + let res = block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.into())) + .map(|o| o.hash()); assert_eq!(pool.validated_pool().status().ready, 0); assert_eq!(pool.validated_pool().status().future, 0); @@ -593,7 +596,8 @@ mod tests { let uxt = ExtrinsicBuilder::new_include_data(vec![42]).build(); // when - let res = block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.into())); + let res = block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.into())) + .map(|o| o.hash()); // then assert_matches!(res.unwrap_err(), error::Error::Unactionable); @@ -621,7 +625,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); let hash1 = block_on( pool.submit_one( &han_of_block0, @@ -635,7 +640,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); // future doesn't count let _hash = block_on( pool.submit_one( @@ -650,7 +656,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); assert_eq!(pool.validated_pool().status().ready, 2); assert_eq!(pool.validated_pool().status().future, 1); @@ -683,7 +690,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); let hash2 = block_on( pool.submit_one( &han_of_block0, @@ -697,7 +705,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); let hash3 = block_on( pool.submit_one( &han_of_block0, @@ -711,7 +720,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); // when pool.validated_pool.clear_stale(&api.expect_hash_and_number(5)); @@ -743,7 +753,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); // when block_on(pool.prune_tags(&api.expect_hash_and_number(1), vec![vec![0]], vec![hash1])); @@ -771,8 +782,9 @@ mod tests { let api = Arc::new(TestApi::default()); let pool = Pool::new(options, true.into(), api.clone()); - let hash1 = - block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, xt.into())).unwrap(); + let hash1 = block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, xt.into())) + .unwrap() + .hash(); assert_eq!(pool.validated_pool().status().future, 1); // when @@ -789,7 +801,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .hash(); // then assert_eq!(pool.validated_pool().status().future, 1); @@ -821,6 +834,7 @@ mod tests { .into(), ), ) + .map(|o| o.hash()) .unwrap_err(); // then @@ -847,6 +861,7 @@ mod tests { .into(), ), ) + .map(|o| o.hash()) .unwrap_err(); // then @@ -875,7 +890,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 1); assert_eq!(pool.validated_pool().status().future, 0); @@ -912,7 +928,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 1); assert_eq!(pool.validated_pool().status().future, 0); @@ -951,7 +968,8 @@ mod tests { .into(), ), ) - .unwrap(); + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 0); assert_eq!(pool.validated_pool().status().future, 1); @@ -990,7 +1008,8 @@ mod tests { }); let watcher = block_on(pool.submit_and_watch(&api.expect_hash_and_number(0), SOURCE, uxt.into())) - .unwrap(); + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 1); // when @@ -1015,7 +1034,8 @@ mod tests { }); let watcher = block_on(pool.submit_and_watch(&api.expect_hash_and_number(0), SOURCE, uxt.into())) - .unwrap(); + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 1); // when @@ -1048,7 +1068,8 @@ mod tests { }); let watcher = block_on(pool.submit_and_watch(&api.expect_hash_and_number(0), SOURCE, xt.into())) - .unwrap(); + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 1); // when @@ -1115,7 +1136,9 @@ mod tests { // after validation `IncludeData` will have priority set to 9001 // (validate_transaction mock) let xt = ExtrinsicBuilder::new_include_data(Vec::new()).build(); - block_on(pool.submit_and_watch(&han_of_block0, SOURCE, xt.into())).unwrap(); + block_on(pool.submit_and_watch(&han_of_block0, SOURCE, xt.into())) + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 1); // after validation `Transfer` will have priority set to 4 (validate_transaction @@ -1126,8 +1149,9 @@ mod tests { amount: 5, nonce: 0, }); - let watcher = - block_on(pool.submit_and_watch(&han_of_block0, SOURCE, xt.into())).unwrap(); + let watcher = block_on(pool.submit_and_watch(&han_of_block0, SOURCE, xt.into())) + .unwrap() + .expect_watcher(); assert_eq!(pool.validated_pool().status().ready, 2); // when diff --git a/substrate/client/transaction-pool/src/graph/ready.rs b/substrate/client/transaction-pool/src/graph/ready.rs index 860bcff0bace..b8aef99e638d 100644 --- a/substrate/client/transaction-pool/src/graph/ready.rs +++ b/substrate/client/transaction-pool/src/graph/ready.rs @@ -232,12 +232,10 @@ impl ReadyTransactions { Ok(replaced) } - /// Fold a list of ready transactions to compute a single value. - pub fn fold, &ReadyTx) -> Option>( - &mut self, - f: F, - ) -> Option { - self.ready.read().values().fold(None, f) + /// Fold a list of ready transactions to compute a single value using initial value of + /// accumulator. + pub fn fold) -> R>(&self, init: R, f: F) -> R { + self.ready.read().values().fold(init, f) } /// Returns true if given transaction is part of the queue. @@ -589,7 +587,6 @@ fn remove_item(vec: &mut Vec, item: &T) { #[cfg(test)] mod tests { use super::*; - use sp_runtime::transaction_validity::TransactionSource as Source; fn tx(id: u8) -> Transaction> { Transaction { @@ -601,7 +598,7 @@ mod tests { requires: vec![vec![1], vec![2]], provides: vec![vec![3], vec![4]], propagate: true, - source: Source::External, + source: crate::TimedTransactionSource::new_external(false), } } @@ -711,7 +708,7 @@ mod tests { requires: vec![tx1.provides[0].clone()], provides: vec![], propagate: true, - source: Source::External, + source: crate::TimedTransactionSource::new_external(false), }; // when diff --git a/substrate/client/transaction-pool/src/graph/rotator.rs b/substrate/client/transaction-pool/src/graph/rotator.rs index 61a26fb4138c..9a2e269b5eed 100644 --- a/substrate/client/transaction-pool/src/graph/rotator.rs +++ b/substrate/client/transaction-pool/src/graph/rotator.rs @@ -106,7 +106,6 @@ impl PoolRotator { #[cfg(test)] mod tests { use super::*; - use sp_runtime::transaction_validity::TransactionSource; type Hash = u64; type Ex = (); @@ -126,7 +125,7 @@ mod tests { requires: vec![], provides: vec![], propagate: true, - source: TransactionSource::External, + source: crate::TimedTransactionSource::new_external(false), }; (hash, tx) @@ -192,7 +191,7 @@ mod tests { requires: vec![], provides: vec![], propagate: true, - source: TransactionSource::External, + source: crate::TimedTransactionSource::new_external(false), } } diff --git a/substrate/client/transaction-pool/src/graph/tracked_map.rs b/substrate/client/transaction-pool/src/graph/tracked_map.rs index 6c3bbbf34b55..818c1ebe544f 100644 --- a/substrate/client/transaction-pool/src/graph/tracked_map.rs +++ b/substrate/client/transaction-pool/src/graph/tracked_map.rs @@ -173,6 +173,16 @@ where pub fn len(&mut self) -> usize { self.inner_guard.len() } + + /// Returns an iterator over all values. + pub fn values(&self) -> std::collections::hash_map::Values { + self.inner_guard.values() + } + + /// Returns an iterator over all key-value pairs. + pub fn iter(&self) -> Iter<'_, K, V> { + self.inner_guard.iter() + } } #[cfg(test)] diff --git a/substrate/client/transaction-pool/src/graph/validated_pool.rs b/substrate/client/transaction-pool/src/graph/validated_pool.rs index d7f55198a40a..42a7b275cbf0 100644 --- a/substrate/client/transaction-pool/src/graph/validated_pool.rs +++ b/substrate/client/transaction-pool/src/graph/validated_pool.rs @@ -18,25 +18,22 @@ use std::{ collections::{HashMap, HashSet}, - hash, sync::Arc, }; use crate::{common::log_xt::log_xt_trace, LOG_TARGET}; use futures::channel::mpsc::{channel, Sender}; use parking_lot::{Mutex, RwLock}; -use sc_transaction_pool_api::{error, PoolStatus, ReadyTransactions}; -use serde::Serialize; +use sc_transaction_pool_api::{error, PoolStatus, ReadyTransactions, TransactionPriority}; use sp_blockchain::HashAndNumber; use sp_runtime::{ - traits::{self, SaturatedConversion}, - transaction_validity::{TransactionSource, TransactionTag as Tag, ValidTransaction}, + traits::SaturatedConversion, + transaction_validity::{TransactionTag as Tag, ValidTransaction}, }; use std::time::Instant; use super::{ base_pool::{self as base, PruneStatus}, - listener::Listener, pool::{ BlockHash, ChainApi, EventStream, ExtrinsicFor, ExtrinsicHash, Options, TransactionFor, }, @@ -62,7 +59,7 @@ impl ValidatedTransaction { pub fn valid_at( at: u64, hash: Hash, - source: TransactionSource, + source: base::TimedTransactionSource, data: Ex, bytes: usize, validity: ValidTransaction, @@ -79,12 +76,23 @@ impl ValidatedTransaction { valid_till: at.saturated_into::().saturating_add(validity.longevity), }) } + + /// Returns priority for valid transaction, None if transaction is not valid. + pub fn priority(&self) -> Option { + match self { + ValidatedTransaction::Valid(base::Transaction { priority, .. }) => Some(*priority), + _ => None, + } + } } -/// A type of validated transaction stored in the pool. +/// A type of validated transaction stored in the validated pool. pub type ValidatedTransactionFor = ValidatedTransaction, ExtrinsicFor, ::Error>; +/// A type alias representing ValidatedPool listener for given ChainApi type. +pub type Listener = super::listener::Listener, B>; + /// A closure that returns true if the local node is a validator that can author blocks. #[derive(Clone)] pub struct IsValidator(Arc bool + Send + Sync>>); @@ -101,12 +109,56 @@ impl From bool + Send + Sync>> for IsValidator { } } +/// Represents the result of `submit` or `submit_and_watch` operations. +pub struct BaseSubmitOutcome { + /// The hash of the submitted transaction. + hash: ExtrinsicHash, + /// A transaction watcher. This is `Some` for `submit_and_watch` and `None` for `submit`. + watcher: Option, + + /// The priority of the transaction. Defaults to None if unknown. + priority: Option, +} + +/// Type alias to outcome of submission to `ValidatedPool`. +pub type ValidatedPoolSubmitOutcome = + BaseSubmitOutcome, ExtrinsicHash>>; + +impl BaseSubmitOutcome { + /// Creates a new instance with given hash and priority. + pub fn new(hash: ExtrinsicHash, priority: Option) -> Self { + Self { hash, priority, watcher: None } + } + + /// Sets the transaction watcher. + pub fn with_watcher(mut self, watcher: W) -> Self { + self.watcher = Some(watcher); + self + } + + /// Provides priority of submitted transaction. + pub fn priority(&self) -> Option { + self.priority + } + + /// Provides hash of submitted transaction. + pub fn hash(&self) -> ExtrinsicHash { + self.hash + } + + /// Provides a watcher. Should only be called on outcomes of `submit_and_watch`. Otherwise will + /// panic (that would mean logical error in program). + pub fn expect_watcher(&mut self) -> W { + self.watcher.take().expect("watcher was set in submit_and_watch. qed") + } +} + /// Pool that deals with validated transactions. pub struct ValidatedPool { api: Arc, is_validator: IsValidator, options: Options, - listener: RwLock, B>>, + listener: RwLock>, pub(crate) pool: RwLock, ExtrinsicFor>>, import_notification_sinks: Mutex>>>, rotator: PoolRotator>, @@ -126,6 +178,8 @@ impl Clone for ValidatedPool { } } +type BaseTransactionFor = base::Transaction, ExtrinsicFor>; + impl ValidatedPool { /// Create a new transaction pool. pub fn new(options: Options, is_validator: IsValidator, api: Arc) -> Self { @@ -175,7 +229,7 @@ impl ValidatedPool { pub fn submit( &self, txs: impl IntoIterator>, - ) -> Vec, B::Error>> { + ) -> Vec, B::Error>> { let results = txs .into_iter() .map(|validated_tx| self.submit_one(validated_tx)) @@ -191,7 +245,7 @@ impl ValidatedPool { results .into_iter() .map(|res| match res { - Ok(ref hash) if removed.contains(hash) => + Ok(outcome) if removed.contains(&outcome.hash) => Err(error::Error::ImmediatelyDropped.into()), other => other, }) @@ -199,9 +253,13 @@ impl ValidatedPool { } /// Submit single pre-validated transaction to the pool. - fn submit_one(&self, tx: ValidatedTransactionFor) -> Result, B::Error> { + fn submit_one( + &self, + tx: ValidatedTransactionFor, + ) -> Result, B::Error> { match tx { ValidatedTransaction::Valid(tx) => { + let priority = tx.priority; log::trace!(target: LOG_TARGET, "[{:?}] ValidatedPool::submit_one", tx.hash); if !tx.propagate && !(self.is_validator.0)() { return Err(error::Error::Unactionable.into()) @@ -229,7 +287,7 @@ impl ValidatedPool { let mut listener = self.listener.write(); fire_events(&mut *listener, &imported); - Ok(*imported.hash()) + Ok(ValidatedPoolSubmitOutcome::new(*imported.hash(), Some(priority))) }, ValidatedTransaction::Invalid(hash, err) => { log::trace!(target: LOG_TARGET, "[{:?}] ValidatedPool::submit_one invalid: {:?}", hash, err); @@ -280,7 +338,7 @@ impl ValidatedPool { // run notifications let mut listener = self.listener.write(); for h in &removed { - listener.dropped(h, None, true); + listener.limits_enforced(h); } removed @@ -293,7 +351,7 @@ impl ValidatedPool { pub fn submit_and_watch( &self, tx: ValidatedTransactionFor, - ) -> Result, ExtrinsicHash>, B::Error> { + ) -> Result, B::Error> { match tx { ValidatedTransaction::Valid(tx) => { let hash = self.api.hash_and_length(&tx.data).0; @@ -301,7 +359,7 @@ impl ValidatedPool { self.submit(std::iter::once(ValidatedTransaction::Valid(tx))) .pop() .expect("One extrinsic passed; one result returned; qed") - .map(|_| watcher) + .map(|outcome| outcome.with_watcher(watcher)) }, ValidatedTransaction::Invalid(hash, err) => { self.rotator.ban(&Instant::now(), std::iter::once(hash)); @@ -453,7 +511,7 @@ impl ValidatedPool { match final_status { Status::Future => listener.future(&hash), Status::Ready => listener.ready(&hash, None), - Status::Dropped => listener.dropped(&hash, None, false), + Status::Dropped => listener.dropped(&hash), Status::Failed => listener.invalid(&hash), } } @@ -492,7 +550,7 @@ impl ValidatedPool { fire_events(&mut *listener, promoted); } for f in &status.failed { - listener.dropped(f, None, false); + listener.dropped(f); } } @@ -666,23 +724,98 @@ impl ValidatedPool { self.listener.write().retracted(block_hash) } + //todo: doc + rename! + //MultiTransactionStatusStream pub fn create_dropped_by_limits_stream( &self, ) -> super::listener::DroppedByLimitsStream, BlockHash> { self.listener.write().create_dropped_by_limits_stream() } + + /// Resends ready and future events for all the ready and future transactions that are already + /// in the pool. + /// + /// Intended to be called after cloning the instance of `ValidatedPool`. + pub fn retrigger_notifications(&self) { + let pool = self.pool.read(); + let mut listener = self.listener.write(); + pool.ready().for_each(|r| { + listener.ready(&r.hash, None); + }); + pool.futures().for_each(|f| { + listener.future(&f.hash); + }); + } + + /// Searches the base pool for a transaction with a lower priority than the given one. + /// + /// Returns the hash of a lower-priority transaction if found, otherwise `None`. + pub fn find_transaction_with_lower_prio( + &self, + tx: ValidatedTransactionFor, + ) -> Option> { + match tx { + ValidatedTransaction::Valid(base::Transaction { priority, .. }) => { + let pool = self.pool.read(); + pool.fold_ready( + Some((priority, Option::>>::None)), + |acc, current| { + let (priority, _) = acc.as_ref().expect("acc is initialized. qed"); + if current.priority < *priority { + return Some((current.priority, Some(current.clone()))) + } else { + return acc + } + }, + ) + .map(|r| r.1.map(|tx| tx.hash))? + }, + _ => None, + } + } + + /// Removes a transaction subtree from the pool, starting from the given transaction hash. + /// + /// This function traverses the dependency graph of transactions and removes the specified + /// transaction along with all its descendant transactions from the pool. + /// + /// A `listener_action` callback function is invoked for every transaction that is removed, + /// providing a reference to the pool's listener and the hash of the removed transaction. This + /// allows to trigger the required events. + /// + /// Returns a vector containing the hashes of all removed transactions, including the root + /// transaction specified by `tx_hash`. + pub fn remove_subtree( + &self, + tx_hash: ExtrinsicHash, + listener_action: F, + ) -> Vec> + where + F: Fn(&mut Listener, ExtrinsicHash), + { + self.pool + .write() + .remove_subtree(&[tx_hash]) + .into_iter() + .map(|tx| { + let removed_tx_hash = tx.hash; + let mut listener = self.listener.write(); + listener_action(&mut *listener, removed_tx_hash); + removed_tx_hash + }) + .collect::>() + } } -fn fire_events(listener: &mut Listener, imported: &base::Imported) +fn fire_events(listener: &mut Listener, imported: &base::Imported, Ex>) where - H: hash::Hash + Eq + traits::Member + Serialize, B: ChainApi, { match *imported { base::Imported::Ready { ref promoted, ref failed, ref removed, ref hash } => { listener.ready(hash, None); failed.iter().for_each(|f| listener.invalid(f)); - removed.iter().for_each(|r| listener.dropped(&r.hash, Some(hash), false)); + removed.iter().for_each(|r| listener.usurped(&r.hash, hash)); promoted.iter().for_each(|p| listener.ready(p, None)); }, base::Imported::Future { ref hash } => listener.future(hash), diff --git a/substrate/client/transaction-pool/src/graph/watcher.rs b/substrate/client/transaction-pool/src/graph/watcher.rs index fb7cf99d4dc6..2fd31e772fd8 100644 --- a/substrate/client/transaction-pool/src/graph/watcher.rs +++ b/substrate/client/transaction-pool/src/graph/watcher.rs @@ -113,6 +113,12 @@ impl Sender { } /// Transaction has been dropped from the pool because of the limit. + pub fn limit_enforced(&mut self) { + self.send(TransactionStatus::Dropped); + self.is_finalized = true; + } + + /// Transaction has been dropped from the pool. pub fn dropped(&mut self) { self.send(TransactionStatus::Dropped); self.is_finalized = true; diff --git a/substrate/client/transaction-pool/src/lib.rs b/substrate/client/transaction-pool/src/lib.rs index 3d3d596c291f..366d91a973d2 100644 --- a/substrate/client/transaction-pool/src/lib.rs +++ b/substrate/client/transaction-pool/src/lib.rs @@ -36,7 +36,10 @@ pub use api::FullChainApi; pub use builder::{Builder, TransactionPoolHandle, TransactionPoolOptions, TransactionPoolType}; pub use common::notification_future; pub use fork_aware_txpool::{ForkAwareTxPool, ForkAwareTxPoolTask}; -pub use graph::{base_pool::Limit as PoolLimit, ChainApi, Options, Pool}; +pub use graph::{ + base_pool::{Limit as PoolLimit, TimedTransactionSource}, + ChainApi, Options, Pool, +}; use single_state_txpool::prune_known_txs_for_block; pub use single_state_txpool::{BasicPool, RevalidationType}; pub use transaction_pool_wrapper::TransactionPoolWrapper; diff --git a/substrate/client/transaction-pool/src/single_state_txpool/revalidation.rs b/substrate/client/transaction-pool/src/single_state_txpool/revalidation.rs index 5ef726c9f7d3..830dacd642ba 100644 --- a/substrate/client/transaction-pool/src/single_state_txpool/revalidation.rs +++ b/substrate/client/transaction-pool/src/single_state_txpool/revalidation.rs @@ -88,7 +88,7 @@ async fn batch_revalidate( let validation_results = futures::future::join_all(batch.into_iter().filter_map(|ext_hash| { pool.validated_pool().ready_by_hash(&ext_hash).map(|ext| { - api.validate_transaction(at, ext.source, ext.data.clone()) + api.validate_transaction(at, ext.source.source, ext.data.clone()) .map(move |validation_result| (validation_result, ext_hash, ext)) }) })) @@ -121,7 +121,7 @@ async fn batch_revalidate( ValidatedTransaction::valid_at( block_number.saturated_into::(), ext_hash, - ext.source, + ext.source.clone(), ext.data.clone(), api.hash_and_length(&ext.data).1, validity, @@ -375,9 +375,9 @@ mod tests { use crate::{ common::tests::{uxt, TestApi}, graph::Pool, + TimedTransactionSource, }; use futures::executor::block_on; - use sc_transaction_pool_api::TransactionSource; use substrate_test_runtime::{AccountId, Transfer, H256}; use substrate_test_runtime_client::AccountKeyring::{Alice, Bob}; @@ -398,10 +398,11 @@ mod tests { let uxt_hash = block_on(pool.submit_one( &han_of_block0, - TransactionSource::External, + TimedTransactionSource::new_external(false), uxt.clone().into(), )) - .expect("Should be valid"); + .expect("Should be valid") + .hash(); block_on(queue.revalidate_later(han_of_block0.hash, vec![uxt_hash])); @@ -433,14 +434,15 @@ mod tests { let han_of_block0 = api.expect_hash_and_number(0); let unknown_block = H256::repeat_byte(0x13); - let uxt_hashes = block_on(pool.submit_at( - &han_of_block0, - TransactionSource::External, - vec![uxt0.into(), uxt1.into()], - )) - .into_iter() - .map(|r| r.expect("Should be valid")) - .collect::>(); + let source = TimedTransactionSource::new_external(false); + let uxt_hashes = + block_on(pool.submit_at( + &han_of_block0, + vec![(source.clone(), uxt0.into()), (source, uxt1.into())], + )) + .into_iter() + .map(|r| r.expect("Should be valid").hash()) + .collect::>(); assert_eq!(api.validation_requests().len(), 2); assert_eq!(pool.validated_pool().status().ready, 2); diff --git a/substrate/client/transaction-pool/src/single_state_txpool/single_state_txpool.rs b/substrate/client/transaction-pool/src/single_state_txpool/single_state_txpool.rs index b29630b563bb..984f7e8e9b3e 100644 --- a/substrate/client/transaction-pool/src/single_state_txpool/single_state_txpool.rs +++ b/substrate/client/transaction-pool/src/single_state_txpool/single_state_txpool.rs @@ -29,7 +29,7 @@ use crate::{ error, log_xt::log_xt_trace, }, - graph::{self, ExtrinsicHash, IsValidator}, + graph::{self, base_pool::TimedTransactionSource, ExtrinsicHash, IsValidator}, ReadyIteratorFor, LOG_TARGET, }; use async_trait::async_trait; @@ -254,14 +254,24 @@ where xts: Vec>, ) -> Result, Self::Error>>, Self::Error> { let pool = self.pool.clone(); - let xts = xts.into_iter().map(Arc::from).collect::>(); + let xts = xts + .into_iter() + .map(|xt| { + (TimedTransactionSource::from_transaction_source(source, false), Arc::from(xt)) + }) + .collect::>(); self.metrics .report(|metrics| metrics.submitted_transactions.inc_by(xts.len() as u64)); let number = self.api.resolve_block_number(at); let at = HashAndNumber { hash: at, number: number? }; - Ok(pool.submit_at(&at, source, xts).await) + Ok(pool + .submit_at(&at, xts) + .await + .into_iter() + .map(|result| result.map(|outcome| outcome.hash())) + .collect()) } async fn submit_one( @@ -277,7 +287,9 @@ where let number = self.api.resolve_block_number(at); let at = HashAndNumber { hash: at, number: number? }; - pool.submit_one(&at, source, xt).await + pool.submit_one(&at, TimedTransactionSource::from_transaction_source(source, false), xt) + .await + .map(|outcome| outcome.hash()) } async fn submit_and_watch( @@ -294,9 +306,13 @@ where let number = self.api.resolve_block_number(at); let at = HashAndNumber { hash: at, number: number? }; - let watcher = pool.submit_and_watch(&at, source, xt).await?; - - Ok(watcher.into_stream().boxed()) + pool.submit_and_watch( + &at, + TimedTransactionSource::from_transaction_source(source, false), + xt, + ) + .await + .map(|mut outcome| outcome.expect_watcher().into_stream().boxed()) } fn remove_invalid(&self, hashes: &[TxHash]) -> Vec> { @@ -458,13 +474,17 @@ where let validated = ValidatedTransaction::valid_at( block_number.saturated_into::(), hash, - TransactionSource::Local, + TimedTransactionSource::new_local(false), Arc::from(xt), bytes, validity, ); - self.pool.validated_pool().submit(vec![validated]).remove(0) + self.pool + .validated_pool() + .submit(vec![validated]) + .remove(0) + .map(|outcome| outcome.hash()) } } @@ -662,23 +682,34 @@ where resubmit_transactions.extend( //todo: arctx - we need to get ref from somewhere - block_transactions.into_iter().map(Arc::from).filter(|tx| { - let tx_hash = pool.hash_of(tx); - let contains = pruned_log.contains(&tx_hash); - - // need to count all transactions, not just filtered, here - resubmitted_to_report += 1; - - if !contains { - log::trace!( - target: LOG_TARGET, - "[{:?}]: Resubmitting from retracted block {:?}", - tx_hash, - hash, - ); - } - !contains - }), + block_transactions + .into_iter() + .map(Arc::from) + .filter(|tx| { + let tx_hash = pool.hash_of(tx); + let contains = pruned_log.contains(&tx_hash); + + // need to count all transactions, not just filtered, here + resubmitted_to_report += 1; + + if !contains { + log::trace!( + target: LOG_TARGET, + "[{:?}]: Resubmitting from retracted block {:?}", + tx_hash, + hash, + ); + } + !contains + }) + .map(|tx| { + ( + // These transactions are coming from retracted blocks, we should + // simply consider them external. + TimedTransactionSource::new_external(false), + tx, + ) + }), ); self.metrics.report(|metrics| { @@ -686,14 +717,7 @@ where }); } - pool.resubmit_at( - &hash_and_number, - // These transactions are coming from retracted blocks, we should - // simply consider them external. - TransactionSource::External, - resubmit_transactions, - ) - .await; + pool.resubmit_at(&hash_and_number, resubmit_transactions).await; } let extra_pool = pool.clone(); diff --git a/substrate/client/transaction-pool/tests/fatp.rs b/substrate/client/transaction-pool/tests/fatp.rs index 9f343a9bd029..c51ca6e17663 100644 --- a/substrate/client/transaction-pool/tests/fatp.rs +++ b/substrate/client/transaction-pool/tests/fatp.rs @@ -2267,19 +2267,13 @@ fn fatp_avoid_stuck_transaction() { assert_pool_status!(header06.hash(), &pool, 0, 0); - // Import enough blocks to make xt4i revalidated - let mut prev_header = header03; - // wait 10 blocks for revalidation - for n in 7..=11 { - let header = api.push_block(n, vec![], true); - let event = finalized_block_event(&pool, prev_header.hash(), header.hash()); - block_on(pool.maintain(event)); - prev_header = header; - } + let header07 = api.push_block(7, vec![], true); + let event = finalized_block_event(&pool, header03.hash(), header07.hash()); + block_on(pool.maintain(event)); let xt4i_events = futures::executor::block_on_stream(xt4i_watcher).collect::>(); log::debug!("xt4i_events: {:#?}", xt4i_events); - assert_eq!(xt4i_events, vec![TransactionStatus::Future, TransactionStatus::Invalid]); + assert_eq!(xt4i_events, vec![TransactionStatus::Future, TransactionStatus::Dropped]); assert_eq!(pool.mempool_len(), (0, 0)); } diff --git a/substrate/client/transaction-pool/tests/fatp_common/mod.rs b/substrate/client/transaction-pool/tests/fatp_common/mod.rs index 15f2b7f79c14..d9bc7d30b13b 100644 --- a/substrate/client/transaction-pool/tests/fatp_common/mod.rs +++ b/substrate/client/transaction-pool/tests/fatp_common/mod.rs @@ -192,12 +192,35 @@ macro_rules! assert_ready_iterator { let output: Vec<_> = ready_iterator.collect(); log::debug!(target:LOG_TARGET, "expected: {:#?}", expected); log::debug!(target:LOG_TARGET, "output: {:#?}", output); + let output = output.into_iter().map(|t|t.hash).collect::>(); assert_eq!(expected.len(), output.len()); - assert!( - output.iter().zip(expected.iter()).all(|(o,e)| { - o.hash == *e - }) - ); + assert_eq!(output,expected); + }}; +} + +#[macro_export] +macro_rules! assert_future_iterator { + ($hash:expr, $pool:expr, [$( $xt:expr ),*]) => {{ + let futures = $pool.futures_at($hash).unwrap(); + let expected = vec![ $($pool.api().hash_and_length(&$xt).0),*]; + log::debug!(target:LOG_TARGET, "expected: {:#?}", futures); + log::debug!(target:LOG_TARGET, "output: {:#?}", expected); + assert_eq!(expected.len(), futures.len()); + let hsf = futures.iter().map(|a| a.hash).collect::>(); + let hse = expected.into_iter().collect::>(); + assert_eq!(hse,hsf); + }}; +} + +#[macro_export] +macro_rules! assert_watcher_stream { + ($stream:ident, [$( $event:expr ),*]) => {{ + let expected = vec![ $($event),*]; + log::debug!(target:LOG_TARGET, "expected: {:#?} {}, block now:", expected, expected.len()); + let output = futures::executor::block_on_stream($stream).take(expected.len()).collect::>(); + log::debug!(target:LOG_TARGET, "output: {:#?}", output); + assert_eq!(expected.len(), output.len()); + assert_eq!(output, expected); }}; } diff --git a/substrate/client/transaction-pool/tests/fatp_limits.rs b/substrate/client/transaction-pool/tests/fatp_limits.rs index 03792fd89dfa..afd8183957a8 100644 --- a/substrate/client/transaction-pool/tests/fatp_limits.rs +++ b/substrate/client/transaction-pool/tests/fatp_limits.rs @@ -641,3 +641,192 @@ fn fatp_limits_future_size_works() { assert_pool_status!(header01.hash(), &pool, 0, 3); assert_eq!(pool.mempool_len().0, 3); } + +#[test] +fn fatp_limits_watcher_ready_transactions_are_not_droped_when_view_is_dropped() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(6).with_ready_count(2).build(); + api.set_nonce(api.genesis_hash(), Bob.into(), 300); + api.set_nonce(api.genesis_hash(), Charlie.into(), 400); + api.set_nonce(api.genesis_hash(), Dave.into(), 500); + api.set_nonce(api.genesis_hash(), Eve.into(), 600); + api.set_nonce(api.genesis_hash(), Ferdie.into(), 700); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 300); + let xt2 = uxt(Charlie, 400); + + let xt3 = uxt(Dave, 500); + let xt4 = uxt(Eve, 600); + let xt5 = uxt(Ferdie, 700); + + let _xt0_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let _xt1_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().1, 2); + + let header02 = api.push_block_with_parent(header01.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header02.hash()))); + + let _xt2_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + let _xt3_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt3.clone())).unwrap(); + + assert_pool_status!(header02.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().1, 4); + + let header03 = api.push_block_with_parent(header02.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header02.hash()), header03.hash()))); + + let _xt4_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt4.clone())).unwrap(); + let _xt5_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt5.clone())).unwrap(); + + assert_pool_status!(header03.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().1, 6); + + let header04 = + api.push_block_with_parent(header03.hash(), vec![xt4.clone(), xt5.clone()], true); + api.set_nonce(header04.hash(), Alice.into(), 201); + api.set_nonce(header04.hash(), Bob.into(), 301); + api.set_nonce(header04.hash(), Charlie.into(), 401); + api.set_nonce(header04.hash(), Dave.into(), 501); + api.set_nonce(header04.hash(), Eve.into(), 601); + api.set_nonce(header04.hash(), Ferdie.into(), 701); + block_on(pool.maintain(new_best_block_event(&pool, Some(header03.hash()), header04.hash()))); + + assert_ready_iterator!(header01.hash(), pool, [xt0, xt1]); + assert_ready_iterator!(header02.hash(), pool, [xt2, xt3]); + assert_ready_iterator!(header03.hash(), pool, [xt4, xt5]); + assert_ready_iterator!(header04.hash(), pool, []); + + block_on(pool.maintain(finalized_block_event(&pool, api.genesis_hash(), header01.hash()))); + assert!(!pool.status_all().contains_key(&header01.hash())); + + block_on(pool.maintain(finalized_block_event(&pool, header01.hash(), header02.hash()))); + assert!(!pool.status_all().contains_key(&header02.hash())); + + //view 01 was dropped + assert!(pool.ready_at(header01.hash()).now_or_never().is_none()); + assert_eq!(pool.mempool_len().1, 6); + + block_on(pool.maintain(finalized_block_event(&pool, header02.hash(), header03.hash()))); + + //no revalidation has happened yet, all txs are kept + assert_eq!(pool.mempool_len().1, 6); + + //view 03 is still there + assert!(!pool.status_all().contains_key(&header03.hash())); + + //view 02 was dropped + assert!(pool.ready_at(header02.hash()).now_or_never().is_none()); + + let mut prev_header = header03; + for n in 5..=11 { + let header = api.push_block(n, vec![], true); + let event = finalized_block_event(&pool, prev_header.hash(), header.hash()); + block_on(pool.maintain(event)); + prev_header = header; + } + + //now revalidation has happened, all txs are dropped + assert_eq!(pool.mempool_len().1, 0); +} + +#[test] +fn fatp_limits_watcher_future_transactions_are_droped_when_view_is_dropped() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(6).with_future_count(2).build(); + api.set_nonce(api.genesis_hash(), Bob.into(), 300); + api.set_nonce(api.genesis_hash(), Charlie.into(), 400); + api.set_nonce(api.genesis_hash(), Dave.into(), 500); + api.set_nonce(api.genesis_hash(), Eve.into(), 600); + api.set_nonce(api.genesis_hash(), Ferdie.into(), 700); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 201); + let xt1 = uxt(Bob, 301); + let xt2 = uxt(Charlie, 401); + + let xt3 = uxt(Dave, 501); + let xt4 = uxt(Eve, 601); + let xt5 = uxt(Ferdie, 701); + + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let xt1_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 0, 2); + assert_eq!(pool.mempool_len().1, 2); + assert_future_iterator!(header01.hash(), pool, [xt0, xt1]); + + let header02 = api.push_block_with_parent(header01.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header02.hash()))); + + let xt2_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + let xt3_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt3.clone())).unwrap(); + + assert_pool_status!(header02.hash(), &pool, 0, 2); + assert_eq!(pool.mempool_len().1, 4); + assert_future_iterator!(header02.hash(), pool, [xt2, xt3]); + + let header03 = api.push_block_with_parent(header02.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header02.hash()), header03.hash()))); + + let xt4_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt4.clone())).unwrap(); + let xt5_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt5.clone())).unwrap(); + + assert_pool_status!(header03.hash(), &pool, 0, 2); + assert_eq!(pool.mempool_len().1, 6); + assert_future_iterator!(header03.hash(), pool, [xt4, xt5]); + + let header04 = api.push_block_with_parent(header03.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header03.hash()), header04.hash()))); + + assert_pool_status!(header04.hash(), &pool, 0, 2); + assert_eq!(pool.futures().len(), 2); + assert_future_iterator!(header04.hash(), pool, [xt4, xt5]); + + block_on(pool.maintain(finalized_block_event(&pool, api.genesis_hash(), header04.hash()))); + assert_eq!(pool.active_views_count(), 1); + assert_eq!(pool.inactive_views_count(), 0); + //todo: can we do better? We don't have API to check if event was processed internally. + let mut counter = 0; + while pool.mempool_len().1 != 2 { + sleep(std::time::Duration::from_millis(1)); + counter = counter + 1; + if counter > 20 { + assert!(false, "timeout {}", pool.mempool_len().1); + } + } + assert_eq!(pool.mempool_len().1, 2); + assert_pool_status!(header04.hash(), &pool, 0, 2); + assert_eq!(pool.futures().len(), 2); + + let to_be_checked = vec![xt0_watcher, xt1_watcher, xt2_watcher, xt3_watcher]; + for x in to_be_checked { + let x_status = futures::executor::block_on_stream(x).take(2).collect::>(); + assert_eq!(x_status, vec![TransactionStatus::Future, TransactionStatus::Dropped]); + } + + let to_be_checked = vec![xt4_watcher, xt5_watcher]; + for x in to_be_checked { + let x_status = futures::executor::block_on_stream(x).take(1).collect::>(); + assert_eq!(x_status, vec![TransactionStatus::Future]); + } +} diff --git a/substrate/client/transaction-pool/tests/fatp_prios.rs b/substrate/client/transaction-pool/tests/fatp_prios.rs new file mode 100644 index 000000000000..abd8ebec2296 --- /dev/null +++ b/substrate/client/transaction-pool/tests/fatp_prios.rs @@ -0,0 +1,502 @@ +// This file is part of Substrate. + +// Copyright (C) Parity Technologies (UK) Ltd. +// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0 + +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with this program. If not, see . + +//! Tests of priorities for fork-aware transaction pool. + +pub mod fatp_common; + +use fatp_common::{invalid_hash, new_best_block_event, TestPoolBuilder, LOG_TARGET, SOURCE}; +use futures::{executor::block_on, FutureExt}; +use sc_transaction_pool::ChainApi; +use sc_transaction_pool_api::{ + error::Error as TxPoolError, MaintainedTransactionPool, TransactionPool, TransactionStatus, +}; +use substrate_test_runtime_client::AccountKeyring::*; +use substrate_test_runtime_transaction_pool::uxt; +#[test] +fn fatp_prio_ready_higher_evicts_lower() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(3).with_ready_count(2).build(); + + let header01 = api.push_block(1, vec![], true); + + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 200); + + api.set_priority(&xt0, 2); + api.set_priority(&xt1, 3); + + let result0 = block_on(pool.submit_one(header01.hash(), SOURCE, xt0.clone())); + let result1 = block_on(pool.submit_one(header01.hash(), SOURCE, xt1.clone())); + + log::info!("r0 => {:?}", result0); + log::info!("r1 => {:?}", result1); + log::info!("len: {:?}", pool.mempool_len()); + log::info!("len: {:?}", pool.status_all()[&header01.hash()]); + assert_ready_iterator!(header01.hash(), pool, [xt1]); + assert_pool_status!(header01.hash(), &pool, 1, 0); +} + +#[test] +fn fatp_prio_watcher_ready_higher_evicts_lower() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(3).with_ready_count(2).build(); + + let header01 = api.push_block(1, vec![], true); + + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 200); + + api.set_priority(&xt0, 2); + api.set_priority(&xt1, 3); + + let xt0_watcher = + block_on(pool.submit_and_watch(header01.hash(), SOURCE, xt0.clone())).unwrap(); + let xt1_watcher = + block_on(pool.submit_and_watch(header01.hash(), SOURCE, xt1.clone())).unwrap(); + + let xt0_status = futures::executor::block_on_stream(xt0_watcher).take(2).collect::>(); + assert_eq!( + xt0_status, + vec![TransactionStatus::Ready, TransactionStatus::Usurped(api.hash_and_length(&xt1).0)] + ); + let xt1_status = futures::executor::block_on_stream(xt1_watcher).take(1).collect::>(); + assert_eq!(xt1_status, vec![TransactionStatus::Ready]); + + log::info!("len: {:?}", pool.mempool_len()); + log::info!("len: {:?}", pool.status_all()[&header01.hash()]); + assert_ready_iterator!(header01.hash(), pool, [xt1]); + assert_pool_status!(header01.hash(), &pool, 1, 0); +} + +#[test] +fn fatp_prio_watcher_future_higher_evicts_lower() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(3).with_ready_count(3).build(); + + let header01 = api.push_block(1, vec![], true); + + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 201); + let xt1 = uxt(Alice, 201); + let xt2 = uxt(Alice, 200); + + api.set_priority(&xt0, 2); + api.set_priority(&xt1, 3); + + let xt0_watcher = + block_on(pool.submit_and_watch(header01.hash(), SOURCE, xt0.clone())).unwrap(); + let xt1_watcher = + block_on(pool.submit_and_watch(header01.hash(), SOURCE, xt1.clone())).unwrap(); + let xt2_watcher = + block_on(pool.submit_and_watch(header01.hash(), SOURCE, xt2.clone())).unwrap(); + + let xt0_status = futures::executor::block_on_stream(xt0_watcher).take(2).collect::>(); + + assert_eq!( + xt0_status, + vec![TransactionStatus::Future, TransactionStatus::Usurped(api.hash_and_length(&xt2).0)] + ); + let xt1_status = futures::executor::block_on_stream(xt1_watcher).take(2).collect::>(); + assert_eq!(xt1_status, vec![TransactionStatus::Future, TransactionStatus::Ready]); + let xt2_status = futures::executor::block_on_stream(xt2_watcher).take(1).collect::>(); + assert_eq!(xt2_status, vec![TransactionStatus::Ready]); + + assert_eq!(pool.mempool_len().1, 2); + assert_ready_iterator!(header01.hash(), pool, [xt2, xt1]); + assert_pool_status!(header01.hash(), &pool, 2, 0); +} + +#[test] +fn fatp_prio_watcher_ready_lower_prio_gets_dropped_from_all_views() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(3).with_ready_count(2).build(); + + let header01 = api.push_block(1, vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, None, header01.hash()))); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 200); + + api.set_priority(&xt0, 2); + api.set_priority(&xt1, 3); + + let xt0_watcher = + block_on(pool.submit_and_watch(header01.hash(), SOURCE, xt0.clone())).unwrap(); + + let header02 = api.push_block_with_parent(header01.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header02.hash()))); + + let header03a = api.push_block_with_parent(header02.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header03a.hash()))); + + let header03b = api.push_block_with_parent(header02.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header03a.hash()), header03b.hash()))); + + assert_pool_status!(header03a.hash(), &pool, 1, 0); + assert_ready_iterator!(header03a.hash(), pool, [xt0]); + assert_pool_status!(header03b.hash(), &pool, 1, 0); + assert_ready_iterator!(header03b.hash(), pool, [xt0]); + assert_ready_iterator!(header01.hash(), pool, [xt0]); + assert_ready_iterator!(header02.hash(), pool, [xt0]); + + let xt1_watcher = + block_on(pool.submit_and_watch(header01.hash(), SOURCE, xt1.clone())).unwrap(); + + let xt1_status = futures::executor::block_on_stream(xt1_watcher).take(1).collect::>(); + assert_eq!(xt1_status, vec![TransactionStatus::Ready]); + let xt0_status = futures::executor::block_on_stream(xt0_watcher).take(2).collect::>(); + assert_eq!( + xt0_status, + vec![TransactionStatus::Ready, TransactionStatus::Usurped(api.hash_and_length(&xt1).0)] + ); + assert_ready_iterator!(header03a.hash(), pool, [xt1]); + assert_ready_iterator!(header03b.hash(), pool, [xt1]); + assert_ready_iterator!(header01.hash(), pool, [xt1]); + assert_ready_iterator!(header02.hash(), pool, [xt1]); +} + +#[test] +fn fatp_prio_watcher_future_lower_prio_gets_dropped_from_all_views() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(3).with_ready_count(2).build(); + + let header01 = api.push_block(1, vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, None, header01.hash()))); + + let xt0 = uxt(Alice, 201); + let xt1 = uxt(Alice, 201); + let xt2 = uxt(Alice, 200); + + api.set_priority(&xt0, 2); + api.set_priority(&xt1, 3); + + let xt0_watcher = + block_on(pool.submit_and_watch(header01.hash(), SOURCE, xt0.clone())).unwrap(); + + let xt1_watcher = + block_on(pool.submit_and_watch(header01.hash(), SOURCE, xt1.clone())).unwrap(); + + let header02 = api.push_block_with_parent(header01.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header02.hash()))); + + let header03a = api.push_block_with_parent(header02.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header03a.hash()))); + + let header03b = api.push_block_with_parent(header02.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header03a.hash()), header03b.hash()))); + + assert_pool_status!(header03a.hash(), &pool, 0, 2); + assert_future_iterator!(header03a.hash(), pool, [xt0, xt1]); + assert_pool_status!(header03b.hash(), &pool, 0, 2); + assert_future_iterator!(header03b.hash(), pool, [xt0, xt1]); + assert_future_iterator!(header01.hash(), pool, [xt0, xt1]); + assert_future_iterator!(header02.hash(), pool, [xt0, xt1]); + + let xt2_watcher = + block_on(pool.submit_and_watch(header01.hash(), SOURCE, xt2.clone())).unwrap(); + + let xt2_status = futures::executor::block_on_stream(xt2_watcher).take(1).collect::>(); + assert_eq!(xt2_status, vec![TransactionStatus::Ready]); + let xt1_status = futures::executor::block_on_stream(xt1_watcher).take(1).collect::>(); + assert_eq!(xt1_status, vec![TransactionStatus::Future]); + let xt0_status = futures::executor::block_on_stream(xt0_watcher).take(2).collect::>(); + assert_eq!( + xt0_status, + vec![TransactionStatus::Future, TransactionStatus::Usurped(api.hash_and_length(&xt2).0)] + ); + assert_future_iterator!(header03a.hash(), pool, []); + assert_future_iterator!(header03b.hash(), pool, []); + assert_future_iterator!(header01.hash(), pool, []); + assert_future_iterator!(header02.hash(), pool, []); + + assert_ready_iterator!(header03a.hash(), pool, [xt2, xt1]); + assert_ready_iterator!(header03b.hash(), pool, [xt2, xt1]); + assert_ready_iterator!(header01.hash(), pool, [xt2, xt1]); + assert_ready_iterator!(header02.hash(), pool, [xt2, xt1]); +} + +#[test] +fn fatp_prios_watcher_full_mempool_higher_prio_is_accepted() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(4).with_ready_count(2).build(); + api.set_nonce(api.genesis_hash(), Bob.into(), 300); + api.set_nonce(api.genesis_hash(), Charlie.into(), 400); + api.set_nonce(api.genesis_hash(), Dave.into(), 500); + api.set_nonce(api.genesis_hash(), Eve.into(), 600); + api.set_nonce(api.genesis_hash(), Ferdie.into(), 700); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 300); + let xt2 = uxt(Charlie, 400); + + let xt3 = uxt(Dave, 500); + + let xt4 = uxt(Eve, 600); + let xt5 = uxt(Ferdie, 700); + + api.set_priority(&xt0, 1); + api.set_priority(&xt1, 2); + api.set_priority(&xt2, 3); + api.set_priority(&xt3, 4); + + api.set_priority(&xt4, 5); + api.set_priority(&xt5, 6); + + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let xt1_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().1, 2); + + let header02 = api.push_block_with_parent(header01.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header02.hash()))); + + let xt2_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + let xt3_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt3.clone())).unwrap(); + + assert_pool_status!(header02.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().1, 4); + + let header03 = api.push_block_with_parent(header02.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header02.hash()), header03.hash()))); + + let _xt4_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt4.clone())).unwrap(); + let _xt5_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt5.clone())).unwrap(); + + assert_pool_status!(header03.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().1, 4); + + // let header04 = api.push_block_with_parent(header03.hash(), vec![], true); + // block_on(pool.maintain(new_best_block_event(&pool, Some(header03.hash()), header04.hash()))); + + assert_watcher_stream!(xt0_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt1_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + + assert_ready_iterator!(header01.hash(), pool, []); + assert_ready_iterator!(header02.hash(), pool, [xt3, xt2]); + assert_ready_iterator!(header03.hash(), pool, [xt5, xt4]); +} + +#[test] +fn fatp_prios_watcher_full_mempool_higher_prio_is_accepted_with_subtree() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(4).with_ready_count(4).build(); + api.set_nonce(api.genesis_hash(), Bob.into(), 300); + api.set_nonce(api.genesis_hash(), Charlie.into(), 400); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 201); + let xt2 = uxt(Alice, 202); + let xt3 = uxt(Bob, 300); + let xt4 = uxt(Charlie, 400); + + api.set_priority(&xt0, 1); + api.set_priority(&xt1, 3); + api.set_priority(&xt2, 3); + api.set_priority(&xt3, 2); + api.set_priority(&xt4, 2); + + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let xt1_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let xt2_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + let xt3_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt3.clone())).unwrap(); + + assert_ready_iterator!(header01.hash(), pool, [xt3, xt0, xt1, xt2]); + assert_pool_status!(header01.hash(), &pool, 4, 0); + assert_eq!(pool.mempool_len().1, 4); + + // let header02 = api.push_block_with_parent(header01.hash(), vec![], true); + // block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header02.hash()))); + + let xt4_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt4.clone())).unwrap(); + assert_pool_status!(header01.hash(), &pool, 2, 0); + assert_ready_iterator!(header01.hash(), pool, [xt3, xt4]); + + assert_watcher_stream!(xt0_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt1_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt2_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt3_watcher, [TransactionStatus::Ready]); + assert_watcher_stream!(xt4_watcher, [TransactionStatus::Ready]); +} + +#[test] +fn fatp_prios_watcher_full_mempool_higher_prio_is_accepted_with_subtree2() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(4).with_ready_count(4).build(); + api.set_nonce(api.genesis_hash(), Bob.into(), 300); + api.set_nonce(api.genesis_hash(), Charlie.into(), 400); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 201); + let xt2 = uxt(Alice, 202); + let xt3 = uxt(Bob, 300); + let xt4 = uxt(Charlie, 400); + + api.set_priority(&xt0, 1); + api.set_priority(&xt1, 3); + api.set_priority(&xt2, 3); + api.set_priority(&xt3, 2); + api.set_priority(&xt4, 2); + + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let xt1_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let xt2_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + let xt3_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt3.clone())).unwrap(); + + assert_ready_iterator!(header01.hash(), pool, [xt3, xt0, xt1, xt2]); + assert_pool_status!(header01.hash(), &pool, 4, 0); + assert_eq!(pool.mempool_len().1, 4); + + let header02 = api.push_block_with_parent(header01.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header02.hash()))); + + let xt4_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt4.clone())).unwrap(); + assert_ready_iterator!(header01.hash(), pool, [xt3]); + assert_pool_status!(header02.hash(), &pool, 2, 0); + assert_ready_iterator!(header02.hash(), pool, [xt3, xt4]); + + assert_watcher_stream!(xt0_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt1_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt2_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt3_watcher, [TransactionStatus::Ready]); + assert_watcher_stream!(xt4_watcher, [TransactionStatus::Ready]); +} + +#[test] +fn fatp_prios_watcher_full_mempool_lower_prio_gets_rejected() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(2).with_ready_count(2).build(); + api.set_nonce(api.genesis_hash(), Bob.into(), 300); + api.set_nonce(api.genesis_hash(), Charlie.into(), 400); + api.set_nonce(api.genesis_hash(), Dave.into(), 500); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 300); + let xt2 = uxt(Charlie, 400); + let xt3 = uxt(Dave, 500); + + api.set_priority(&xt0, 2); + api.set_priority(&xt1, 2); + api.set_priority(&xt2, 2); + api.set_priority(&xt3, 1); + + let _xt0_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let _xt1_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().1, 2); + + let header02 = api.push_block_with_parent(header01.hash(), vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header02.hash()))); + + assert_pool_status!(header02.hash(), &pool, 2, 0); + assert_eq!(pool.mempool_len().1, 2); + + assert_ready_iterator!(header01.hash(), pool, [xt0, xt1]); + assert_ready_iterator!(header02.hash(), pool, [xt0, xt1]); + + let result2 = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).map(|_| ()); + assert!(matches!(result2.as_ref().unwrap_err().0, TxPoolError::ImmediatelyDropped)); + let result3 = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt3.clone())).map(|_| ()); + assert!(matches!(result3.as_ref().unwrap_err().0, TxPoolError::ImmediatelyDropped)); +} + +#[test] +fn fatp_prios_watcher_full_mempool_does_not_keep_dropped_transaction() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(4).with_ready_count(2).build(); + api.set_nonce(api.genesis_hash(), Bob.into(), 300); + api.set_nonce(api.genesis_hash(), Charlie.into(), 400); + api.set_nonce(api.genesis_hash(), Dave.into(), 500); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 300); + let xt2 = uxt(Charlie, 400); + let xt3 = uxt(Dave, 500); + + api.set_priority(&xt0, 2); + api.set_priority(&xt1, 2); + api.set_priority(&xt2, 2); + api.set_priority(&xt3, 2); + + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let xt1_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let xt2_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + let xt3_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt3.clone())).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 2, 0); + assert_ready_iterator!(header01.hash(), pool, [xt2, xt3]); + + assert_watcher_stream!(xt0_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt1_watcher, [TransactionStatus::Ready, TransactionStatus::Dropped]); + assert_watcher_stream!(xt2_watcher, [TransactionStatus::Ready]); + assert_watcher_stream!(xt3_watcher, [TransactionStatus::Ready]); +} diff --git a/substrate/client/transaction-pool/tests/pool.rs b/substrate/client/transaction-pool/tests/pool.rs index ed0fd7d4e655..e4fd1b37c1f1 100644 --- a/substrate/client/transaction-pool/tests/pool.rs +++ b/substrate/client/transaction-pool/tests/pool.rs @@ -80,12 +80,14 @@ fn create_basic_pool(test_api: TestApi) -> BasicPool { create_basic_pool_with_genesis(Arc::from(test_api)).0 } +const TSOURCE: TimedTransactionSource = + TimedTransactionSource { source: TransactionSource::External, timestamp: None }; const SOURCE: TransactionSource = TransactionSource::External; #[test] fn submission_should_work() { let (pool, api) = pool(); - block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 209).into())) + block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt(Alice, 209).into())) .unwrap(); let pending: Vec<_> = pool @@ -99,9 +101,9 @@ fn submission_should_work() { #[test] fn multiple_submission_should_work() { let (pool, api) = pool(); - block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 209).into())) + block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt(Alice, 209).into())) .unwrap(); - block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 210).into())) + block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt(Alice, 210).into())) .unwrap(); let pending: Vec<_> = pool @@ -116,7 +118,7 @@ fn multiple_submission_should_work() { fn early_nonce_should_be_culled() { sp_tracing::try_init_simple(); let (pool, api) = pool(); - block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 208).into())) + block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt(Alice, 208).into())) .unwrap(); log::debug!("-> {:?}", pool.validated_pool().status()); @@ -132,7 +134,7 @@ fn early_nonce_should_be_culled() { fn late_nonce_should_be_queued() { let (pool, api) = pool(); - block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 210).into())) + block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt(Alice, 210).into())) .unwrap(); let pending: Vec<_> = pool .validated_pool() @@ -141,7 +143,7 @@ fn late_nonce_should_be_queued() { .collect(); assert_eq!(pending, Vec::::new()); - block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 209).into())) + block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt(Alice, 209).into())) .unwrap(); let pending: Vec<_> = pool .validated_pool() @@ -155,9 +157,10 @@ fn late_nonce_should_be_queued() { fn prune_tags_should_work() { let (pool, api) = pool(); let hash209 = - block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 209).into())) + block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt(Alice, 209).into())) + .map(|o| o.hash()) .unwrap(); - block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 210).into())) + block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt(Alice, 210).into())) .unwrap(); let pending: Vec<_> = pool @@ -182,10 +185,13 @@ fn prune_tags_should_work() { fn should_ban_invalid_transactions() { let (pool, api) = pool(); let uxt = Arc::from(uxt(Alice, 209)); - let hash = - block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.clone())).unwrap(); + let hash = block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt.clone())) + .unwrap() + .hash(); pool.validated_pool().remove_invalid(&[hash]); - block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.clone())).unwrap_err(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt.clone())) + .map(|_| ()) + .unwrap_err(); // when let pending: Vec<_> = pool @@ -196,7 +202,9 @@ fn should_ban_invalid_transactions() { assert_eq!(pending, Vec::::new()); // then - block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.clone())).unwrap_err(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, uxt.clone())) + .map(|_| ()) + .unwrap_err(); } #[test] @@ -224,7 +232,7 @@ fn should_correctly_prune_transactions_providing_more_than_one_tag() { })); let pool = Pool::new(Default::default(), true.into(), api.clone()); let xt0 = Arc::from(uxt(Alice, 209)); - block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, xt0.clone())) + block_on(pool.submit_one(&api.expect_hash_and_number(0), TSOURCE, xt0.clone())) .expect("1. Imported"); assert_eq!(pool.validated_pool().status().ready, 1); assert_eq!(api.validation_requests().len(), 1); @@ -242,7 +250,7 @@ fn should_correctly_prune_transactions_providing_more_than_one_tag() { api.increment_nonce(Alice.into()); api.push_block(2, Vec::new(), true); let xt1 = uxt(Alice, 211); - block_on(pool.submit_one(&api.expect_hash_and_number(2), SOURCE, xt1.clone().into())) + block_on(pool.submit_one(&api.expect_hash_and_number(2), TSOURCE, xt1.clone().into())) .expect("2. Imported"); assert_eq!(api.validation_requests().len(), 3); assert_eq!(pool.validated_pool().status().ready, 1);