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 8de10555ea330a522f00f6414deca62a91564959..c0a2f470d83d3d8797b848f0043260317c09f3b6 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
@@ -191,8 +191,9 @@ where
 	) -> Option<DroppedTransaction<ExtrinsicHash<C>>> {
 		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::<Vec<_>>(),
 		);
 		let (tx_hash, status) = event;
@@ -219,23 +220,8 @@ where
 					return Some(DroppedTransaction::new_enforced_by_limts(tx_hash))
 				}
 			},
-			// todo:
-			// 1. usurpued shall be sent unconditionally
-			// 2. fatp shall act for every usurped message - it should remove tx from every view and
-			//    replace it with new one (also in mempool).
-			TransactionStatus::Usurped(by) => {
-				if let Entry::Occupied(mut views_keeping_tx_valid) =
-					self.ready_transaction_views.entry(tx_hash)
-				{
-					views_keeping_tx_valid.get_mut().remove(&block_hash);
-					if views_keeping_tx_valid.get().is_empty() {
-						return Some(DroppedTransaction::new_usurped(tx_hash, by))
-					}
-				} else {
-					debug!("[{:?}] dropped_watcher: removing (non-tracked) tx", tx_hash);
-					return Some(DroppedTransaction::new_usurped(tx_hash, by))
-				}
-			},
+			TransactionStatus::Usurped(by) =>
+				return Some(DroppedTransaction::new_usurped(tx_hash, by)),
 			_ => {},
 		};
 		None
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 d90f5f1fedd687608bb0cafb741ebe515fe44527..4dad7cb6bdad7e80b98de746ff77e4506371268d 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
@@ -31,7 +31,7 @@ use crate::{
 	api::FullChainApi,
 	common::log_xt::log_xt_trace,
 	enactment_state::{EnactmentAction, EnactmentState},
-	fork_aware_txpool::revalidation_worker,
+	fork_aware_txpool::{dropped_watcher::DroppedReason, revalidation_worker},
 	graph::{
 		self,
 		base_pool::{TimedTransactionSource, Transaction},
@@ -201,9 +201,14 @@ where
 
 		let (dropped_stream_controller, dropped_stream) =
 			MultiViewDroppedWatcherController::<ChainApi>::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(),
 		);
 
@@ -220,8 +225,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,
@@ -237,14 +242,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<ChainApi>,
 		mempool: Arc<TxMemPool<ChainApi, Block>>,
+		view_store: Arc<ViewStore<ChainApi, Block>>,
 		import_notification_sink: MultiViewImportNotificationSink<
 			Block::Hash,
 			ExtrinsicHash<ChainApi>,
@@ -255,10 +263,33 @@ where
 				log::debug!(target: LOG_TARGET, "fatp::dropped_monitor_task: terminated...");
 				break;
 			};
-			log::trace!(target: LOG_TARGET, "[{:?}] fatp::dropped notification, removing", dropped);
-			let tx_hash = dropped.tx_hash;
-			mempool.remove_dropped_transaction(dropped).await;
-			import_notification_sink.clean_notified_items(&[tx_hash]);
+			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_dropped_transaction(&dropped_tx_hash).await;
+			view_store.listener.transaction_dropped(dropped);
+			import_notification_sink.clean_notified_items(&[dropped_tx_hash]);
 		}
 	}
 
@@ -293,9 +324,13 @@ where
 
 		let (dropped_stream_controller, dropped_stream) =
 			MultiViewDroppedWatcherController::<ChainApi>::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(),
 		);
 
@@ -311,8 +346,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,
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 52acb22d8a6b33f528349ea7d79c98b97746120c..a00234a99808238de77ec72a4e37531fcd4e3d6a 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
@@ -285,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<ChainApi>) -> 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
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 09b81be7766363b081e92a86abb16cd95febaf16..463ff938a7fbc6a694a928697a829fcdc9d32f36 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
@@ -98,7 +98,7 @@ 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
 	}
 
@@ -328,15 +328,13 @@ where
 		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.
+	/// Removes transaction from the memory pool which are specified by the given list of hashes.
 	pub(super) async fn remove_dropped_transaction(
 		&self,
-		dropped: DroppedTransaction<ExtrinsicHash<ChainApi>>,
-	) {
+		dropped: &ExtrinsicHash<ChainApi>,
+	) -> Option<Arc<TxInMemPool<ChainApi, Block>>> {
 		log::debug!(target: LOG_TARGET, "[{:?}] mempool::remove_dropped_transaction", dropped);
-		self.transactions.write().remove(&dropped.tx_hash);
-		self.listener.transaction_dropped(dropped);
+		self.transactions.write().remove(dropped)
 	}
 
 	/// Clones and returns a `HashMap` of references to all unwatched transactions in the memory
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 52855994d9b9d8e050750aecdbd783ad3ffffff5..dd3cf6bddcc6540e106514fcc529cf62dc08cea1 100644
--- a/substrate/client/transaction-pool/src/fork_aware_txpool/view.rs
+++ b/substrate/client/transaction-pool/src/fork_aware_txpool/view.rs
@@ -454,4 +454,10 @@ where
 			);
 		}
 	}
+
+	/// Returns true if the transaction hash is already imported into the view
+	pub(super) fn is_imported(&self, tx_hash: &ExtrinsicHash<ChainApi>) -> bool {
+		const IGNORE_BANNED: bool = false;
+		self.pool.validated_pool().check_is_known(tx_hash, IGNORE_BANNED).is_ok()
+	}
 }
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 41e3fff29f5fe50d1457d357a23c884c413377a0..47d2f12b40339076bba12de0631dbc1ad122daf3 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
@@ -37,7 +37,38 @@ use parking_lot::RwLock;
 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},
+	sync::Arc,
+	time::Instant,
+};
+
+/// Helper struct to keep the context for transaction replacements.
+#[derive(Clone)]
+struct PendingTxReplacement<ChainApi>
+where
+	ChainApi: graph::ChainApi,
+{
+	/// Indicates if the new transaction was already submitted to all the views in the view_store.
+	/// If true, it can be removed after inserting any new view.
+	processed: bool,
+	/// New transaction replacing the old one.
+	xt: ExtrinsicFor<ChainApi>,
+	/// Source of the transaction.
+	source: TimedTransactionSource,
+	/// Inidicates if transaction is watched.
+	watched: bool,
+}
+
+impl<ChainApi> PendingTxReplacement<ChainApi>
+where
+	ChainApi: graph::ChainApi,
+{
+	/// Creates new unprocessed instance of pending transaction replacement.
+	fn new(xt: ExtrinsicFor<ChainApi>, source: TimedTransactionSource, watched: bool) -> Self {
+		Self { processed: false, xt, source, watched }
+	}
+}
 
 /// The helper structure encapsulates all the views.
 pub(super) struct ViewStore<ChainApi, Block>
@@ -65,6 +96,13 @@ where
 	pub(super) most_recent_view: RwLock<Option<Block::Hash>>,
 	/// The controller of multi view dropped stream.
 	pub(super) dropped_stream_controller: MultiViewDroppedWatcherController<ChainApi>,
+	/// 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 replaced extrinsic.
+	pending_txs_replacements:
+		RwLock<HashMap<ExtrinsicHash<ChainApi>, PendingTxReplacement<ChainApi>>>,
 }
 
 impl<ChainApi, Block> ViewStore<ChainApi, Block>
@@ -86,6 +124,7 @@ where
 			listener,
 			most_recent_view: RwLock::from(None),
 			dropped_stream_controller,
+			pending_txs_replacements: Default::default(),
 		}
 	}
 
@@ -332,12 +371,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<View<ChainApi>>,
 		tree_route: &TreeRoute<Block>,
 	) {
+		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();
@@ -389,8 +432,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,
@@ -399,11 +444,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,
@@ -445,6 +487,7 @@ where
 		//clean up older then finalized
 		{
 			let mut active_views = self.active_views.write();
+			let mut inactive_views = self.inactive_views.write();
 			active_views.retain(|hash, v| {
 				let retain = match finalized_number {
 					Err(_) | Ok(None) => *hash == finalized_hash,
@@ -456,10 +499,7 @@ where
 				}
 				retain
 			});
-		}
 
-		{
-			let mut inactive_views = self.inactive_views.write();
 			inactive_views.retain(|hash, v| {
 				let retain = match finalized_number {
 					Err(_) | Ok(None) => false,
@@ -507,4 +547,139 @@ 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<ChainApi>,
+		replaced: ExtrinsicHash<ChainApi>,
+		watched: bool,
+	) {
+		if let Entry::Vacant(entry) = self.pending_txs_replacements.write().entry(replaced) {
+			entry.insert(PendingTxReplacement::new(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_replacements.write().get_mut(&replaced) {
+			replacement.processed = true;
+		}
+	}
+
+	/// 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<View<ChainApi>>) {
+		let mut futures = vec![];
+		for replacement in self.pending_txs_replacements.read().values() {
+			let xt_hash = self.api.hash_and_length(&replacement.xt).0;
+			futures.push(self.replace_transaction_in_view(
+				view.clone(),
+				replacement.source.clone(),
+				replacement.xt.clone(),
+				xt_hash,
+				replacement.watched,
+			));
+		}
+		let _results = futures::future::join_all(futures).await;
+		self.pending_txs_replacements.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<View<ChainApi>>,
+		source: TimedTransactionSource,
+		xt: ExtrinsicFor<ChainApi>,
+		xt_hash: ExtrinsicHash<ChainApi>,
+		watched: bool,
+	) {
+		if watched {
+			match view.submit_and_watch(source, xt).await {
+				Ok(watcher) => {
+					self.listener.add_view_watcher_for_tx(
+						xt_hash,
+						view.at.hash,
+						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<ChainApi>,
+		xt_hash: ExtrinsicHash<ChainApi>,
+		replaced: ExtrinsicHash<ChainApi>,
+		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::<Vec<_>>()
+		};
+		let _results = futures::future::join_all(submit_futures).await;
+	}
 }
diff --git a/substrate/client/transaction-pool/tests/fatp_prios.rs b/substrate/client/transaction-pool/tests/fatp_prios.rs
index e254a3d8171d37af775a95380fd961be9a03d0fd..544ea108306af2bdc67f0d002b8e09ad972831c6 100644
--- a/substrate/client/transaction-pool/tests/fatp_prios.rs
+++ b/substrate/client/transaction-pool/tests/fatp_prios.rs
@@ -20,15 +20,10 @@
 
 pub mod fatp_common;
 
-use fatp_common::{
-	finalized_block_event, invalid_hash, new_best_block_event, TestPoolBuilder, LOG_TARGET, SOURCE,
-};
+use fatp_common::{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 std::{thread::sleep, time::Duration};
+use sc_transaction_pool_api::{MaintainedTransactionPool, TransactionPool, TransactionStatus};
 use substrate_test_runtime_client::AccountKeyring::*;
 use substrate_test_runtime_transaction_pool::uxt;
 
@@ -59,28 +54,6 @@ fn fatp_prio_ready_higher_evicts_lower() {
 	log::info!("len: {:?}", pool.status_all()[&header01.hash()]);
 	assert_ready_iterator!(header01.hash(), pool, [xt1]);
 	assert_pool_status!(header01.hash(), &pool, 1, 0);
-
-	// let results = block_on(futures::future::join_all(submissions));
-	// assert!(results.iter().all(Result::is_ok));
-	// //charlie was not included into view:
-	// assert_pool_status!(header01.hash(), &pool, 2, 0);
-
-	// //branch with alice transactions:
-	// let header02b = api.push_block(2, vec![xt1.clone(), xt2.clone()], true);
-	// let event = new_best_block_event(&pool, Some(header01.hash()), header02b.hash());
-	// block_on(pool.maintain(event));
-	// assert_eq!(pool.mempool_len().0, 2);
-	// assert_pool_status!(header02b.hash(), &pool, 0, 0);
-	// assert_ready_iterator!(header02b.hash(), pool, []);
-	//
-	// //branch with alice/charlie transactions shall also work:
-	// let header02a = api.push_block(2, vec![xt0.clone(), xt1.clone()], true);
-	// api.set_nonce(header02a.hash(), Alice.into(), 201);
-	// let event = new_best_block_event(&pool, Some(header02b.hash()), header02a.hash());
-	// block_on(pool.maintain(event));
-	// assert_eq!(pool.mempool_len().0, 2);
-	// // assert_pool_status!(header02a.hash(), &pool, 1, 0);
-	// assert_ready_iterator!(header02a.hash(), pool, [xt2]);
 }
 
 #[test]
@@ -161,3 +134,54 @@ fn fatp_prio_watcher_future_higher_evicts_lower() {
 	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::<Vec<_>>();
+	assert_eq!(xt1_status, vec![TransactionStatus::Ready]);
+	let xt0_status = futures::executor::block_on_stream(xt0_watcher).take(2).collect::<Vec<_>>();
+	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]);
+}