diff --git a/Cargo.lock b/Cargo.lock index a5aa286485f6..b823690125fc 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -18394,6 +18394,7 @@ dependencies = [ "sc-service", "sc-telemetry", "sc-tracing", + "sc-transaction-pool", "sc-utils", "serde", "serde_json", @@ -19738,6 +19739,8 @@ dependencies = [ "criterion", "futures", "futures-timer", + "indexmap 2.2.3", + "itertools 0.11.0", "linked-hash-map", "log", "parity-scale-codec", @@ -19760,6 +19763,8 @@ dependencies = [ "substrate-test-runtime-client", "substrate-test-runtime-transaction-pool", "thiserror", + "tokio", + "tokio-stream", ] [[package]] @@ -23862,6 +23867,7 @@ name = "substrate-test-runtime-transaction-pool" version = "2.0.0" dependencies = [ "futures", + "log", "parity-scale-codec", "parking_lot 0.12.3", "sc-transaction-pool", diff --git a/cumulus/client/service/src/lib.rs b/cumulus/client/service/src/lib.rs index 92dc64371f34..25b8ee10a931 100644 --- a/cumulus/client/service/src/lib.rs +++ b/cumulus/client/service/src/lib.rs @@ -417,7 +417,7 @@ pub struct BuildNetworkParams< pub net_config: sc_network::config::FullNetworkConfiguration::Hash, Network>, pub client: Arc, - pub transaction_pool: Arc>, + pub transaction_pool: Arc>, pub para_id: ParaId, pub relay_chain_interface: RCInterface, pub spawn_handle: SpawnTaskHandle, diff --git a/cumulus/polkadot-omni-node/lib/src/common/rpc.rs b/cumulus/polkadot-omni-node/lib/src/common/rpc.rs index 85665c9b220f..4879bd1eb7f4 100644 --- a/cumulus/polkadot-omni-node/lib/src/common/rpc.rs +++ b/cumulus/polkadot-omni-node/lib/src/common/rpc.rs @@ -46,7 +46,7 @@ impl BuildRpcExtensions< ParachainClient, ParachainBackend, - sc_transaction_pool::FullPool>, + sc_transaction_pool::TransactionPoolHandle>, > for BuildParachainRpcExtensions where RuntimeApi: @@ -57,7 +57,9 @@ where fn build_rpc_extensions( client: Arc>, backend: Arc>, - pool: Arc>>, + pool: Arc< + sc_transaction_pool::TransactionPoolHandle>, + >, ) -> sc_service::error::Result { let build = || -> Result> { let mut module = RpcExtension::new(()); diff --git a/cumulus/polkadot-omni-node/lib/src/common/spec.rs b/cumulus/polkadot-omni-node/lib/src/common/spec.rs index dca28b3c28f7..8397cb778dcf 100644 --- a/cumulus/polkadot-omni-node/lib/src/common/spec.rs +++ b/cumulus/polkadot-omni-node/lib/src/common/spec.rs @@ -40,7 +40,7 @@ use sc_service::{Configuration, ImportQueue, PartialComponents, TaskManager}; use sc_sysinfo::HwBench; use sc_telemetry::{TelemetryHandle, TelemetryWorker}; use sc_tracing::tracing::Instrument; -use sc_transaction_pool::FullPool; +use sc_transaction_pool::TransactionPoolHandle; use sp_keystore::KeystorePtr; use std::{future::Future, pin::Pin, sync::Arc, time::Duration}; @@ -65,7 +65,7 @@ where telemetry: Option, task_manager: &TaskManager, relay_chain_interface: Arc, - transaction_pool: Arc>>, + transaction_pool: Arc>>, keystore: KeystorePtr, relay_chain_slot_duration: Duration, para_id: ParaId, @@ -149,12 +149,15 @@ pub(crate) trait BaseNodeSpec { telemetry }); - let transaction_pool = sc_transaction_pool::BasicPool::new_full( - config.transaction_pool.clone(), - config.role.is_authority().into(), - config.prometheus_registry(), - task_manager.spawn_essential_handle(), - client.clone(), + let transaction_pool = Arc::from( + sc_transaction_pool::Builder::new( + task_manager.spawn_essential_handle(), + client.clone(), + config.role.is_authority().into(), + ) + .with_options(config.transaction_pool.clone()) + .with_prometheus(config.prometheus_registry()) + .build(), ); let block_import = ParachainBlockImport::new(client.clone(), backend.clone()); @@ -184,7 +187,7 @@ pub(crate) trait NodeSpec: BaseNodeSpec { type BuildRpcExtensions: BuildRpcExtensions< ParachainClient, ParachainBackend, - FullPool>, + TransactionPoolHandle>, >; type StartConsensus: StartConsensus; diff --git a/cumulus/polkadot-omni-node/lib/src/common/types.rs b/cumulus/polkadot-omni-node/lib/src/common/types.rs index 9cfdcb22451c..4bc58dc9db7e 100644 --- a/cumulus/polkadot-omni-node/lib/src/common/types.rs +++ b/cumulus/polkadot-omni-node/lib/src/common/types.rs @@ -20,7 +20,7 @@ use sc_consensus::DefaultImportQueue; use sc_executor::WasmExecutor; use sc_service::{PartialComponents, TFullBackend, TFullClient}; use sc_telemetry::{Telemetry, TelemetryWorkerHandle}; -use sc_transaction_pool::FullPool; +use sc_transaction_pool::TransactionPoolHandle; use sp_runtime::{generic, traits::BlakeTwo256}; use std::sync::Arc; @@ -51,6 +51,6 @@ pub type ParachainService = PartialComponents< ParachainBackend, (), DefaultImportQueue, - FullPool>, + TransactionPoolHandle>, (ParachainBlockImport, Option, Option), >; diff --git a/cumulus/polkadot-omni-node/lib/src/nodes/aura.rs b/cumulus/polkadot-omni-node/lib/src/nodes/aura.rs index cf1ee91cbab5..ec5d0a439ec4 100644 --- a/cumulus/polkadot-omni-node/lib/src/nodes/aura.rs +++ b/cumulus/polkadot-omni-node/lib/src/nodes/aura.rs @@ -52,7 +52,7 @@ use sc_consensus::{ }; use sc_service::{Configuration, Error, TaskManager}; use sc_telemetry::TelemetryHandle; -use sc_transaction_pool::FullPool; +use sc_transaction_pool::TransactionPoolHandle; use sp_api::ProvideRuntimeApi; use sp_inherents::CreateInherentDataProviders; use sp_keystore::KeystorePtr; @@ -291,7 +291,7 @@ where telemetry: Option, task_manager: &TaskManager, relay_chain_interface: Arc, - transaction_pool: Arc>>, + transaction_pool: Arc>>, keystore: KeystorePtr, _relay_chain_slot_duration: Duration, para_id: ParaId, @@ -387,7 +387,7 @@ where telemetry: Option, task_manager: &TaskManager, relay_chain_interface: Arc, - transaction_pool: Arc>>, + transaction_pool: Arc>>, keystore: KeystorePtr, relay_chain_slot_duration: Duration, para_id: ParaId, diff --git a/cumulus/test/service/src/lib.rs b/cumulus/test/service/src/lib.rs index 9f93572e9cea..a13399d3a40e 100644 --- a/cumulus/test/service/src/lib.rs +++ b/cumulus/test/service/src/lib.rs @@ -134,7 +134,7 @@ pub type Backend = TFullBackend; pub type ParachainBlockImport = TParachainBlockImport, Backend>; /// Transaction pool type used by the test service -pub type TransactionPool = Arc>; +pub type TransactionPool = Arc>; /// Recovery handle that fails regularly to simulate unavailable povs. pub struct FailingRecoveryHandle { @@ -183,7 +183,7 @@ pub type Service = PartialComponents< Backend, (), sc_consensus::import_queue::BasicQueue, - sc_transaction_pool::FullPool, + sc_transaction_pool::TransactionPoolHandle, ParachainBlockImport, >; @@ -219,12 +219,15 @@ pub fn new_partial( let block_import = ParachainBlockImport::new(client.clone(), backend.clone()); - let transaction_pool = sc_transaction_pool::BasicPool::new_full( - config.transaction_pool.clone(), - config.role.is_authority().into(), - config.prometheus_registry(), - task_manager.spawn_essential_handle(), - client.clone(), + let transaction_pool = Arc::from( + sc_transaction_pool::Builder::new( + task_manager.spawn_essential_handle(), + client.clone(), + config.role.is_authority().into(), + ) + .with_options(config.transaction_pool.clone()) + .with_prometheus(config.prometheus_registry()) + .build(), ); let slot_duration = sc_consensus_aura::slot_duration(&*client)?; diff --git a/cumulus/zombienet/tests/0008-main.js b/cumulus/zombienet/tests/0008-main.js new file mode 100644 index 000000000000..31c01324a77e --- /dev/null +++ b/cumulus/zombienet/tests/0008-main.js @@ -0,0 +1,18 @@ +// Allows to manually submit extrinsic to collator. +// Usage: +// zombienet-linux -p native spwan 0008-parachain-extrinsic-gets-finalized.toml +// node 0008-main.js + +global.zombie = null + +const fs = require('fs'); +const test = require('./0008-transaction_gets_finalized.js'); + +if (process.argv.length == 2) { + console.error('Path to zombie.json (generated by zombienet-linux spawn command shall be given)!'); + process.exit(1); +} + +let networkInfo = JSON.parse(fs.readFileSync(process.argv[2])); + +test.run("charlie", networkInfo).then(process.exit) diff --git a/cumulus/zombienet/tests/0008-parachain_extrinsic_gets_finalized.toml b/cumulus/zombienet/tests/0008-parachain_extrinsic_gets_finalized.toml new file mode 100644 index 000000000000..a295d3960bfe --- /dev/null +++ b/cumulus/zombienet/tests/0008-parachain_extrinsic_gets_finalized.toml @@ -0,0 +1,25 @@ +[relaychain] +default_image = "{{RELAY_IMAGE}}" +default_command = "polkadot" +chain = "rococo-local" + + [[relaychain.nodes]] + name = "alice" + validator = true + + [[relaychain.nodes]] + name = "bob" + validator = true + +[[parachains]] +id = 2000 +cumulus_based = true +chain = "asset-hub-rococo-local" + + # run charlie as parachain collator + [[parachains.collators]] + name = "charlie" + validator = true + image = "{{POLKADOT_PARACHAIN_IMAGE}}" + command = "polkadot-parachain" + args = ["--force-authoring", "-ltxpool=trace", "--pool-type=fork-aware"] diff --git a/cumulus/zombienet/tests/0008-parachain_extrinsic_gets_finalized.zndsl b/cumulus/zombienet/tests/0008-parachain_extrinsic_gets_finalized.zndsl new file mode 100644 index 000000000000..5aab1bd923a5 --- /dev/null +++ b/cumulus/zombienet/tests/0008-parachain_extrinsic_gets_finalized.zndsl @@ -0,0 +1,20 @@ +Description: Block building +Network: ./0008-parachain_extrinsic_gets_finalized.toml +Creds: config + +alice: reports node_roles is 4 +bob: reports node_roles is 4 +charlie: reports node_roles is 4 + +alice: reports peers count is at least 1 +bob: reports peers count is at least 1 + +alice: reports block height is at least 5 within 60 seconds +bob: reports block height is at least 5 within 60 seconds +charlie: reports block height is at least 2 within 120 seconds + +alice: count of log lines containing "error" is 0 within 2 seconds +bob: count of log lines containing "error" is 0 within 2 seconds +charlie: count of log lines containing "error" is 0 within 2 seconds + +charlie: js-script ./0008-transaction_gets_finalized.js within 600 seconds diff --git a/cumulus/zombienet/tests/0008-transaction_gets_finalized.js b/cumulus/zombienet/tests/0008-transaction_gets_finalized.js new file mode 100644 index 000000000000..3031c45e3a4b --- /dev/null +++ b/cumulus/zombienet/tests/0008-transaction_gets_finalized.js @@ -0,0 +1,69 @@ +//based on: https://polkadot.js.org/docs/api/examples/promise/transfer-events + +const assert = require("assert"); + +async function run(nodeName, networkInfo, args) { + const {wsUri, userDefinedTypes} = networkInfo.nodesByName[nodeName]; + // Create the API and wait until ready + var api = null; + var keyring = null; + if (zombie == null) { + const testKeyring = require('@polkadot/keyring/testing'); + const { WsProvider, ApiPromise } = require('@polkadot/api'); + const provider = new WsProvider(wsUri); + api = await ApiPromise.create({provider}); + // Construct the keyring after the API (crypto has an async init) + keyring = testKeyring.createTestKeyring({ type: "sr25519" }); + } else { + keyring = new zombie.Keyring({ type: "sr25519" }); + api = await zombie.connect(wsUri, userDefinedTypes); + } + + + // Add Alice to our keyring with a hard-derivation path (empty phrase, so uses dev) + const alice = keyring.addFromUri('//Alice'); + + // Create an extrinsic: + const extrinsic = api.tx.system.remark("xxx"); + + let extrinsic_success_event = false; + try { + await new Promise( async (resolve, reject) => { + const unsubscribe = await extrinsic + .signAndSend(alice, { nonce: -1 }, ({ events = [], status }) => { + console.log('Extrinsic status:', status.type); + + if (status.isInBlock) { + console.log('Included at block hash', status.asInBlock.toHex()); + console.log('Events:'); + + events.forEach(({ event: { data, method, section }, phase }) => { + console.log('\t', phase.toString(), `: ${section}.${method}`, data.toString()); + + if (section=="system" && method =="ExtrinsicSuccess") { + extrinsic_success_event = true; + } + }); + } else if (status.isFinalized) { + console.log('Finalized block hash', status.asFinalized.toHex()); + unsubscribe(); + if (extrinsic_success_event) { + resolve(); + } else { + reject("ExtrinsicSuccess has not been seen"); + } + } else if (status.isError) { + unsubscribe(); + reject("Extrinsic status.isError"); + } + + }); + }); + } catch (error) { + assert.fail("Transfer promise failed, error: " + error); + } + + assert.ok("test passed"); +} + +module.exports = { run } diff --git a/polkadot/node/service/src/lib.rs b/polkadot/node/service/src/lib.rs index 9515dd231138..da3ab760ed22 100644 --- a/polkadot/node/service/src/lib.rs +++ b/polkadot/node/service/src/lib.rs @@ -450,7 +450,7 @@ fn new_partial( FullBackend, ChainSelection, sc_consensus::DefaultImportQueue, - sc_transaction_pool::FullPool, + sc_transaction_pool::TransactionPoolHandle, ( impl Fn( polkadot_rpc::SubscriptionTaskExecutor, @@ -478,12 +478,15 @@ fn new_partial( where ChainSelection: 'static + SelectChain, { - let transaction_pool = sc_transaction_pool::BasicPool::new_full( - config.transaction_pool.clone(), - config.role.is_authority().into(), - config.prometheus_registry(), - task_manager.spawn_essential_handle(), - client.clone(), + let transaction_pool = Arc::from( + sc_transaction_pool::Builder::new( + task_manager.spawn_essential_handle(), + client.clone(), + config.role.is_authority().into(), + ) + .with_options(config.transaction_pool.clone()) + .with_prometheus(config.prometheus_registry()) + .build(), ); let grandpa_hard_forks = if config.chain_spec.is_kusama() { diff --git a/prdoc/pr_4639.prdoc b/prdoc/pr_4639.prdoc new file mode 100644 index 000000000000..dfdd60f2bdb2 --- /dev/null +++ b/prdoc/pr_4639.prdoc @@ -0,0 +1,69 @@ +title: "Added the fork-aware transaction pool implementation" + +doc: + - audience: Node Dev + description: | + Most important changes introduced by this PR: + - The transaction pool references spread across codebase are now wrapper to a transaction pool trait object, + - The fork-aware pool implementation was added. + - The `sc-transaction-pool` refactored, + - Trasnaction pool builder was introduced to allow to instantiation of either old or new transaction pool. Refer to PR description for + more details on how to enable fork-aware pool in the custom node. + - audience: Node Operator + description: | + - New command line option was added, allowing to select implementation of transaction pool: + - `--pool-type=fork-aware` - new fork aware transaction pool, + - `--pool-type=single-state` - old transaction pool implementation which is still default, + +crates: + - name: sc-basic-authorship + bump: patch + - name: sc-cli + bump: major + - name: sc-consensus-manual-seal + bump: patch + - name: sc-network-transactions + bump: none + - name: sc-rpc + bump: patch + - name: sc-rpc-spec-v2 + bump: patch + - name: sc-offchain + bump: patch + - name: sc-service + bump: patch + - name: sc-service-test + bump: minor + - name: sc-transaction-pool + bump: major + - name: sc-transaction-pool-api + bump: major + validate: false + - name: sp-runtime + bump: patch + - name: substrate-test-runtime-transaction-pool + bump: minor + - name: staging-node-cli + bump: minor + - name: node-bench + bump: patch + - name: node-rpc + bump: minor + - name: substrate-prometheus-endpoint + bump: patch + - name: substrate-frame-rpc-system + bump: patch + - name: minimal-template-node + bump: minor + - name: parachain-template-node + bump: minor + - name: solochain-template-node + bump: minor + - name: polkadot-service + bump: patch + - name: cumulus-client-service + bump: patch + - name: cumulus-test-service + bump: major + - name: polkadot-omni-node-lib + bump: patch diff --git a/substrate/bin/node/bench/src/construct.rs b/substrate/bin/node/bench/src/construct.rs index 23d0a0cc1ee5..bed6e3d914c2 100644 --- a/substrate/bin/node/bench/src/construct.rs +++ b/substrate/bin/node/bench/src/construct.rs @@ -35,7 +35,7 @@ use sc_transaction_pool_api::{ }; use sp_consensus::{Environment, Proposer}; use sp_inherents::InherentDataProvider; -use sp_runtime::{traits::NumberFor, OpaqueExtrinsic}; +use sp_runtime::OpaqueExtrinsic; use crate::{ common::SizeType, @@ -165,18 +165,18 @@ impl core::Benchmark for ConstructionBenchmark { #[derive(Clone, Debug)] pub struct PoolTransaction { - data: OpaqueExtrinsic, + data: Arc, hash: node_primitives::Hash, } impl From for PoolTransaction { fn from(e: OpaqueExtrinsic) -> Self { - PoolTransaction { data: e, hash: node_primitives::Hash::zero() } + PoolTransaction { data: Arc::from(e), hash: node_primitives::Hash::zero() } } } impl sc_transaction_pool_api::InPoolTransaction for PoolTransaction { - type Transaction = OpaqueExtrinsic; + type Transaction = Arc; type Hash = node_primitives::Hash; fn data(&self) -> &Self::Transaction { @@ -261,7 +261,7 @@ impl sc_transaction_pool_api::TransactionPool for Transactions { fn ready_at( &self, - _at: NumberFor, + _at: Self::Hash, ) -> Pin< Box< dyn Future< @@ -305,4 +305,19 @@ impl sc_transaction_pool_api::TransactionPool for Transactions { fn ready_transaction(&self, _hash: &TxHash) -> Option> { unimplemented!() } + + fn ready_at_with_timeout( + &self, + _at: Self::Hash, + _timeout: std::time::Duration, + ) -> Pin< + Box< + dyn Future< + Output = Box> + Send>, + > + Send + + '_, + >, + > { + unimplemented!() + } } diff --git a/substrate/bin/node/cli/benches/transaction_pool.rs b/substrate/bin/node/cli/benches/transaction_pool.rs index efec081427f4..c07cb3ec0d13 100644 --- a/substrate/bin/node/cli/benches/transaction_pool.rs +++ b/substrate/bin/node/cli/benches/transaction_pool.rs @@ -16,15 +16,16 @@ // You should have received a copy of the GNU General Public License // along with this program. If not, see . -use polkadot_sdk::*; -use std::time::Duration; - use criterion::{criterion_group, criterion_main, BatchSize, Criterion, Throughput}; use futures::{future, StreamExt}; use kitchensink_runtime::{constants::currency::*, BalancesCall, SudoCall}; use node_cli::service::{create_extrinsic, fetch_nonce, FullClient, TransactionPool}; use node_primitives::AccountId; -use polkadot_sdk::sc_service::config::{ExecutorConfiguration, RpcConfiguration}; +use polkadot_sdk::{ + sc_service::config::{ExecutorConfiguration, RpcConfiguration}, + sc_transaction_pool_api::TransactionPool as _, + *, +}; use sc_service::{ config::{ BlocksPruning, DatabaseSource, KeystoreConfig, NetworkConfiguration, OffchainWorkerConfig, @@ -32,8 +33,7 @@ use sc_service::{ }, BasePath, Configuration, Role, }; -use sc_transaction_pool::PoolLimit; -use sc_transaction_pool_api::{TransactionPool as _, TransactionSource, TransactionStatus}; +use sc_transaction_pool_api::{TransactionSource, TransactionStatus}; use sp_core::{crypto::Pair, sr25519}; use sp_keyring::Sr25519Keyring; use sp_runtime::OpaqueExtrinsic; @@ -58,12 +58,7 @@ fn new_node(tokio_handle: Handle) -> node_cli::service::NewFullBase { impl_version: "1.0".into(), role: Role::Authority, tokio_handle: tokio_handle.clone(), - transaction_pool: TransactionPoolOptions { - ready: PoolLimit { count: 100_000, total_bytes: 100 * 1024 * 1024 }, - future: PoolLimit { count: 100_000, total_bytes: 100 * 1024 * 1024 }, - reject_future_transactions: false, - ban_time: Duration::from_secs(30 * 60), - }, + transaction_pool: TransactionPoolOptions::new_for_benchmarks(), network: network_config, keystore: KeystoreConfig::InMemory, database: DatabaseSource::RocksDb { path: root.join("db"), cache_size: 128 }, diff --git a/substrate/bin/node/cli/src/service.rs b/substrate/bin/node/cli/src/service.rs index 69e953f54e42..4eb1db185e9b 100644 --- a/substrate/bin/node/cli/src/service.rs +++ b/substrate/bin/node/cli/src/service.rs @@ -42,6 +42,7 @@ use sc_network_sync::{strategy::warp::WarpSyncConfig, SyncingService}; use sc_service::{config::Configuration, error::Error as ServiceError, RpcHandlers, TaskManager}; use sc_statement_store::Store as StatementStore; use sc_telemetry::{Telemetry, TelemetryWorker}; +use sc_transaction_pool::TransactionPoolHandle; use sc_transaction_pool_api::OffchainTransactionPoolFactory; use sp_api::ProvideRuntimeApi; use sp_core::crypto::Pair; @@ -80,7 +81,7 @@ type FullBeefyBlockImport = beefy::import::BeefyBlockImport< >; /// The transaction pool type definition. -pub type TransactionPool = sc_transaction_pool::FullPool; +pub type TransactionPool = sc_transaction_pool::TransactionPoolHandle; /// The minimum period of blocks on which justifications will be /// imported and generated. @@ -175,7 +176,7 @@ pub fn new_partial( FullBackend, FullSelectChain, sc_consensus::DefaultImportQueue, - sc_transaction_pool::FullPool, + sc_transaction_pool::TransactionPoolHandle, ( impl Fn( sc_rpc::SubscriptionTaskExecutor, @@ -226,12 +227,15 @@ pub fn new_partial( let select_chain = sc_consensus::LongestChain::new(backend.clone()); - let transaction_pool = sc_transaction_pool::BasicPool::new_full( - config.transaction_pool.clone(), - config.role.is_authority().into(), - config.prometheus_registry(), - task_manager.spawn_essential_handle(), - client.clone(), + let transaction_pool = Arc::from( + sc_transaction_pool::Builder::new( + task_manager.spawn_essential_handle(), + client.clone(), + config.role.is_authority().into(), + ) + .with_options(config.transaction_pool.clone()) + .with_prometheus(config.prometheus_registry()) + .build(), ); let (grandpa_block_import, grandpa_link) = grandpa::block_import( @@ -385,7 +389,7 @@ pub struct NewFullBase { /// The syncing service of the node. pub sync: Arc>, /// The transaction pool of the node. - pub transaction_pool: Arc, + pub transaction_pool: Arc>, /// The rpc handlers of the node. pub rpc_handlers: RpcHandlers, } @@ -865,14 +869,14 @@ mod tests { Address, BalancesCall, RuntimeCall, UncheckedExtrinsic, }; use node_primitives::{Block, DigestItem, Signature}; - use polkadot_sdk::*; + use polkadot_sdk::{sc_transaction_pool_api::MaintainedTransactionPool, *}; use sc_client_api::BlockBackend; use sc_consensus::{BlockImport, BlockImportParams, ForkChoiceStrategy}; use sc_consensus_babe::{BabeIntermediate, CompatibleDigestItem, INTERMEDIATE_KEY}; use sc_consensus_epochs::descendent_query; use sc_keystore::LocalKeystore; use sc_service_test::TestNetNode; - use sc_transaction_pool_api::{ChainEvent, MaintainedTransactionPool}; + use sc_transaction_pool_api::ChainEvent; use sp_consensus::{BlockOrigin, Environment, Proposer}; use sp_core::crypto::Pair; use sp_inherents::InherentDataProvider; diff --git a/substrate/client/basic-authorship/src/basic_authorship.rs b/substrate/client/basic-authorship/src/basic_authorship.rs index 527a3d12d9e7..79e6fddae99f 100644 --- a/substrate/client/basic-authorship/src/basic_authorship.rs +++ b/substrate/client/basic-authorship/src/basic_authorship.rs @@ -25,7 +25,6 @@ use futures::{ channel::oneshot, future, future::{Future, FutureExt}, - select, }; use log::{debug, error, info, trace, warn}; use sc_block_builder::{BlockBuilderApi, BlockBuilderBuilder}; @@ -416,26 +415,13 @@ where let mut skipped = 0; let mut unqueue_invalid = Vec::new(); - let mut t1 = self.transaction_pool.ready_at(self.parent_number).fuse(); - let mut t2 = - futures_timer::Delay::new(deadline.saturating_duration_since((self.now)()) / 8).fuse(); - - let mut pending_iterator = select! { - res = t1 => res, - _ = t2 => { - warn!(target: LOG_TARGET, - "Timeout fired waiting for transaction pool at block #{}. \ - Proceeding with production.", - self.parent_number, - ); - self.transaction_pool.ready() - }, - }; + let delay = deadline.saturating_duration_since((self.now)()) / 8; + let mut pending_iterator = + self.transaction_pool.ready_at_with_timeout(self.parent_hash, delay).await; let block_size_limit = block_size_limit.unwrap_or(self.default_block_size_limit); - debug!(target: LOG_TARGET, "Attempting to push transactions from the pool."); - debug!(target: LOG_TARGET, "Pool status: {:?}", self.transaction_pool.status()); + debug!(target: LOG_TARGET, "Attempting to push transactions from the pool at {:?}.", self.parent_hash); let mut transaction_pushed = false; let end_reason = loop { @@ -460,7 +446,7 @@ where break EndProposingReason::HitDeadline } - let pending_tx_data = pending_tx.data().clone(); + let pending_tx_data = (**pending_tx.data()).clone(); let pending_tx_hash = pending_tx.hash().clone(); let block_size = @@ -524,7 +510,7 @@ where pending_iterator.report_invalid(&pending_tx); debug!( target: LOG_TARGET, - "[{:?}] Invalid transaction: {}", pending_tx_hash, e + "[{:?}] Invalid transaction: {} at: {}", pending_tx_hash, e, self.parent_hash ); unqueue_invalid.push(pending_tx_hash); }, @@ -577,13 +563,25 @@ where ) }; - info!( - "🎁 Prepared block for proposing at {} ({} ms) [hash: {:?}; parent_hash: {}; {extrinsics_summary}", - block.header().number(), - block_took.as_millis(), - ::Hash::from(block.header().hash()), - block.header().parent_hash(), - ); + if log::log_enabled!(log::Level::Info) { + info!( + "🎁 Prepared block for proposing at {} ({} ms) [hash: {:?}; parent_hash: {}; extrinsics_count: {}", + block.header().number(), + block_took.as_millis(), + ::Hash::from(block.header().hash()), + block.header().parent_hash(), + extrinsics.len() + ) + } else if log::log_enabled!(log::Level::Debug) { + debug!( + "🎁 Prepared block for proposing at {} ({} ms) [hash: {:?}; parent_hash: {}; {extrinsics_summary}", + block.header().number(), + block_took.as_millis(), + ::Hash::from(block.header().hash()), + block.header().parent_hash(), + ); + } + telemetry!( self.telemetry; CONSENSUS_INFO; @@ -643,22 +641,20 @@ mod tests { // given let client = Arc::new(substrate_test_runtime_client::new()); let spawner = sp_core::testing::TaskExecutor::new(); - let txpool = BasicPool::new_full( + let txpool = Arc::from(BasicPool::new_full( Default::default(), true.into(), None, spawner.clone(), client.clone(), - ); + )); let hashof0 = client.info().genesis_hash; block_on(txpool.submit_at(hashof0, SOURCE, vec![extrinsic(0), extrinsic(1)])).unwrap(); block_on( txpool.maintain(chain_event( - client - .expect_header(client.info().genesis_hash) - .expect("there should be header"), + client.expect_header(hashof0).expect("there should be header"), )), ); @@ -698,13 +694,13 @@ mod tests { fn should_not_panic_when_deadline_is_reached() { let client = Arc::new(substrate_test_runtime_client::new()); let spawner = sp_core::testing::TaskExecutor::new(); - let txpool = BasicPool::new_full( + let txpool = Arc::from(BasicPool::new_full( Default::default(), true.into(), None, spawner.clone(), client.clone(), - ); + )); let mut proposer_factory = ProposerFactory::new(spawner.clone(), client.clone(), txpool.clone(), None, None); @@ -735,13 +731,13 @@ mod tests { let (client, backend) = TestClientBuilder::new().build_with_backend(); let client = Arc::new(client); let spawner = sp_core::testing::TaskExecutor::new(); - let txpool = BasicPool::new_full( + let txpool = Arc::from(BasicPool::new_full( Default::default(), true.into(), None, spawner.clone(), client.clone(), - ); + )); let genesis_hash = client.info().best_hash; @@ -791,13 +787,13 @@ mod tests { // given let client = Arc::new(substrate_test_runtime_client::new()); let spawner = sp_core::testing::TaskExecutor::new(); - let txpool = BasicPool::new_full( + let txpool = Arc::from(BasicPool::new_full( Default::default(), true.into(), None, spawner.clone(), client.clone(), - ); + )); let medium = |nonce| { ExtrinsicBuilder::new_fill_block(Perbill::from_parts(MEDIUM)) @@ -871,27 +867,27 @@ mod tests { // let's create one block and import it let block = propose_block(&client, 0, 2, 7); - import_and_maintain(client.clone(), block); + import_and_maintain(client.clone(), block.clone()); assert_eq!(txpool.ready().count(), 5); // now let's make sure that we can still make some progress let block = propose_block(&client, 1, 1, 5); - import_and_maintain(client.clone(), block); + import_and_maintain(client.clone(), block.clone()); assert_eq!(txpool.ready().count(), 4); // again let's make sure that we can still make some progress let block = propose_block(&client, 2, 1, 4); - import_and_maintain(client.clone(), block); + import_and_maintain(client.clone(), block.clone()); assert_eq!(txpool.ready().count(), 3); // again let's make sure that we can still make some progress let block = propose_block(&client, 3, 1, 3); - import_and_maintain(client.clone(), block); + import_and_maintain(client.clone(), block.clone()); assert_eq!(txpool.ready().count(), 2); // again let's make sure that we can still make some progress let block = propose_block(&client, 4, 2, 2); - import_and_maintain(client.clone(), block); + import_and_maintain(client.clone(), block.clone()); assert_eq!(txpool.ready().count(), 0); } @@ -899,13 +895,13 @@ mod tests { fn should_cease_building_block_when_block_limit_is_reached() { let client = Arc::new(substrate_test_runtime_client::new()); let spawner = sp_core::testing::TaskExecutor::new(); - let txpool = BasicPool::new_full( + let txpool = Arc::from(BasicPool::new_full( Default::default(), true.into(), None, spawner.clone(), client.clone(), - ); + )); let genesis_hash = client.info().genesis_hash; let genesis_header = client.expect_header(genesis_hash).expect("there should be header"); @@ -1004,13 +1000,13 @@ mod tests { // given let client = Arc::new(substrate_test_runtime_client::new()); let spawner = sp_core::testing::TaskExecutor::new(); - let txpool = BasicPool::new_full( + let txpool = Arc::from(BasicPool::new_full( Default::default(), true.into(), None, spawner.clone(), client.clone(), - ); + )); let genesis_hash = client.info().genesis_hash; let tiny = |nonce| { @@ -1073,13 +1069,13 @@ mod tests { // given let client = Arc::new(substrate_test_runtime_client::new()); let spawner = sp_core::testing::TaskExecutor::new(); - let txpool = BasicPool::new_full( + let txpool = Arc::from(BasicPool::new_full( Default::default(), true.into(), None, spawner.clone(), client.clone(), - ); + )); let genesis_hash = client.info().genesis_hash; let tiny = |who| { diff --git a/substrate/client/basic-authorship/src/lib.rs b/substrate/client/basic-authorship/src/lib.rs index 8f47c2ea00e6..adea7a3571dd 100644 --- a/substrate/client/basic-authorship/src/lib.rs +++ b/substrate/client/basic-authorship/src/lib.rs @@ -32,13 +32,13 @@ //! # use sc_transaction_pool::{BasicPool, FullChainApi}; //! # let client = Arc::new(substrate_test_runtime_client::new()); //! # let spawner = sp_core::testing::TaskExecutor::new(); -//! # let txpool = BasicPool::new_full( +//! # let txpool = Arc::from(BasicPool::new_full( //! # Default::default(), //! # true.into(), //! # None, //! # spawner.clone(), //! # client.clone(), -//! # ); +//! # )); //! // The first step is to create a `ProposerFactory`. //! let mut proposer_factory = ProposerFactory::new( //! spawner, diff --git a/substrate/client/cli/Cargo.toml b/substrate/client/cli/Cargo.toml index b7d29aebc3d7..f0b9f8f9b905 100644 --- a/substrate/client/cli/Cargo.toml +++ b/substrate/client/cli/Cargo.toml @@ -43,6 +43,7 @@ sc-network = { workspace = true, default-features = true } sc-service = { workspace = true } sc-telemetry = { workspace = true, default-features = true } sc-tracing = { workspace = true, default-features = true } +sc-transaction-pool = { workspace = true, default-features = true } sc-utils = { workspace = true, default-features = true } sp-blockchain = { workspace = true, default-features = true } sp-core = { workspace = true, default-features = true } diff --git a/substrate/client/cli/src/params/transaction_pool_params.rs b/substrate/client/cli/src/params/transaction_pool_params.rs index 48b2e5b1572b..9cf738f58b6b 100644 --- a/substrate/client/cli/src/params/transaction_pool_params.rs +++ b/substrate/client/cli/src/params/transaction_pool_params.rs @@ -16,8 +16,28 @@ // You should have received a copy of the GNU General Public License // along with this program. If not, see . -use clap::Args; -use sc_service::config::TransactionPoolOptions; +use clap::{Args, ValueEnum}; +use sc_transaction_pool::TransactionPoolOptions; + +/// Type of transaction pool to be used +#[derive(Debug, Clone, Copy, ValueEnum)] +#[value(rename_all = "kebab-case")] +pub enum TransactionPoolType { + /// Uses a legacy, single-state transaction pool. + SingleState, + /// Uses a fork-aware transaction pool. + ForkAware, +} + +impl Into for TransactionPoolType { + fn into(self) -> sc_transaction_pool::TransactionPoolType { + match self { + TransactionPoolType::SingleState => + sc_transaction_pool::TransactionPoolType::SingleState, + TransactionPoolType::ForkAware => sc_transaction_pool::TransactionPoolType::ForkAware, + } + } +} /// Parameters used to create the pool configuration. #[derive(Debug, Clone, Args)] @@ -35,30 +55,21 @@ pub struct TransactionPoolParams { /// If it is considered invalid. Defaults to 1800s. #[arg(long, value_name = "SECONDS")] pub tx_ban_seconds: Option, + + /// The type of transaction pool to be instantiated. + #[arg(long, value_enum, default_value_t = TransactionPoolType::SingleState)] + pub pool_type: TransactionPoolType, } impl TransactionPoolParams { /// Fill the given `PoolConfiguration` by looking at the cli parameters. pub fn transaction_pool(&self, is_dev: bool) -> TransactionPoolOptions { - let mut opts = TransactionPoolOptions::default(); - - // ready queue - opts.ready.count = self.pool_limit; - opts.ready.total_bytes = self.pool_kbytes * 1024; - - // future queue - let factor = 10; - opts.future.count = self.pool_limit / factor; - opts.future.total_bytes = self.pool_kbytes * 1024 / factor; - - opts.ban_time = if let Some(ban_seconds) = self.tx_ban_seconds { - std::time::Duration::from_secs(ban_seconds) - } else if is_dev { - std::time::Duration::from_secs(0) - } else { - std::time::Duration::from_secs(30 * 60) - }; - - opts + TransactionPoolOptions::new_with_params( + self.pool_limit, + self.pool_kbytes * 1024, + self.tx_ban_seconds, + self.pool_type.into(), + is_dev, + ) } } diff --git a/substrate/client/network/transactions/src/lib.rs b/substrate/client/network/transactions/src/lib.rs index a241041968fd..2b5297fe0e13 100644 --- a/substrate/client/network/transactions/src/lib.rs +++ b/substrate/client/network/transactions/src/lib.rs @@ -462,6 +462,8 @@ where if let Some(transaction) = self.transaction_pool.transaction(hash) { let propagated_to = self.do_propagate_transactions(&[(hash.clone(), transaction)]); self.transaction_pool.on_broadcasted(propagated_to); + } else { + debug!(target: "sync", "Propagating transaction failure [{:?}]", hash); } } diff --git a/substrate/client/offchain/src/lib.rs b/substrate/client/offchain/src/lib.rs index 7cee64e6ce7e..3d5728aad17d 100644 --- a/substrate/client/offchain/src/lib.rs +++ b/substrate/client/offchain/src/lib.rs @@ -446,8 +446,13 @@ mod tests { let client = Arc::new(substrate_test_runtime_client::new()); let spawner = sp_core::testing::TaskExecutor::new(); - let pool = - BasicPool::new_full(Default::default(), true.into(), None, spawner, client.clone()); + let pool = Arc::from(BasicPool::new_full( + Default::default(), + true.into(), + None, + spawner, + client.clone(), + )); let network = Arc::new(TestNetwork()); let header = client.header(client.chain_info().genesis_hash).unwrap().unwrap(); diff --git a/substrate/client/rpc-spec-v2/src/transaction/tests/middleware_pool.rs b/substrate/client/rpc-spec-v2/src/transaction/tests/middleware_pool.rs index aa8ac572dec9..adcc987f9c39 100644 --- a/substrate/client/rpc-spec-v2/src/transaction/tests/middleware_pool.rs +++ b/substrate/client/rpc-spec-v2/src/transaction/tests/middleware_pool.rs @@ -27,7 +27,7 @@ use sc_transaction_pool_api::{ use crate::hex_string; use futures::{FutureExt, StreamExt}; -use sp_runtime::traits::{Block as BlockT, NumberFor}; +use sp_runtime::traits::Block as BlockT; use std::{collections::HashMap, pin::Pin, sync::Arc}; use substrate_test_runtime_transaction_pool::TestApi; use tokio::sync::mpsc; @@ -166,7 +166,7 @@ impl TransactionPool for MiddlewarePool { fn ready_at( &self, - at: NumberFor, + at: ::Hash, ) -> Pin< Box< dyn Future< @@ -184,4 +184,19 @@ impl TransactionPool for MiddlewarePool { fn futures(&self) -> Vec { self.inner_pool.futures() } + + fn ready_at_with_timeout( + &self, + at: ::Hash, + _timeout: std::time::Duration, + ) -> Pin< + Box< + dyn Future< + Output = Box> + Send>, + > + Send + + '_, + >, + > { + self.inner_pool.ready_at(at) + } } diff --git a/substrate/client/rpc/src/author/tests.rs b/substrate/client/rpc/src/author/tests.rs index bde60960eaf4..ab0b8bdab699 100644 --- a/substrate/client/rpc/src/author/tests.rs +++ b/substrate/client/rpc/src/author/tests.rs @@ -66,8 +66,13 @@ impl Default for TestSetup { let client = Arc::new(substrate_test_runtime_client::TestClientBuilder::new().build()); let spawner = sp_core::testing::TaskExecutor::new(); - let pool = - BasicPool::new_full(Default::default(), true.into(), None, spawner, client.clone()); + let pool = Arc::from(BasicPool::new_full( + Default::default(), + true.into(), + None, + spawner, + client.clone(), + )); TestSetup { client, keystore, pool } } } diff --git a/substrate/client/service/src/config.rs b/substrate/client/service/src/config.rs index 6f65c2e2d81b..fb9e9264dfe7 100644 --- a/substrate/client/service/src/config.rs +++ b/substrate/client/service/src/config.rs @@ -37,7 +37,7 @@ pub use sc_rpc_server::{ IpNetwork, RpcEndpoint, RpcMethods, SubscriptionIdProvider as RpcSubscriptionIdProvider, }; pub use sc_telemetry::TelemetryEndpoints; -pub use sc_transaction_pool::Options as TransactionPoolOptions; +pub use sc_transaction_pool::TransactionPoolOptions; use sp_core::crypto::SecretString; use std::{ io, iter, diff --git a/substrate/client/service/src/lib.rs b/substrate/client/service/src/lib.rs index b6acdb8ed002..54e847791cff 100644 --- a/substrate/client/service/src/lib.rs +++ b/substrate/client/service/src/lib.rs @@ -94,7 +94,7 @@ pub use sc_network_sync::WarpSyncConfig; pub use sc_network_transactions::config::{TransactionImport, TransactionImportFuture}; pub use sc_rpc::{RandomIntegerSubscriptionId, RandomStringSubscriptionId}; pub use sc_tracing::TracingReceiver; -pub use sc_transaction_pool::Options as TransactionPoolOptions; +pub use sc_transaction_pool::TransactionPoolOptions; pub use sc_transaction_pool_api::{error::IntoPoolError, InPoolTransaction, TransactionPool}; #[doc(hidden)] pub use std::{ops::Deref, result::Result, sync::Arc}; @@ -484,7 +484,7 @@ where .filter(|t| t.is_propagable()) .map(|t| { let hash = t.hash().clone(); - let ex: B::Extrinsic = t.data().clone(); + let ex: B::Extrinsic = (**t.data()).clone(); (hash, ex) }) .collect() @@ -523,6 +523,7 @@ where }, }; + let start = std::time::Instant::now(); let import_future = self.pool.submit_one( self.client.info().best_hash, sc_transaction_pool_api::TransactionSource::External, @@ -530,16 +531,16 @@ where ); Box::pin(async move { match import_future.await { - Ok(_) => TransactionImport::NewGood, + Ok(_) => { + let elapsed = start.elapsed(); + debug!(target: sc_transaction_pool::LOG_TARGET, "import transaction: {elapsed:?}"); + TransactionImport::NewGood + }, Err(e) => match e.into_pool_error() { Ok(sc_transaction_pool_api::error::Error::AlreadyImported(_)) => TransactionImport::KnownGood, - Ok(e) => { - debug!("Error adding transaction to the pool: {:?}", e); - TransactionImport::Bad - }, - Err(e) => { - debug!("Error converting pool error: {}", e); + Ok(_) => TransactionImport::Bad, + Err(_) => { // it is not bad at least, just some internal node logic error, so peer is // innocent. TransactionImport::KnownGood @@ -556,7 +557,7 @@ where fn transaction(&self, hash: &H) -> Option { self.pool.ready_transaction(hash).and_then( // Only propagable transactions should be resolved for network service. - |tx| if tx.is_propagable() { Some(tx.data().clone()) } else { None }, + |tx| if tx.is_propagable() { Some((**tx.data()).clone()) } else { None }, ) } } @@ -578,8 +579,13 @@ mod tests { let (client, longest_chain) = TestClientBuilder::new().build_with_longest_chain(); let client = Arc::new(client); let spawner = sp_core::testing::TaskExecutor::new(); - let pool = - BasicPool::new_full(Default::default(), true.into(), None, spawner, client.clone()); + let pool = Arc::from(BasicPool::new_full( + Default::default(), + true.into(), + None, + spawner, + client.clone(), + )); let source = sp_runtime::transaction_validity::TransactionSource::External; let best = block_on(longest_chain.best_chain()).unwrap(); let transaction = Transfer { diff --git a/substrate/client/transaction-pool/Cargo.toml b/substrate/client/transaction-pool/Cargo.toml index 98994cc742ff..d346add93a64 100644 --- a/substrate/client/transaction-pool/Cargo.toml +++ b/substrate/client/transaction-pool/Cargo.toml @@ -20,6 +20,8 @@ async-trait = { workspace = true } codec = { workspace = true, default-features = true } futures = { workspace = true } futures-timer = { workspace = true } +indexmap = { workspace = true } +itertools = { workspace = true } linked-hash-map = { workspace = true } log = { workspace = true, default-features = true } parking_lot = { workspace = true, default-features = true } @@ -36,6 +38,8 @@ sp-crypto-hashing = { workspace = true, default-features = true } sp-runtime = { workspace = true, default-features = true } sp-tracing = { workspace = true, default-features = true } sp-transaction-pool = { workspace = true, default-features = true } +tokio-stream = { workspace = true } +tokio = { workspace = true, default-features = true, features = ["macros", "time"] } [dev-dependencies] array-bytes = { workspace = true, default-features = true } diff --git a/substrate/client/transaction-pool/api/src/error.rs b/substrate/client/transaction-pool/api/src/error.rs index d0744bfa3e19..e81955ebe54c 100644 --- a/substrate/client/transaction-pool/api/src/error.rs +++ b/substrate/client/transaction-pool/api/src/error.rs @@ -38,7 +38,7 @@ pub enum Error { /// The transaction validity returned no "provides" tag. /// /// Such transactions are not accepted to the pool, since we use those tags - /// to define identity of transactions (occupance of the same "slot"). + /// to define identity of transactions (occupancy of the same "slot"). #[error("Transaction does not provide any tags, so the pool can't identify it")] NoTagsProvided, diff --git a/substrate/client/transaction-pool/api/src/lib.rs b/substrate/client/transaction-pool/api/src/lib.rs index 0a313c5b782d..3ac1a79a0c28 100644 --- a/substrate/client/transaction-pool/api/src/lib.rs +++ b/substrate/client/transaction-pool/api/src/lib.rs @@ -26,7 +26,7 @@ use codec::Codec; use futures::{Future, Stream}; use serde::{de::DeserializeOwned, Deserialize, Serialize}; use sp_core::offchain::TransactionPoolExt; -use sp_runtime::traits::{Block as BlockT, Member, NumberFor}; +use sp_runtime::traits::{Block as BlockT, Member}; use std::{collections::HashMap, hash::Hash, marker::PhantomData, pin::Pin, sync::Arc}; const LOG_TARGET: &str = "txpool::api"; @@ -36,7 +36,7 @@ pub use sp_runtime::transaction_validity::{ }; /// Transaction pool status. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct PoolStatus { /// Number of transactions in the ready queue. pub ready: usize, @@ -49,7 +49,7 @@ pub struct PoolStatus { } impl PoolStatus { - /// Returns true if the are no transactions in the pool. + /// Returns true if there are no transactions in the pool. pub fn is_empty(&self) -> bool { self.ready == 0 && self.future == 0 } @@ -57,7 +57,7 @@ impl PoolStatus { /// Possible transaction status events. /// -/// This events are being emitted by `TransactionPool` watchers, +/// These events are being emitted by `TransactionPool` watchers, /// which are also exposed over RPC. /// /// The status events can be grouped based on their kinds as: @@ -144,7 +144,7 @@ pub enum TransactionStatus { /// Maximum number of finality watchers has been reached, /// old watchers are being removed. FinalityTimeout(BlockHash), - /// Transaction has been finalized by a finality-gadget, e.g GRANDPA. + /// Transaction has been finalized by a finality-gadget, e.g. GRANDPA. #[serde(with = "v1_compatible")] Finalized((BlockHash, TxIndex)), /// Transaction has been replaced in the pool, by another transaction @@ -245,7 +245,7 @@ pub trait TransactionPool: Send + Sync { type Hash: Hash + Eq + Member + Serialize + DeserializeOwned + Codec; /// In-pool transaction type. type InPoolTransaction: InPoolTransaction< - Transaction = TransactionFor, + Transaction = Arc>, Hash = TxHash, >; /// Error type. @@ -269,7 +269,7 @@ pub trait TransactionPool: Send + Sync { xt: TransactionFor, ) -> PoolFuture, Self::Error>; - /// Returns a future that import a single transaction and starts to watch their progress in the + /// Returns a future that imports a single transaction and starts to watch their progress in the /// pool. fn submit_and_watch( &self, @@ -285,7 +285,7 @@ pub trait TransactionPool: Send + Sync { /// Guarantees to return immediately when `None` is passed. fn ready_at( &self, - at: NumberFor, + at: ::Hash, ) -> Pin< Box< dyn Future< @@ -321,6 +321,23 @@ pub trait TransactionPool: Send + Sync { /// Return specific ready transaction by hash, if there is one. fn ready_transaction(&self, hash: &TxHash) -> Option>; + + /// Returns set of ready transaction at given block within given timeout. + /// + /// If the timeout is hit during method execution then the best effort set of ready transactions + /// for given block, without executing full maintain process is returned. + fn ready_at_with_timeout( + &self, + at: ::Hash, + timeout: std::time::Duration, + ) -> Pin< + Box< + dyn Future< + Output = Box> + Send>, + > + Send + + '_, + >, + >; } /// An iterator of ready transactions. @@ -345,6 +362,7 @@ impl ReadyTransactions for std::iter::Empty { } /// Events that the transaction pool listens for. +#[derive(Debug)] pub enum ChainEvent { /// New best block have been added to the chain. NewBestBlock { @@ -441,7 +459,7 @@ impl OffchainSubmitTransaction for TP at: ::Hash, extrinsic: ::Extrinsic, ) -> Result<(), ()> { - log::debug!( + log::trace!( target: LOG_TARGET, "(offchain call) Submitting a transaction to the pool: {:?}", extrinsic diff --git a/substrate/client/transaction-pool/benches/basics.rs b/substrate/client/transaction-pool/benches/basics.rs index 65c83f090535..2db34bc3f32f 100644 --- a/substrate/client/transaction-pool/benches/basics.rs +++ b/substrate/client/transaction-pool/benches/basics.rs @@ -24,6 +24,7 @@ use futures::{ future::{ready, Ready}, }; use sc_transaction_pool::*; +use sp_blockchain::HashAndNumber; use sp_crypto_hashing::blake2_256; use sp_runtime::{ generic::BlockId, @@ -64,8 +65,9 @@ impl ChainApi for TestApi { &self, at: ::Hash, _source: TransactionSource, - uxt: ::Extrinsic, + uxt: Arc<::Extrinsic>, ) -> Self::ValidationFuture { + let uxt = (*uxt).clone(); let transfer = TransferData::try_from(&uxt) .expect("uxt is expected to be bench_call (carrying TransferData)"); let nonce = transfer.nonce; @@ -144,6 +146,10 @@ fn bench_configured(pool: Pool, number: u64, api: Arc) { let source = TransactionSource::External; let mut futures = Vec::new(); let mut tags = Vec::new(); + let at = HashAndNumber { + hash: api.block_id_to_hash(&BlockId::Number(1)).unwrap().unwrap(), + number: 1, + }; for nonce in 1..=number { let xt = uxt(TransferData { @@ -151,15 +157,12 @@ fn bench_configured(pool: Pool, number: u64, api: Arc) { to: AccountId::from_h256(H256::from_low_u64_be(2)), amount: 5, nonce, - }); + }) + .into(); tags.push(to_tag(nonce, AccountId::from_h256(H256::from_low_u64_be(1)))); - futures.push(pool.submit_one( - api.block_id_to_hash(&BlockId::Number(1)).unwrap().unwrap(), - source, - xt, - )); + futures.push(pool.submit_one(&at, source, xt)); } let res = block_on(futures::future::join_all(futures.into_iter())); @@ -170,12 +173,11 @@ fn bench_configured(pool: Pool, number: u64, api: Arc) { // Prune all transactions. let block_num = 6; - block_on(pool.prune_tags( - api.block_id_to_hash(&BlockId::Number(block_num)).unwrap().unwrap(), - tags, - vec![], - )) - .expect("Prune failed"); + let at = HashAndNumber { + hash: api.block_id_to_hash(&BlockId::Number(block_num)).unwrap().unwrap(), + number: block_num, + }; + block_on(pool.prune_tags(&at, tags, vec![])); // pool is empty assert_eq!(pool.validated_pool().status().ready, 0); diff --git a/substrate/client/transaction-pool/src/builder.rs b/substrate/client/transaction-pool/src/builder.rs new file mode 100644 index 000000000000..e1fddcdd8952 --- /dev/null +++ b/substrate/client/transaction-pool/src/builder.rs @@ -0,0 +1,245 @@ +// 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 . + +//! Utility for building substrate transaction pool trait object. + +use crate::{ + common::api::FullChainApi, + fork_aware_txpool::ForkAwareTxPool as ForkAwareFullPool, + graph::{base_pool::Transaction, ChainApi, ExtrinsicFor, ExtrinsicHash, IsValidator, Options}, + single_state_txpool::BasicPool as SingleStateFullPool, + TransactionPoolWrapper, LOG_TARGET, +}; +use prometheus_endpoint::Registry as PrometheusRegistry; +use sc_transaction_pool_api::{LocalTransactionPool, MaintainedTransactionPool}; +use sp_core::traits::SpawnEssentialNamed; +use sp_runtime::traits::Block as BlockT; +use std::{marker::PhantomData, sync::Arc, time::Duration}; + +/// The type of transaction pool. +#[derive(Debug, Clone)] +pub enum TransactionPoolType { + /// Single-state transaction pool + SingleState, + /// Fork-aware transaction pool + ForkAware, +} + +/// Transaction pool options. +#[derive(Debug, Clone)] +pub struct TransactionPoolOptions { + txpool_type: TransactionPoolType, + options: Options, +} + +impl Default for TransactionPoolOptions { + fn default() -> Self { + Self { txpool_type: TransactionPoolType::SingleState, options: Default::default() } + } +} + +impl TransactionPoolOptions { + /// Creates the options for the transaction pool using given parameters. + pub fn new_with_params( + pool_limit: usize, + pool_bytes: usize, + tx_ban_seconds: Option, + txpool_type: TransactionPoolType, + is_dev: bool, + ) -> TransactionPoolOptions { + let mut options = Options::default(); + + // ready queue + options.ready.count = pool_limit; + options.ready.total_bytes = pool_bytes; + + // future queue + let factor = 10; + options.future.count = pool_limit / factor; + options.future.total_bytes = pool_bytes / factor; + + options.ban_time = if let Some(ban_seconds) = tx_ban_seconds { + Duration::from_secs(ban_seconds) + } else if is_dev { + Duration::from_secs(0) + } else { + Duration::from_secs(30 * 60) + }; + + TransactionPoolOptions { options, txpool_type } + } + + /// Creates predefined options for benchmarking + pub fn new_for_benchmarks() -> TransactionPoolOptions { + TransactionPoolOptions { + options: Options { + ready: crate::graph::base_pool::Limit { + count: 100_000, + total_bytes: 100 * 1024 * 1024, + }, + future: crate::graph::base_pool::Limit { + count: 100_000, + total_bytes: 100 * 1024 * 1024, + }, + reject_future_transactions: false, + ban_time: Duration::from_secs(30 * 60), + }, + txpool_type: TransactionPoolType::SingleState, + } + } +} + +/// `FullClientTransactionPool` is a trait that combines the functionality of +/// `MaintainedTransactionPool` and `LocalTransactionPool` for a given `Client` and `Block`. +/// +/// This trait defines the requirements for a full client transaction pool, ensuring +/// that it can handle transactions submission and maintenance. +pub trait FullClientTransactionPool: + MaintainedTransactionPool< + Block = Block, + Hash = ExtrinsicHash>, + InPoolTransaction = Transaction< + ExtrinsicHash>, + ExtrinsicFor>, + >, + Error = as ChainApi>::Error, + > + LocalTransactionPool< + Block = Block, + Hash = ExtrinsicHash>, + Error = as ChainApi>::Error, + > +where + Block: BlockT, + Client: sp_api::ProvideRuntimeApi + + sc_client_api::BlockBackend + + sc_client_api::blockchain::HeaderBackend + + sp_runtime::traits::BlockIdTo + + sp_blockchain::HeaderMetadata + + 'static, + Client::Api: sp_transaction_pool::runtime_api::TaggedTransactionQueue, +{ +} + +impl FullClientTransactionPool for P +where + Block: BlockT, + Client: sp_api::ProvideRuntimeApi + + sc_client_api::BlockBackend + + sc_client_api::blockchain::HeaderBackend + + sp_runtime::traits::BlockIdTo + + sp_blockchain::HeaderMetadata + + 'static, + Client::Api: sp_transaction_pool::runtime_api::TaggedTransactionQueue, + P: MaintainedTransactionPool< + Block = Block, + Hash = ExtrinsicHash>, + InPoolTransaction = Transaction< + ExtrinsicHash>, + ExtrinsicFor>, + >, + Error = as ChainApi>::Error, + > + LocalTransactionPool< + Block = Block, + Hash = ExtrinsicHash>, + Error = as ChainApi>::Error, + >, +{ +} + +/// The public type alias for the actual type providing the implementation of +/// `FullClientTransactionPool` with the given `Client` and `Block` types. +/// +/// This handle abstracts away the specific type of the transaction pool. Should be used +/// externally to keep reference to transaction pool. +pub type TransactionPoolHandle = TransactionPoolWrapper; + +/// Builder allowing to create specific instance of transaction pool. +pub struct Builder<'a, Block, Client> { + options: TransactionPoolOptions, + is_validator: IsValidator, + prometheus: Option<&'a PrometheusRegistry>, + client: Arc, + spawner: Box, + _phantom: PhantomData<(Client, Block)>, +} + +impl<'a, Client, Block> Builder<'a, Block, Client> +where + Block: BlockT, + Client: sp_api::ProvideRuntimeApi + + sc_client_api::BlockBackend + + sc_client_api::blockchain::HeaderBackend + + sp_runtime::traits::BlockIdTo + + sc_client_api::ExecutorProvider + + sc_client_api::UsageProvider + + sp_blockchain::HeaderMetadata + + Send + + Sync + + 'static, + ::Hash: std::marker::Unpin, + Client::Api: sp_transaction_pool::runtime_api::TaggedTransactionQueue, +{ + /// Creates new instance of `Builder` + pub fn new( + spawner: impl SpawnEssentialNamed + 'static, + client: Arc, + is_validator: IsValidator, + ) -> Builder<'a, Block, Client> { + Builder { + options: Default::default(), + _phantom: Default::default(), + spawner: Box::new(spawner), + client, + is_validator, + prometheus: None, + } + } + + /// Sets the options used for creating a transaction pool instance. + pub fn with_options(mut self, options: TransactionPoolOptions) -> Self { + self.options = options; + self + } + + /// Sets the prometheus endpoint used in a transaction pool instance. + pub fn with_prometheus(mut self, prometheus: Option<&'a PrometheusRegistry>) -> Self { + self.prometheus = prometheus; + self + } + + /// Creates an instance of transaction pool. + pub fn build(self) -> TransactionPoolHandle { + log::info!(target:LOG_TARGET, " creating {:?} txpool {:?}/{:?}.", self.options.txpool_type, self.options.options.ready, self.options.options.future); + TransactionPoolWrapper::(match self.options.txpool_type { + TransactionPoolType::SingleState => Box::new(SingleStateFullPool::new_full( + self.options.options, + self.is_validator, + self.prometheus, + self.spawner, + self.client, + )), + TransactionPoolType::ForkAware => Box::new(ForkAwareFullPool::new_full( + self.options.options, + self.is_validator, + self.prometheus, + self.spawner, + self.client, + )), + }) + } +} diff --git a/substrate/client/transaction-pool/src/api.rs b/substrate/client/transaction-pool/src/common/api.rs similarity index 87% rename from substrate/client/transaction-pool/src/api.rs rename to substrate/client/transaction-pool/src/common/api.rs index cccaad7c8994..a5185ba606ef 100644 --- a/substrate/client/transaction-pool/src/api.rs +++ b/substrate/client/transaction-pool/src/common/api.rs @@ -40,18 +40,18 @@ use sp_runtime::{ }; use sp_transaction_pool::runtime_api::TaggedTransactionQueue; -use crate::{ +use super::{ error::{self, Error}, - graph, metrics::{ApiMetrics, ApiMetricsExt}, }; +use crate::graph; /// The transaction pool logic for full client. pub struct FullChainApi { client: Arc, _marker: PhantomData, metrics: Option>, - validation_pool: Arc + Send>>>>>, + validation_pool: mpsc::Sender + Send>>>, } /// Spawn a validation task that will be used by the transaction pool to validate transactions. @@ -101,12 +101,7 @@ impl FullChainApi { spawn_validation_pool_task("transaction-pool-task-0", receiver.clone(), spawner); spawn_validation_pool_task("transaction-pool-task-1", receiver, spawner); - FullChainApi { - client, - validation_pool: Arc::new(Mutex::new(sender)), - _marker: Default::default(), - metrics, - } + FullChainApi { client, validation_pool: sender, _marker: Default::default(), metrics } } } @@ -139,25 +134,25 @@ where ) -> Self::ValidationFuture { let (tx, rx) = oneshot::channel(); let client = self.client.clone(); - let validation_pool = self.validation_pool.clone(); + let mut validation_pool = self.validation_pool.clone(); let metrics = self.metrics.clone(); async move { metrics.report(|m| m.validations_scheduled.inc()); - validation_pool - .lock() - .await - .send( - async move { - let res = validate_transaction_blocking(&*client, at, source, uxt); - let _ = tx.send(res); - metrics.report(|m| m.validations_finished.inc()); - } - .boxed(), - ) - .await - .map_err(|e| Error::RuntimeApi(format!("Validation pool down: {:?}", e)))?; + { + validation_pool + .send( + async move { + let res = validate_transaction_blocking(&*client, at, source, uxt); + let _ = tx.send(res); + metrics.report(|m| m.validations_finished.inc()); + } + .boxed(), + ) + .await + .map_err(|e| Error::RuntimeApi(format!("Validation pool down: {:?}", e)))?; + } match rx.await { Ok(r) => r, @@ -183,7 +178,7 @@ where fn hash_and_length( &self, - ex: &graph::ExtrinsicFor, + ex: &graph::RawExtrinsicFor, ) -> (graph::ExtrinsicHash, usize) { ex.using_encoded(|x| ( as traits::Hash>::hash(x), x.len())) } @@ -222,7 +217,10 @@ where Client: Send + Sync + 'static, Client::Api: TaggedTransactionQueue, { - sp_tracing::within_span!(sp_tracing::Level::TRACE, "validate_transaction"; + let s = std::time::Instant::now(); + let h = uxt.using_encoded(|x| as traits::Hash>::hash(x)); + + let result = sp_tracing::within_span!(sp_tracing::Level::TRACE, "validate_transaction"; { let runtime_api = client.runtime_api(); let api_version = sp_tracing::within_span! { sp_tracing::Level::TRACE, "check_version"; @@ -240,7 +238,7 @@ where sp_tracing::Level::TRACE, "runtime::validate_transaction"; { if api_version >= 3 { - runtime_api.validate_transaction(at, source, uxt, at) + runtime_api.validate_transaction(at, source, (*uxt).clone(), at) .map_err(|e| Error::RuntimeApi(e.to_string())) } else { let block_number = client.to_number(&BlockId::Hash(at)) @@ -260,16 +258,19 @@ where if api_version == 2 { #[allow(deprecated)] // old validate_transaction - runtime_api.validate_transaction_before_version_3(at, source, uxt) + runtime_api.validate_transaction_before_version_3(at, source, (*uxt).clone()) .map_err(|e| Error::RuntimeApi(e.to_string())) } else { #[allow(deprecated)] // old validate_transaction - runtime_api.validate_transaction_before_version_2(at, uxt) + runtime_api.validate_transaction_before_version_2(at, (*uxt).clone()) .map_err(|e| Error::RuntimeApi(e.to_string())) } } }) - }) + }); + log::trace!(target: LOG_TARGET, "[{h:?}] validate_transaction_blocking: at:{at:?} took:{:?}", s.elapsed()); + + result } impl FullChainApi diff --git a/substrate/client/transaction-pool/src/enactment_state.rs b/substrate/client/transaction-pool/src/common/enactment_state.rs similarity index 94% rename from substrate/client/transaction-pool/src/enactment_state.rs rename to substrate/client/transaction-pool/src/common/enactment_state.rs index 85c572c127e8..a7eb6a3687c6 100644 --- a/substrate/client/transaction-pool/src/enactment_state.rs +++ b/substrate/client/transaction-pool/src/common/enactment_state.rs @@ -34,7 +34,7 @@ const SKIP_MAINTENANCE_THRESHOLD: u16 = 20; /// is to figure out which phases (enactment / finalization) of transaction pool /// maintenance are needed. /// -/// Given the following chain: +/// Example: given the following chain: /// /// B1-C1-D1-E1 /// / @@ -42,8 +42,8 @@ const SKIP_MAINTENANCE_THRESHOLD: u16 = 20; /// \ /// B2-C2-D2-E2 /// -/// Some scenarios and expected behavior for sequence of `NewBestBlock` (`nbb`) and `Finalized` -/// (`f`) events: +/// the list presents scenarios and expected behavior for sequence of `NewBestBlock` (`nbb`) +/// and `Finalized` (`f`) events. true/false means if enactiment is required: /// /// - `nbb(C1)`, `f(C1)` -> false (enactment was already performed in `nbb(C1))` /// - `f(C1)`, `nbb(C1)` -> false (enactment was already performed in `f(C1))` @@ -103,7 +103,7 @@ where let new_hash = event.hash(); let finalized = event.is_finalized(); - // do not proceed with txpool maintain if block distance is to high + // do not proceed with txpool maintain if block distance is too high let skip_maintenance = match (hash_to_number(new_hash), hash_to_number(self.recent_best_block)) { (Ok(Some(new)), Ok(Some(current))) => @@ -112,14 +112,14 @@ where }; if skip_maintenance { - log::debug!(target: LOG_TARGET, "skip maintain: tree_route would be too long"); + log::trace!(target: LOG_TARGET, "skip maintain: tree_route would be too long"); self.force_update(event); return Ok(EnactmentAction::Skip) } // block was already finalized if self.recent_finalized_block == new_hash { - log::debug!(target: LOG_TARGET, "handle_enactment: block already finalized"); + log::trace!(target: LOG_TARGET, "handle_enactment: block already finalized"); return Ok(EnactmentAction::Skip) } @@ -127,7 +127,7 @@ where // it instead of tree_route provided with event let tree_route = tree_route(self.recent_best_block, new_hash)?; - log::debug!( + log::trace!( target: LOG_TARGET, "resolve hash: {new_hash:?} finalized: {finalized:?} \ tree_route: (common {:?}, last {:?}) best_block: {:?} finalized_block:{:?}", @@ -141,7 +141,7 @@ where // happening if we first received a finalization event and then a new // best event for some old stale best head. if tree_route.retracted().iter().any(|x| x.hash == self.recent_finalized_block) { - log::debug!( + log::trace!( target: LOG_TARGET, "Recently finalized block {} would be retracted by ChainEvent {}, skipping", self.recent_finalized_block, @@ -180,7 +180,7 @@ where ChainEvent::NewBestBlock { hash, .. } => self.recent_best_block = *hash, ChainEvent::Finalized { hash, .. } => self.recent_finalized_block = *hash, }; - log::debug!( + log::trace!( target: LOG_TARGET, "forced update: {:?}, {:?}", self.recent_best_block, @@ -296,7 +296,7 @@ mod enactment_state_tests { use super::*; /// asserts that tree routes are equal - fn assert_treeroute_eq( + fn assert_tree_route_eq( expected: Result, String>, result: Result, String>, ) { @@ -323,56 +323,56 @@ mod enactment_state_tests { fn tree_route_mock_test_01() { let result = tree_route(b1().hash, a().hash); let expected = TreeRoute::new(vec![b1(), a()], 1); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] fn tree_route_mock_test_02() { let result = tree_route(a().hash, b1().hash); let expected = TreeRoute::new(vec![a(), b1()], 0); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] fn tree_route_mock_test_03() { let result = tree_route(a().hash, c2().hash); let expected = TreeRoute::new(vec![a(), b2(), c2()], 0); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] fn tree_route_mock_test_04() { let result = tree_route(e2().hash, a().hash); let expected = TreeRoute::new(vec![e2(), d2(), c2(), b2(), a()], 4); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] fn tree_route_mock_test_05() { let result = tree_route(d1().hash, b1().hash); let expected = TreeRoute::new(vec![d1(), c1(), b1()], 2); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] fn tree_route_mock_test_06() { let result = tree_route(d2().hash, b2().hash); let expected = TreeRoute::new(vec![d2(), c2(), b2()], 2); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] fn tree_route_mock_test_07() { let result = tree_route(b1().hash, d1().hash); let expected = TreeRoute::new(vec![b1(), c1(), d1()], 0); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] fn tree_route_mock_test_08() { let result = tree_route(b2().hash, d2().hash); let expected = TreeRoute::new(vec![b2(), c2(), d2()], 0); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] @@ -380,7 +380,7 @@ mod enactment_state_tests { let result = tree_route(e2().hash, e1().hash); let expected = TreeRoute::new(vec![e2(), d2(), c2(), b2(), a(), b1(), c1(), d1(), e1()], 4); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] @@ -388,55 +388,55 @@ mod enactment_state_tests { let result = tree_route(e1().hash, e2().hash); let expected = TreeRoute::new(vec![e1(), d1(), c1(), b1(), a(), b2(), c2(), d2(), e2()], 4); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] fn tree_route_mock_test_11() { let result = tree_route(b1().hash, c2().hash); let expected = TreeRoute::new(vec![b1(), a(), b2(), c2()], 1); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] fn tree_route_mock_test_12() { let result = tree_route(d2().hash, b1().hash); let expected = TreeRoute::new(vec![d2(), c2(), b2(), a(), b1()], 3); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] fn tree_route_mock_test_13() { let result = tree_route(c2().hash, e1().hash); let expected = TreeRoute::new(vec![c2(), b2(), a(), b1(), c1(), d1(), e1()], 2); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] fn tree_route_mock_test_14() { let result = tree_route(b1().hash, b1().hash); let expected = TreeRoute::new(vec![b1()], 0); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] fn tree_route_mock_test_15() { let result = tree_route(b2().hash, b2().hash); let expected = TreeRoute::new(vec![b2()], 0); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] fn tree_route_mock_test_16() { let result = tree_route(a().hash, a().hash); let expected = TreeRoute::new(vec![a()], 0); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } #[test] fn tree_route_mock_test_17() { let result = tree_route(x2().hash, b1().hash); let expected = TreeRoute::new(vec![x2(), e2(), d2(), c2(), b2(), a(), b1()], 5); - assert_treeroute_eq(result, expected); + assert_tree_route_eq(result, expected); } } diff --git a/substrate/client/transaction-pool/src/error.rs b/substrate/client/transaction-pool/src/common/error.rs similarity index 100% rename from substrate/client/transaction-pool/src/error.rs rename to substrate/client/transaction-pool/src/common/error.rs diff --git a/substrate/client/transaction-pool/src/common/log_xt.rs b/substrate/client/transaction-pool/src/common/log_xt.rs new file mode 100644 index 000000000000..6c3752c1d50e --- /dev/null +++ b/substrate/client/transaction-pool/src/common/log_xt.rs @@ -0,0 +1,54 @@ +// 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 . + +//! Utility for logging transaction collections. + +/// Logs every transaction from given `tx_collection` with given level. +macro_rules! log_xt { + (data: hash, target: $target:expr, $level:expr, $tx_collection:expr, $text_with_format:expr) => { + if log::log_enabled!(target: $target, $level) { + for tx in $tx_collection { + log::log!(target: $target, $level, $text_with_format, tx); + } + } + }; + (data: hash, target: $target:expr, $level:expr, $tx_collection:expr, $text_with_format:expr, $($arg:expr),*) => { + if log::log_enabled!(target: $target, $level) { + for tx in $tx_collection { + log::log!(target: $target, $level, $text_with_format, tx, $($arg),*); + } + } + }; + (data: tuple, target: $target:expr, $level:expr, $tx_collection:expr, $text_with_format:expr) => { + if log::log_enabled!(target: $target, $level) { + for tx in $tx_collection { + log::log!(target: $target, $level, $text_with_format, tx.0, tx.1) + } + } + }; +} + +/// Logs every transaction from given `tx_collection` with trace level. +macro_rules! log_xt_trace { + (data: $datatype:ident, target: $target:expr, $($arg:tt)+) => ($crate::common::log_xt::log_xt!(data: $datatype, target: $target, log::Level::Trace, $($arg)+)); + (target: $target:expr, $tx_collection:expr, $text_with_format:expr) => ($crate::common::log_xt::log_xt!(data: hash, target: $target, log::Level::Trace, $tx_collection, $text_with_format)); + (target: $target:expr, $tx_collection:expr, $text_with_format:expr, $($arg:expr)*) => ($crate::common::log_xt::log_xt!(data: hash, target: $target, log::Level::Trace, $tx_collection, $text_with_format, $($arg)*)); +} + +pub(crate) use log_xt; +pub(crate) use log_xt_trace; diff --git a/substrate/client/transaction-pool/src/metrics.rs b/substrate/client/transaction-pool/src/common/metrics.rs similarity index 58% rename from substrate/client/transaction-pool/src/metrics.rs rename to substrate/client/transaction-pool/src/common/metrics.rs index 170bface9647..0ec3b511fa0e 100644 --- a/substrate/client/transaction-pool/src/metrics.rs +++ b/substrate/client/transaction-pool/src/common/metrics.rs @@ -16,76 +16,52 @@ // You should have received a copy of the GNU General Public License // along with this program. If not, see . -//! Transaction pool Prometheus metrics. +//! Transaction pool Prometheus metrics for implementation of Chain API. +use prometheus_endpoint::{register, Counter, PrometheusError, Registry, U64}; use std::sync::Arc; -use prometheus_endpoint::{register, Counter, PrometheusError, Registry, U64}; +use crate::LOG_TARGET; -#[derive(Clone, Default)] -pub struct MetricsLink(Arc>); +/// Provides interface to register the specific metrics in the Prometheus register. +pub(crate) trait MetricsRegistrant { + /// Registers the metrics at given Prometheus registry. + fn register(registry: &Registry) -> Result, PrometheusError>; +} -impl MetricsLink { +/// Generic structure to keep a link to metrics register. +pub(crate) struct GenericMetricsLink(Arc>>); + +impl Default for GenericMetricsLink { + fn default() -> Self { + Self(Arc::from(None)) + } +} + +impl Clone for GenericMetricsLink { + fn clone(&self) -> Self { + Self(self.0.clone()) + } +} + +impl GenericMetricsLink { pub fn new(registry: Option<&Registry>) -> Self { Self(Arc::new(registry.and_then(|registry| { - Metrics::register(registry) + M::register(registry) .map_err(|err| { - log::warn!("Failed to register prometheus metrics: {}", err); + log::warn!(target: LOG_TARGET, "Failed to register prometheus metrics: {}", err); }) .ok() }))) } - pub fn report(&self, do_this: impl FnOnce(&Metrics)) { + pub fn report(&self, do_this: impl FnOnce(&M)) { if let Some(metrics) = self.0.as_ref() { - do_this(metrics); + do_this(&**metrics); } } } -/// Transaction pool Prometheus metrics. -pub struct Metrics { - pub submitted_transactions: Counter, - pub validations_invalid: Counter, - pub block_transactions_pruned: Counter, - pub block_transactions_resubmitted: Counter, -} - -impl Metrics { - pub fn register(registry: &Registry) -> Result { - Ok(Self { - submitted_transactions: register( - Counter::new( - "substrate_sub_txpool_submitted_transactions", - "Total number of transactions submitted", - )?, - registry, - )?, - validations_invalid: register( - Counter::new( - "substrate_sub_txpool_validations_invalid", - "Total number of transactions that were removed from the pool as invalid", - )?, - registry, - )?, - block_transactions_pruned: register( - Counter::new( - "substrate_sub_txpool_block_transactions_pruned", - "Total number of transactions that was requested to be pruned by block events", - )?, - registry, - )?, - block_transactions_resubmitted: register( - Counter::new( - "substrate_sub_txpool_block_transactions_resubmitted", - "Total number of transactions that was requested to be resubmitted by block events", - )?, - registry, - )?, - }) - } -} - /// Transaction pool api Prometheus metrics. pub struct ApiMetrics { pub validations_scheduled: Counter, diff --git a/substrate/client/transaction-pool/src/common/mod.rs b/substrate/client/transaction-pool/src/common/mod.rs new file mode 100644 index 000000000000..fb280e8780ad --- /dev/null +++ b/substrate/client/transaction-pool/src/common/mod.rs @@ -0,0 +1,48 @@ +// 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 . + +//! Common components re-used across different txpool implementations. + +pub(crate) mod api; +pub(crate) mod enactment_state; +pub(crate) mod error; +pub(crate) mod log_xt; +pub(crate) mod metrics; +#[cfg(test)] +pub(crate) mod tests; + +use futures::StreamExt; +use std::sync::Arc; + +/// Inform the transaction pool about imported and finalized blocks. +pub async fn notification_future(client: Arc, txpool: Arc) +where + Block: sp_runtime::traits::Block, + Client: sc_client_api::BlockchainEvents, + Pool: sc_transaction_pool_api::MaintainedTransactionPool, +{ + let import_stream = client + .import_notification_stream() + .filter_map(|n| futures::future::ready(n.try_into().ok())) + .fuse(); + let finality_stream = client.finality_notification_stream().map(Into::into).fuse(); + + futures::stream::select(import_stream, finality_stream) + .for_each(|evt| txpool.maintain(evt)) + .await +} diff --git a/substrate/client/transaction-pool/src/tests.rs b/substrate/client/transaction-pool/src/common/tests.rs similarity index 94% rename from substrate/client/transaction-pool/src/tests.rs rename to substrate/client/transaction-pool/src/common/tests.rs index 325add3fb1c5..1cbabf8b5fde 100644 --- a/substrate/client/transaction-pool/src/tests.rs +++ b/substrate/client/transaction-pool/src/common/tests.rs @@ -18,11 +18,11 @@ //! Testing related primitives for internal usage in this crate. -use crate::graph::{BlockHash, ChainApi, ExtrinsicFor, NumberFor, Pool}; +use crate::graph::{BlockHash, ChainApi, ExtrinsicFor, NumberFor, Pool, RawExtrinsicFor}; use codec::Encode; use parking_lot::Mutex; use sc_transaction_pool_api::error; -use sp_blockchain::TreeRoute; +use sp_blockchain::{HashAndNumber, TreeRoute}; use sp_runtime::{ generic::BlockId, traits::{Block as BlockT, Hash}, @@ -58,6 +58,10 @@ impl TestApi { pub fn expect_hash_from_number(&self, n: BlockNumber) -> H256 { self.block_id_to_hash(&BlockId::Number(n)).unwrap().unwrap() } + + pub fn expect_hash_and_number(&self, n: BlockNumber) -> HashAndNumber { + HashAndNumber { hash: self.expect_hash_from_number(n), number: n } + } } impl ChainApi for TestApi { @@ -73,6 +77,7 @@ impl ChainApi for TestApi { _source: TransactionSource, uxt: ExtrinsicFor, ) -> Self::ValidationFuture { + let uxt = (*uxt).clone(); self.validation_requests.lock().push(uxt.clone()); let hash = self.hash_and_length(&uxt).0; let block_number = self.block_id_to_number(&BlockId::Hash(at)).unwrap().unwrap(); @@ -176,7 +181,7 @@ impl ChainApi for TestApi { } /// Hash the extrinsic. - fn hash_and_length(&self, uxt: &ExtrinsicFor) -> (BlockHash, usize) { + fn hash_and_length(&self, uxt: &RawExtrinsicFor) -> (BlockHash, usize) { let encoded = uxt.encode(); let len = encoded.len(); (Hashing::hash(&encoded), len) 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 new file mode 100644 index 000000000000..2dd5836c570f --- /dev/null +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/dropped_watcher.rs @@ -0,0 +1,533 @@ +// 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 . + +//! Multi-view pool dropped events listener provides means to combine streams from multiple pool +//! views into a single event stream. It allows management of dropped transaction events, adding new +//! views, and removing views as needed, ensuring that transactions which are no longer referenced +//! by any view are detected and properly notified. + +use crate::{ + common::log_xt::log_xt_trace, + fork_aware_txpool::stream_map_util::next_event, + graph::{BlockHash, ChainApi, ExtrinsicHash}, + LOG_TARGET, +}; +use futures::stream::StreamExt; +use log::{debug, trace}; +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}, + fmt::{self, Debug, Formatter}, + pin::Pin, +}; +use tokio_stream::StreamMap; + +/// Dropped-logic related event from the single view. +pub type ViewStreamEvent = crate::graph::DroppedByLimitsEvent, BlockHash>; + +/// Dropped-logic stream of events coming from the single view. +type ViewStream = Pin> + Send>>; + +/// Stream of extrinsic hashes that were dropped by the views and have no references by existing +/// views. +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 +/// [`MultiViewDropWatcherContext`]. +type Controller = mpsc::TracingUnboundedSender; + +/// A type alias for a receiver used as the commands receiver in the +/// [`MultiViewDropWatcherContext`]. +type CommandReceiver = mpsc::TracingUnboundedReceiver; + +/// Commands to control the instance of dropped transactions stream [`StreamOfDropped`]. +enum Command +where + C: ChainApi, +{ + /// Adds a new stream of dropped-related events originating in a view with a specific block + /// hash + AddView(BlockHash, ViewStream), + /// Removes an existing view's stream associated with a specific block hash. + RemoveView(BlockHash), + /// Adds initial views for given extrinsics hashes. + /// + /// This message should be sent when the external submission of a transaction occures. It + /// provides the list of initial views for given extrinsics hashes. + /// The dropped notification is not sent if it comes from the initial views. It allows to keep + /// transaction in the mempool, even if all the views are full at the time of submitting + /// transaction to the pool. + AddInitialViews(Vec>, BlockHash), + /// Removes all initial views for given extrinsic hashes. + /// + /// Intended to ba called on finalization. + RemoveFinalizedTxs(Vec>), +} + +impl Debug for Command +where + C: ChainApi, +{ + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + match self { + Command::AddView(..) => write!(f, "AddView"), + Command::RemoveView(..) => write!(f, "RemoveView"), + Command::AddInitialViews(..) => write!(f, "AddInitialViews"), + Command::RemoveFinalizedTxs(..) => write!(f, "RemoveFinalizedTxs"), + } + } +} + +/// Manages the state and logic for handling events related to dropped transactions across multiple +/// views. +/// +/// 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 +where + C: 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>, + /// 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>, + + /// For each transaction hash we keep the set of hashes representing the views that see this + /// transaction as ready or future. + /// + /// Once transaction is dropped, dropping view is removed from the set. + transaction_states: HashMap, HashSet>>, + + /// The list of initial view for every extrinsic. + /// + /// Dropped notifications from initial views will be silenced. This allows to accept the + /// transaction into the mempool, even if all the views are full at the time of submitting new + /// transaction. + initial_views: HashMap, HashSet>>, +} + +impl MultiViewDropWatcherContext +where + C: ChainApi + 'static, + <::Block as BlockT>::Hash: Unpin, +{ + /// Processes a `ViewStreamEvent` from a specific view and updates the internal state + /// accordingly. + /// + /// If the event indicates that a transaction has been dropped and is no longer referenced by + /// any active views, the transaction hash is returned. Otherwise `None` is returned. + fn handle_event( + &mut self, + block_hash: BlockHash, + event: ViewStreamEvent, + ) -> Option> { + trace!( + target: LOG_TARGET, + "dropped_watcher: handle_event: event:{:?} views:{:?}, ", + event, + 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::Dropped | TransactionStatus::Usurped(_) => { + if let Entry::Occupied(mut views_keeping_tx_valid) = + self.transaction_states.entry(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 self + .initial_views + .get(&tx_hash) + .map(|list| !list.contains(&block_hash)) + .unwrap_or(true) + .then(|| { + debug!("[{:?}] dropped_watcher: removing tx", tx_hash); + tx_hash + }) + } + } else { + debug!("[{:?}] dropped_watcher: removing (non-tracked) tx", tx_hash); + return Some(tx_hash) + } + }, + _ => {}, + }; + None + } + + /// Creates a new `StreamOfDropped` and its associated event stream controller. + /// + /// This method initializes the internal structures and unfolds the stream of dropped + /// transactions. Returns a tuple containing this stream and the controller for managing + /// this stream. + fn event_stream() -> (StreamOfDropped, Controller>) { + //note: 64 allows to avoid warning messages during execution of unit tests. + const CHANNEL_SIZE: usize = 64; + let (sender, command_receiver) = sc_utils::mpsc::tracing_unbounded::>( + "tx-pool-dropped-watcher-cmd-stream", + CHANNEL_SIZE, + ); + + let ctx = Self { + stream_map: StreamMap::new(), + command_receiver, + transaction_states: Default::default(), + initial_views: Default::default(), + }; + + let stream_map = futures::stream::unfold(ctx, |mut ctx| async move { + loop { + tokio::select! { + biased; + cmd = ctx.command_receiver.next() => { + match cmd? { + Command::AddView(key,stream) => { + trace!(target: LOG_TARGET,"dropped_watcher: Command::AddView {key:?} views:{:?}",ctx.stream_map.keys().collect::>()); + ctx.stream_map.insert(key,stream); + }, + Command::RemoveView(key) => { + trace!(target: LOG_TARGET,"dropped_watcher: Command::RemoveView {key:?} views:{:?}",ctx.stream_map.keys().collect::>()); + ctx.stream_map.remove(&key); + }, + Command::AddInitialViews(xts,block_hash) => { + log_xt_trace!(target: LOG_TARGET, xts.clone(), "[{:?}] dropped_watcher: xt initial view added {block_hash:?}"); + xts.into_iter().for_each(|xt| { + ctx.initial_views.entry(xt).or_default().insert(block_hash); + }); + }, + Command::RemoveFinalizedTxs(xts) => { + log_xt_trace!(target: LOG_TARGET, xts.clone(), "[{:?}] dropped_watcher: finalized xt removed"); + xts.iter().for_each(|xt| { + ctx.initial_views.remove(xt); + ctx.transaction_states.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)); + } + } + } + } + }) + .boxed(); + + (stream_map, sender) + } +} + +/// The controller for manipulating the state of the [`StreamOfDropped`]. +/// +/// This struct provides methods to add and remove streams associated with views to and from the +/// stream. +pub struct MultiViewDroppedWatcherController { + /// A controller allowing to update the state of the associated [`StreamOfDropped`]. + controller: Controller>, +} + +impl Clone for MultiViewDroppedWatcherController { + fn clone(&self) -> Self { + Self { controller: self.controller.clone() } + } +} + +impl MultiViewDroppedWatcherController +where + C: 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(); + (Self { controller: ctrl }, stream_map.boxed()) + } + + /// Notifies the [`StreamOfDropped`] that new view was created. + 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}"); + }); + } + + /// Notifies the [`StreamOfDropped`] that the view was destroyed and shall be removed the + /// stream map. + 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}"); + }); + } + + /// Adds the initial view for the given transactions hashes. + /// + /// This message should be called when the external submission of a transaction occures. It + /// provides the list of initial views for given extrinsics hashes. + /// + /// The dropped notification is not sent if it comes from the initial views. It allows to keep + /// transaction in the mempool, even if all the views are full at the time of submitting + /// transaction to the pool. + pub fn add_initial_views( + &self, + xts: impl IntoIterator> + Clone, + block_hash: BlockHash, + ) { + let _ = self + .controller + .unbounded_send(Command::AddInitialViews(xts.into_iter().collect(), block_hash)) + .map_err(|e| { + trace!(target: LOG_TARGET, "dropped_watcher: add_initial_views_ send message failed: {e}"); + }); + } + + /// Removes all initial views for finalized transactions. + pub fn remove_finalized_txs(&self, xts: impl IntoIterator> + Clone) { + let _ = self + .controller + .unbounded_send(Command::RemoveFinalizedTxs(xts.into_iter().collect())) + .map_err(|e| { + trace!(target: LOG_TARGET, "dropped_watcher: remove_initial_views send message failed: {e}"); + }); + } +} + +#[cfg(test)] +mod dropped_watcher_tests { + use super::*; + use crate::common::tests::TestApi; + use futures::{stream::pending, FutureExt, StreamExt}; + use sp_core::H256; + + type MultiViewDroppedWatcher = super::MultiViewDroppedWatcherController; + + #[tokio::test] + async fn test01() { + sp_tracing::try_init_simple(); + let (watcher, output_stream) = MultiViewDroppedWatcher::new(); + + let block_hash = H256::repeat_byte(0x01); + let tx_hash = H256::repeat_byte(0x0a); + + let view_stream = futures::stream::iter(vec![ + (tx_hash, TransactionStatus::Ready), + (tx_hash, TransactionStatus::Dropped), + ]) + .boxed(); + + 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]); + } + + #[tokio::test] + async fn test02() { + sp_tracing::try_init_simple(); + let (watcher, mut output_stream) = MultiViewDroppedWatcher::new(); + + let block_hash0 = H256::repeat_byte(0x01); + let block_hash1 = H256::repeat_byte(0x02); + let tx_hash = H256::repeat_byte(0x0a); + + let view_stream0 = futures::stream::iter(vec![(tx_hash, TransactionStatus::Future)]) + .chain(pending()) + .boxed(); + let view_stream1 = futures::stream::iter(vec![ + (tx_hash, TransactionStatus::Ready), + (tx_hash, TransactionStatus::Dropped), + ]) + .boxed(); + + watcher.add_view(block_hash0, view_stream0); + + assert!(output_stream.next().now_or_never().is_none()); + watcher.add_view(block_hash1, view_stream1); + assert!(output_stream.next().now_or_never().is_none()); + } + + #[tokio::test] + async fn test03() { + sp_tracing::try_init_simple(); + let (watcher, output_stream) = MultiViewDroppedWatcher::new(); + + let block_hash0 = H256::repeat_byte(0x01); + let block_hash1 = H256::repeat_byte(0x02); + let tx_hash0 = H256::repeat_byte(0x0a); + let tx_hash1 = H256::repeat_byte(0x0b); + + let view_stream0 = futures::stream::iter(vec![(tx_hash0, TransactionStatus::Future)]) + .chain(pending()) + .boxed(); + let view_stream1 = futures::stream::iter(vec![ + (tx_hash1, TransactionStatus::Ready), + (tx_hash1, TransactionStatus::Dropped), + ]) + .boxed(); + + 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]); + } + + #[tokio::test] + async fn test04() { + sp_tracing::try_init_simple(); + let (watcher, mut output_stream) = MultiViewDroppedWatcher::new(); + + let block_hash0 = H256::repeat_byte(0x01); + let block_hash1 = H256::repeat_byte(0x02); + let tx_hash = H256::repeat_byte(0x0b); + + let view_stream0 = futures::stream::iter(vec![ + (tx_hash, TransactionStatus::Future), + (tx_hash, TransactionStatus::InBlock((block_hash1, 0))), + ]) + .boxed(); + let view_stream1 = futures::stream::iter(vec![ + (tx_hash, TransactionStatus::Ready), + (tx_hash, TransactionStatus::Dropped), + ]) + .boxed(); + + watcher.add_view(block_hash0, view_stream0); + assert!(output_stream.next().now_or_never().is_none()); + + 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]); + } + + #[tokio::test] + async fn test05() { + sp_tracing::try_init_simple(); + let (watcher, mut output_stream) = MultiViewDroppedWatcher::new(); + assert!(output_stream.next().now_or_never().is_none()); + + let block_hash0 = H256::repeat_byte(0x01); + let block_hash1 = H256::repeat_byte(0x02); + let tx_hash = H256::repeat_byte(0x0b); + + let view_stream0 = futures::stream::iter(vec![ + (tx_hash, TransactionStatus::Future), + (tx_hash, TransactionStatus::InBlock((block_hash1, 0))), + ]) + .boxed(); + watcher.add_view(block_hash0, view_stream0); + assert!(output_stream.next().now_or_never().is_none()); + + let view_stream1 = futures::stream::iter(vec![ + (tx_hash, TransactionStatus::Ready), + (tx_hash, TransactionStatus::InBlock((block_hash0, 0))), + ]) + .boxed(); + + watcher.add_view(block_hash1, view_stream1); + assert!(output_stream.next().now_or_never().is_none()); + assert!(output_stream.next().now_or_never().is_none()); + assert!(output_stream.next().now_or_never().is_none()); + assert!(output_stream.next().now_or_never().is_none()); + assert!(output_stream.next().now_or_never().is_none()); + + let tx_hash = H256::repeat_byte(0x0c); + let view_stream2 = futures::stream::iter(vec![ + (tx_hash, TransactionStatus::Future), + (tx_hash, TransactionStatus::Dropped), + ]) + .boxed(); + 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]); + } + + #[tokio::test] + async fn test06() { + sp_tracing::try_init_simple(); + let (watcher, mut output_stream) = MultiViewDroppedWatcher::new(); + assert!(output_stream.next().now_or_never().is_none()); + + let block_hash0 = H256::repeat_byte(0x01); + let block_hash1 = H256::repeat_byte(0x02); + let tx_hash = H256::repeat_byte(0x0b); + + let view_stream0 = futures::stream::iter(vec![ + (tx_hash, TransactionStatus::Future), + (tx_hash, TransactionStatus::InBlock((block_hash1, 0))), + ]) + .boxed(); + watcher.add_view(block_hash0, view_stream0); + assert!(output_stream.next().now_or_never().is_none()); + + let view_stream1 = futures::stream::iter(vec![ + (tx_hash, TransactionStatus::Ready), + (tx_hash, TransactionStatus::Dropped), + ]) + .boxed(); + + watcher.add_view(block_hash1, view_stream1); + watcher.add_initial_views(vec![tx_hash], block_hash1); + assert!(output_stream.next().now_or_never().is_none()); + } + + #[tokio::test] + async fn test07() { + sp_tracing::try_init_simple(); + let (watcher, mut output_stream) = MultiViewDroppedWatcher::new(); + assert!(output_stream.next().now_or_never().is_none()); + + let block_hash0 = H256::repeat_byte(0x01); + let block_hash1 = H256::repeat_byte(0x02); + let tx_hash = H256::repeat_byte(0x0b); + + let view_stream0 = futures::stream::iter(vec![ + (tx_hash, TransactionStatus::Future), + (tx_hash, TransactionStatus::InBlock((block_hash1, 0))), + ]) + .boxed(); + watcher.add_view(block_hash0, view_stream0); + watcher.add_initial_views(vec![tx_hash], block_hash0); + assert!(output_stream.next().now_or_never().is_none()); + + let view_stream1 = futures::stream::iter(vec![ + (tx_hash, TransactionStatus::Ready), + (tx_hash, TransactionStatus::Dropped), + ]) + .boxed(); + 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]); + } +} 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 new file mode 100644 index 000000000000..404225167e57 --- /dev/null +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/fork_aware_txpool.rs @@ -0,0 +1,1563 @@ +// 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 . + +//! Substrate fork-aware transaction pool implementation. + +use super::{ + dropped_watcher::{MultiViewDroppedWatcherController, StreamOfDropped}, + import_notification_sink::MultiViewImportNotificationSink, + metrics::MetricsLink as PrometheusMetrics, + multi_view_listener::MultiViewListener, + tx_mem_pool::{TxInMemPool, TxMemPool, TXMEMPOOL_TRANSACTION_LIMIT_MULTIPLIER}, + view::View, + view_store::ViewStore, +}; +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}, + PolledIterator, ReadyIteratorFor, LOG_TARGET, +}; +use async_trait::async_trait; +use futures::{ + channel::oneshot, + future::{self}, + prelude::*, + FutureExt, +}; +use parking_lot::Mutex; +use prometheus_endpoint::Registry as PrometheusRegistry; +use sc_transaction_pool_api::{ + error::{Error, IntoPoolError}, + ChainEvent, ImportNotificationStream, MaintainedTransactionPool, PoolFuture, PoolStatus, + TransactionFor, TransactionPool, TransactionSource, TransactionStatusStreamFor, TxHash, +}; +use sp_blockchain::{HashAndNumber, TreeRoute}; +use sp_core::traits::SpawnEssentialNamed; +use sp_runtime::{ + generic::BlockId, + traits::{Block as BlockT, Extrinsic, NumberFor}, +}; +use std::{ + collections::{HashMap, HashSet}, + pin::Pin, + sync::Arc, + time::Instant, +}; +use tokio::select; + +/// Fork aware transaction pool task, that needs to be polled. +pub type ForkAwareTxPoolTask = Pin + Send>>; + +/// A structure that maintains a collection of pollers associated with specific block hashes +/// (views). +struct ReadyPoll +where + Block: BlockT, +{ + pollers: HashMap>>, +} + +impl ReadyPoll +where + Block: BlockT, +{ + /// Creates a new `ReadyPoll` instance with an empty collection of pollers. + fn new() -> Self { + Self { pollers: Default::default() } + } + + /// Adds a new poller for a specific block hash and returns the `Receiver` end of the created + /// oneshot channel which will be used to deliver polled result. + fn add(&mut self, at: ::Hash) -> oneshot::Receiver { + let (s, r) = oneshot::channel(); + self.pollers.entry(at).or_default().push(s); + r + } + + /// Triggers all pollers associated with a specific block by sending the polled result through + /// each oneshot channel. + /// + /// `ready_iterator` is a closure that generates the result data to be sent to the pollers. + fn trigger(&mut self, at: Block::Hash, ready_iterator: impl Fn() -> T) { + log::trace!(target: LOG_TARGET, "fatp::trigger {at:?} pending keys: {:?}", self.pollers.keys()); + let Some(pollers) = self.pollers.remove(&at) else { return }; + pollers.into_iter().for_each(|p| { + log::debug!(target: LOG_TARGET, "trigger ready signal at block {}", at); + let _ = p.send(ready_iterator()); + }); + } + + /// Removes pollers that have their oneshot channels cancelled. + fn remove_cancelled(&mut self) { + self.pollers.retain(|_, v| v.iter().any(|sender| !sender.is_canceled())); + } +} + +/// The fork-aware transaction pool. +/// +/// It keeps track of every fork and provides the set of transactions that is valid for every fork. +pub struct ForkAwareTxPool +where + Block: BlockT, + ChainApi: graph::ChainApi + 'static, +{ + /// The reference to the `ChainApi` provided by client/backend. + api: Arc, + + /// Intermediate buffer for the incoming transaction. + mempool: Arc>, + + /// The store for all the views. + view_store: Arc>, + + /// Utility for managing pollers of `ready_at` future. + ready_poll: Arc, Block>>>, + + /// Prometheus's metrics endpoint. + metrics: PrometheusMetrics, + + /// Util tracking best and finalized block. + enactment_state: Arc>>, + + /// The channel allowing to send revalidation jobs to the background thread. + revalidation_queue: Arc>, + + /// Util providing an aggregated stream of transactions that were imported to ready queue in + /// any view. + import_notification_sink: MultiViewImportNotificationSink>, + + /// Externally provided pool options. + options: Options, + + /// Is node the validator. + is_validator: IsValidator, +} + +impl ForkAwareTxPool +where + Block: BlockT, + ChainApi: graph::ChainApi + 'static, + ::Hash: Unpin, +{ + /// Create new fork aware transaction pool with provided shared instance of `ChainApi` intended + /// for tests. + pub fn new_test( + pool_api: Arc, + best_block_hash: Block::Hash, + finalized_hash: Block::Hash, + ) -> (Self, ForkAwareTxPoolTask) { + Self::new_test_with_limits( + pool_api, + best_block_hash, + finalized_hash, + Options::default().ready, + Options::default().future, + usize::MAX, + ) + } + + /// Create new fork aware transaction pool with given limits and with provided shared instance + /// of `ChainApi` intended for tests. + pub fn new_test_with_limits( + pool_api: Arc, + best_block_hash: Block::Hash, + finalized_hash: Block::Hash, + ready_limits: crate::PoolLimit, + future_limits: crate::PoolLimit, + mempool_max_transactions_count: usize, + ) -> (Self, ForkAwareTxPoolTask) { + let listener = Arc::from(MultiViewListener::new()); + let (import_notification_sink, import_notification_sink_task) = + MultiViewImportNotificationSink::new_with_worker(); + + let mempool = Arc::from(TxMemPool::new( + pool_api.clone(), + listener.clone(), + Default::default(), + mempool_max_transactions_count, + )); + + let (dropped_stream_controller, dropped_stream) = + MultiViewDroppedWatcherController::::new(); + let dropped_monitor_task = Self::dropped_monitor_task( + dropped_stream, + mempool.clone(), + import_notification_sink.clone(), + ); + + let combined_tasks = async move { + tokio::select! { + _ = import_notification_sink_task => {}, + _ = dropped_monitor_task => {} + } + } + .boxed(); + + let options = Options { ready: ready_limits, future: future_limits, ..Default::default() }; + + ( + Self { + mempool, + api: pool_api.clone(), + view_store: Arc::new(ViewStore::new(pool_api, listener, dropped_stream_controller)), + ready_poll: Arc::from(Mutex::from(ReadyPoll::new())), + enactment_state: Arc::new(Mutex::new(EnactmentState::new( + best_block_hash, + finalized_hash, + ))), + revalidation_queue: Arc::from(revalidation_worker::RevalidationQueue::new()), + import_notification_sink, + options, + is_validator: false.into(), + metrics: Default::default(), + }, + combined_tasks, + ) + } + + /// Monitors the stream of dropped transactions and removes them from the mempool. + /// + /// 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. + async fn dropped_monitor_task( + mut dropped_stream: StreamOfDropped, + mempool: Arc>, + import_notification_sink: MultiViewImportNotificationSink< + Block::Hash, + ExtrinsicHash, + >, + ) { + loop { + let Some(dropped) = dropped_stream.next().await else { + 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]); + } + } + + /// Creates new fork aware transaction pool with the background revalidation worker. + /// + /// The txpool essential tasks (including a revalidation worker) are spawned using provided + /// spawner. + pub fn new_with_background_worker( + options: Options, + is_validator: IsValidator, + pool_api: Arc, + prometheus: Option<&PrometheusRegistry>, + spawner: impl SpawnEssentialNamed, + best_block_hash: Block::Hash, + finalized_hash: Block::Hash, + ) -> Self { + let metrics = PrometheusMetrics::new(prometheus); + let listener = Arc::from(MultiViewListener::new()); + let (revalidation_queue, revalidation_task) = + revalidation_worker::RevalidationQueue::new_with_worker(); + + let (import_notification_sink, import_notification_sink_task) = + MultiViewImportNotificationSink::new_with_worker(); + + let mempool = Arc::from(TxMemPool::new( + pool_api.clone(), + listener.clone(), + metrics.clone(), + TXMEMPOOL_TRANSACTION_LIMIT_MULTIPLIER * (options.ready.count + options.future.count), + )); + + let (dropped_stream_controller, dropped_stream) = + MultiViewDroppedWatcherController::::new(); + let dropped_monitor_task = Self::dropped_monitor_task( + dropped_stream, + mempool.clone(), + import_notification_sink.clone(), + ); + + let combined_tasks = async move { + tokio::select! { + _ = revalidation_task => {}, + _ = import_notification_sink_task => {}, + _ = dropped_monitor_task => {} + } + } + .boxed(); + spawner.spawn_essential("txpool-background", Some("transaction-pool"), combined_tasks); + + Self { + mempool, + api: pool_api.clone(), + view_store: Arc::new(ViewStore::new(pool_api, listener, dropped_stream_controller)), + ready_poll: Arc::from(Mutex::from(ReadyPoll::new())), + enactment_state: Arc::new(Mutex::new(EnactmentState::new( + best_block_hash, + finalized_hash, + ))), + revalidation_queue: Arc::from(revalidation_queue), + import_notification_sink, + options, + metrics, + is_validator, + } + } + + /// Get access to the underlying api + pub fn api(&self) -> &ChainApi { + &self.api + } + + /// Provides a status for all views at the tips of the forks. + pub fn status_all(&self) -> HashMap { + self.view_store.status() + } + + /// Provides a number of views at the tips of the forks. + pub fn active_views_count(&self) -> usize { + self.view_store.active_views.read().len() + } + + /// Provides a number of views at the tips of the forks. + pub fn inactive_views_count(&self) -> usize { + self.view_store.inactive_views.read().len() + } + + /// Provides internal views statistics. + /// + /// Provides block number, count of ready, count of future transactions for every view. It is + /// suitable for printing log information. + fn views_stats(&self) -> Vec<(NumberFor, usize, usize)> { + self.view_store + .active_views + .read() + .iter() + .map(|v| (v.1.at.number, v.1.status().ready, v.1.status().future)) + .collect() + } + + /// Checks if there is a view at the tip of the fork with given hash. + pub fn has_view(&self, hash: &Block::Hash) -> bool { + self.view_store.active_views.read().contains_key(hash) + } + + /// Returns a number of unwatched and watched transactions in internal mempool. + /// + /// Intended for use in unit tests. + pub fn mempool_len(&self) -> (usize, usize) { + self.mempool.unwatched_and_watched_count() + } + + /// Returns a best-effort set of ready transactions for a given block, without executing full + /// maintain process. + /// + /// The method attempts to build a temporary view and create an iterator of ready transactions + /// for a specific `at` hash. If a valid view is found, it collects and prunes + /// transactions already included in the blocks and returns the valid set. + /// + /// Pruning is just rebuilding the underlying transactions graph, no validations are executed, + /// so this process shall be fast. + pub fn ready_at_light(&self, at: Block::Hash) -> PolledIterator { + let start = Instant::now(); + let api = self.api.clone(); + log::trace!(target: LOG_TARGET, "fatp::ready_at_light {:?}", at); + + let Ok(block_number) = self.api.resolve_block_number(at) else { + let empty: ReadyIteratorFor = Box::new(std::iter::empty()); + return Box::pin(async { empty }) + }; + + let best_result = { + api.tree_route(self.enactment_state.lock().recent_finalized_block(), at).map( + |tree_route| { + if let Some((index, view)) = + tree_route.enacted().iter().enumerate().rev().skip(1).find_map(|(i, b)| { + self.view_store.get_view_at(b.hash, true).map(|(view, _)| (i, view)) + }) { + let e = tree_route.enacted()[index..].to_vec(); + (TreeRoute::new(e, 0).ok(), Some(view)) + } else { + (None, None) + } + }, + ) + }; + + Box::pin(async move { + if let Ok((Some(best_tree_route), Some(best_view))) = best_result { + let tmp_view: View = View::new_from_other( + &best_view, + &HashAndNumber { hash: at, number: block_number }, + ); + + let mut all_extrinsics = vec![]; + + for h in best_tree_route.enacted() { + let extrinsics = api + .block_body(h.hash) + .await + .unwrap_or_else(|e| { + log::warn!(target: LOG_TARGET, "Compute ready light transactions: error request: {}", e); + None + }) + .unwrap_or_default() + .into_iter() + .map(|t| api.hash_and_length(&t).0); + all_extrinsics.extend(extrinsics); + } + + let before_count = tmp_view.pool.validated_pool().status().ready; + let tags = tmp_view + .pool + .validated_pool() + .extrinsics_tags(&all_extrinsics) + .into_iter() + .flatten() + .flatten() + .collect::>(); + let _ = tmp_view.pool.validated_pool().prune_tags(tags); + + let after_count = tmp_view.pool.validated_pool().status().ready; + log::debug!(target: LOG_TARGET, + "fatp::ready_at_light {} from {} before: {} to be removed: {} after: {} took:{:?}", + at, + best_view.at.hash, + before_count, + all_extrinsics.len(), + after_count, + start.elapsed() + ); + Box::new(tmp_view.pool.validated_pool().ready()) + } else { + let empty: ReadyIteratorFor = Box::new(std::iter::empty()); + log::debug!(target: LOG_TARGET, "fatp::ready_at_light {} -> empty, took:{:?}", at, start.elapsed()); + empty + } + }) + } + + /// Waits for the set of ready transactions for a given block up to a specified timeout. + /// + /// This method combines two futures: + /// - The `ready_at` future, which waits for the ready transactions resulting from the full + /// maintenance process to be available. + /// - The `ready_at_light` future, used as a fallback if the timeout expires before `ready_at` + /// completes. This provides a best-effort, ready set of transactions as a result light + /// maintain. + /// + /// Returns a future resolving to a ready iterator of transactions. + fn ready_at_with_timeout_internal( + &self, + at: Block::Hash, + timeout: std::time::Duration, + ) -> PolledIterator { + log::debug!(target: LOG_TARGET, "fatp::ready_at_with_timeout at {:?} allowed delay: {:?}", at, timeout); + + let timeout = futures_timer::Delay::new(timeout); + let (view_already_exists, ready_at) = self.ready_at_internal(at); + + if view_already_exists { + return ready_at; + } + + let maybe_ready = async move { + select! { + ready = ready_at => Some(ready), + _ = timeout => { + log::warn!(target: LOG_TARGET, + "Timeout fired waiting for transaction pool at block: ({:?}). \ + Proceeding with production.", + at, + ); + None + } + } + }; + + let fall_back_ready = self.ready_at_light(at); + Box::pin(async { + let (maybe_ready, fall_back_ready) = + futures::future::join(maybe_ready.boxed(), fall_back_ready.boxed()).await; + maybe_ready.unwrap_or(fall_back_ready) + }) + } + + fn ready_at_internal(&self, at: Block::Hash) -> (bool, PolledIterator) { + let mut ready_poll = self.ready_poll.lock(); + + if let Some((view, inactive)) = self.view_store.get_view_at(at, true) { + log::debug!(target: LOG_TARGET, "fatp::ready_at {at:?} (inactive:{inactive:?})"); + let iterator: ReadyIteratorFor = Box::new(view.pool.validated_pool().ready()); + return (true, async move { iterator }.boxed()); + } + + let pending = ready_poll + .add(at) + .map(|received| { + received.unwrap_or_else(|e| { + log::warn!(target: LOG_TARGET, "Error receiving ready-set iterator: {:?}", e); + Box::new(std::iter::empty()) + }) + }) + .boxed(); + log::debug!(target: LOG_TARGET, + "fatp::ready_at {at:?} pending keys: {:?}", + ready_poll.pollers.keys() + ); + (false, pending) + } +} + +/// Converts the input view-to-statuses map into the output vector of statuses. +/// +/// The result of importing a bunch of transactions into a single view is the vector of statuses. +/// Every item represents a status for single transaction. The input is the map that associates +/// hash-views with vectors indicating the statuses of transactions imports. +/// +/// Import to multiple views result in two-dimensional array of statuses, which is provided as +/// input map. +/// +/// This function converts the map into the vec of results, according to the following rules: +/// - for given transaction if at least one status is success, then output vector contains success, +/// - if given transaction status is error for every view, then output vector contains error. +/// +/// The results for transactions are in the same order for every view. An output vector preserves +/// this order. +/// +/// ```skip +/// in: +/// view | xt0 status | xt1 status | xt2 status +/// h1 -> [ Ok(xth0), Ok(xth1), Err ] +/// h2 -> [ Ok(xth0), Err, Err ] +/// h3 -> [ Ok(xth0), Ok(xth1), Err ] +/// +/// out: +/// [ Ok(xth0), Ok(xth1), Err ] +/// ``` +fn reduce_multiview_result(input: HashMap>>) -> Vec> { + let mut values = input.values(); + let Some(first) = values.next() else { + return Default::default(); + }; + let length = first.len(); + debug_assert!(values.all(|x| length == x.len())); + + input + .into_values() + .reduce(|mut agg_results, results| { + agg_results.iter_mut().zip(results.into_iter()).for_each(|(agg_r, r)| { + if agg_r.is_err() { + *agg_r = r; + } + }); + agg_results + }) + .unwrap_or_default() +} + +impl TransactionPool for ForkAwareTxPool +where + Block: BlockT, + ChainApi: 'static + graph::ChainApi, + ::Hash: Unpin, +{ + type Block = ChainApi::Block; + type Hash = ExtrinsicHash; + type InPoolTransaction = Transaction, ExtrinsicFor>; + type Error = ChainApi::Error; + + /// Submits multiple transactions and returns a future resolving to the submission results. + /// + /// Actual transactions submission process is delegated to the `ViewStore` internal instance. + /// + /// The internal limits of the pool are checked. The results of submissions to individual views + /// are reduced to single result. Refer to `reduce_multiview_result` for more details. + fn submit_at( + &self, + _: ::Hash, + source: TransactionSource, + xts: Vec>, + ) -> PoolFuture, Self::Error>>, Self::Error> { + let view_store = self.view_store.clone(); + log::debug!(target: LOG_TARGET, "fatp::submit_at count:{} views:{}", xts.len(), self.active_views_count()); + log_xt_trace!(target: LOG_TARGET, xts.iter().map(|xt| self.tx_hash(xt)), "[{:?}] fatp::submit_at"); + let xts = xts.into_iter().map(Arc::from).collect::>(); + let mempool_result = self.mempool.extend_unwatched(source, xts.clone()); + + if view_store.is_empty() { + return future::ready(Ok(mempool_result)).boxed() + } + + let (hashes, to_be_submitted): (Vec>, Vec>) = + mempool_result + .iter() + .zip(xts) + .filter_map(|(result, xt)| result.as_ref().ok().map(|xt_hash| (xt_hash, xt))) + .unzip(); + + self.metrics + .report(|metrics| metrics.submitted_transactions.inc_by(to_be_submitted.len() as _)); + + let mempool = self.mempool.clone(); + async move { + let results_map = view_store.submit(source, to_be_submitted.into_iter(), hashes).await; + let mut submission_results = reduce_multiview_result(results_map).into_iter(); + + Ok(mempool_result + .into_iter() + .map(|result| { + result.and_then(|xt_hash| { + let result = submission_results + .next() + .expect("The number of Ok results in mempool is exactly the same as the size of to-views-submission result. qed."); + result.or_else(|error| { + let error = error.into_pool_error(); + match error { + Ok( + // The transaction is still in mempool it may get included into the view for the next block. + Error::ImmediatelyDropped + ) => Ok(xt_hash), + Ok(e) => { + mempool.remove(xt_hash); + Err(e.into()) + }, + Err(e) => Err(e), + } + }) + }) + }) + .collect::>()) + } + .boxed() + } + + /// Submits a single transaction and returns a future resolving to the submission results. + /// + /// Actual transaction submission process is delegated to the `submit_at` function. + fn submit_one( + &self, + _at: ::Hash, + source: TransactionSource, + xt: TransactionFor, + ) -> PoolFuture, Self::Error> { + log::trace!(target: LOG_TARGET, "[{:?}] fatp::submit_one views:{}", self.tx_hash(&xt), self.active_views_count()); + let result_future = self.submit_at(_at, source, vec![xt]); + async move { + let result = result_future.await; + match result { + Ok(mut v) => + v.pop().expect("There is exactly one element in result of submit_at. qed."), + Err(e) => Err(e), + } + } + .boxed() + } + + /// Submits a transaction and starts to watch its progress in the pool, returning a stream of + /// status updates. + /// + /// Actual transaction submission process is delegated to the `ViewStore` internal instance. + fn submit_and_watch( + &self, + at: ::Hash, + source: TransactionSource, + xt: TransactionFor, + ) -> PoolFuture>>, 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 future::ready(Err(e)).boxed(), + }; + + self.metrics.report(|metrics| metrics.submitted_transactions.inc()); + + let view_store = self.view_store.clone(); + let mempool = self.mempool.clone(); + async move { + let result = view_store.submit_and_watch(at, source, xt).await; + let result = result.or_else(|(e, maybe_watcher)| { + let error = e.into_pool_error(); + match (error, maybe_watcher) { + ( + Ok( + // The transaction is still in mempool it may get included into the + // view for the next block. + Error::ImmediatelyDropped, + ), + Some(watcher), + ) => Ok(watcher), + (Ok(e), _) => { + mempool.remove(xt_hash); + Err(e.into()) + }, + (Err(e), _) => Err(e), + } + }); + result + } + .boxed() + } + + /// Intended to remove transactions identified by the given hashes, and any dependent + /// transactions, from the pool. In current implementation this function only outputs the error. + /// Seems that API change is needed here to make this call reasonable. + // todo [#5491]: api change? we need block hash here (assuming we need it at all - could be + // useful for verification for debugging purposes). + fn remove_invalid(&self, hashes: &[TxHash]) -> Vec> { + if !hashes.is_empty() { + log::debug!(target: LOG_TARGET, "fatp::remove_invalid {}", hashes.len()); + log_xt_trace!(target:LOG_TARGET, hashes, "[{:?}] fatp::remove_invalid"); + self.metrics + .report(|metrics| metrics.removed_invalid_txs.inc_by(hashes.len() as _)); + } + Default::default() + } + + // todo [#5491]: api change? + // status(Hash) -> Option + /// Returns the pool status which includes information like the number of ready and future + /// transactions. + /// + /// Currently the status for the most recently notified best block is returned (for which + /// maintain process was accomplished). + fn status(&self) -> PoolStatus { + self.view_store + .most_recent_view + .read() + .map(|hash| self.view_store.status()[&hash].clone()) + .unwrap_or(PoolStatus { ready: 0, ready_bytes: 0, future: 0, future_bytes: 0 }) + } + + /// Return an event stream of notifications when transactions are imported to the pool. + /// + /// Consumers of this stream should use the `ready` method to actually get the + /// pending transactions in the right order. + fn import_notification_stream(&self) -> ImportNotificationStream> { + self.import_notification_sink.event_stream() + } + + /// Returns the hash of a given transaction. + fn hash_of(&self, xt: &TransactionFor) -> TxHash { + self.api().hash_and_length(xt).0 + } + + /// Notifies the pool about the broadcasting status of transactions. + fn on_broadcasted(&self, propagations: HashMap, Vec>) { + self.view_store.listener.transactions_broadcasted(propagations); + } + + /// Return specific ready transaction by hash, if there is one. + /// + /// Currently the ready transaction is returned if it exists for the most recently notified best + /// block (for which maintain process was accomplished). + // todo [#5491]: api change: we probably should have at here? + fn ready_transaction(&self, tx_hash: &TxHash) -> Option> { + let most_recent_view = self.view_store.most_recent_view.read(); + let result = most_recent_view + .map(|block_hash| self.view_store.ready_transaction(block_hash, tx_hash)) + .flatten(); + log::trace!( + target: LOG_TARGET, + "[{tx_hash:?}] ready_transaction: {} {:?}", + result.is_some(), + most_recent_view + ); + result + } + + /// Returns an iterator for ready transactions at a specific block, ordered by priority. + fn ready_at(&self, at: ::Hash) -> PolledIterator { + let (_, result) = self.ready_at_internal(at); + result + } + + /// Returns an iterator for ready transactions, ordered by priority. + /// + /// Currently the set of ready transactions is returned if it exists for the most recently + /// notified best block (for which maintain process was accomplished). + fn ready(&self) -> ReadyIteratorFor { + self.view_store.ready() + } + + /// Returns a list of future transactions in the pool. + /// + /// Currently the set of future transactions is returned if it exists for the most recently + /// notified best block (for which maintain process was accomplished). + fn futures(&self) -> Vec { + self.view_store.futures() + } + + /// Returns a set of ready transactions at a given block within the specified timeout. + /// + /// If the timeout expires before the maintain process is accomplished, a best-effort + /// set of transactions is returned (refer to `ready_at_light`). + fn ready_at_with_timeout( + &self, + at: ::Hash, + timeout: std::time::Duration, + ) -> PolledIterator { + self.ready_at_with_timeout_internal(at, timeout) + } +} + +impl sc_transaction_pool_api::LocalTransactionPool + for ForkAwareTxPool, Block> +where + Block: BlockT, + ::Hash: Unpin, + Client: sp_api::ProvideRuntimeApi + + sc_client_api::BlockBackend + + sc_client_api::blockchain::HeaderBackend + + sp_runtime::traits::BlockIdTo + + sp_blockchain::HeaderMetadata, + Client: Send + Sync + 'static, + Client::Api: sp_transaction_pool::runtime_api::TaggedTransactionQueue, +{ + type Block = Block; + type Hash = ExtrinsicHash>; + type Error = as graph::ChainApi>::Error; + + fn submit_local( + &self, + _at: Block::Hash, + _xt: sc_transaction_pool_api::LocalTransactionFor, + ) -> Result { + //todo [#5493] + //looks like view_store / view needs non async submit_local method ?. + let e = Err(sc_transaction_pool_api::error::Error::Unactionable.into()); + log::warn!( + target: LOG_TARGET, + "LocalTransactionPool::submit_local is not implemented for ForkAwareTxPool, returning error: {e:?}", + ); + e + } +} + +impl ForkAwareTxPool +where + Block: BlockT, + ChainApi: graph::ChainApi + 'static, + ::Hash: Unpin, +{ + /// Handles a new block notification. + /// + /// It is responsible for handling a newly notified block. It executes some sanity checks, find + /// the best view to clone from and executes the new view build procedure for the notified + /// block. + /// + /// If the view is correctly created, `ready_at` pollers for this block will be triggered. + async fn handle_new_block(&self, tree_route: &TreeRoute) { + let hash_and_number = match tree_route.last() { + Some(hash_and_number) => hash_and_number, + None => { + log::warn!( + target: LOG_TARGET, + "Skipping ChainEvent - no last block in tree route {:?}", + tree_route, + ); + return + }, + }; + + if self.has_view(&hash_and_number.hash) { + log::trace!( + target: LOG_TARGET, + "view already exists for block: {:?}", + hash_and_number, + ); + return + } + + let best_view = self.view_store.find_best_view(tree_route); + let new_view = self.build_new_view(best_view, hash_and_number, tree_route).await; + + if let Some(view) = new_view { + { + let view = view.clone(); + self.ready_poll.lock().trigger(hash_and_number.hash, move || { + Box::from(view.pool.validated_pool().ready()) + }); + } + + View::start_background_revalidation(view, self.revalidation_queue.clone()).await; + } + } + + /// Builds a new view. + /// + /// If `origin_view` is provided, the new view will be cloned from it. Otherwise an empty view + /// will be created. + /// + /// The new view will be updated with transactions from the tree_route and the mempool, all + /// required events will be triggered, it will be inserted to the view store. + /// + /// This method will also update multi-view listeners with newly created view. + async fn build_new_view( + &self, + origin_view: Option>>, + at: &HashAndNumber, + tree_route: &TreeRoute, + ) -> Option>> { + log::debug!( + target: LOG_TARGET, + "build_new_view: for: {:?} from: {:?} tree_route: {:?}", + at, + origin_view.as_ref().map(|v| v.at.clone()), + tree_route + ); + let mut view = if let Some(origin_view) = origin_view { + let mut view = View::new_from_other(&origin_view, at); + if !tree_route.retracted().is_empty() { + view.pool.clear_recently_pruned(); + } + view + } else { + log::debug!(target: LOG_TARGET, "creating non-cloned view: for: {at:?}"); + View::new( + self.api.clone(), + at.clone(), + self.options.clone(), + self.metrics.clone(), + self.is_validator.clone(), + ) + }; + + // 1. Capture all import notification from the very beginning, so first register all + //the listeners. + self.import_notification_sink.add_view( + view.at.hash, + view.pool.validated_pool().import_notification_stream().boxed(), + ); + + self.view_store.dropped_stream_controller.add_view( + view.at.hash, + view.pool.validated_pool().create_dropped_by_limits_stream().boxed(), + ); + + let start = Instant::now(); + let watched_xts = self.register_listeners(&mut view).await; + let duration = start.elapsed(); + log::debug!(target: LOG_TARGET, "register_listeners: at {at:?} took {duration:?}"); + + // 2. Handle transactions from the tree route. Pruning transactions from the view first + // will make some space for mempool transactions in case we are at the view's limits. + let start = Instant::now(); + self.update_view_with_fork(&view, tree_route, at.clone()).await; + let duration = start.elapsed(); + log::debug!(target: LOG_TARGET, "update_view_with_fork: at {at:?} took {duration:?}"); + + // 3. Finally, submit transactions from the mempool. + let start = Instant::now(); + self.update_view_with_mempool(&mut view, watched_xts).await; + let duration = start.elapsed(); + log::debug!(target: LOG_TARGET, "update_view_with_mempool: at {at:?} took {duration:?}"); + + let view = Arc::from(view); + self.view_store.insert_new_view(view.clone(), tree_route).await; + Some(view) + } + + /// Returns the list of xts included in all block ancestors, including the block itself. + /// + /// Example: for the following chain `F<-B1<-B2<-B3` xts from `F,B1,B2,B3` will be returned. + async fn extrinsics_included_since_finalized(&self, at: Block::Hash) -> HashSet> { + let start = Instant::now(); + let recent_finalized_block = self.enactment_state.lock().recent_finalized_block(); + + let Ok(tree_route) = self.api.tree_route(recent_finalized_block, at) else { + return Default::default() + }; + + let api = self.api.clone(); + let mut all_extrinsics = HashSet::new(); + + for h in tree_route.enacted().iter().rev() { + api.block_body(h.hash) + .await + .unwrap_or_else(|e| { + log::warn!(target: LOG_TARGET, "Compute ready light transactions: error request: {}", e); + None + }) + .unwrap_or_default() + .into_iter() + .map(|t| self.hash_of(&t)) + .for_each(|tx_hash| { + all_extrinsics.insert(tx_hash); + }); + } + + log::debug!(target: LOG_TARGET, + "fatp::extrinsics_included_since_finalized {} from {} count: {} took:{:?}", + at, + recent_finalized_block, + all_extrinsics.len(), + start.elapsed() + ); + all_extrinsics + } + + /// For every watched transaction in the mempool registers a transaction listener in the view. + /// + /// The transaction listener for a given view is also added to multi-view listener. This allows + /// to track aggreagated progress of the transaction within the transaction pool. + /// + /// Function returns a list of currently watched transactions in the mempool. + async fn register_listeners( + &self, + view: &View, + ) -> Vec<(ExtrinsicHash, Arc>)> { + log::debug!( + target: LOG_TARGET, + "register_listeners: {:?} xts:{:?} v:{}", + view.at, + self.mempool.unwatched_and_watched_count(), + self.active_views_count() + ); + + //todo [#5495]: maybe we don't need to register listener in view? We could use + // multi_view_listener.transaction_in_block + let results = self + .mempool + .clone_watched() + .into_iter() + .map(|(tx_hash, tx)| { + let watcher = view.create_watcher(tx_hash); + let at = view.at.clone(); + async move { + log::trace!(target: LOG_TARGET, "[{:?}] adding watcher {:?}", tx_hash, at.hash); + self.view_store.listener.add_view_watcher_for_tx( + tx_hash, + at.hash, + watcher.into_stream().boxed(), + ); + (tx_hash, tx) + } + }) + .collect::>(); + + future::join_all(results).await + } + + /// Updates the given view with the transaction from the internal mempol. + /// + /// All transactions from the mempool (excluding those which are either already imported or + /// already included in blocks since recently finalized block) are submitted to the + /// view. + /// + /// If there are no views, and mempool transaction is reported as invalid for the given view, + /// the transaction is reported as invalid and removed from the mempool. This does not apply to + /// stale and temporarily banned transactions. + /// + /// As the listeners for watched transactions were registered at the very beginning of maintain + /// procedure (`register_listeners`), this function accepts the list of watched transactions + /// from the mempool for which listener was actually registered to avoid submit/maintain races. + async fn update_view_with_mempool( + &self, + view: &View, + watched_xts: Vec<(ExtrinsicHash, Arc>)>, + ) { + log::debug!( + target: LOG_TARGET, + "update_view_with_mempool: {:?} xts:{:?} v:{}", + view.at, + self.mempool.unwatched_and_watched_count(), + 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 + .into_iter() + .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)) + }); + + 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); + } + + log::debug!(target: LOG_TARGET, "update_view_with_mempool: at {:?} watched {}/{}", view.at.hash, watched_submitted_count, self.mempool_len().1); + + 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))); + + // if there are no views yet, and a single newly created view is reporting error, just send + // out the invalid event, and remove transaction. + if self.view_store.is_empty() { + for result in watched_results { + match result { + Err(tx_hash) => { + self.view_store.listener.invalidate_transactions(&[tx_hash]); + self.mempool.remove(tx_hash); + }, + Ok(_) => {}, + } + } + } + } + + /// Updates the view with the transactions from the given tree route. + /// + /// Transactions from the retracted blocks are resubmitted to the given view. Tags for + /// transactions included in blocks on enacted fork are pruned from the provided view. + async fn update_view_with_fork( + &self, + view: &View, + tree_route: &TreeRoute, + hash_and_number: HashAndNumber, + ) { + log::debug!(target: LOG_TARGET, "update_view_with_fork tree_route: {:?} {tree_route:?}", view.at); + let api = self.api.clone(); + + // We keep track of everything we prune so that later we won't add + // transactions with those hashes from the retracted blocks. + let mut pruned_log = HashSet::>::new(); + + future::join_all( + tree_route + .enacted() + .iter() + .map(|h| crate::prune_known_txs_for_block(h, &*api, &view.pool)), + ) + .await + .into_iter() + .for_each(|enacted_log| { + pruned_log.extend(enacted_log); + }); + + //resubmit + { + let mut resubmit_transactions = Vec::new(); + + for retracted in tree_route.retracted() { + let hash = retracted.hash; + + let block_transactions = api + .block_body(hash) + .await + .unwrap_or_else(|e| { + log::warn!(target: LOG_TARGET, "Failed to fetch block body: {}", e); + None + }) + .unwrap_or_default() + .into_iter() + .filter(|tx| tx.is_signed().unwrap_or(true)); + + let mut resubmitted_to_report = 0; + + resubmit_transactions.extend( + block_transactions + .into_iter() + .map(|tx| (self.hash_of(&tx), tx)) + .filter(|(tx_hash, _)| { + 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_hash, tx)| { + //find arc if tx is known + self.mempool.get_by_hash(tx_hash).unwrap_or_else(|| Arc::from(tx)) + }), + ); + + self.metrics.report(|metrics| { + metrics.resubmitted_retracted_txs.inc_by(resubmitted_to_report) + }); + } + + 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; + } + } + + /// Executes the maintainance for the finalized event. + /// + /// Performs a house-keeping required for finalized event. This includes: + /// - executing the on finalized procedure for the view store, + /// - purging finalized transactions from the mempool and triggering mempool revalidation, + async fn handle_finalized(&self, finalized_hash: Block::Hash, tree_route: &[Block::Hash]) { + let finalized_number = self.api.block_id_to_number(&BlockId::Hash(finalized_hash)); + log::debug!(target: LOG_TARGET, "handle_finalized {finalized_number:?} tree_route: {tree_route:?} views_count:{}", self.active_views_count()); + + let finalized_xts = self.view_store.handle_finalized(finalized_hash, tree_route).await; + + self.mempool.purge_finalized_transactions(&finalized_xts).await; + self.import_notification_sink.clean_notified_items(&finalized_xts); + + self.metrics + .report(|metrics| metrics.finalized_txs.inc_by(finalized_xts.len() as _)); + + if let Ok(Some(finalized_number)) = finalized_number { + self.revalidation_queue + .revalidate_mempool( + self.mempool.clone(), + HashAndNumber { hash: finalized_hash, number: finalized_number }, + ) + .await; + } else { + log::trace!(target: LOG_TARGET, "purge_transactions_later skipped, cannot find block number {finalized_number:?}"); + } + + self.ready_poll.lock().remove_cancelled(); + log::trace!(target: LOG_TARGET, "handle_finalized after views_count:{:?}", self.active_views_count()); + } + + /// Computes a hash of the provided transaction + fn tx_hash(&self, xt: &TransactionFor) -> TxHash { + self.api.hash_and_length(xt).0 + } +} + +#[async_trait] +impl MaintainedTransactionPool for ForkAwareTxPool +where + Block: BlockT, + ChainApi: 'static + graph::ChainApi, + ::Hash: Unpin, +{ + /// Executes the maintainance for the given chain event. + async fn maintain(&self, event: ChainEvent) { + let start = Instant::now(); + log::debug!(target: LOG_TARGET, "processing event: {event:?}"); + + self.view_store.finish_background_revalidations().await; + + let prev_finalized_block = self.enactment_state.lock().recent_finalized_block(); + + let compute_tree_route = |from, to| -> Result, String> { + match self.api.tree_route(from, to) { + Ok(tree_route) => Ok(tree_route), + Err(e) => + return Err(format!( + "Error occurred while computing tree_route from {from:?} to {to:?}: {e}" + )), + } + }; + let block_id_to_number = + |hash| self.api.block_id_to_number(&BlockId::Hash(hash)).map_err(|e| format!("{}", e)); + + let result = + self.enactment_state + .lock() + .update(&event, &compute_tree_route, &block_id_to_number); + + match result { + Err(msg) => { + log::trace!(target: LOG_TARGET, "enactment_state::update error: {msg}"); + self.enactment_state.lock().force_update(&event); + }, + Ok(EnactmentAction::Skip) => return, + Ok(EnactmentAction::HandleFinalization) => { + // todo [#5492]: in some cases handle_new_block is actually needed (new_num > + // tips_of_forks) let hash = event.hash(); + // if !self.has_view(hash) { + // if let Ok(tree_route) = compute_tree_route(prev_finalized_block, hash) { + // self.handle_new_block(&tree_route).await; + // } + // } + }, + Ok(EnactmentAction::HandleEnactment(tree_route)) => { + if matches!(event, ChainEvent::Finalized { .. }) { + self.view_store.handle_pre_finalized(event.hash()).await; + }; + self.handle_new_block(&tree_route).await; + }, + }; + + match event { + ChainEvent::NewBestBlock { .. } => {}, + ChainEvent::Finalized { hash, ref tree_route } => { + self.handle_finalized(hash, tree_route).await; + + log::trace!( + target: LOG_TARGET, + "on-finalized enacted: {tree_route:?}, previously finalized: \ + {prev_finalized_block:?}", + ); + }, + } + + let maintain_duration = start.elapsed(); + + log::info!( + target: LOG_TARGET, + "maintain: txs:{:?} views:[{};{:?}] event:{event:?} took:{:?}", + self.mempool_len(), + self.active_views_count(), + self.views_stats(), + maintain_duration + ); + + self.metrics.report(|metrics| { + let (unwatched, watched) = self.mempool_len(); + let _ = ( + self.active_views_count().try_into().map(|v| metrics.active_views.set(v)), + self.inactive_views_count().try_into().map(|v| metrics.inactive_views.set(v)), + watched.try_into().map(|v| metrics.watched_txs.set(v)), + unwatched.try_into().map(|v| metrics.unwatched_txs.set(v)), + ); + metrics.maintain_duration.observe(maintain_duration.as_secs_f64()); + }); + } +} + +impl ForkAwareTxPool, Block> +where + Block: BlockT, + Client: sp_api::ProvideRuntimeApi + + sc_client_api::BlockBackend + + sc_client_api::blockchain::HeaderBackend + + sp_runtime::traits::BlockIdTo + + sc_client_api::ExecutorProvider + + sc_client_api::UsageProvider + + sp_blockchain::HeaderMetadata + + Send + + Sync + + 'static, + Client::Api: sp_transaction_pool::runtime_api::TaggedTransactionQueue, + ::Hash: std::marker::Unpin, +{ + /// Create new fork aware transaction pool for a full node with the provided api. + pub fn new_full( + options: Options, + is_validator: IsValidator, + prometheus: Option<&PrometheusRegistry>, + spawner: impl SpawnEssentialNamed, + client: Arc, + ) -> Self { + let pool_api = Arc::new(FullChainApi::new(client.clone(), prometheus, &spawner)); + let pool = Self::new_with_background_worker( + options, + is_validator, + pool_api, + prometheus, + spawner, + client.usage_info().chain.best_hash, + client.usage_info().chain.finalized_hash, + ); + + pool + } +} + +#[cfg(test)] +mod reduce_multiview_result_tests { + use super::*; + use sp_core::H256; + #[derive(Debug, PartialEq, Clone)] + enum Error { + Custom(u8), + } + + #[test] + fn empty() { + sp_tracing::try_init_simple(); + let input = HashMap::default(); + let r = reduce_multiview_result::(input); + assert!(r.is_empty()); + } + + #[test] + fn errors_only() { + sp_tracing::try_init_simple(); + let v: Vec<(H256, Vec>)> = vec![ + ( + H256::repeat_byte(0x13), + vec![ + Err(Error::Custom(10)), + Err(Error::Custom(11)), + Err(Error::Custom(12)), + Err(Error::Custom(13)), + ], + ), + ( + H256::repeat_byte(0x14), + vec![ + Err(Error::Custom(20)), + Err(Error::Custom(21)), + Err(Error::Custom(22)), + Err(Error::Custom(23)), + ], + ), + ( + H256::repeat_byte(0x15), + vec![ + Err(Error::Custom(30)), + Err(Error::Custom(31)), + Err(Error::Custom(32)), + Err(Error::Custom(33)), + ], + ), + ]; + let input = HashMap::from_iter(v.clone()); + let r = reduce_multiview_result(input); + + //order in HashMap is random, the result shall be one of: + assert!(r == v[0].1 || r == v[1].1 || r == v[2].1); + } + + #[test] + #[should_panic] + #[cfg(debug_assertions)] + fn invalid_lengths() { + sp_tracing::try_init_simple(); + let v: Vec<(H256, Vec>)> = vec![ + (H256::repeat_byte(0x13), vec![Err(Error::Custom(12)), Err(Error::Custom(13))]), + (H256::repeat_byte(0x14), vec![Err(Error::Custom(23))]), + ]; + let input = HashMap::from_iter(v); + let _ = reduce_multiview_result(input); + } + + #[test] + fn only_hashes() { + sp_tracing::try_init_simple(); + + let v: Vec<(H256, Vec>)> = vec![ + ( + H256::repeat_byte(0x13), + vec![Ok(H256::repeat_byte(0x13)), Ok(H256::repeat_byte(0x14))], + ), + ( + H256::repeat_byte(0x14), + vec![Ok(H256::repeat_byte(0x13)), Ok(H256::repeat_byte(0x14))], + ), + ]; + let input = HashMap::from_iter(v); + let r = reduce_multiview_result(input); + + assert_eq!(r, vec![Ok(H256::repeat_byte(0x13)), Ok(H256::repeat_byte(0x14))]); + } + + #[test] + fn one_view() { + sp_tracing::try_init_simple(); + let v: Vec<(H256, Vec>)> = vec![( + H256::repeat_byte(0x13), + vec![Ok(H256::repeat_byte(0x10)), Err(Error::Custom(11))], + )]; + let input = HashMap::from_iter(v); + let r = reduce_multiview_result(input); + + assert_eq!(r, vec![Ok(H256::repeat_byte(0x10)), Err(Error::Custom(11))]); + } + + #[test] + fn mix() { + sp_tracing::try_init_simple(); + let v: Vec<(H256, Vec>)> = vec![ + ( + H256::repeat_byte(0x13), + vec![ + Ok(H256::repeat_byte(0x10)), + Err(Error::Custom(11)), + Err(Error::Custom(12)), + Err(Error::Custom(33)), + ], + ), + ( + H256::repeat_byte(0x14), + vec![ + Err(Error::Custom(20)), + Ok(H256::repeat_byte(0x21)), + Err(Error::Custom(22)), + Err(Error::Custom(33)), + ], + ), + ( + H256::repeat_byte(0x15), + vec![ + Err(Error::Custom(30)), + Err(Error::Custom(31)), + Ok(H256::repeat_byte(0x32)), + Err(Error::Custom(33)), + ], + ), + ]; + let input = HashMap::from_iter(v); + let r = reduce_multiview_result(input); + + assert_eq!( + r, + vec![ + Ok(H256::repeat_byte(0x10)), + Ok(H256::repeat_byte(0x21)), + Ok(H256::repeat_byte(0x32)), + Err(Error::Custom(33)) + ] + ); + } +} 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 new file mode 100644 index 000000000000..7fbdcade63b8 --- /dev/null +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/import_notification_sink.rs @@ -0,0 +1,396 @@ +// 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 . + +//! Multi view import notification sink. This module provides a unified stream of transactions that +//! have been notified as ready by any of the active views maintained by the transaction pool. It +//! combines streams (`import_notification_stream`) from multiple views into a single stream. Events +//! coming from this stream are dynamically dispatched to many external watchers. + +use crate::{fork_aware_txpool::stream_map_util::next_event, LOG_TARGET}; +use futures::{ + channel::mpsc::{channel, Receiver as EventStream, Sender as ExternalSink}, + stream::StreamExt, + Future, FutureExt, +}; +use log::trace; +use parking_lot::RwLock; +use sc_utils::mpsc; +use std::{ + collections::HashSet, + fmt::{self, Debug, Formatter}, + hash::Hash, + pin::Pin, + sync::Arc, +}; +use tokio_stream::StreamMap; + +/// A type alias for a pinned, boxed stream of items of type `I`. +/// This alias is particularly useful for defining the types of the incoming streams from various +/// views, and is intended to build the stream of transaction hashes that become ready. +/// +/// Note: generic parameter allows better testing of all types involved. +type StreamOf = Pin + Send>>; + +/// A type alias for a tracing unbounded sender used as the command channel controller. +/// Used to send control commands to the [`AggregatedStreamContext`]. +type Controller = mpsc::TracingUnboundedSender; + +/// A type alias for a tracing unbounded receiver used as the command channel receiver. +/// Used to receive control commands in the [`AggregatedStreamContext`]. +type CommandReceiver = mpsc::TracingUnboundedReceiver; + +/// An enum representing commands that can be sent to the multi-sinks context. +/// +/// This enum contains variants that encapsulate control commands used to manage multiple streams +/// within the `AggregatedStreamContext`. +enum Command { + /// Adds a new view with a unique key and a stream of items of type `I`. + AddView(K, StreamOf), +} + +impl Debug for Command { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + match self { + Command::AddView(..) => write!(f, "AddView"), + } + } +} + +/// A context used to unfold the single stream of items aggregated from the multiple +/// streams. +/// +/// The `AggregatedStreamContext` continuously monitors both the command receiver and the stream +/// map, ensuring new views can be dynamically added and events from any active view can be +/// processed. +struct AggregatedStreamContext { + /// A map of streams identified by unique keys, + stream_map: StreamMap>, + /// A receiver for handling control commands, such as adding new views. + command_receiver: CommandReceiver>, +} + +impl AggregatedStreamContext +where + K: Send + Debug + Unpin + Clone + Default + Hash + Eq + 'static, + I: Send + Sync + 'static + PartialEq + Eq + Hash + Clone + Debug, +{ + /// Creates a new aggregated stream of items and its command controller. + /// + /// This function sets up the initial context with an empty stream map. The aggregated output + /// stream of items (e.g. hashes of transactions that become ready) is unfolded. + /// + /// It returns a tuple containing the output stream and the command controller, allowing + /// external components to control this stream. + fn event_stream() -> (StreamOf, Controller>) { + let (sender, receiver) = + sc_utils::mpsc::tracing_unbounded::>("import-notification-sink", 16); + + let ctx = Self { stream_map: StreamMap::new(), command_receiver: receiver }; + + let output_stream = futures::stream::unfold(ctx, |mut ctx| async move { + loop { + tokio::select! { + biased; + cmd = ctx.command_receiver.next() => { + match cmd? { + Command::AddView(key,stream) => { + trace!(target: LOG_TARGET,"Command::AddView {key:?}"); + ctx.stream_map.insert(key,stream); + }, + } + }, + + Some(event) = next_event(&mut ctx.stream_map) => { + trace!(target: LOG_TARGET, "import_notification_sink: select_next_some -> {:?}", event); + return Some((event.1, ctx)); + } + } + } + }) + .boxed(); + + (output_stream, sender) + } +} + +/// A struct that facilitates the relaying notifications of ready transactions from multiple views +/// to many external sinks. +/// +/// `MultiViewImportNotificationSink` provides mechanisms to dynamically add new views, filter +/// notifications of imported transactions hashes and relay them to the multiple external sinks. +#[derive(Clone)] +pub struct MultiViewImportNotificationSink { + /// A controller used to send commands to the internal [`AggregatedStreamContext`]. + controller: Controller>, + /// A vector of the external sinks, each receiving a copy of the merged stream of ready + /// transaction hashes. + external_sinks: Arc>>>, + /// A set of already notified items, ensuring that each item (transaction hash) is only + /// sent out once. + already_notified_items: Arc>>, +} + +/// An asynchronous task responsible for dispatching aggregated import notifications to multiple +/// sinks (created by [`MultiViewImportNotificationSink::event_stream`]). +pub type ImportNotificationTask = Pin + Send>>; + +impl MultiViewImportNotificationSink +where + K: 'static + Clone + Send + Debug + Default + Unpin + Eq + Hash, + I: 'static + Clone + Send + Debug + Sync + PartialEq + Eq + Hash, +{ + /// Creates a new [`MultiViewImportNotificationSink`] along with its associated worker task. + /// + /// This function initializes the sink and provides the worker task that listens for events from + /// the aggregated stream, relaying them to the external sinks. The task shall be polled by + /// caller. + /// + /// Returns a tuple containing the [`MultiViewImportNotificationSink`] and the + /// [`ImportNotificationTask`]. + pub fn new_with_worker() -> (MultiViewImportNotificationSink, ImportNotificationTask) { + let (output_stream, controller) = AggregatedStreamContext::::event_stream(); + let output_stream_controller = Self { + controller, + external_sinks: Default::default(), + already_notified_items: Default::default(), + }; + let external_sinks = output_stream_controller.external_sinks.clone(); + let already_notified_items = output_stream_controller.already_notified_items.clone(); + + let import_notifcation_task = output_stream + .for_each(move |event| { + let external_sinks = external_sinks.clone(); + let already_notified_items = already_notified_items.clone(); + async move { + if already_notified_items.write().insert(event.clone()) { + external_sinks.write().retain_mut(|sink| { + trace!(target: LOG_TARGET, "[{:?}] import_sink_worker sending out imported", event); + if let Err(e) = sink.try_send(event.clone()) { + trace!(target: LOG_TARGET, "import_sink_worker sending message failed: {e}"); + false + } else { + true + } + }); + } + } + }) + .boxed(); + (output_stream_controller, import_notifcation_task) + } + + /// Adds a new stream associated with the view identified by specified key. + /// + /// The new view's stream is added to the internal aggregated stream context by sending command + /// to its `command_receiver`. + pub fn add_view(&self, key: K, view: StreamOf) { + let _ = self + .controller + .unbounded_send(Command::AddView(key.clone(), view)) + .map_err(|e| { + trace!(target: LOG_TARGET, "add_view {key:?} send message failed: {e}"); + }); + } + + /// Creates and returns a new external stream of ready transactions hashes notifications. + pub fn event_stream(&self) -> EventStream { + const CHANNEL_BUFFER_SIZE: usize = 1024; + let (sender, receiver) = channel(CHANNEL_BUFFER_SIZE); + self.external_sinks.write().push(sender); + receiver + } + + /// Removes specified items from the `already_notified_items` set. + /// + /// Intended to be called once transactions are finalized. + pub fn clean_notified_items(&self, items_to_be_removed: &[I]) { + let mut already_notified_items = self.already_notified_items.write(); + items_to_be_removed.iter().for_each(|i| { + already_notified_items.remove(i); + }); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use core::time::Duration; + use tokio::task::JoinHandle; + + #[derive(Debug, Clone)] + struct Event { + delay: u64, + value: I, + } + + impl From<(u64, I)> for Event { + fn from(event: (u64, I)) -> Self { + Self { delay: event.0, value: event.1 } + } + } + + struct View { + scenario: Vec>, + sinks: Arc>>>, + } + + impl View { + fn new(scenario: Vec<(u64, I)>) -> Self { + Self { + scenario: scenario.into_iter().map(Into::into).collect(), + sinks: Default::default(), + } + } + + async fn event_stream(&self) -> EventStream { + let (sender, receiver) = channel(32); + self.sinks.write().push(sender); + receiver + } + + fn play(&mut self) -> JoinHandle<()> { + let mut scenario = self.scenario.clone(); + let sinks = self.sinks.clone(); + tokio::spawn(async move { + loop { + if scenario.is_empty() { + for sink in &mut *sinks.write() { + sink.close_channel(); + } + break; + }; + let x = scenario.remove(0); + tokio::time::sleep(Duration::from_millis(x.delay)).await; + for sink in &mut *sinks.write() { + sink.try_send(x.value.clone()).unwrap(); + } + } + }) + } + } + + #[tokio::test] + async fn deduplicating_works() { + sp_tracing::try_init_simple(); + + let (ctrl, runnable) = MultiViewImportNotificationSink::::new_with_worker(); + + let j0 = tokio::spawn(runnable); + + let stream = ctrl.event_stream(); + + let mut v1 = View::new(vec![(0, 1), (0, 2), (0, 3)]); + let mut v2 = View::new(vec![(0, 1), (0, 2), (0, 6)]); + let mut v3 = View::new(vec![(0, 1), (0, 2), (0, 3)]); + + let j1 = v1.play(); + let j2 = v2.play(); + let j3 = v3.play(); + + let o1 = v1.event_stream().await.boxed(); + let o2 = v2.event_stream().await.boxed(); + let o3 = v3.event_stream().await.boxed(); + + ctrl.add_view(1000, o1); + ctrl.add_view(2000, o2); + ctrl.add_view(3000, o3); + + let out = stream.take(4).collect::>().await; + assert!(out.iter().all(|v| vec![1, 2, 3, 6].contains(v))); + drop(ctrl); + + futures::future::join_all(vec![j0, j1, j2, j3]).await; + } + + #[tokio::test] + async fn dedup_filter_reset_works() { + sp_tracing::try_init_simple(); + + let (ctrl, runnable) = MultiViewImportNotificationSink::::new_with_worker(); + + let j0 = tokio::spawn(runnable); + + let stream = 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)]); + let mut v3 = View::new(vec![(20, 1), (20, 2), (20, 3)]); + + let j1 = v1.play(); + let j2 = v2.play(); + let j3 = v3.play(); + + let o1 = v1.event_stream().await.boxed(); + let o2 = v2.event_stream().await.boxed(); + let o3 = v3.event_stream().await.boxed(); + + 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(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; + } + + #[tokio::test] + async fn many_output_streams_are_supported() { + sp_tracing::try_init_simple(); + + let (ctrl, runnable) = MultiViewImportNotificationSink::::new_with_worker(); + + let j0 = tokio::spawn(runnable); + + let stream0 = ctrl.event_stream(); + let stream1 = ctrl.event_stream(); + + let mut v1 = View::new(vec![(0, 1), (0, 2), (0, 3)]); + let mut v2 = View::new(vec![(0, 1), (0, 2), (0, 6)]); + let mut v3 = View::new(vec![(0, 1), (0, 2), (0, 3)]); + + let j1 = v1.play(); + let j2 = v2.play(); + let j3 = v3.play(); + + let o1 = v1.event_stream().await.boxed(); + let o2 = v2.event_stream().await.boxed(); + let o3 = v3.event_stream().await.boxed(); + + ctrl.add_view(1000, o1); + ctrl.add_view(2000, o2); + ctrl.add_view(3000, o3); + + let out0 = stream0.take(4).collect::>().await; + let out1 = stream1.take(4).collect::>().await; + assert!(out0.iter().all(|v| vec![1, 2, 3, 6].contains(v))); + assert!(out1.iter().all(|v| vec![1, 2, 3, 6].contains(v))); + drop(ctrl); + + futures::future::join_all(vec![j0, j1, j2, j3]).await; + } +} diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/metrics.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/metrics.rs new file mode 100644 index 000000000000..73d45ac43051 --- /dev/null +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/metrics.rs @@ -0,0 +1,176 @@ +// 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 . + +//! Prometheus's metrics for a fork-aware transaction pool. + +use crate::common::metrics::{GenericMetricsLink, MetricsRegistrant}; +use prometheus_endpoint::{ + histogram_opts, linear_buckets, register, Counter, Gauge, Histogram, PrometheusError, Registry, + U64, +}; + +/// A helper alias for the Prometheus's metrics endpoint. +pub type MetricsLink = GenericMetricsLink; + +/// Transaction pool Prometheus metrics. +pub struct Metrics { + /// Total number of transactions submitted. + pub submitted_transactions: Counter, + /// Total number of currently maintained views. + pub active_views: Gauge, + /// Total number of current inactive views. + pub inactive_views: Gauge, + /// Total number of watched transactions in txpool. + pub watched_txs: Gauge, + /// Total number of unwatched transactions in txpool. + pub unwatched_txs: Gauge, + /// Total number of transactions reported as invalid. + pub removed_invalid_txs: Counter, + /// Total number of finalized transactions. + pub finalized_txs: Counter, + /// Histogram of maintain durations. + pub maintain_duration: Histogram, + /// Total number of transactions resubmitted from retracted forks. + pub resubmitted_retracted_txs: Counter, + /// Total number of transactions submitted from mempool to views. + pub submitted_from_mempool_txs: Counter, + /// Total number of transactions found as invalid during mempool revalidation. + pub mempool_revalidation_invalid_txs: Counter, + /// Total number of transactions found as invalid during view revalidation. + pub view_revalidation_invalid_txs: Counter, + /// Total number of valid transactions processed during view revalidation. + pub view_revalidation_resubmitted_txs: Counter, + /// Histogram of view revalidation durations. + pub view_revalidation_duration: Histogram, + /// Total number of the views created w/o cloning existing view. + pub non_cloned_views: Counter, +} + +impl MetricsRegistrant for Metrics { + fn register(registry: &Registry) -> Result, PrometheusError> { + Ok(Box::from(Self { + submitted_transactions: register( + Counter::new( + "substrate_sub_txpool_submitted_txs_total", + "Total number of transactions submitted", + )?, + registry, + )?, + active_views: register( + Gauge::new( + "substrate_sub_txpool_active_views", + "Total number of currently maintained views.", + )?, + registry, + )?, + inactive_views: register( + Gauge::new( + "substrate_sub_txpool_inactive_views", + "Total number of current inactive views.", + )?, + registry, + )?, + watched_txs: register( + Gauge::new( + "substrate_sub_txpool_watched_txs", + "Total number of watched transactions in txpool.", + )?, + registry, + )?, + unwatched_txs: register( + Gauge::new( + "substrate_sub_txpool_unwatched_txs", + "Total number of unwatched transactions in txpool.", + )?, + registry, + )?, + removed_invalid_txs: register( + Counter::new( + "substrate_sub_txpool_removed_invalid_txs_total", + "Total number of transactions reported as invalid.", + )?, + registry, + )?, + finalized_txs: register( + Counter::new( + "substrate_sub_txpool_finalized_txs_total", + "Total number of finalized transactions.", + )?, + registry, + )?, + maintain_duration: register( + Histogram::with_opts(histogram_opts!( + "substrate_sub_txpool_maintain_duration_seconds", + "Histogram of maintain durations.", + linear_buckets(0.0, 0.25, 13).unwrap() + ))?, + registry, + )?, + resubmitted_retracted_txs: register( + Counter::new( + "substrate_sub_txpool_resubmitted_retracted_txs_total", + "Total number of transactions resubmitted from retracted forks.", + )?, + registry, + )?, + submitted_from_mempool_txs: register( + Counter::new( + "substrate_sub_txpool_submitted_from_mempool_txs_total", + "Total number of transactions submitted from mempool to views.", + )?, + registry, + )?, + mempool_revalidation_invalid_txs: register( + Counter::new( + "substrate_sub_txpool_mempool_revalidation_invalid_txs_total", + "Total number of transactions found as invalid during mempool revalidation.", + )?, + registry, + )?, + view_revalidation_invalid_txs: register( + Counter::new( + "substrate_sub_txpool_view_revalidation_invalid_txs_total", + "Total number of transactions found as invalid during view revalidation.", + )?, + registry, + )?, + view_revalidation_resubmitted_txs: register( + Counter::new( + "substrate_sub_txpool_view_revalidation_resubmitted_txs_total", + "Total number of valid transactions processed during view revalidation.", + )?, + registry, + )?, + view_revalidation_duration: register( + Histogram::with_opts(histogram_opts!( + "substrate_sub_txpool_view_revalidation_duration_seconds", + "Histogram of view revalidation durations.", + linear_buckets(0.0, 0.25, 13).unwrap() + ))?, + registry, + )?, + non_cloned_views: register( + Counter::new( + "substrate_sub_txpool_non_cloned_views_total", + "Total number of the views created w/o cloning existing view.", + )?, + registry, + )?, + })) + } +} diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/mod.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/mod.rs new file mode 100644 index 000000000000..9f979e216b6d --- /dev/null +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/mod.rs @@ -0,0 +1,376 @@ +// 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 . + +//! Substrate fork aware transaction pool implementation. +//! +//! # Top level overview. +//! This documentation provides high level overview of the main structures and the main flows within +//! the fork-aware transaction pool. +//! +//! ## Structures. +//! ### View. +//! #### Purpose. +//! The main responsibility of the [`View`] is to provide the valid set of ready transactions at +//! the given block. [`ForkAwareTxPool`] keeps the number of recent views for all the blocks +//! notified since recently finalized block. +//! +//! The views associated with blocks at the tips of the forks are actively updated with all newly +//! incoming transactions, while intermediate views are not updated (they still provide transactions +//! ready to be included at that block) due to performance reasons, since every transaction +//! submitted to the view needs to be [validated][runtime_api::validate]. +//! Building upon the older blocks happens relatively rare so this does not affect blocks filling. +//! +//! The view is wrapper around [`Pool`] and exposes its functionality, including the ability +//! of [tracking][`Watcher`] the progress of every transaction. +//! +//! #### Views: active, inactive. +//! All the views are stored in [`ViewStore`] structure. In this documentation the views at the tips +//! of the forks are referred as [`active_views`], while the intermediate views as +//! [`inactive_views`]. +//! +//! +//! #### The life cycle of the [`View`]. +//! Views are created when the new [`ChainEvent`] is notified to the pool. The view that is +//! [closest][find_best_view] to the newly notified block is chosen to clone from. Once built and +//! updated the newly created view is placed in [`active_views`]. Detailed description of view +//! creation is described in [the material to follow](#handling-the-new-best-block). When the view +//! is no longer at the tip of the forks, it is moved to the [`inactive_views`]. When the block +//! number of the view is lower then the finalized block, the view is permanently removed. +//! +//! +//! *Example*: +//! The following chain: +//! ```text +//! C2 - C3 - C4 +//! / +//! B1 +//! \ +//! B2 - B3 - B4 +//! ``` +//! and the following set of events: +//! ```text +//! New best block: B1, C3, C4, B4 +//! ``` +//! will result in the following set of views within the [`ViewStore`]: +//! ```text +//! active: C4, B4 +//! inactive: B1, C3 +//! ``` +//! Please note that views are only created for the notified blocks. +//! +//! +//! ### View store. +//! [`ViewStore`] is the helper structure that provides means to perform some actions like +//! [`submit`] or [`submit_and_watch`] on every view. It keeps track of both active and inactive +//! views. +//! +//! It also keeps tracks of the `most_recent_view` which is used to implement some methods of +//! [TransactionPool API], see [API considerations](#api-considerations) section. +//! +//! ### Multi-view listeners +//! There is a number of event streams that are provided by individual views: +//! - [transaction status][`Watcher`], +//! - [ready notification][`vp::import_notification_stream`] (see [networking +//! section](#networking)), +//! - [dropped notification][`create_dropped_by_limits_stream`]. +//! +//! These streams need to be merged into a single stream exposed by transaction pool (or used +//! internally). Those aggregators are often referred as multi-view listeners and they implement +//! stream-specific or event-specific logic. +//! +//! The most important is [`MultiViewListener`] which is owned by view store. +//! More information about it is provided in [transaction +//! route](#transaction-route-submit_and_watch) section. +//! +//! +//! ### Intermediate transactions buffer: [`TxMemPool`] +//! The main purpose of an internal [`TxMemPool`] (referred to as *mempool*) is to prevent a +//! transaction from being lost, e.g. due to race condition when the new transaction submission +//! occurs just before the new view is created. This could also happen when a transaction is invalid +//! on one fork and could be valid on another which is not yet fully processed by the maintain +//! procedure. Additionally, it allows the pool to accept transactions when no blocks have been +//! reported yet. +//! +//! Since watched and non-watched transactions require a different treatment, the *mempool* keeps a +//! track on how the transaction was submitted. The [transaction source][`TransactionSource`] used +//! to submit transactions also needs to be kept in the *mempool*. The *mempool* transaction is a +//! simple [wrapper][`TxInMemPool`] around the [`Arc`] reference to the actual extrinsic body. +//! +//! Once the view is created, all transactions from *mempool* are submitted to and validated at this +//! view. +//! +//! The *mempool* removes its transactions when they get finalized. The transactions in *mempool* +//! are also periodically verified at every finalized block and removed from the *mempool* if no +//! longer valid. This is process is called [*mempool* revalidation](#mempool-pruningrevalidation). +//! +//! ## Flows +//! +//! The transaction pool internally is executing numerous tasks. This includes handling submitted +//! transactions and tracking their progress, listening to [`ChainEvent`]s and executing the +//! maintain process, which aims to provide the set of ready transactions. On the other side +//! transaction pool provides a [`ready_at`] future that resolves to the iterator of ready +//! transactions. On top of that pool performs background revalidation jobs. +//! +//! This section provides a top level overview of all flows within the fork aware transaction pool. +//! +//! ### Transaction route: [`submit`][`api_submit`] +//! This flow is simple. Transaction is added to the mempool and if it is not rejected by it (due to +//! size limits), it is also [submitted][`submit`] into every view in [`active_views`]. +//! +//! When the newly created view does not contain this transaction yet, it is +//! [re-submitted][ForkAwareTxPool::update_view_with_mempool] from [`TxMemPool`] into this view. +//! +//! ### Transaction route: [`submit_and_watch`][`api_submit_and_watch`] +//! +//! The [`submit_and_watch`] function allows to submit the transaction and track its +//! [status][`TransactionStatus`] within the pool. Every view is providing an independent +//! [stream][`View::submit_and_watch`] of events, which needs to be merged into the single stream +//! exposed to the [external listener][`TransactionStatusStreamFor`]. For majority of events simple +//! forwarding of events would not work (e.g. we could get multiple [`Ready`] events, or [`Ready`] / +//! [`Future`] mix). Some additional stateful logic is required to filter and process the views' +//! events. It is also easier to trigger some events (e.g. [`Finalized`], [`Invalid`], and +//! [`Broadcast`]) using some side-channel and simply ignoring these events from the view. All the +//! before mentioned functionality is provided by the [`MultiViewListener`]. +//! +//! When watched transaction is submitted to the pool it is added the *mempool* with watched +//! flag. The external stream for the transaction is created in a [`MultiViewListener`]. Then +//! transaction is submitted to every active [`View`] (using +//! [`submit_and_watch`][`View::submit_and_watch`]) and the resulting +//! views' stream is connected to the [`MultiViewListener`]. +//! +//! ### Maintain +//! The transaction pool exposes the [task][`notification_future`] that listens to the +//! finalized and best block streams and executes the [`maintain`] procedure. +//! +//! The [`maintain`] is the main procedure of the transaction pool. It handles incoming +//! [`ChainEvent`]s, as described in the following two sub-sections. +//! +//! #### Handling the new (best) block +//! If the new block actually needs to be handled, the following steps are +//! executed: +//! - [find][find_best_view] the best view and clone it to [create a new +//! view][crate::ForkAwareTxPool::build_new_view], +//! - [update the view][ForkAwareTxPool::update_view_with_mempool] with the transactions from the +//! *mempool* +//! - all transactions from the *mempool* (with some obvious filtering applied) are submitted to +//! the view, +//! - for all watched transactions from the *mempool* the watcher is registered in the new view, +//! and it is connected to the multi-view-listener, +//! - [update the view][ForkAwareTxPool::update_view_with_fork] with the transactions from the [tree +//! route][`TreeRoute`] (which is computed from the recent best block to newly notified one by +//! [enactment state][`EnactmentState`] helper): +//! - resubmit the transactions from the retracted blocks, +//! - prune extrinsic from the enacted blocks, and trigger [`InBlock`] events, +//! - insert the newly created and updated view into the view store. +//! +//! +//! #### Handling the finalized block +//! The following actions are taken on every finalized block: +//! - send [`Finalized`] events for every transactions on the finalized [tree route][`TreeRoute`], +//! - remove all the views (both active and inactive) that are lower then finalized block from the +//! view store, +//! - removal of finalized transaction from the *mempool*, +//! - trigger [*mempool* background revalidation](#mempool-pruningrevalidation). +//! - clean up of multi-view listeners which is required to avoid ever-growing structures, +//! +//! ### Light maintain +//! The [maintain](#maintain) procedure can sometimes be quite heavy, and it may not be accomplished +//! within the time window expected by the block builder. On top of that block builder may want to +//! build few blocks in the raw, not giving the pool enough time to accomplish possible ongoing +//! maintain process. +//! +//! To address this, there is a [light version][`ready_at_light`] of the maintain procedure. It +//! [finds the best view][find_best_view], clones it and prunes all the transactions that were +//! included in enacted part of [tree route][`TreeRoute`] from the base view to the block at which a +//! ready iterator was requested. No new [transaction validations][runtime_api::validate] are +//! required to accomplish it. +//! +//! ### Providing ready transactions: `ready_at` +//! The [`ready_at`] function returns a [future][`crate::PolledIterator`] that resolves to the +//! [ready transactions iterator][`ReadyTransactions`]. The block builder shall wait either for the +//! future to be resolved or for timeout to be hit. To avoid building empty blocks in case of +//! timeout, the waiting for timeout functionality was moved into the transaction pool, and new API +//! function was added: [`ready_at_with_timeout`]. This function also provides a fall back ready +//! iterator which is result of [light maintain](#light-maintain). +//! +//! New function internally waits either for [maintain](#maintain) process triggered for requested +//! block to be accomplished or for the timeout. If timeout hits then the result of [light +//! maintain](#light-maintain) is returned. Light maintain is always executed at the beginning of +//! [`ready_at_with_timeout`] to make sure that it is available w/ o additional delay. +//! +//! If the maintain process for the requested block was accomplished before the `ready_at` functions +//! are called both of them immediately provide the ready transactions iterator (which is simply +//! requested on the appropriate instance of the [`View`]). +//! +//! The little [`ReadyPoll`] helper contained within [`ForkAwareTxPool`] as ([`ready_poll`]) +//! implements the futures management. +//! +//! ### Background tasks +//! The [maintain](#maintain) procedure shall be as quick as possible, so heavy revalidation job is +//! delegated to the background worker. These includes view and *mempool* revalidation which are +//! both handled by the [`RevalidationQueue`] which simply sends revalidation requests to the +//! background thread. +//! +//! #### View revalidation +//! View revalidation is performed in the background thread. Revalidation is executed for every +//! view. All the transaction from the view are [revalidated][`view::revalidate`]. +//! +//! The fork-aware pool utilizes two threads to execute maintain and revalidation process +//! exclusively, ensuring maintain performance without overlapping with revalidation. +//! +//! The view revalidation process is [triggered][`start_background_revalidation`] at the very end of +//! the [maintain][`maintain`] process, and [stopped][`finish_background_revalidations`] at the +//! very beginning of the next maintenance execution (upon the next [`ChainEvent`] reception). The +//! results from the revalidation are immediately applied once the revalidation is +//! [terminated][crate::fork_aware_txpool::view::View::finish_revalidation]. +//! ```text +//! time: ----------------------> +//! maintenance thread: M----M------M--M-M--- +//! revalidation thread: -RRRR-RR-----RR-R-RRR +//! ``` +//! +//! #### Mempool pruning/revalidation +//! Transactions within *mempool* are constantly revalidated in the background. The +//! [revalidation][`mp::revalidate`] is performed in [batches][`batch_size`], and transactions that +//! were validated as latest, are revalidated first in the next iteration. The revalidation is +//! triggered on every finalized block. If a transaction is found to be invalid, the [`Invalid`] +//! event is sent and transaction is removed from the *mempool*. +//! +//! NOTE: There is one exception: if transaction is referenced by any view as ready, then it is +//! removed from the *mempool*, but not removed from the view. The [`Invalid`] event is not sent. +//! This case is not likely to happen, however it may need some extra attention. +//! +//! ### Networking +//! The pool is exposing [`ImportNotificationStream`][`import_notification_stream`], the dedicated +//! channel over which all ready transactions are notified. Internally this channel needs to merge +//! all ready events from every view. This functionality is implemented by +//! [`MultiViewImportNotificationSink`]. +//! +//! The networking module is utilizing this channel to receive info about new ready transactions +//! which later will be propagated over the network. On the other side, when a transaction is +//! received networking submits transaction to the pool using [`submit`][`api_submit`]. +//! +//! ### Handling invalid transactions +//! Refer to *mempool* revalidation [section](#mempool-pruningrevalidation). +//! +//! ## Pool limits +//! Every [`View`] has the [limits][`Options`] for the number or size of transactions it can hold. +//! Obviously the number of transactions in every view is not distributed equally, so some views +//! might be fully filled while others not. +//! +//! On the other hand the size of internal *mempool* shall also be capped, but transactions that are +//! still referenced by views should not be removed. +//! +//! When the [`View`] is at its limits, it can either reject the transaction during +//! submission process, or it can accept the transaction and drop different transaction which is +//! already in the pool during the [`enforce_limits`][`vp::enforce_limits`] process. +//! +//! The [`StreamOfDropped`] stream aggregating [per-view][`create_dropped_by_limits_stream`] streams +//! allows to monitor the transactions that were dropped by all the views (or dropped by some views +//! while not referenced by the others), what means that transaction can also be +//! [removed][`dropped_monitor_task`] from the *mempool*. +//! +//! +//! ## API Considerations +//! Refer to github issue: +//! +//! [`View`]: crate::fork_aware_txpool::view::View +//! [`view::revalidate`]: crate::fork_aware_txpool::view::View::revalidate +//! [`start_background_revalidation`]: crate::fork_aware_txpool::view::View::start_background_revalidation +//! [`View::submit_and_watch`]: crate::fork_aware_txpool::view::View::submit_and_watch +//! [`ViewStore`]: crate::fork_aware_txpool::view_store::ViewStore +//! [`finish_background_revalidations`]: crate::fork_aware_txpool::view_store::ViewStore::finish_background_revalidations +//! [find_best_view]: crate::fork_aware_txpool::view_store::ViewStore::find_best_view +//! [`active_views`]: crate::fork_aware_txpool::view_store::ViewStore::active_views +//! [`inactive_views`]: crate::fork_aware_txpool::view_store::ViewStore::inactive_views +//! [`TxMemPool`]: crate::fork_aware_txpool::tx_mem_pool::TxMemPool +//! [`mp::revalidate`]: crate::fork_aware_txpool::tx_mem_pool::TxMemPool::revalidate +//! [`batch_size`]: crate::fork_aware_txpool::tx_mem_pool::TXMEMPOOL_MAX_REVALIDATION_BATCH_SIZE +//! [`TxInMemPool`]: crate::fork_aware_txpool::tx_mem_pool::TxInMemPool +//! [`MultiViewListener`]: crate::fork_aware_txpool::multi_view_listener::MultiViewListener +//! [`Pool`]: crate::graph::Pool +//! [`Watcher`]: crate::graph::watcher::Watcher +//! [`Options`]: crate::graph::Options +//! [`vp::import_notification_stream`]: ../graph/validated_pool/struct.ValidatedPool.html#method.import_notification_stream +//! [`vp::enforce_limits`]: ../graph/validated_pool/struct.ValidatedPool.html#method.enforce_limits +//! [`create_dropped_by_limits_stream`]: ../graph/validated_pool/struct.ValidatedPool.html#method.create_dropped_by_limits_stream +//! [`ChainEvent`]: sc_transaction_pool_api::ChainEvent +//! [`TransactionStatusStreamFor`]: sc_transaction_pool_api::TransactionStatusStreamFor +//! [`api_submit`]: sc_transaction_pool_api::TransactionPool::submit_at +//! [`api_submit_and_watch`]: sc_transaction_pool_api::TransactionPool::submit_and_watch +//! [`ready_at_with_timeout`]: sc_transaction_pool_api::TransactionPool::ready_at_with_timeout +//! [`TransactionSource`]: sc_transaction_pool_api::TransactionSource +//! [TransactionPool API]: sc_transaction_pool_api::TransactionPool +//! [`TransactionStatus`]:sc_transaction_pool_api::TransactionStatus +//! [`Ready`]:sc_transaction_pool_api::TransactionStatus::Ready +//! [`Future`]:sc_transaction_pool_api::TransactionStatus::Future +//! [`Broadcast`]:sc_transaction_pool_api::TransactionStatus::Broadcast +//! [`Invalid`]:sc_transaction_pool_api::TransactionStatus::Invalid +//! [`InBlock`]:sc_transaction_pool_api::TransactionStatus::InBlock +//! [`Finalized`]:sc_transaction_pool_api::TransactionStatus::Finalized +//! [`ReadyTransactions`]:sc_transaction_pool_api::ReadyTransactions +//! [`dropped_monitor_task`]: ForkAwareTxPool::dropped_monitor_task +//! [`ready_poll`]: ForkAwareTxPool::ready_poll +//! [`ready_at_light`]: ForkAwareTxPool::ready_at_light +//! [`ready_at`]: ../struct.ForkAwareTxPool.html#method.ready_at +//! [`import_notification_stream`]: ../struct.ForkAwareTxPool.html#method.import_notification_stream +//! [`maintain`]: ../struct.ForkAwareTxPool.html#method.maintain +//! [`submit`]: ../struct.ForkAwareTxPool.html#method.submit_at +//! [`submit_and_watch`]: ../struct.ForkAwareTxPool.html#method.submit_and_watch +//! [`ReadyPoll`]: ../fork_aware_txpool/fork_aware_txpool/struct.ReadyPoll.html +//! [`TreeRoute`]: sp_blockchain::TreeRoute +//! [runtime_api::validate]: sp_transaction_pool::runtime_api::TaggedTransactionQueue::validate_transaction +//! [`notification_future`]: crate::common::notification_future +//! [`EnactmentState`]: crate::common::enactment_state::EnactmentState +//! [`MultiViewImportNotificationSink`]: crate::fork_aware_txpool::import_notification_sink::MultiViewImportNotificationSink +//! [`RevalidationQueue`]: crate::fork_aware_txpool::revalidation_worker::RevalidationQueue +//! [`StreamOfDropped`]: crate::fork_aware_txpool::dropped_watcher::StreamOfDropped +//! [`Arc`]: std::sync::Arc + +mod dropped_watcher; +pub(crate) mod fork_aware_txpool; +mod import_notification_sink; +mod metrics; +mod multi_view_listener; +mod revalidation_worker; +mod tx_mem_pool; +mod view; +mod view_store; + +pub use fork_aware_txpool::{ForkAwareTxPool, ForkAwareTxPoolTask}; + +mod stream_map_util { + use futures::Stream; + use std::marker::Unpin; + use tokio_stream::StreamMap; + + pub async fn next_event( + stream_map: &mut StreamMap, + ) -> Option<(K, ::Item)> + where + K: Clone + Unpin, + V: Stream + Unpin, + { + if stream_map.is_empty() { + // yield pending to prevent busy-loop on an empty map + futures::pending!() + } + + futures::StreamExt::next(stream_map).await + } +} 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 new file mode 100644 index 000000000000..8d0e69db2e9a --- /dev/null +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/multi_view_listener.rs @@ -0,0 +1,736 @@ +// 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 . + +//! `MultiViewListener` and `ExternalWatcherContext` manage view streams and status updates for +//! transactions, providing control commands to manage transaction states, and create external +//! aggregated streams of transaction events. + +use crate::{ + fork_aware_txpool::stream_map_util::next_event, + graph::{self, BlockHash, ExtrinsicHash}, + LOG_TARGET, +}; +use futures::StreamExt; +use log::{debug, trace}; +use sc_transaction_pool_api::{TransactionStatus, TransactionStatusStream, TxIndex}; +use sc_utils::mpsc; +use sp_runtime::traits::Block as BlockT; +use std::{ + collections::{hash_map::Entry, HashMap, HashSet}, + pin::Pin, +}; +use tokio_stream::StreamMap; + +/// A side channel allowing to control the external stream instance (one per transaction) with +/// [`ControllerCommand`]. +/// +/// Set of instances of [`Controller`] lives within the [`MultiViewListener`]. +type Controller = mpsc::TracingUnboundedSender; + +/// A receiver of [`ControllerCommand`] instances allowing to control the external stream. +/// +/// Lives within the [`ExternalWatcherContext`] instance. +type CommandReceiver = mpsc::TracingUnboundedReceiver; + +/// The stream of the transaction events. +/// +/// It can represent both a single view's stream and an external watcher stream. +pub type TxStatusStream = Pin, BlockHash>>>; + +/// Commands to control the single external stream living within the multi view listener. +enum ControllerCommand { + /// Adds a new stream of transaction statuses originating in the view associated with a + /// specific block hash. + AddViewStream(BlockHash, TxStatusStream), + + /// Removes an existing view's stream associated with a specific block hash. + RemoveViewStream(BlockHash), + + /// Marks a transaction as invalidated. + /// + /// If all pre-conditions are met, an external invalid event will be sent out. + TransactionInvalidated, + + /// Notifies that a transaction was finalized in a specific block hash and transaction index. + /// + /// Send out an external finalized event. + FinalizeTransaction(BlockHash, TxIndex), + + /// Notifies that a transaction was broadcasted with a list of peer addresses. + /// + /// Sends out an external broadcasted event. + TransactionBroadcasted(Vec), + + /// Notifies that a transaction was dropped from the pool. + /// + /// If all preconditions are met, an external dropped event will be sent out. + TransactionDropped, +} + +impl std::fmt::Debug for ControllerCommand +where + ChainApi: graph::ChainApi, +{ + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + ControllerCommand::AddViewStream(h, _) => write!(f, "ListenerAction::AddView({h})"), + ControllerCommand::RemoveViewStream(h) => write!(f, "ListenerAction::RemoveView({h})"), + ControllerCommand::TransactionInvalidated => { + write!(f, "ListenerAction::TransactionInvalidated") + }, + ControllerCommand::FinalizeTransaction(h, i) => { + write!(f, "ListenerAction::FinalizeTransaction({h},{i})") + }, + ControllerCommand::TransactionBroadcasted(_) => { + write!(f, "ListenerAction::TransactionBroadcasted(...)") + }, + ControllerCommand::TransactionDropped => { + write!(f, "ListenerAction::TransactionDropped") + }, + } + } +} + +/// This struct allows to create and control listener for multiple transactions. +/// +/// For every transaction the view's stream generating its own events can be added. The events are +/// flattened and sent out to the external listener. (The *external* term here means that it can be +/// exposed to [`sc_transaction_pool_api::TransactionPool`] API client e.g. over RPC.) +/// +/// The listener allows to add and remove view's stream (per transaction). +/// +/// The listener provides a side channel that allows triggering specific events (finalized, dropped, +/// invalid) independently of the view's stream. +pub struct MultiViewListener { + /// Provides the set of controllers for the events streams corresponding to individual + /// transactions identified by transaction hashes. + controllers: parking_lot::RwLock< + HashMap, Controller>>, + >, +} + +/// The external stream unfolding context. +/// +/// This context is used to unfold the external events stream for a single transaction, it +/// facilitates the logic of converting single view's events to the external events stream. +struct ExternalWatcherContext { + /// The hash of the transaction being monitored within this context. + tx_hash: ExtrinsicHash, + /// A stream map of transaction status streams coming from individual views, keyed by + /// block hash associated with view. + status_stream_map: StreamMap, TxStatusStream>, + /// A receiver for controller commands. + command_receiver: CommandReceiver>, + /// A flag indicating whether the context should terminate. + terminate: bool, + /// A flag indicating if a `Future` status has been encountered. + future_seen: bool, + /// A flag indicating if a `Ready` status has been encountered. + ready_seen: bool, + + /// A hash set of block hashes from views that consider the transaction valid. + views_keeping_tx_valid: HashSet>, +} + +impl ExternalWatcherContext +where + <::Block as BlockT>::Hash: Unpin, +{ + /// Creates new `ExternalWatcherContext` for particular transaction identified by `tx_hash` + /// + /// The `command_receiver` is a side channel for receiving controller's commands. + fn new( + tx_hash: ExtrinsicHash, + command_receiver: CommandReceiver>, + ) -> Self { + Self { + tx_hash, + status_stream_map: StreamMap::new(), + command_receiver, + terminate: false, + future_seen: false, + ready_seen: false, + views_keeping_tx_valid: Default::default(), + } + } + + /// Handles various transaction status updates and manages internal states based on the status. + /// + /// Function may set the context termination flag, which will close the stream. + /// + /// Returns `Some` with the `event` to forward or `None`. + fn handle( + &mut self, + status: TransactionStatus, BlockHash>, + hash: BlockHash, + ) -> Option, BlockHash>> { + trace!( + target: LOG_TARGET, "[{:?}] mvl handle event from {hash:?}: {status:?} views:{:?}", self.tx_hash, + self.status_stream_map.keys().collect::>() + ); + match status { + TransactionStatus::Future => { + self.views_keeping_tx_valid.insert(hash); + if self.ready_seen || self.future_seen { + None + } else { + self.future_seen = true; + Some(status) + } + }, + TransactionStatus::Ready => { + self.views_keeping_tx_valid.insert(hash); + if self.ready_seen { + None + } else { + self.ready_seen = true; + Some(status) + } + }, + TransactionStatus::Broadcast(_) => None, + TransactionStatus::InBlock((..)) => { + self.views_keeping_tx_valid.insert(hash); + if !(self.ready_seen || self.future_seen) { + self.ready_seen = true; + Some(status) + } else { + Some(status) + } + }, + TransactionStatus::Retracted(_) => None, + TransactionStatus::FinalityTimeout(_) => Some(status), + TransactionStatus::Finalized(_) => { + self.terminate = true; + Some(status) + }, + TransactionStatus::Usurped(_) | + TransactionStatus::Dropped | + TransactionStatus::Invalid => None, + } + } + + /// Handles transaction invalidation sent via side channel. + /// + /// Function may set the context termination flag, which will close the stream. + /// + /// Returns true if the event should be sent out, and false if the invalidation request should + /// be skipped. + fn handle_invalidate_transaction(&mut self) -> bool { + let keys = HashSet::>::from_iter( + self.status_stream_map.keys().map(Clone::clone), + ); + trace!( + target: LOG_TARGET, + "[{:?}] got invalidate_transaction: views:{:?}", self.tx_hash, + self.status_stream_map.keys().collect::>() + ); + if self.views_keeping_tx_valid.is_disjoint(&keys) { + self.terminate = true; + true + } else { + //todo [#5477] + // - handle corner case: this may happen when tx is invalid for mempool, but somehow + // some view still sees it as ready/future. In that case we don't send the invalid + // event, as transaction can still be included. Probably we should set some flag here + // and allow for invalid sent from the view. + // - add debug / metrics, + false + } + } + + /// Adds a new transaction status stream. + /// + /// Inserts a new view's transaction status stream associated with a specific block hash into + /// the stream map. + fn add_stream(&mut self, block_hash: BlockHash, stream: TxStatusStream) { + self.status_stream_map.insert(block_hash, stream); + trace!(target: LOG_TARGET, "[{:?}] AddView view: {:?} views:{:?}", self.tx_hash, block_hash, self.status_stream_map.keys().collect::>()); + } + + /// Removes an existing transaction status stream. + /// + /// Removes a transaction status stream associated with a specific block hash from the + /// stream map. + fn remove_view(&mut self, block_hash: BlockHash) { + self.status_stream_map.remove(&block_hash); + trace!(target: LOG_TARGET, "[{:?}] RemoveView view: {:?} views:{:?}", self.tx_hash, block_hash, self.status_stream_map.keys().collect::>()); + } +} + +impl MultiViewListener +where + ChainApi: graph::ChainApi + 'static, + <::Block as BlockT>::Hash: Unpin, +{ + /// Creates new instance of `MultiViewListener`. + pub fn new() -> Self { + Self { controllers: Default::default() } + } + + /// Creates an external aggregated stream of events for given transaction. + /// + /// This method initializes an `ExternalWatcherContext` for the provided transaction hash, sets + /// up the necessary communication channels, and unfolds an external (meaning that it can be + /// exposed to [`sc_transaction_pool_api::TransactionPool`] API client e.g. rpc) stream of + /// transaction status events. If an external watcher is already present for the given + /// transaction, it returns `None`. + pub(crate) fn create_external_watcher_for_tx( + &self, + tx_hash: ExtrinsicHash, + ) -> Option> { + let mut controllers = self.controllers.write(); + if controllers.contains_key(&tx_hash) { + return None + } + + trace!(target: LOG_TARGET, "[{:?}] create_external_watcher_for_tx", tx_hash); + + let (tx, rx) = mpsc::tracing_unbounded("txpool-multi-view-listener", 32); + controllers.insert(tx_hash, tx); + + let ctx = ExternalWatcherContext::new(tx_hash, rx); + + Some( + futures::stream::unfold(ctx, |mut ctx| async move { + if ctx.terminate { + return None + } + loop { + tokio::select! { + biased; + Some((view_hash, status)) = next_event(&mut ctx.status_stream_map) => { + if let Some(new_status) = ctx.handle(status, view_hash) { + log::trace!(target: LOG_TARGET, "[{:?}] mvl sending out: {new_status:?}", ctx.tx_hash); + return Some((new_status, ctx)) + } + }, + cmd = ctx.command_receiver.next() => { + log::trace!(target: LOG_TARGET, "[{:?}] select::rx views:{:?}", + ctx.tx_hash, + ctx.status_stream_map.keys().collect::>() + ); + match cmd? { + ControllerCommand::AddViewStream(h,stream) => { + ctx.add_stream(h, stream); + }, + ControllerCommand::RemoveViewStream(h) => { + ctx.remove_view(h); + }, + ControllerCommand::TransactionInvalidated => { + if ctx.handle_invalidate_transaction() { + log::trace!(target: LOG_TARGET, "[{:?}] mvl sending out: Invalid", ctx.tx_hash); + return Some((TransactionStatus::Invalid, ctx)) + } + }, + ControllerCommand::FinalizeTransaction(block, index) => { + log::trace!(target: LOG_TARGET, "[{:?}] mvl sending out: Finalized", ctx.tx_hash); + ctx.terminate = true; + return Some((TransactionStatus::Finalized((block, index)), ctx)) + }, + ControllerCommand::TransactionBroadcasted(peers) => { + log::trace!(target: LOG_TARGET, "[{:?}] mvl sending out: Broadcasted", ctx.tx_hash); + return Some((TransactionStatus::Broadcast(peers), ctx)) + }, + ControllerCommand::TransactionDropped => { + log::trace!(target: LOG_TARGET, "[{:?}] mvl sending out: Dropped", ctx.tx_hash); + ctx.terminate = true; + return Some((TransactionStatus::Dropped, ctx)) + }, + } + }, + }; + } + }) + .boxed(), + ) + } + + /// Adds a view's transaction status stream for particular transaction. + /// + /// This method sends a `AddViewStream` command to the controller of each transaction to + /// remove the view's stream corresponding to the given block hash. + pub(crate) fn add_view_watcher_for_tx( + &self, + tx_hash: ExtrinsicHash, + block_hash: BlockHash, + stream: TxStatusStream, + ) { + let mut controllers = self.controllers.write(); + + if let Entry::Occupied(mut tx) = controllers.entry(tx_hash) { + if let Err(e) = tx + .get_mut() + .unbounded_send(ControllerCommand::AddViewStream(block_hash, stream)) + { + trace!(target: LOG_TARGET, "[{:?}] add_view_watcher_for_tx: send message failed: {:?}", tx_hash, e); + tx.remove(); + } + } + } + + /// Removes a view's stream associated with a specific view hash across all transactions. + /// + /// This method sends a `RemoveViewStream` command to the controller of each transaction to + /// remove the view's stream corresponding to the given block hash. + pub(crate) fn remove_view(&self, block_hash: BlockHash) { + self.controllers.write().retain(|tx_hash, sender| { + sender + .unbounded_send(ControllerCommand::RemoveViewStream(block_hash)) + .map_err(|e| { + log::trace!(target: LOG_TARGET, "[{:?}] remove_view: send message failed: {:?}", tx_hash, e); + e + }) + .is_ok() + }); + } + + /// Invalidate given transaction. + /// + /// This method sends a `TransactionInvalidated` command to the controller of each transaction + /// provided to process the invalidation request. + /// + /// The external event will be sent if no view is referencing the transaction as `Ready` or + /// `Future`. + pub(crate) fn invalidate_transactions(&self, invalid_hashes: &[ExtrinsicHash]) { + let mut controllers = self.controllers.write(); + invalid_hashes.iter().for_each(|tx_hash| { + if let Entry::Occupied(mut tx) = controllers.entry(*tx_hash) { + trace!(target: LOG_TARGET, "[{:?}] invalidate_transaction", tx_hash); + if let Err(e) = + tx.get_mut().unbounded_send(ControllerCommand::TransactionInvalidated) + { + trace!(target: LOG_TARGET, "[{:?}] invalidate_transaction: send message failed: {:?}", tx_hash, e); + tx.remove(); + } + } + }); + } + + /// Send `Broadcasted` event to listeners of all transactions. + /// + /// This method sends a `TransactionBroadcasted` command to the controller of each transaction + /// provided prompting the external `Broadcasted` event. + pub(crate) fn transactions_broadcasted( + &self, + propagated: HashMap, Vec>, + ) { + let mut controllers = self.controllers.write(); + propagated.into_iter().for_each(|(tx_hash, peers)| { + if let Entry::Occupied(mut tx) = controllers.entry(tx_hash) { + trace!(target: LOG_TARGET, "[{:?}] transaction_broadcasted", tx_hash); + if let Err(e) = tx.get_mut().unbounded_send(ControllerCommand::TransactionBroadcasted(peers)) { + trace!(target: LOG_TARGET, "[{:?}] transactions_broadcasted: send message failed: {:?}", tx_hash, e); + tx.remove(); + } + } + }); + } + + /// Send `Dropped` event to listeners of transactions. + /// + /// 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]) { + 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); + }; + } + } + } + + /// Send `Finalized` event for given transaction at given block. + /// + /// This will send `Finalized` event to the external watcher. + pub(crate) fn finalize_transaction( + &self, + tx_hash: ExtrinsicHash, + block: BlockHash, + idx: TxIndex, + ) { + let mut controllers = self.controllers.write(); + if let Some(tx) = controllers.remove(&tx_hash) { + trace!(target: LOG_TARGET, "[{:?}] finalize_transaction", tx_hash); + if let Err(e) = tx.unbounded_send(ControllerCommand::FinalizeTransaction(block, idx)) { + trace!(target: LOG_TARGET, "[{:?}] finalize_transaction: send message failed: {:?}", tx_hash, e); + } + }; + } + + /// Removes stale controllers. + pub(crate) fn remove_stale_controllers(&self) { + self.controllers.write().retain(|_, c| !c.is_closed()); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::common::tests::TestApi; + use futures::{stream, StreamExt}; + use sp_core::H256; + + type MultiViewListener = super::MultiViewListener; + + #[tokio::test] + async fn test01() { + sp_tracing::try_init_simple(); + let listener = MultiViewListener::new(); + + let block_hash = H256::repeat_byte(0x01); + let events = vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((block_hash, 0)), + TransactionStatus::Finalized((block_hash, 0)), + ]; + + let tx_hash = H256::repeat_byte(0x0a); + let external_watcher = listener.create_external_watcher_for_tx(tx_hash).unwrap(); + let handle = tokio::spawn(async move { external_watcher.collect::>().await }); + + let view_stream = futures::stream::iter(events.clone()); + + listener.add_view_watcher_for_tx(tx_hash, block_hash, view_stream.boxed()); + + let out = handle.await.unwrap(); + assert_eq!(out, events); + log::debug!("out: {:#?}", out); + } + + #[tokio::test] + async fn test02() { + sp_tracing::try_init_simple(); + let listener = MultiViewListener::new(); + + let block_hash0 = H256::repeat_byte(0x01); + let events0 = vec![ + TransactionStatus::Future, + TransactionStatus::Ready, + TransactionStatus::InBlock((block_hash0, 0)), + ]; + + let block_hash1 = H256::repeat_byte(0x02); + let events1 = vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((block_hash1, 0)), + TransactionStatus::Finalized((block_hash1, 0)), + ]; + + let tx_hash = H256::repeat_byte(0x0a); + let external_watcher = listener.create_external_watcher_for_tx(tx_hash).unwrap(); + + let view_stream0 = futures::stream::iter(events0.clone()); + let view_stream1 = futures::stream::iter(events1.clone()); + + let handle = tokio::spawn(async move { external_watcher.collect::>().await }); + + listener.add_view_watcher_for_tx(tx_hash, block_hash0, view_stream0.boxed()); + listener.add_view_watcher_for_tx(tx_hash, block_hash1, view_stream1.boxed()); + + let out = handle.await.unwrap(); + + log::debug!("out: {:#?}", out); + assert!(out.iter().all(|v| vec![ + TransactionStatus::Future, + TransactionStatus::Ready, + TransactionStatus::InBlock((block_hash0, 0)), + TransactionStatus::InBlock((block_hash1, 0)), + TransactionStatus::Finalized((block_hash1, 0)), + ] + .contains(v))); + assert_eq!(out.len(), 5); + } + + #[tokio::test] + async fn test03() { + sp_tracing::try_init_simple(); + let listener = MultiViewListener::new(); + + let block_hash0 = H256::repeat_byte(0x01); + let events0 = vec![ + TransactionStatus::Future, + TransactionStatus::Ready, + TransactionStatus::InBlock((block_hash0, 0)), + ]; + + let block_hash1 = H256::repeat_byte(0x02); + let events1 = vec![TransactionStatus::Future]; + + let tx_hash = H256::repeat_byte(0x0a); + let external_watcher = listener.create_external_watcher_for_tx(tx_hash).unwrap(); + let handle = tokio::spawn(async move { external_watcher.collect::>().await }); + + let view_stream0 = futures::stream::iter(events0.clone()); + let view_stream1 = futures::stream::iter(events1.clone()); + + listener.add_view_watcher_for_tx(tx_hash, block_hash0, view_stream0.boxed()); + listener.add_view_watcher_for_tx(tx_hash, block_hash1, view_stream1.boxed()); + + listener.invalidate_transactions(&[tx_hash]); + + let out = handle.await.unwrap(); + log::debug!("out: {:#?}", out); + assert!(out.iter().all(|v| vec![ + TransactionStatus::Future, + TransactionStatus::Ready, + TransactionStatus::InBlock((block_hash0, 0)), + TransactionStatus::Invalid + ] + .contains(v))); + assert_eq!(out.len(), 4); + } + + #[tokio::test] + async fn test032() { + sp_tracing::try_init_simple(); + let listener = MultiViewListener::new(); + + let block_hash0 = H256::repeat_byte(0x01); + let events0_tx0 = vec![TransactionStatus::Future]; + let events0_tx1 = vec![TransactionStatus::Ready]; + + let block_hash1 = H256::repeat_byte(0x02); + let events1_tx0 = + vec![TransactionStatus::Ready, TransactionStatus::InBlock((block_hash1, 0))]; + let events1_tx1 = vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((block_hash1, 1)), + TransactionStatus::Finalized((block_hash1, 1)), + ]; + + let tx0_hash = H256::repeat_byte(0x0a); + let tx1_hash = H256::repeat_byte(0x0b); + let external_watcher_tx0 = listener.create_external_watcher_for_tx(tx0_hash).unwrap(); + let external_watcher_tx1 = listener.create_external_watcher_for_tx(tx1_hash).unwrap(); + + let handle0 = tokio::spawn(async move { external_watcher_tx0.collect::>().await }); + let handle1 = tokio::spawn(async move { external_watcher_tx1.collect::>().await }); + + let view0_tx0_stream = futures::stream::iter(events0_tx0.clone()); + let view0_tx1_stream = futures::stream::iter(events0_tx1.clone()); + + let view1_tx0_stream = futures::stream::iter(events1_tx0.clone()); + let view1_tx1_stream = futures::stream::iter(events1_tx1.clone()); + + listener.add_view_watcher_for_tx(tx0_hash, block_hash0, view0_tx0_stream.boxed()); + listener.add_view_watcher_for_tx(tx0_hash, block_hash1, view1_tx0_stream.boxed()); + listener.add_view_watcher_for_tx(tx1_hash, block_hash0, view0_tx1_stream.boxed()); + listener.add_view_watcher_for_tx(tx1_hash, block_hash1, view1_tx1_stream.boxed()); + + listener.invalidate_transactions(&[tx0_hash]); + listener.invalidate_transactions(&[tx1_hash]); + + let out_tx0 = handle0.await.unwrap(); + let out_tx1 = handle1.await.unwrap(); + + log::debug!("out_tx0: {:#?}", out_tx0); + log::debug!("out_tx1: {:#?}", out_tx1); + assert!(out_tx0.iter().all(|v| vec![ + TransactionStatus::Future, + TransactionStatus::Ready, + TransactionStatus::InBlock((block_hash1, 0)), + TransactionStatus::Invalid + ] + .contains(v))); + + assert!(out_tx1.iter().all(|v| vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((block_hash1, 1)), + TransactionStatus::Finalized((block_hash1, 1)) + ] + .contains(v))); + assert_eq!(out_tx0.len(), 4); + assert_eq!(out_tx1.len(), 3); + } + + #[tokio::test] + async fn test04() { + sp_tracing::try_init_simple(); + let listener = MultiViewListener::new(); + + let block_hash0 = H256::repeat_byte(0x01); + let events0 = vec![ + TransactionStatus::Future, + TransactionStatus::Ready, + TransactionStatus::InBlock((block_hash0, 0)), + ]; + + let block_hash1 = H256::repeat_byte(0x02); + let events1 = vec![TransactionStatus::Future]; + + let tx_hash = H256::repeat_byte(0x0a); + let external_watcher = listener.create_external_watcher_for_tx(tx_hash).unwrap(); + + //views will keep transaction valid, invalidation shall not happen + let view_stream0 = futures::stream::iter(events0.clone()).chain(stream::pending().boxed()); + let view_stream1 = futures::stream::iter(events1.clone()).chain(stream::pending().boxed()); + + let handle = tokio::spawn(async move { + // views are still there, we need to fetch 3 events + external_watcher.take(3).collect::>().await + }); + + listener.add_view_watcher_for_tx(tx_hash, block_hash0, view_stream0.boxed()); + listener.add_view_watcher_for_tx(tx_hash, block_hash1, view_stream1.boxed()); + + listener.invalidate_transactions(&[tx_hash]); + + let out = handle.await.unwrap(); + log::debug!("out: {:#?}", out); + + // invalid shall not be sent + assert!(out.iter().all(|v| vec![ + TransactionStatus::Future, + TransactionStatus::Ready, + TransactionStatus::InBlock((block_hash0, 0)), + ] + .contains(v))); + assert_eq!(out.len(), 3); + } + + #[tokio::test] + async fn test05() { + sp_tracing::try_init_simple(); + let listener = MultiViewListener::new(); + + let block_hash0 = H256::repeat_byte(0x01); + let events0 = vec![TransactionStatus::Invalid]; + + let tx_hash = H256::repeat_byte(0x0a); + let external_watcher = listener.create_external_watcher_for_tx(tx_hash).unwrap(); + let handle = tokio::spawn(async move { external_watcher.collect::>().await }); + + let view_stream0 = futures::stream::iter(events0.clone()).chain(stream::pending().boxed()); + + // Note: this generates actual Invalid event. + // Invalid event from View's stream is intentionally ignored. + listener.invalidate_transactions(&[tx_hash]); + + listener.add_view_watcher_for_tx(tx_hash, block_hash0, view_stream0.boxed()); + + let out = handle.await.unwrap(); + log::debug!("out: {:#?}", out); + + assert!(out.iter().all(|v| vec![TransactionStatus::Invalid].contains(v))); + assert_eq!(out.len(), 1); + } +} 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 new file mode 100644 index 000000000000..9464ab3f5766 --- /dev/null +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/revalidation_worker.rs @@ -0,0 +1,240 @@ +// 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 . + +//! The background worker for the [`View`] and [`TxMemPool`] revalidation. +//! +//! The [*Background tasks*](../index.html#background-tasks) section provides some extra details on +//! revalidation process. + +use std::{marker::PhantomData, pin::Pin, sync::Arc}; + +use crate::{graph::ChainApi, LOG_TARGET}; +use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; +use sp_blockchain::HashAndNumber; +use sp_runtime::traits::Block as BlockT; + +use super::tx_mem_pool::TxMemPool; +use futures::prelude::*; + +use super::view::{FinishRevalidationWorkerChannels, View}; + +/// Revalidation request payload sent from the queue to the worker. +enum WorkerPayload +where + Block: BlockT, + Api: ChainApi + 'static, +{ + /// Request to revalidated the given instance of the [`View`] + /// + /// Communication channels with maintain thread are also provided. + RevalidateView(Arc>, FinishRevalidationWorkerChannels), + /// Request to revalidated the given instance of the [`TxMemPool`] at provided block hash. + RevalidateMempool(Arc>, HashAndNumber), +} + +/// The background revalidation worker. +struct RevalidationWorker { + _phantom: PhantomData, +} + +impl RevalidationWorker +where + Block: BlockT, + ::Hash: Unpin, +{ + /// Create a new instance of the background worker. + fn new() -> Self { + Self { _phantom: Default::default() } + } + + /// A background worker main loop. + /// + /// Waits for and dispatches the [`WorkerPayload`] messages sent from the + /// [`RevalidationQueue`]. + pub async fn run + 'static>( + self, + from_queue: TracingUnboundedReceiver>, + ) { + let mut from_queue = from_queue.fuse(); + + loop { + let Some(payload) = from_queue.next().await else { + // R.I.P. worker! + break; + }; + match payload { + WorkerPayload::RevalidateView(view, worker_channels) => + view.revalidate(worker_channels).await, + WorkerPayload::RevalidateMempool(mempool, finalized_hash_and_number) => + mempool.revalidate(finalized_hash_and_number).await, + }; + } + } +} + +/// A Revalidation queue. +/// +/// Allows to send the revalidation requests to the [`RevalidationWorker`]. +pub struct RevalidationQueue +where + Api: ChainApi + 'static, + Block: BlockT, +{ + background: Option>>, +} + +impl RevalidationQueue +where + Api: ChainApi + 'static, + Block: BlockT, + ::Hash: Unpin, +{ + /// New revalidation queue without background worker. + /// + /// All validation requests will be blocking. + pub fn new() -> Self { + Self { background: None } + } + + /// New revalidation queue with background worker. + /// + /// All validation requests will be executed in the background. + pub fn new_with_worker() -> (Self, Pin + Send>>) { + let (to_worker, from_queue) = tracing_unbounded("mpsc_revalidation_queue", 100_000); + (Self { background: Some(to_worker) }, RevalidationWorker::new().run(from_queue).boxed()) + } + + /// Queue the view for later revalidation. + /// + /// If the queue is configured with background worker, this will return immediately. + /// If the queue is configured without background worker, this will resolve after + /// revalidation is actually done. + /// + /// Schedules execution of the [`View::revalidate`]. + pub async fn revalidate_view( + &self, + view: Arc>, + finish_revalidation_worker_channels: FinishRevalidationWorkerChannels, + ) { + log::trace!( + target: LOG_TARGET, + "revalidation_queue::revalidate_view: Sending view to revalidation queue at {}", + view.at.hash + ); + + if let Some(ref to_worker) = self.background { + if let Err(e) = to_worker.unbounded_send(WorkerPayload::RevalidateView( + view, + finish_revalidation_worker_channels, + )) { + log::warn!(target: LOG_TARGET, "revalidation_queue::revalidate_view: Failed to update background worker: {:?}", e); + } + } else { + view.revalidate(finish_revalidation_worker_channels).await + } + } + + /// Revalidates the given mempool instance. + /// + /// If queue configured with background worker, this will return immediately. + /// If queue configured without background worker, this will resolve after + /// revalidation is actually done. + /// + /// Schedules execution of the [`TxMemPool::revalidate`]. + pub async fn revalidate_mempool( + &self, + mempool: Arc>, + finalized_hash: HashAndNumber, + ) { + log::trace!( + target: LOG_TARGET, + "Sent mempool to revalidation queue at hash: {:?}", + finalized_hash + ); + + if let Some(ref to_worker) = self.background { + if let Err(e) = + to_worker.unbounded_send(WorkerPayload::RevalidateMempool(mempool, finalized_hash)) + { + log::warn!(target: LOG_TARGET, "Failed to update background worker: {:?}", e); + } + } else { + mempool.revalidate(finalized_hash).await + } + } +} + +#[cfg(test)] +//todo: add more tests [#5480] +mod tests { + use super::*; + use crate::{ + common::tests::{uxt, TestApi}, + fork_aware_txpool::view::FinishRevalidationLocalChannels, + }; + 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] + fn revalidation_queue_works() { + let api = Arc::new(TestApi::default()); + let block0 = api.expect_hash_and_number(0); + + let view = Arc::new(View::new( + api.clone(), + block0, + Default::default(), + Default::default(), + false.into(), + )); + let queue = Arc::new(RevalidationQueue::new()); + + let uxt = uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: 0, + }); + + let _ = block_on( + view.submit_many(TransactionSource::External, std::iter::once(uxt.clone().into())), + ); + assert_eq!(api.validation_requests().len(), 1); + + let (finish_revalidation_request_tx, finish_revalidation_request_rx) = + tokio::sync::mpsc::channel(1); + let (revalidation_result_tx, revalidation_result_rx) = tokio::sync::mpsc::channel(1); + + let finish_revalidation_worker_channels = FinishRevalidationWorkerChannels::new( + finish_revalidation_request_rx, + revalidation_result_tx, + ); + + let _finish_revalidation_local_channels = FinishRevalidationLocalChannels::new( + finish_revalidation_request_tx, + revalidation_result_rx, + ); + + block_on(queue.revalidate_view(view.clone(), finish_revalidation_worker_channels)); + + assert_eq!(api.validation_requests().len(), 2); + // number of ready + assert_eq!(view.status().ready, 1); + } +} 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 new file mode 100644 index 000000000000..86ea27dcf451 --- /dev/null +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/tx_mem_pool.rs @@ -0,0 +1,535 @@ +// 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 . + +//! Transaction memory pool, container for watched and unwatched transactions. +//! Acts as a buffer which collect transactions before importing them to the views. Following are +//! the crucial use cases when it is needed: +//! - empty pool (no views yet) +//! - potential races between creation of view and submitting transaction (w/o intermediary buffer +//! some transactions could be lost) +//! - the transaction can be invalid on some forks (and thus the associated views may not contain +//! 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 crate::{ + common::log_xt::log_xt_trace, + graph, + graph::{ExtrinsicFor, ExtrinsicHash}, + LOG_TARGET, +}; +use futures::FutureExt; +use itertools::Itertools; +use parking_lot::RwLock; +use sc_transaction_pool_api::TransactionSource; +use sp_blockchain::HashAndNumber; +use sp_runtime::{ + traits::Block as BlockT, + transaction_validity::{InvalidTransaction, TransactionValidityError}, +}; +use std::{ + collections::{hash_map::Entry, HashMap}, + sync::{atomic, atomic::AtomicU64, Arc}, + time::Instant, +}; + +/// The minimum interval between single transaction revalidations. Given in blocks. +pub(crate) const TXMEMPOOL_REVALIDATION_PERIOD: u64 = 10; + +/// The number of transactions revalidated in single revalidation batch. +pub(crate) const TXMEMPOOL_MAX_REVALIDATION_BATCH_SIZE: usize = 1000; + +/// The maximum number of transactions kept in the mem pool. Given as multiple of +/// the view's total limit. +pub const TXMEMPOOL_TRANSACTION_LIMIT_MULTIPLIER: usize = 4; + +/// Represents the transaction in the intermediary buffer. +#[derive(Debug)] +pub(crate) struct TxInMemPool +where + Block: BlockT, + ChainApi: graph::ChainApi + 'static, +{ + //todo: add listener for updating listeners with events [#5495] + /// Is the progress of transaction watched. + /// + /// Was transaction sent with `submit_and_watch`. + watched: bool, + /// Extrinsic actual body. + tx: ExtrinsicFor, + /// Transaction source. + source: TransactionSource, + /// When the transaction was revalidated, used to periodically revalidate the mem pool buffer. + validated_at: 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. + // For sure priority is one of them, but some additional criteria maybe required. + // + // The other maybe simple solution for this could be just obeying 10% limit for future in + // tx_mem_pool. Oldest future transaction could be just dropped. *(Status at finalized would + // also be needed). Probably is_future_at_finalized:Option flag will be enought +} + +impl TxInMemPool +where + Block: BlockT, + ChainApi: graph::ChainApi + 'static, +{ + /// Shall the progress of transaction be watched. + /// + /// Was transaction sent with `submit_and_watch`. + fn is_watched(&self) -> bool { + self.watched + } + + /// Creates a new instance of wrapper for unwatched transaction. + fn new_unwatched(source: TransactionSource, tx: ExtrinsicFor) -> Self { + Self { watched: false, tx, source, validated_at: AtomicU64::new(0) } + } + + /// Creates a new instance of wrapper for watched transaction. + fn new_watched(source: TransactionSource, tx: ExtrinsicFor) -> Self { + Self { watched: true, tx, source, validated_at: AtomicU64::new(0) } + } + + /// Provides a clone of actual transaction body. + /// + /// Operation is cheap, as the body is `Arc`. + pub(crate) fn tx(&self) -> ExtrinsicFor { + self.tx.clone() + } + + /// Returns the source of the transaction. + pub(crate) fn source(&self) -> TransactionSource { + self.source + } +} + +type InternalTxMemPoolMap = + HashMap, Arc>>; +type InternalTxMemPoolMapEntry<'a, ChainApi, Block> = + Entry<'a, ExtrinsicHash, Arc>>; + +/// An intermediary transactions buffer. +/// +/// Keeps all the transaction which are potentially valid. Transactions that were finalized or +/// transactions that are invalid at finalized blocks are removed, either while handling the +/// `Finalized` event, or during revalidation process. +/// +/// All transactions from a`TxMemPool` are submitted to the newly created views. +/// +/// All newly submitted transactions goes into the `TxMemPool`. +pub(super) struct TxMemPool +where + Block: BlockT, + ChainApi: graph::ChainApi + 'static, +{ + /// A shared API instance necessary for blockchain related operations. + api: Arc, + + /// A shared instance of the `MultiViewListener`. + /// + /// Provides a side-channel allowing to send per-transaction state changes notification. + //todo: could be removed after removing watched field (and adding listener into tx) [#5495] + listener: Arc>, + + /// A map that stores the transactions currently in the memory pool. + /// + /// The key is the hash of the transaction, and the value is a wrapper + /// structure, which contains the mempool specific details of the transaction. + transactions: RwLock>, + + /// Prometheus's metrics endpoint. + metrics: PrometheusMetrics, + + /// Indicates the maximum number of transactions that can be maintained in the memory pool. + max_transactions_count: usize, +} + +impl TxMemPool +where + Block: BlockT, + ChainApi: graph::ChainApi + 'static, + ::Hash: Unpin, +{ + /// Creates a new `TxMemPool` instance with the given API, listener, metrics, + /// and max transaction count. + pub(super) fn new( + api: Arc, + listener: Arc>, + metrics: PrometheusMetrics, + max_transactions_count: usize, + ) -> Self { + Self { api, listener, transactions: Default::default(), metrics, max_transactions_count } + } + + /// Creates a new `TxMemPool` instance for testing purposes. + #[allow(dead_code)] + fn new_test(api: Arc, max_transactions_count: usize) -> Self { + Self { + api, + listener: Arc::from(MultiViewListener::new()), + transactions: Default::default(), + metrics: Default::default(), + max_transactions_count, + } + } + + /// Retrieves a transaction by its hash if it exists in the memory pool. + pub(super) fn get_by_hash( + &self, + hash: ExtrinsicHash, + ) -> Option> { + self.transactions.read().get(&hash).map(|t| t.tx()) + } + + /// Returns a tuple with the count of unwatched and watched transactions in the memory pool. + pub(super) fn unwatched_and_watched_count(&self) -> (usize, usize) { + let transactions = self.transactions.read(); + let watched_count = transactions.values().filter(|t| t.is_watched()).count(); + (transactions.len() - watched_count, watched_count) + } + + /// Attempts to insert a transaction into the memory pool, ensuring it does not + /// exceed the maximum allowed transaction count. + fn try_insert( + &self, + current_len: usize, + entry: InternalTxMemPoolMapEntry<'_, ChainApi, Block>, + hash: ExtrinsicHash, + tx: TxInMemPool, + ) -> Result, ChainApi::Error> { + //todo: obey size limits [#5476] + let result = match (current_len < self.max_transactions_count, entry) { + (true, Entry::Vacant(v)) => { + v.insert(Arc::from(tx)); + Ok(hash) + }, + (_, Entry::Occupied(_)) => + Err(sc_transaction_pool_api::error::Error::AlreadyImported(Box::new(hash)).into()), + (false, _) => Err(sc_transaction_pool_api::error::Error::ImmediatelyDropped.into()), + }; + log::trace!(target: LOG_TARGET, "[{:?}] mempool::try_insert: {:?}", hash, result); + + result + } + + /// 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 + /// vector. + pub(super) fn extend_unwatched( + &self, + source: TransactionSource, + xts: Vec>, + ) -> Vec, ChainApi::Error>> { + let mut transactions = self.transactions.write(); + let result = xts + .into_iter() + .map(|xt| { + let hash = self.api.hash_and_length(&xt).0; + self.try_insert( + transactions.len(), + transactions.entry(hash), + hash, + TxInMemPool::new_unwatched(source, xt.clone()), + ) + }) + .collect::>(); + result + } + + /// Adds a new watched transaction to the memory pool if it does not exceed the maximum allowed + /// transaction count. + pub(super) fn push_watched( + &self, + source: TransactionSource, + xt: ExtrinsicFor, + ) -> Result, ChainApi::Error> { + let mut transactions = self.transactions.write(); + let hash = self.api.hash_and_length(&xt).0; + self.try_insert( + transactions.len(), + transactions.entry(hash), + hash, + TxInMemPool::new_watched(source, xt.clone()), + ) + } + + /// 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( + &self, + ) -> HashMap, Arc>> { + self.transactions + .read() + .iter() + .filter_map(|(hash, tx)| (!tx.is_watched()).then(|| (*hash, tx.clone()))) + .collect::>() + } + + /// Clones and returns a `HashMap` of references to all watched transactions in the memory pool. + pub(super) fn clone_watched( + &self, + ) -> HashMap, Arc>> { + self.transactions + .read() + .iter() + .filter_map(|(hash, tx)| (tx.is_watched()).then(|| (*hash, tx.clone()))) + .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); + } + + /// Revalidates a batch of transactions against the provided finalized block. + /// + /// Returns a vector of invalid transaction hashes. + async fn revalidate_inner(&self, finalized_block: HashAndNumber) -> Vec { + log::trace!(target: LOG_TARGET, "mempool::revalidate at:{finalized_block:?}"); + let start = Instant::now(); + + let (count, input) = { + let transactions = self.transactions.read(); + + ( + transactions.len(), + transactions + .clone() + .into_iter() + .filter(|xt| { + let finalized_block_number = finalized_block.number.into().as_u64(); + xt.1.validated_at.load(atomic::Ordering::Relaxed) + + TXMEMPOOL_REVALIDATION_PERIOD < + finalized_block_number + }) + .sorted_by_key(|tx| tx.1.validated_at.load(atomic::Ordering::Relaxed)) + .take(TXMEMPOOL_MAX_REVALIDATION_BATCH_SIZE), + ) + }; + + 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| { + 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(); + + let duration = start.elapsed(); + + let invalid_hashes = validation_results + .into_iter() + .filter_map(|(xt_hash, validation_result)| match validation_result { + Ok(Ok(_)) | + Ok(Err(TransactionValidityError::Invalid(InvalidTransaction::Future))) => None, + Err(_) | + Ok(Err(TransactionValidityError::Unknown(_))) | + Ok(Err(TransactionValidityError::Invalid(_))) => { + log::trace!( + target: LOG_TARGET, + "[{:?}]: Purging: invalid: {:?}", + xt_hash, + validation_result, + ); + Some(xt_hash) + }, + }) + .collect::>(); + + log::debug!( + target: LOG_TARGET, + "mempool::revalidate: at {finalized_block:?} count:{input_len}/{count} purged:{} took {duration:?}", invalid_hashes.len(), + ); + + invalid_hashes + } + + /// Removes the finalized transactions from the memory pool, using a provided list of hashes. + pub(super) async fn purge_finalized_transactions( + &self, + finalized_xts: &Vec>, + ) { + log::debug!(target: LOG_TARGET, "purge_finalized_transactions count:{:?}", finalized_xts.len()); + log_xt_trace!(target: LOG_TARGET, finalized_xts, "[{:?}] purged finalized transactions"); + let mut transactions = self.transactions.write(); + finalized_xts.iter().for_each(|t| { + transactions.remove(t); + }); + } + + /// Revalidates transactions in the memory pool against a given finalized block and removes + /// invalid ones. + pub(super) async fn revalidate(&self, finalized_block: HashAndNumber) { + log::trace!(target: LOG_TARGET, "purge_transactions at:{:?}", finalized_block); + let invalid_hashes = self.revalidate_inner(finalized_block.clone()).await; + + self.metrics.report(|metrics| { + metrics.mempool_revalidation_invalid_txs.inc_by(invalid_hashes.len() as _) + }); + + let mut transactions = self.transactions.write(); + invalid_hashes.iter().for_each(|i| { + transactions.remove(i); + }); + self.listener.invalidate_transactions(&invalid_hashes); + } +} + +#[cfg(test)] +mod tx_mem_pool_tests { + use super::*; + use crate::common::tests::TestApi; + use substrate_test_runtime::{AccountId, Extrinsic, Transfer, H256}; + use substrate_test_runtime_client::AccountKeyring::*; + fn uxt(nonce: u64) -> Extrinsic { + crate::common::tests::uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce, + }) + } + + #[test] + fn extend_unwatched_obeys_limit() { + let max = 10; + let api = Arc::from(TestApi::default()); + let mempool = TxMemPool::new_test(api, max); + + let xts = (0..max + 1).map(|x| Arc::from(uxt(x as _))).collect::>(); + + let results = mempool.extend_unwatched(TransactionSource::External, xts); + assert!(results.iter().take(max).all(Result::is_ok)); + assert!(matches!( + results.into_iter().last().unwrap().unwrap_err(), + sc_transaction_pool_api::error::Error::ImmediatelyDropped + )); + } + + #[test] + fn extend_unwatched_detects_already_imported() { + sp_tracing::try_init_simple(); + let max = 10; + let api = Arc::from(TestApi::default()); + let mempool = TxMemPool::new_test(api, max); + + let mut xts = (0..max - 1).map(|x| Arc::from(uxt(x as _))).collect::>(); + xts.push(xts.iter().last().unwrap().clone()); + + let results = mempool.extend_unwatched(TransactionSource::External, xts); + assert!(results.iter().take(max - 1).all(Result::is_ok)); + assert!(matches!( + results.into_iter().last().unwrap().unwrap_err(), + sc_transaction_pool_api::error::Error::AlreadyImported(_) + )); + } + + #[test] + fn push_obeys_limit() { + let max = 10; + let api = Arc::from(TestApi::default()); + let mempool = TxMemPool::new_test(api, max); + + let xts = (0..max).map(|x| Arc::from(uxt(x as _))).collect::>(); + + let results = mempool.extend_unwatched(TransactionSource::External, xts); + assert!(results.iter().all(Result::is_ok)); + + let xt = Arc::from(uxt(98)); + let result = mempool.push_watched(TransactionSource::External, xt); + assert!(matches!( + result.unwrap_err(), + sc_transaction_pool_api::error::Error::ImmediatelyDropped + )); + let xt = Arc::from(uxt(99)); + let mut result = mempool.extend_unwatched(TransactionSource::External, vec![xt]); + assert!(matches!( + result.pop().unwrap().unwrap_err(), + sc_transaction_pool_api::error::Error::ImmediatelyDropped + )); + } + + #[test] + fn push_detects_already_imported() { + let max = 10; + let api = Arc::from(TestApi::default()); + let mempool = TxMemPool::new_test(api, 2 * max); + + let xts = (0..max).map(|x| Arc::from(uxt(x as _))).collect::>(); + let xt0 = xts.iter().last().unwrap().clone(); + let xt1 = xts.iter().next().unwrap().clone(); + + let results = mempool.extend_unwatched(TransactionSource::External, xts); + assert!(results.iter().all(Result::is_ok)); + + let result = mempool.push_watched(TransactionSource::External, xt0); + assert!(matches!( + result.unwrap_err(), + sc_transaction_pool_api::error::Error::AlreadyImported(_) + )); + let mut result = mempool.extend_unwatched(TransactionSource::External, vec![xt1]); + assert!(matches!( + result.pop().unwrap().unwrap_err(), + sc_transaction_pool_api::error::Error::AlreadyImported(_) + )); + } + + #[test] + fn count_works() { + let max = 100; + let api = Arc::from(TestApi::default()); + let mempool = TxMemPool::new_test(api, max); + + let xts0 = (0..10).map(|x| Arc::from(uxt(x as _))).collect::>(); + + let results = mempool.extend_unwatched(TransactionSource::External, xts0); + assert!(results.iter().all(Result::is_ok)); + + let xts1 = (0..5).map(|x| Arc::from(uxt(2 * x))).collect::>(); + let results = xts1 + .into_iter() + .map(|t| mempool.push_watched(TransactionSource::External, t)) + .collect::>(); + assert!(results.iter().all(Result::is_ok)); + assert_eq!(mempool.unwatched_and_watched_count(), (10, 5)); + } +} diff --git a/substrate/client/transaction-pool/src/fork_aware_txpool/view.rs b/substrate/client/transaction-pool/src/fork_aware_txpool/view.rs new file mode 100644 index 000000000000..fd5bfa8312c0 --- /dev/null +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/view.rs @@ -0,0 +1,415 @@ +// 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 . + +//! Transaction pool view. +//! +//! The View represents the state of the transaction pool at given block. The view is created when +//! new block is notified to transaction pool. Views are removed on finalization. +//! +//! Refer to [*View*](../index.html#view) section for more details. + +use super::metrics::MetricsLink as PrometheusMetrics; +use crate::{ + common::log_xt::log_xt_trace, + graph::{ + self, watcher::Watcher, ExtrinsicFor, ExtrinsicHash, IsValidator, ValidatedTransaction, + ValidatedTransactionFor, + }, + LOG_TARGET, +}; +use parking_lot::Mutex; +use sc_transaction_pool_api::{PoolStatus, TransactionSource}; +use sp_blockchain::HashAndNumber; +use sp_runtime::{ + traits::Block as BlockT, transaction_validity::TransactionValidityError, SaturatedConversion, +}; +use std::{collections::HashMap, sync::Arc, time::Instant}; + +pub(super) struct RevalidationResult { + revalidated: HashMap, ValidatedTransactionFor>, + invalid_hashes: Vec>, +} + +/// Used to obtain result from RevalidationWorker on View side. +pub(super) type RevalidationResultReceiver = + tokio::sync::mpsc::Receiver>; + +/// Used to send revalidation result from RevalidationWorker to View. +pub(super) type RevalidationResultSender = + tokio::sync::mpsc::Sender>; + +/// Used to receive finish-revalidation-request from View on RevalidationWorker side. +pub(super) type FinishRevalidationRequestReceiver = tokio::sync::mpsc::Receiver<()>; + +/// Used to send finish-revalidation-request from View to RevalidationWorker. +pub(super) type FinishRevalidationRequestSender = tokio::sync::mpsc::Sender<()>; + +/// Endpoints of channels used on View side (maintain thread) +pub(super) struct FinishRevalidationLocalChannels { + /// Used to send finish revalidation request. + finish_revalidation_request_tx: Option, + /// Used to receive revalidation results. + revalidation_result_rx: RevalidationResultReceiver, +} + +impl FinishRevalidationLocalChannels { + /// Creates a new instance of endpoints for channels used on View side + pub fn new( + finish_revalidation_request_tx: FinishRevalidationRequestSender, + revalidation_result_rx: RevalidationResultReceiver, + ) -> Self { + Self { + finish_revalidation_request_tx: Some(finish_revalidation_request_tx), + revalidation_result_rx, + } + } + + /// Removes a finish revalidation sender + /// + /// Should be called when revalidation was already terminated and finish revalidation message is + /// no longer expected. + fn remove_sender(&mut self) { + self.finish_revalidation_request_tx = None; + } +} + +/// Endpoints of channels used on `RevalidationWorker` side (background thread) +pub(super) struct FinishRevalidationWorkerChannels { + /// Used to receive finish revalidation request. + finish_revalidation_request_rx: FinishRevalidationRequestReceiver, + /// Used to send revalidation results. + revalidation_result_tx: RevalidationResultSender, +} + +impl FinishRevalidationWorkerChannels { + /// Creates a new instance of endpoints for channels used on `RevalidationWorker` side + pub fn new( + finish_revalidation_request_rx: FinishRevalidationRequestReceiver, + revalidation_result_tx: RevalidationResultSender, + ) -> Self { + Self { finish_revalidation_request_rx, revalidation_result_tx } + } +} + +/// Represents the state of transaction pool for given block. +/// +/// Refer to [*View*](../index.html#view) section for more details on the purpose and life cycle of +/// the `View`. +pub(super) struct View { + /// The internal pool keeping the set of ready and future transaction at the given block. + pub(super) pool: graph::Pool, + /// The hash and number of the block with which this view is associated. + pub(super) at: HashAndNumber, + /// Endpoints of communication channel with background worker. + revalidation_worker_channels: Mutex>>, + /// Prometheus's metrics endpoint. + metrics: PrometheusMetrics, +} + +impl View +where + ChainApi: graph::ChainApi, + ::Hash: Unpin, +{ + /// Creates a new empty view. + pub(super) fn new( + api: Arc, + at: HashAndNumber, + options: graph::Options, + metrics: PrometheusMetrics, + is_validator: IsValidator, + ) -> Self { + metrics.report(|metrics| metrics.non_cloned_views.inc()); + Self { + pool: graph::Pool::new(options, is_validator, api), + at, + revalidation_worker_channels: Mutex::from(None), + metrics, + } + } + + /// Creates a copy of the other view. + pub(super) fn new_from_other(&self, at: &HashAndNumber) -> Self { + View { + at: at.clone(), + pool: self.pool.deep_clone(), + revalidation_worker_channels: Mutex::from(None), + metrics: self.metrics.clone(), + } + } + + /// Imports many unvalidated extrinsics into the view. + pub(super) async fn submit_many( + &self, + source: TransactionSource, + 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 + } else { + self.pool.submit_at(&self.at, source, 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, + xt: ExtrinsicFor, + ) -> Result, ExtrinsicHash>, 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 + } + + /// Status of the pool associated with the view. + pub(super) fn status(&self) -> PoolStatus { + self.pool.validated_pool().status() + } + + /// Creates a watcher for given transaction. + /// + /// Intended to be called for the transaction that already exists in the pool + pub(super) fn create_watcher( + &self, + tx_hash: ExtrinsicHash, + ) -> Watcher, ExtrinsicHash> { + //todo(minor): some assert could be added here - to make sure that transaction actually + // exists in the view. + self.pool.validated_pool().create_watcher(tx_hash) + } + + /// Revalidates some part of transaction from the internal pool. + /// + /// Intended to be called from the revalidation worker. The revalidation process can be + /// terminated by sending a message to the `rx` channel provided within + /// `finish_revalidation_worker_channels`. Revalidation results are sent back over the `tx` + /// channels and shall be applied in maintain thread. + /// + /// View revalidation currently is not throttled, and until not terminated it will revalidate + /// all the transactions. Note: this can be improved if CPU usage due to revalidation becomes a + /// problem. + pub(super) async fn revalidate( + &self, + finish_revalidation_worker_channels: FinishRevalidationWorkerChannels, + ) { + let FinishRevalidationWorkerChannels { + mut finish_revalidation_request_rx, + revalidation_result_tx, + } = finish_revalidation_worker_channels; + + log::trace!(target:LOG_TARGET, "view::revalidate: at {} starting", self.at.hash); + let start = Instant::now(); + let validated_pool = self.pool.validated_pool(); + let api = validated_pool.api(); + + let batch: Vec<_> = validated_pool.ready().collect(); + let batch_len = batch.len(); + + //todo: sort batch by revalidation timestamp | maybe not needed at all? xts will be getting + //out of the view... + //todo: revalidate future, remove if invalid [#5496] + + let mut invalid_hashes = Vec::new(); + let mut revalidated = HashMap::new(); + + let mut validation_results = vec![]; + let mut batch_iter = batch.into_iter(); + loop { + let mut should_break = false; + tokio::select! { + _ = finish_revalidation_request_rx.recv() => { + log::trace!(target: LOG_TARGET, "view::revalidate: finish revalidation request received at {}.", self.at.hash); + break + } + _ = 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); + validation_results.push(validation_result); + } else { + { + self.revalidation_worker_channels.lock().as_mut().map(|ch| ch.remove_sender()); + } + should_break = true; + } + } => {} + } + + if should_break { + break; + } + } + + let revalidation_duration = start.elapsed(); + self.metrics.report(|metrics| { + metrics.view_revalidation_duration.observe(revalidation_duration.as_secs_f64()); + }); + log::debug!( + target:LOG_TARGET, + "view::revalidate: at {:?} count: {}/{} took {:?}", + self.at.hash, + validation_results.len(), + batch_len, + revalidation_duration + ); + log_xt_trace!(data:tuple, target:LOG_TARGET, validation_results.iter().map(|x| (x.1, &x.0)), "[{:?}] view::revalidateresult: {:?}"); + + for (validation_result, tx_hash, tx) in validation_results { + match validation_result { + Ok(Err(TransactionValidityError::Invalid(_))) => { + invalid_hashes.push(tx_hash); + }, + Ok(Ok(validity)) => { + revalidated.insert( + tx_hash, + ValidatedTransaction::valid_at( + self.at.number.saturated_into::(), + tx_hash, + tx.source, + tx.data.clone(), + api.hash_and_length(&tx.data).1, + validity, + ), + ); + }, + Ok(Err(TransactionValidityError::Unknown(e))) => { + log::trace!( + target: LOG_TARGET, + "[{:?}]: Removing. Cannot determine transaction validity: {:?}", + tx_hash, + e + ); + invalid_hashes.push(tx_hash); + }, + Err(validation_err) => { + log::trace!( + target: LOG_TARGET, + "[{:?}]: Removing due to error during revalidation: {}", + tx_hash, + validation_err + ); + invalid_hashes.push(tx_hash); + }, + } + } + + log::trace!(target:LOG_TARGET, "view::revalidate: sending revalidation result at {}", self.at.hash); + if let Err(e) = revalidation_result_tx + .send(RevalidationResult { invalid_hashes, revalidated }) + .await + { + log::trace!(target:LOG_TARGET, "view::revalidate: sending revalidation_result at {} failed {:?}", self.at.hash, e); + } + } + + /// Sends revalidation request to the background worker. + /// + /// Creates communication channels required to stop revalidation request and receive the + /// revalidation results and sends the revalidation request to the background worker. + /// + /// Intended to be called from maintain thread, at the very end of the maintain process. + /// + /// Refer to [*View revalidation*](../index.html#view-revalidation) for more details. + pub(super) async fn start_background_revalidation( + view: Arc, + revalidation_queue: Arc< + super::revalidation_worker::RevalidationQueue, + >, + ) { + log::trace!(target:LOG_TARGET,"view::start_background_revalidation: at {}", view.at.hash); + let (finish_revalidation_request_tx, finish_revalidation_request_rx) = + tokio::sync::mpsc::channel(1); + let (revalidation_result_tx, revalidation_result_rx) = tokio::sync::mpsc::channel(1); + + let finish_revalidation_worker_channels = FinishRevalidationWorkerChannels::new( + finish_revalidation_request_rx, + revalidation_result_tx, + ); + + let finish_revalidation_local_channels = FinishRevalidationLocalChannels::new( + finish_revalidation_request_tx, + revalidation_result_rx, + ); + + *view.revalidation_worker_channels.lock() = Some(finish_revalidation_local_channels); + revalidation_queue + .revalidate_view(view.clone(), finish_revalidation_worker_channels) + .await; + } + + /// Terminates a background view revalidation. + /// + /// Receives the results from the background worker and applies them to the internal pool. + /// Intended to be called from the maintain thread, at the very beginning of the maintain + /// process, before the new view is cloned and updated. Applying results before cloning ensures + /// that view contains up-to-date set of revalidated transactions. + /// + /// Refer to [*View revalidation*](../index.html#view-revalidation) for more details. + pub(super) async fn finish_revalidation(&self) { + log::trace!(target:LOG_TARGET,"view::finish_revalidation: at {}", self.at.hash); + let Some(revalidation_worker_channels) = self.revalidation_worker_channels.lock().take() + else { + log::trace!(target:LOG_TARGET, "view::finish_revalidation: no finish_revalidation_request_tx"); + return + }; + + let FinishRevalidationLocalChannels { + finish_revalidation_request_tx, + mut revalidation_result_rx, + } = revalidation_worker_channels; + + if let Some(finish_revalidation_request_tx) = finish_revalidation_request_tx { + if let Err(e) = finish_revalidation_request_tx.send(()).await { + log::trace!(target:LOG_TARGET, "view::finish_revalidation: sending cancellation request at {} failed {:?}", self.at.hash, e); + } + } + + if let Some(revalidation_result) = revalidation_result_rx.recv().await { + let start = Instant::now(); + let revalidated_len = revalidation_result.revalidated.len(); + let validated_pool = self.pool.validated_pool(); + validated_pool.remove_invalid(&revalidation_result.invalid_hashes); + if revalidated_len > 0 { + self.pool.resubmit(revalidation_result.revalidated); + } + + self.metrics.report(|metrics| { + let _ = ( + revalidation_result + .invalid_hashes + .len() + .try_into() + .map(|v| metrics.view_revalidation_invalid_txs.inc_by(v)), + revalidated_len + .try_into() + .map(|v| metrics.view_revalidation_resubmitted_txs.inc_by(v)), + ); + }); + + log::debug!( + target:LOG_TARGET, + "view::finish_revalidation: applying revalidation result invalid: {} revalidated: {} at {:?} took {:?}", + revalidation_result.invalid_hashes.len(), + revalidated_len, + self.at.hash, + start.elapsed() + ); + } + } +} 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 new file mode 100644 index 000000000000..953d6d860338 --- /dev/null +++ b/substrate/client/transaction-pool/src/fork_aware_txpool/view_store.rs @@ -0,0 +1,468 @@ +// 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 . + +//! Transaction pool view store. Basically block hash to view map with some utility methods. + +use super::{ + multi_view_listener::{MultiViewListener, TxStatusStream}, + view::View, +}; +use crate::{ + fork_aware_txpool::dropped_watcher::MultiViewDroppedWatcherController, + graph, + graph::{base_pool::Transaction, ExtrinsicFor, ExtrinsicHash, TransactionFor}, + ReadyIteratorFor, LOG_TARGET, +}; +use futures::prelude::*; +use parking_lot::RwLock; +use sc_transaction_pool_api::{error::Error as PoolError, PoolStatus, TransactionSource}; +use sp_blockchain::TreeRoute; +use sp_runtime::{generic::BlockId, traits::Block as BlockT}; +use std::{collections::HashMap, sync::Arc, time::Instant}; + +/// The helper structure encapsulates all the views. +pub(super) struct ViewStore +where + Block: BlockT, + ChainApi: graph::ChainApi, +{ + /// The blockchain api. + pub(super) api: Arc, + /// Active views at tips of the forks. + /// + /// Active views are updated with incoming transactions. + pub(super) active_views: RwLock>>>, + /// Inactive views at intermediary blocks that are no longer tips of the forks. + /// + /// Inactive views are not updated with incoming transactions, while they can still be used to + /// build new blocks upon them. + pub(super) inactive_views: RwLock>>>, + /// Listener for controlling external watchers of transactions. + /// + /// Provides a side-channel allowing to send per-transaction state changes notification. + pub(super) listener: Arc>, + /// Most recent block processed by tx-pool. Used in the API functions that were not changed to + /// add `at` parameter. + pub(super) most_recent_view: RwLock>, + /// The controller of multi view dropped stream. + pub(super) dropped_stream_controller: MultiViewDroppedWatcherController, +} + +impl ViewStore +where + Block: BlockT, + ChainApi: graph::ChainApi + 'static, + ::Hash: Unpin, +{ + /// Creates a new empty view store. + pub(super) fn new( + api: Arc, + listener: Arc>, + dropped_stream_controller: MultiViewDroppedWatcherController, + ) -> Self { + Self { + api, + active_views: Default::default(), + inactive_views: Default::default(), + listener, + most_recent_view: RwLock::from(None), + dropped_stream_controller, + } + } + + /// Imports a bunch of unverified extrinsics to every active view. + pub(super) async fn submit( + &self, + source: TransactionSource, + xts: impl IntoIterator> + Clone, + xts_hashes: impl IntoIterator> + Clone, + ) -> HashMap, ChainApi::Error>>> { + let submit_futures = { + let active_views = self.active_views.read(); + active_views + .iter() + .map(|(_, view)| { + let view = view.clone(); + let xts = xts.clone(); + self.dropped_stream_controller + .add_initial_views(xts_hashes.clone(), view.at.hash); + async move { (view.at.hash, view.submit_many(source, xts.clone()).await) } + }) + .collect::>() + }; + let results = futures::future::join_all(submit_futures).await; + + HashMap::<_, _>::from_iter(results.into_iter()) + } + + /// Import a single extrinsic and starts to watch its progress in the pool. + /// + /// The extrinsic is imported to every view, and the individual streams providing the progress + /// of this transaction within every view are added to the multi view listener. + /// + /// The external stream of aggregated/processed events provided by the `MultiViewListener` + /// instance is returned. + pub(super) async fn submit_and_watch( + &self, + _at: Block::Hash, + source: TransactionSource, + xt: ExtrinsicFor, + ) -> Result, (ChainApi::Error, Option>)> { + 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(), None)) + }; + let submit_and_watch_futures = { + let active_views = self.active_views.read(); + active_views + .iter() + .map(|(_, view)| { + let view = view.clone(); + let xt = xt.clone(); + self.dropped_stream_controller + .add_initial_views(std::iter::once(tx_hash), view.at.hash); + async move { + match view.submit_and_watch(source, xt).await { + Ok(watcher) => { + self.listener.add_view_watcher_for_tx( + tx_hash, + view.at.hash, + watcher.into_stream().boxed(), + ); + Ok(()) + }, + Err(e) => Err(e), + } + } + }) + .collect::>() + }; + let maybe_error = futures::future::join_all(submit_and_watch_futures) + .await + .into_iter() + .reduce(|mut r, v| { + if r.is_err() && v.is_ok() { + r = v; + } + r + }); + if let Some(Err(err)) = maybe_error { + log::trace!(target: LOG_TARGET, "[{:?}] submit_and_watch: err: {}", tx_hash, err); + return Err((err, Some(external_watcher))); + }; + + Ok(external_watcher) + } + + /// Returns the pool status for every active view. + pub(super) fn status(&self) -> HashMap { + self.active_views.read().iter().map(|(h, v)| (*h, v.status())).collect() + } + + /// Returns true if there are no active views. + pub(super) fn is_empty(&self) -> bool { + self.active_views.read().is_empty() && self.inactive_views.read().is_empty() + } + + /// Finds the best existing active view to clone from along the path. + /// + /// ```text + /// Tree route from R1 to E2. + /// <- R3 <- R2 <- R1 + /// / + /// C + /// \-> E1 -> E2 + /// ``` + /// ```text + /// Search path is: + /// [E1, C, R3, R2, R1] + /// ``` + pub(super) fn find_best_view( + &self, + tree_route: &TreeRoute, + ) -> Option>> { + let active_views = self.active_views.read(); + let best_view = { + tree_route + .retracted() + .iter() + .chain(std::iter::once(tree_route.common_block())) + .chain(tree_route.enacted().iter()) + .rev() + .find(|block| active_views.contains_key(&block.hash)) + }; + best_view.map(|h| { + active_views + .get(&h.hash) + .expect("hash was just found in the map's keys. qed") + .clone() + }) + } + + /// Returns an iterator for ready transactions for the most recently notified best block. + /// + /// The iterator for future transactions is returned if the most recently notified best block, + /// for which maintain process was accomplished, exists. + pub(super) fn ready(&self) -> ReadyIteratorFor { + let ready_iterator = self + .most_recent_view + .read() + .map(|at| self.get_view_at(at, true)) + .flatten() + .map(|(v, _)| v.pool.validated_pool().ready()); + + if let Some(ready_iterator) = ready_iterator { + return Box::new(ready_iterator) + } else { + return Box::new(std::iter::empty()) + } + } + + /// Returns a list of future transactions for the most recently notified best block. + /// + /// The set of future transactions is returned if the most recently notified best block, for + /// which maintain process was accomplished, exists. + pub(super) fn futures( + &self, + ) -> Vec, ExtrinsicFor>> { + self.most_recent_view + .read() + .map(|at| self.get_view_at(at, true)) + .flatten() + .map(|(v, _)| v.pool.validated_pool().pool.read().futures().cloned().collect()) + .unwrap_or_default() + } + + /// Collects all the transactions included in the blocks on the provided `tree_route` and + /// triggers finalization event for them. + /// + /// The finalization event is sent using side-channel of the multi view `listener`. + /// + /// Returns the list of finalized transactions hashes. + pub(super) async fn finalize_route( + &self, + finalized_hash: Block::Hash, + tree_route: &[Block::Hash], + ) -> Vec> { + log::trace!(target: LOG_TARGET, "finalize_route finalized_hash:{finalized_hash:?} tree_route: {tree_route:?}"); + + let mut finalized_transactions = Vec::new(); + + for block in tree_route.iter().chain(std::iter::once(&finalized_hash)) { + let extrinsics = self + .api + .block_body(*block) + .await + .unwrap_or_else(|e| { + log::warn!(target: LOG_TARGET, "Finalize route: error request: {}", e); + None + }) + .unwrap_or_default() + .iter() + .map(|e| self.api.hash_and_length(&e).0) + .collect::>(); + + extrinsics + .iter() + .enumerate() + .for_each(|(i, tx_hash)| self.listener.finalize_transaction(*tx_hash, *block, i)); + + finalized_transactions.extend(extrinsics); + } + + finalized_transactions + } + + /// Return specific ready transaction by hash, if there is one. + /// + /// Currently the ready transaction is returned if it exists for the most recently notified best + /// block (for which maintain process was accomplished). + pub(super) fn ready_transaction( + &self, + at: Block::Hash, + tx_hash: &ExtrinsicHash, + ) -> Option> { + self.active_views + .read() + .get(&at) + .and_then(|v| v.pool.validated_pool().ready_by_hash(tx_hash)) + } + + /// Inserts new view into the view store. + /// + /// All the views associated with the blocks which are on enacted path (including common + /// ancestor) will be: + /// - 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. + pub(super) async fn insert_new_view( + &self, + view: Arc>, + tree_route: &TreeRoute, + ) { + //note: most_recent_view must be synced with changes in in/active_views. + { + let mut most_recent_view_lock = self.most_recent_view.write(); + let mut active_views = self.active_views.write(); + let mut inactive_views = self.inactive_views.write(); + + std::iter::once(tree_route.common_block()) + .chain(tree_route.enacted().iter()) + .map(|block| block.hash) + .for_each(|hash| { + active_views.remove(&hash).map(|view| { + inactive_views.insert(hash, view); + }); + }); + active_views.insert(view.at.hash, view.clone()); + most_recent_view_lock.replace(view.at.hash); + }; + log::trace!(target:LOG_TARGET,"insert_new_view: inactive_views: {:?}", self.inactive_views.read().keys()); + } + + /// Returns an optional reference to the view at given hash. + /// + /// If `allow_retracted` flag is set, inactive views are also searched. + /// + /// If the view at provided hash does not exist `None` is returned. + pub(super) fn get_view_at( + &self, + at: Block::Hash, + allow_inactive: bool, + ) -> Option<(Arc>, bool)> { + if let Some(view) = self.active_views.read().get(&at) { + return Some((view.clone(), false)); + } + if allow_inactive { + if let Some(view) = self.inactive_views.read().get(&at) { + return Some((view.clone(), true)) + } + }; + None + } + + /// The pre-finalization event handle for the view store. + /// + /// This function removes the references to the views that will be removed during finalization + /// from the dropped stream controller. This will allow for correct dispatching of `Dropped` + /// events. + pub(crate) async fn handle_pre_finalized(&self, finalized_hash: Block::Hash) { + let finalized_number = self.api.block_id_to_number(&BlockId::Hash(finalized_hash)); + let mut removed_views = vec![]; + + { + self.active_views + .read() + .iter() + .filter(|(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, + }) + .map(|(_, v)| removed_views.push(v.at.hash)) + .for_each(drop); + } + + { + self.inactive_views + .read() + .iter() + .filter(|(_, v)| !match finalized_number { + Err(_) | Ok(None) => false, + Ok(Some(n)) => v.at.number >= n, + }) + .map(|(_, v)| removed_views.push(v.at.hash)) + .for_each(drop); + } + + log::trace!(target:LOG_TARGET,"handle_pre_finalized: removed_views: {:?}", removed_views); + + removed_views.iter().for_each(|view| { + self.dropped_stream_controller.remove_view(*view); + }); + } + + /// The finalization event handle for the view store. + /// + /// Views that have associated block number less than finalized block number are removed from + /// both active and inactive set. + /// + /// Note: the views with the associated number greater than finalized block number on the forks + /// that are not finalized will stay in the view store. They will be removed in the future, once + /// new finalized blocks will be notified. This is to avoid scanning for common ancestors. + /// + /// All watched transactions in the blocks from the tree_route will be notified with `Finalized` + /// event. + /// + /// Returns the list of hashes of all finalized transactions along the provided `tree_route`. + pub(crate) async fn handle_finalized( + &self, + finalized_hash: Block::Hash, + tree_route: &[Block::Hash], + ) -> Vec> { + let finalized_xts = self.finalize_route(finalized_hash, tree_route).await; + let finalized_number = self.api.block_id_to_number(&BlockId::Hash(finalized_hash)); + + //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(); + inactive_views.retain(|_, v| match finalized_number { + Err(_) | Ok(None) => false, + Ok(Some(n)) => v.at.number >= n, + }); + + log::trace!(target:LOG_TARGET,"handle_finalized: inactive_views: {:?}", inactive_views.keys()); + } + + self.listener.remove_view(finalized_hash); + self.listener.remove_stale_controllers(); + self.dropped_stream_controller.remove_finalized_txs(finalized_xts.clone()); + + finalized_xts + } + + /// Terminates all the ongoing background views revalidations triggered at the end of maintain + /// process. + /// + /// Refer to [*View revalidation*](../index.html#view-revalidation) for more details. + pub(crate) async fn finish_background_revalidations(&self) { + let start = Instant::now(); + let finish_revalidation_futures = { + let active_views = self.active_views.read(); + active_views + .iter() + .map(|(_, view)| { + let view = view.clone(); + async move { view.finish_revalidation().await } + }) + .collect::>() + }; + futures::future::join_all(finish_revalidation_futures).await; + log::trace!(target:LOG_TARGET,"finish_background_revalidations took {:?}", start.elapsed()); + } +} diff --git a/substrate/client/transaction-pool/src/graph/base_pool.rs b/substrate/client/transaction-pool/src/graph/base_pool.rs index 32885622da42..e4c3a6c425a9 100644 --- a/substrate/client/transaction-pool/src/graph/base_pool.rs +++ b/substrate/client/transaction-pool/src/graph/base_pool.rs @@ -23,7 +23,7 @@ use std::{cmp::Ordering, collections::HashSet, fmt, hash, sync::Arc}; use crate::LOG_TARGET; -use log::{debug, trace, warn}; +use log::{trace, warn}; use sc_transaction_pool_api::{error, InPoolTransaction, PoolStatus}; use serde::Serialize; use sp_core::hexdisplay::HexDisplay; @@ -207,7 +207,7 @@ const RECENTLY_PRUNED_TAGS: usize = 2; /// as-is for the second time will fail or produce unwanted results. /// Most likely it is required to revalidate them and recompute set of /// required tags. -#[derive(Debug)] +#[derive(Clone, Debug)] pub struct BasePool { reject_future_transactions: bool, future: FutureTransactions, @@ -238,6 +238,12 @@ impl BasePool BasePool BasePool if first { - debug!(target: LOG_TARGET, "[{:?}] Error importing: {:?}", current_hash, e); + trace!(target: LOG_TARGET, "[{:?}] Error importing: {:?}", current_hash, e); return Err(e) } else { failed.push(current_hash); @@ -347,7 +353,7 @@ impl BasePool BasePool Vec>> { let mut removed = self.ready.remove_subtree(hashes); @@ -463,8 +469,8 @@ impl BasePool) -> PruneStatus { @@ -474,6 +480,9 @@ impl BasePool>(); + let futures_removed = self.future.prune_tags(&tags); + for tag in tags { // make sure to promote any future transactions that could be unlocked to_import.append(&mut self.future.satisfy_tags(std::iter::once(&tag))); @@ -485,6 +494,10 @@ impl BasePool> = Transaction { - data: vec![], - bytes: 1, - hash: 1u64, - priority: 5u64, - valid_till: 64u64, - requires: vec![], - provides: vec![], - propagate: true, - source: Source::External, - }; + fn default_tx() -> Transaction> { + Transaction { + data: vec![], + bytes: 1, + hash: 1u64, + priority: 5u64, + valid_till: 64u64, + requires: vec![], + provides: vec![], + propagate: true, + source: Source::External, + } + } + + #[test] + fn prune_for_ready_works() { + // given + let mut pool = pool(); + + // when + pool.import(Transaction { + data: vec![1u8].into(), + provides: vec![vec![2]], + ..default_tx().clone() + }) + .unwrap(); + + // then + assert_eq!(pool.ready().count(), 1); + assert_eq!(pool.ready.len(), 1); + + let result = pool.prune_tags(vec![vec![2]]); + assert_eq!(pool.ready().count(), 0); + assert_eq!(pool.ready.len(), 0); + assert_eq!(result.pruned.len(), 1); + assert_eq!(result.failed.len(), 0); + assert_eq!(result.promoted.len(), 0); + } + + #[test] + fn prune_for_future_works() { + // given + let mut pool = pool(); + + // when + pool.import(Transaction { + data: vec![1u8].into(), + requires: vec![vec![1]], + provides: vec![vec![2]], + hash: 0xaa, + ..default_tx().clone() + }) + .unwrap(); + + // then + assert_eq!(pool.futures().count(), 1); + assert_eq!(pool.future.len(), 1); + + let result = pool.prune_tags(vec![vec![2]]); + assert_eq!(pool.ready().count(), 0); + assert_eq!(pool.ready.len(), 0); + assert_eq!(pool.futures().count(), 0); + assert_eq!(pool.future.len(), 0); + + assert_eq!(result.pruned.len(), 0); + assert_eq!(result.failed.len(), 1); + assert_eq!(result.failed[0], 0xaa); + assert_eq!(result.promoted.len(), 0); + } #[test] fn should_import_transaction_to_ready() { @@ -557,8 +628,12 @@ mod tests { let mut pool = pool(); // when - pool.import(Transaction { data: vec![1u8], provides: vec![vec![1]], ..DEFAULT_TX.clone() }) - .unwrap(); + pool.import(Transaction { + data: vec![1u8].into(), + provides: vec![vec![1]], + ..default_tx().clone() + }) + .unwrap(); // then assert_eq!(pool.ready().count(), 1); @@ -571,10 +646,18 @@ mod tests { let mut pool = pool(); // when - pool.import(Transaction { data: vec![1u8], provides: vec![vec![1]], ..DEFAULT_TX.clone() }) - .unwrap(); - pool.import(Transaction { data: vec![1u8], provides: vec![vec![1]], ..DEFAULT_TX.clone() }) - .unwrap_err(); + pool.import(Transaction { + data: vec![1u8].into(), + provides: vec![vec![1]], + ..default_tx().clone() + }) + .unwrap(); + pool.import(Transaction { + data: vec![1u8].into(), + provides: vec![vec![1]], + ..default_tx().clone() + }) + .unwrap_err(); // then assert_eq!(pool.ready().count(), 1); @@ -588,19 +671,19 @@ mod tests { // when pool.import(Transaction { - data: vec![1u8], + data: vec![1u8].into(), requires: vec![vec![0]], provides: vec![vec![1]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); assert_eq!(pool.ready().count(), 0); assert_eq!(pool.ready.len(), 0); pool.import(Transaction { - data: vec![2u8], + data: vec![2u8].into(), hash: 2, provides: vec![vec![0]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); @@ -616,33 +699,33 @@ mod tests { // when pool.import(Transaction { - data: vec![1u8], + data: vec![1u8].into(), requires: vec![vec![0]], provides: vec![vec![1]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); pool.import(Transaction { - data: vec![3u8], + data: vec![3u8].into(), hash: 3, requires: vec![vec![2]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); pool.import(Transaction { - data: vec![2u8], + data: vec![2u8].into(), hash: 2, requires: vec![vec![1]], provides: vec![vec![3], vec![2]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); pool.import(Transaction { - data: vec![4u8], + data: vec![4u8].into(), hash: 4, priority: 1_000u64, requires: vec![vec![3], vec![4]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); assert_eq!(pool.ready().count(), 0); @@ -650,10 +733,10 @@ mod tests { let res = pool .import(Transaction { - data: vec![5u8], + data: vec![5u8].into(), hash: 5, provides: vec![vec![0], vec![4]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); @@ -682,18 +765,18 @@ mod tests { // given let mut pool = pool(); pool.import(Transaction { - data: vec![1u8], + data: vec![1u8].into(), requires: vec![vec![0]], provides: vec![vec![1]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); pool.import(Transaction { - data: vec![3u8], + data: vec![3u8].into(), hash: 3, requires: vec![vec![1]], provides: vec![vec![2]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); assert_eq!(pool.ready().count(), 0); @@ -701,11 +784,11 @@ mod tests { // when pool.import(Transaction { - data: vec![2u8], + data: vec![2u8].into(), hash: 2, requires: vec![vec![2]], provides: vec![vec![0]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); @@ -720,11 +803,11 @@ mod tests { // let's close the cycle with one additional transaction let res = pool .import(Transaction { - data: vec![4u8], + data: vec![4u8].into(), hash: 4, priority: 50u64, provides: vec![vec![0]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); let mut it = pool.ready().into_iter().map(|tx| tx.data[0]); @@ -744,18 +827,18 @@ mod tests { // given let mut pool = pool(); pool.import(Transaction { - data: vec![1u8], + data: vec![1u8].into(), requires: vec![vec![0]], provides: vec![vec![1]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); pool.import(Transaction { - data: vec![3u8], + data: vec![3u8].into(), hash: 3, requires: vec![vec![1]], provides: vec![vec![2]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); assert_eq!(pool.ready().count(), 0); @@ -763,11 +846,11 @@ mod tests { // when pool.import(Transaction { - data: vec![2u8], + data: vec![2u8].into(), hash: 2, requires: vec![vec![2]], provides: vec![vec![0]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); @@ -782,11 +865,11 @@ mod tests { // let's close the cycle with one additional transaction let err = pool .import(Transaction { - data: vec![4u8], + data: vec![4u8].into(), hash: 4, priority: 1u64, // lower priority than Tx(2) provides: vec![vec![0]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap_err(); let mut it = pool.ready().into_iter().map(|tx| tx.data[0]); @@ -804,49 +887,49 @@ mod tests { // given let mut pool = pool(); pool.import(Transaction { - data: vec![5u8], + data: vec![5u8].into(), hash: 5, provides: vec![vec![0], vec![4]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); pool.import(Transaction { - data: vec![1u8], + data: vec![1u8].into(), requires: vec![vec![0]], provides: vec![vec![1]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); pool.import(Transaction { - data: vec![3u8], + data: vec![3u8].into(), hash: 3, requires: vec![vec![2]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); pool.import(Transaction { - data: vec![2u8], + data: vec![2u8].into(), hash: 2, requires: vec![vec![1]], provides: vec![vec![3], vec![2]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); pool.import(Transaction { - data: vec![4u8], + data: vec![4u8].into(), hash: 4, priority: 1_000u64, requires: vec![vec![3], vec![4]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); // future pool.import(Transaction { - data: vec![6u8], + data: vec![6u8].into(), hash: 6, priority: 1_000u64, requires: vec![vec![11]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); assert_eq!(pool.ready().count(), 5); @@ -866,39 +949,43 @@ mod tests { let mut pool = pool(); // future (waiting for 0) pool.import(Transaction { - data: vec![5u8], + data: vec![5u8].into(), hash: 5, requires: vec![vec![0]], provides: vec![vec![100]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); // ready - pool.import(Transaction { data: vec![1u8], provides: vec![vec![1]], ..DEFAULT_TX.clone() }) - .unwrap(); pool.import(Transaction { - data: vec![2u8], + data: vec![1u8].into(), + provides: vec![vec![1]], + ..default_tx().clone() + }) + .unwrap(); + pool.import(Transaction { + data: vec![2u8].into(), hash: 2, requires: vec![vec![2]], provides: vec![vec![3]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); pool.import(Transaction { - data: vec![3u8], + data: vec![3u8].into(), hash: 3, requires: vec![vec![1]], provides: vec![vec![2]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); pool.import(Transaction { - data: vec![4u8], + data: vec![4u8].into(), hash: 4, priority: 1_000u64, requires: vec![vec![3], vec![2]], provides: vec![vec![4]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); @@ -927,12 +1014,12 @@ mod tests { format!( "{:?}", Transaction { - data: vec![4u8], + data: vec![4u8].into(), hash: 4, priority: 1_000u64, requires: vec![vec![3], vec![2]], provides: vec![vec![4]], - ..DEFAULT_TX.clone() + ..default_tx().clone() } ), "Transaction { \ @@ -946,12 +1033,12 @@ source: TransactionSource::External, requires: [03, 02], provides: [04], data: [ fn transaction_propagation() { assert_eq!( Transaction { - data: vec![4u8], + data: vec![4u8].into(), hash: 4, priority: 1_000u64, requires: vec![vec![3], vec![2]], provides: vec![vec![4]], - ..DEFAULT_TX.clone() + ..default_tx().clone() } .is_propagable(), true @@ -959,13 +1046,13 @@ source: TransactionSource::External, requires: [03, 02], provides: [04], data: [ assert_eq!( Transaction { - data: vec![4u8], + data: vec![4u8].into(), hash: 4, priority: 1_000u64, requires: vec![vec![3], vec![2]], provides: vec![vec![4]], propagate: false, - ..DEFAULT_TX.clone() + ..default_tx().clone() } .is_propagable(), false @@ -982,10 +1069,10 @@ source: TransactionSource::External, requires: [03, 02], provides: [04], data: [ // then let err = pool.import(Transaction { - data: vec![5u8], + data: vec![5u8].into(), hash: 5, requires: vec![vec![0]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }); if let Err(error::Error::RejectedFutureTransaction) = err { @@ -1001,10 +1088,10 @@ source: TransactionSource::External, requires: [03, 02], provides: [04], data: [ // when pool.import(Transaction { - data: vec![5u8], + data: vec![5u8].into(), hash: 5, requires: vec![vec![0]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); @@ -1027,10 +1114,10 @@ source: TransactionSource::External, requires: [03, 02], provides: [04], data: [ // when let flag_value = pool.with_futures_enabled(|pool, flag| { pool.import(Transaction { - data: vec![5u8], + data: vec![5u8].into(), hash: 5, requires: vec![vec![0]], - ..DEFAULT_TX.clone() + ..default_tx().clone() }) .unwrap(); diff --git a/substrate/client/transaction-pool/src/graph/future.rs b/substrate/client/transaction-pool/src/graph/future.rs index bad466318485..2c1e64c04b7f 100644 --- a/substrate/client/transaction-pool/src/graph/future.rs +++ b/substrate/client/transaction-pool/src/graph/future.rs @@ -27,6 +27,7 @@ use sp_runtime::transaction_validity::TransactionTag as Tag; use std::time::Instant; use super::base_pool::Transaction; +use crate::{common::log_xt::log_xt_trace, LOG_TARGET}; /// Transaction with partially satisfied dependencies. pub struct WaitingTransaction { @@ -105,11 +106,11 @@ impl WaitingTransaction { /// A pool of transactions that are not yet ready to be included in the block. /// -/// Contains transactions that are still awaiting for some other transactions that +/// Contains transactions that are still awaiting some other transactions that /// could provide a tag that they require. -#[derive(Debug)] +#[derive(Clone, Debug)] pub struct FutureTransactions { - /// tags that are not yet provided by any transaction and we await for them + /// tags that are not yet provided by any transaction, and we await for them wanted_tags: HashMap>, /// Transactions waiting for a particular other transaction waiting: HashMap>, @@ -128,7 +129,9 @@ every hash from `wanted_tags` is always present in `waiting`; qed #"; -impl FutureTransactions { +impl + FutureTransactions +{ /// Import transaction to Future queue. /// /// Only transactions that don't have all their tags satisfied should occupy @@ -165,10 +168,30 @@ impl FutureTransactions { .collect() } + /// Removes transactions that provide any of tags in the given list. + /// + /// Returns list of removed transactions. + pub fn prune_tags(&mut self, tags: &Vec) -> Vec>> { + let pruned = self + .waiting + .values() + .filter_map(|tx| { + tx.transaction + .provides + .iter() + .any(|provided_tag| tags.contains(provided_tag)) + .then(|| tx.transaction.hash.clone()) + }) + .collect::>(); + + log_xt_trace!(target: LOG_TARGET, &pruned, "[{:?}] FutureTransactions: removed while pruning tags."); + self.remove(&pruned) + } + /// Satisfies provided tags in transactions that are waiting for them. /// /// Returns (and removes) transactions that became ready after their last tag got - /// satisfied and now we can remove them from Future and move to Ready queue. + /// satisfied, and now we can remove them from Future and move to Ready queue. pub fn satisfy_tags>( &mut self, tags: impl IntoIterator, @@ -218,6 +241,7 @@ impl FutureTransactions { removed.push(waiting_tx.transaction) } } + removed } diff --git a/substrate/client/transaction-pool/src/graph/listener.rs b/substrate/client/transaction-pool/src/graph/listener.rs index 46b7957e0b31..a5593920eec4 100644 --- a/substrate/client/transaction-pool/src/graph/listener.rs +++ b/substrate/client/transaction-pool/src/graph/listener.rs @@ -18,18 +18,31 @@ use std::{collections::HashMap, fmt::Debug, hash}; -use crate::LOG_TARGET; use linked_hash_map::LinkedHashMap; -use log::{debug, trace}; +use log::trace; +use sc_transaction_pool_api::TransactionStatus; +use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; use serde::Serialize; use sp_runtime::traits; use super::{watcher, BlockHash, ChainApi, ExtrinsicHash}; +static LOG_TARGET: &str = "txpool::watcher"; + +/// Single event used in dropped by limits stream. It is one of Ready/Future/Dropped. +pub type DroppedByLimitsEvent = (H, TransactionStatus); +/// Stream of events used to determine if a transaction was dropped. +pub type DroppedByLimitsStream = TracingUnboundedReceiver>; + /// Extrinsic pool default listener. pub struct Listener { - watchers: HashMap>>, + watchers: HashMap>>, finality_watchers: LinkedHashMap, Vec>, + + /// The sink used to notify dropped-by-enforcing-limits transactions. Also ready and future + /// statuses are reported via this channel to allow consumer of the stream tracking actual + /// drops. + dropped_by_limits_sink: Option>>>, } /// Maximum number of blocks awaiting finality at any time. @@ -37,11 +50,15 @@ const MAX_FINALITY_WATCHERS: usize = 512; impl Default for Listener { fn default() -> Self { - Self { watchers: Default::default(), finality_watchers: Default::default() } + Self { + watchers: Default::default(), + finality_watchers: Default::default(), + dropped_by_limits_sink: None, + } } } -impl Listener { +impl Listener { fn fire(&mut self, hash: &H, fun: F) where F: FnOnce(&mut watcher::Sender>), @@ -66,6 +83,15 @@ impl Listener { sender.new_watcher(hash) } + /// Creates a new single stream for entire pool. + /// + /// The stream can be used to subscribe to life-cycle events of all extrinsics in the pool. + pub fn create_dropped_by_limits_stream(&mut self) -> DroppedByLimitsStream> { + let (sender, single_stream) = tracing_unbounded("mpsc_txpool_watcher", 100_000); + self.dropped_by_limits_sink = Some(sender); + single_stream + } + /// Notify the listeners about extrinsic broadcast. pub fn broadcasted(&mut self, hash: &H, peers: Vec) { trace!(target: LOG_TARGET, "[{:?}] Broadcasted", hash); @@ -79,32 +105,55 @@ impl Listener { if let Some(old) = old { self.fire(old, |watcher| watcher.usurped(tx.clone())); } + + if let Some(ref sink) = self.dropped_by_limits_sink { + if let Err(e) = sink.unbounded_send((tx.clone(), TransactionStatus::Ready)) { + trace!(target: LOG_TARGET, "[{:?}] dropped_sink/ready: send message failed: {:?}", tx, e); + } + } } /// New transaction was added to the future pool. pub fn future(&mut self, tx: &H) { trace!(target: LOG_TARGET, "[{:?}] Future", tx); self.fire(tx, |watcher| watcher.future()); + if let Some(ref sink) = self.dropped_by_limits_sink { + if let Err(e) = sink.unbounded_send((tx.clone(), TransactionStatus::Future)) { + trace!(target: LOG_TARGET, "[{:?}] dropped_sink/future: send message failed: {:?}", tx, e); + } + } } /// Transaction was dropped from the pool because of the limit. - pub fn dropped(&mut self, tx: &H, by: Option<&H>) { + /// + /// If the function was actually called due to enforcing limits, the `limits_enforced` flag + /// shall be set to true. + pub fn dropped(&mut self, tx: &H, by: Option<&H>, limits_enforced: bool) { 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); + } + } + } } /// Transaction was removed as invalid. pub fn invalid(&mut self, tx: &H) { - debug!(target: LOG_TARGET, "[{:?}] Extrinsic invalid", tx); + trace!(target: LOG_TARGET, "[{:?}] Extrinsic invalid", tx); self.fire(tx, |watcher| watcher.invalid()); } /// Transaction was pruned from the pool. pub fn pruned(&mut self, block_hash: BlockHash, tx: &H) { - debug!(target: LOG_TARGET, "[{:?}] Pruned at {:?}", tx, block_hash); + trace!(target: LOG_TARGET, "[{:?}] Pruned at {:?}", tx, block_hash); // Get the transactions included in the given block hash. let txs = self.finality_watchers.entry(block_hash).or_insert(vec![]); txs.push(tx.clone()); @@ -135,7 +184,7 @@ impl Listener { pub fn finalized(&mut self, block_hash: BlockHash) { if let Some(hashes) = self.finality_watchers.remove(&block_hash) { for (tx_index, hash) in hashes.into_iter().enumerate() { - log::debug!( + log::trace!( target: LOG_TARGET, "[{:?}] Sent finalization event (block {:?})", hash, @@ -145,4 +194,9 @@ impl Listener { } } } + + /// Provides hashes of all watched transactions. + pub fn watched_transactions(&self) -> impl Iterator { + self.watchers.keys() + } } diff --git a/substrate/client/transaction-pool/src/graph/mod.rs b/substrate/client/transaction-pool/src/graph/mod.rs index 484a6d6cf9f0..c1225d7356d9 100644 --- a/substrate/client/transaction-pool/src/graph/mod.rs +++ b/substrate/client/transaction-pool/src/graph/mod.rs @@ -37,8 +37,10 @@ mod validated_pool; pub mod base_pool; pub mod watcher; -pub use self::{ - base_pool::Transaction, - pool::{BlockHash, ChainApi, ExtrinsicFor, ExtrinsicHash, NumberFor, Options, Pool}, +pub use self::pool::{ + BlockHash, ChainApi, ExtrinsicFor, ExtrinsicHash, NumberFor, Options, Pool, RawExtrinsicFor, + TransactionFor, ValidatedTransactionFor, }; pub use validated_pool::{IsValidator, ValidatedTransaction}; + +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 5305b5f1c12e..6d08a0f0b93c 100644 --- a/substrate/client/transaction-pool/src/graph/pool.rs +++ b/substrate/client/transaction-pool/src/graph/pool.rs @@ -16,12 +16,11 @@ // You should have received a copy of the GNU General Public License // along with this program. If not, see . -use std::{collections::HashMap, sync::Arc, time::Duration}; - -use crate::LOG_TARGET; +use crate::{common::log_xt::log_xt_trace, LOG_TARGET}; use futures::{channel::mpsc::Receiver, Future}; +use indexmap::IndexMap; use sc_transaction_pool_api::error; -use sp_blockchain::TreeRoute; +use sp_blockchain::{HashAndNumber, TreeRoute}; use sp_runtime::{ generic::BlockId, traits::{self, Block as BlockT, SaturatedConversion}, @@ -29,7 +28,11 @@ use sp_runtime::{ TransactionSource, TransactionTag as Tag, TransactionValidity, TransactionValidityError, }, }; -use std::time::Instant; +use std::{ + collections::HashMap, + sync::Arc, + time::{Duration, Instant}, +}; use super::{ base_pool as base, @@ -44,8 +47,10 @@ pub type EventStream = Receiver; pub type BlockHash = <::Block as traits::Block>::Hash; /// Extrinsic hash type for a pool. pub type ExtrinsicHash = <::Block as traits::Block>::Hash; -/// Extrinsic type for a pool. -pub type ExtrinsicFor = <::Block as traits::Block>::Extrinsic; +/// Extrinsic type for a pool (reference counted). +pub type ExtrinsicFor = Arc<<::Block as traits::Block>::Extrinsic>; +/// Extrinsic type for a pool (raw data). +pub type RawExtrinsicFor = <::Block as traits::Block>::Extrinsic; /// Block number type for the ChainApi pub type NumberFor = traits::NumberFor<::Block>; /// A type of transaction stored in the pool @@ -89,7 +94,7 @@ pub trait ChainApi: Send + Sync { ) -> Result::Hash>, Self::Error>; /// Returns hash and encoding length of the extrinsic. - fn hash_and_length(&self, uxt: &ExtrinsicFor) -> (ExtrinsicHash, usize); + fn hash_and_length(&self, uxt: &RawExtrinsicFor) -> (ExtrinsicHash, usize); /// Returns a block body given the block. fn block_body(&self, at: ::Hash) -> Self::BodyFuture; @@ -106,6 +111,16 @@ pub trait ChainApi: Send + Sync { from: ::Hash, to: ::Hash, ) -> Result, Self::Error>; + + /// Resolves block number by id. + fn resolve_block_number( + &self, + at: ::Hash, + ) -> Result, Self::Error> { + self.block_id_to_number(&BlockId::Hash(at)).and_then(|number| { + number.ok_or_else(|| error::Error::InvalidBlockId(format!("{:?}", at)).into()) + }) + } } /// Pool configuration options. @@ -154,13 +169,13 @@ impl Pool { /// Imports a bunch of unverified extrinsics to the pool pub async fn submit_at( &self, - at: ::Hash, + at: &HashAndNumber, source: TransactionSource, xts: impl IntoIterator>, - ) -> Result, B::Error>>, B::Error> { + ) -> Vec, B::Error>> { let xts = xts.into_iter().map(|xt| (source, xt)); - let validated_transactions = self.verify(at, xts, CheckBannedBeforeVerify::Yes).await?; - Ok(self.validated_pool.submit(validated_transactions.into_values())) + let validated_transactions = self.verify(at, xts, CheckBannedBeforeVerify::Yes).await; + self.validated_pool.submit(validated_transactions.into_values()) } /// Resubmit the given extrinsics to the pool. @@ -168,36 +183,35 @@ impl Pool { /// This does not check if a transaction is banned, before we verify it again. pub async fn resubmit_at( &self, - at: ::Hash, + at: &HashAndNumber, source: TransactionSource, xts: impl IntoIterator>, - ) -> Result, B::Error>>, B::Error> { + ) -> Vec, B::Error>> { let xts = xts.into_iter().map(|xt| (source, xt)); - let validated_transactions = self.verify(at, xts, CheckBannedBeforeVerify::No).await?; - Ok(self.validated_pool.submit(validated_transactions.into_values())) + let validated_transactions = self.verify(at, xts, CheckBannedBeforeVerify::No).await; + self.validated_pool.submit(validated_transactions.into_values()) } /// Imports one unverified extrinsic to the pool pub async fn submit_one( &self, - at: ::Hash, + at: &HashAndNumber, source: TransactionSource, xt: ExtrinsicFor, ) -> Result, B::Error> { - let res = self.submit_at(at, source, std::iter::once(xt)).await?.pop(); + let res = self.submit_at(at, source, std::iter::once(xt)).await.pop(); res.expect("One extrinsic passed; one result returned; qed") } /// Import a single extrinsic and starts to watch its progress in the pool. pub async fn submit_and_watch( &self, - at: ::Hash, + at: &HashAndNumber, source: TransactionSource, xt: ExtrinsicFor, ) -> Result, ExtrinsicHash>, B::Error> { - let block_number = self.resolve_block_number(&BlockId::Hash(at))?; let (_, tx) = self - .verify_one(at, block_number, source, xt, CheckBannedBeforeVerify::Yes) + .verify_one(at.hash, at.number, source, xt, CheckBannedBeforeVerify::Yes) .await; self.validated_pool.submit_and_watch(tx) } @@ -209,7 +223,7 @@ impl Pool { ) { let now = Instant::now(); self.validated_pool.resubmit(revalidated_transactions); - log::debug!( + log::trace!( target: LOG_TARGET, "Resubmitted. Took {} ms. Status: {:?}", now.elapsed().as_millis(), @@ -222,34 +236,30 @@ impl Pool { /// Used to clear the pool from transactions that were part of recently imported block. /// The main difference from the `prune` is that we do not revalidate any transactions /// and ignore unknown passed hashes. - pub fn prune_known( - &self, - at: &BlockId, - hashes: &[ExtrinsicHash], - ) -> Result<(), B::Error> { + pub fn prune_known(&self, at: &HashAndNumber, hashes: &[ExtrinsicHash]) { // Get details of all extrinsics that are already in the pool let in_pool_tags = self.validated_pool.extrinsics_tags(hashes).into_iter().flatten().flatten(); // Prune all transactions that provide given tags - let prune_status = self.validated_pool.prune_tags(in_pool_tags)?; + let prune_status = self.validated_pool.prune_tags(in_pool_tags); let pruned_transactions = hashes.iter().cloned().chain(prune_status.pruned.iter().map(|tx| tx.hash)); - self.validated_pool.fire_pruned(at, pruned_transactions) + self.validated_pool.fire_pruned(at, pruned_transactions); } /// Prunes ready transactions. /// /// Used to clear the pool from transactions that were part of recently imported block. /// To perform pruning we need the tags that each extrinsic provides and to avoid calling - /// into runtime too often we first lookup all extrinsics that are in the pool and get + /// into runtime too often we first look up all extrinsics that are in the pool and get /// their provided tags from there. Otherwise we query the runtime at the `parent` block. pub async fn prune( &self, - at: ::Hash, + at: &HashAndNumber, parent: ::Hash, - extrinsics: &[ExtrinsicFor], - ) -> Result<(), B::Error> { + extrinsics: &[RawExtrinsicFor], + ) { log::debug!( target: LOG_TARGET, "Starting pruning of block {:?} (extrinsics: {})", @@ -264,6 +274,7 @@ impl Pool { // Zip the ones from the pool with the full list (we get pairs `(Extrinsic, // Option>)`) let all = extrinsics.iter().zip(in_pool_tags.into_iter()); + let mut validated_counter: usize = 0; let mut future_tags = Vec::new(); for (extrinsic, in_pool_tags) in all { @@ -275,16 +286,19 @@ impl Pool { None => { // Avoid validating block txs if the pool is empty if !self.validated_pool.status().is_empty() { + validated_counter = validated_counter + 1; let validity = self .validated_pool .api() .validate_transaction( parent, TransactionSource::InBlock, - extrinsic.clone(), + Arc::from(extrinsic.clone()), ) .await; + log::trace!(target: LOG_TARGET,"[{:?}] prune::revalidated {:?}", self.validated_pool.api().hash_and_length(&extrinsic.clone()).0, validity); + if let Ok(Ok(validity)) = validity { future_tags.extend(validity.provides); } @@ -298,6 +312,8 @@ impl Pool { } } + log::trace!(target: LOG_TARGET,"prune: validated_counter:{validated_counter}"); + self.prune_tags(at, future_tags, in_pool_hashes).await } @@ -324,13 +340,13 @@ impl Pool { /// prevent importing them in the (near) future. pub async fn prune_tags( &self, - at: ::Hash, + at: &HashAndNumber, tags: impl IntoIterator, known_imported_hashes: impl IntoIterator> + Clone, - ) -> Result<(), B::Error> { - log::debug!(target: LOG_TARGET, "Pruning at {:?}", at); + ) { + log::trace!(target: LOG_TARGET, "Pruning at {:?}", at); // Prune all transactions that provide given tags - let prune_status = self.validated_pool.prune_tags(tags)?; + let prune_status = self.validated_pool.prune_tags(tags); // Make sure that we don't revalidate extrinsics that were part of the recently // imported block. This is especially important for UTXO-like chains cause the @@ -340,18 +356,20 @@ 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_hashes = prune_status.pruned.iter().map(|tx| tx.hash).collect::>(); let pruned_transactions = prune_status.pruned.into_iter().map(|tx| (tx.source, tx.data.clone())); let reverified_transactions = - self.verify(at, pruned_transactions, CheckBannedBeforeVerify::Yes).await?; + self.verify(at, pruned_transactions, CheckBannedBeforeVerify::Yes).await; - log::trace!(target: LOG_TARGET, "Pruning at {:?}. Resubmitting transactions.", at); - // And finally - submit reverified transactions back to the pool + let pruned_hashes = reverified_transactions.keys().map(Clone::clone).collect(); + + log::trace!(target: LOG_TARGET, "Pruning at {:?}. Resubmitting transactions: {}", &at, reverified_transactions.len()); + log_xt_trace!(data: tuple, target: LOG_TARGET, &reverified_transactions, "[{:?}] Resubmitting transaction: {:?}"); + // And finally - submit reverified transactions back to the pool self.validated_pool.resubmit_pruned( - &BlockId::Hash(at), + &at, known_imported_hashes, pruned_hashes, reverified_transactions.into_values().collect(), @@ -359,36 +377,28 @@ impl Pool { } /// Returns transaction hash - pub fn hash_of(&self, xt: &ExtrinsicFor) -> ExtrinsicHash { + pub fn hash_of(&self, xt: &RawExtrinsicFor) -> ExtrinsicHash { self.validated_pool.api().hash_and_length(xt).0 } - /// Resolves block number by id. - fn resolve_block_number(&self, at: &BlockId) -> Result, B::Error> { - self.validated_pool.api().block_id_to_number(at).and_then(|number| { - number.ok_or_else(|| error::Error::InvalidBlockId(format!("{:?}", at)).into()) - }) - } - /// Returns future that validates a bunch of transactions at given block. async fn verify( &self, - at: ::Hash, + at: &HashAndNumber, xts: impl IntoIterator)>, check: CheckBannedBeforeVerify, - ) -> Result, ValidatedTransactionFor>, B::Error> { - // we need a block number to compute tx validity - let block_number = self.resolve_block_number(&BlockId::Hash(at))?; + ) -> IndexMap, ValidatedTransactionFor> { + let HashAndNumber { number, hash } = *at; let res = futures::future::join_all( xts.into_iter() - .map(|(source, xt)| self.verify_one(at, block_number, source, xt, check)), + .map(|(source, xt)| self.verify_one(hash, number, source, xt, check)), ) .await .into_iter() - .collect::>(); + .collect::>(); - Ok(res) + res } /// Returns future that validates single transaction at given block. @@ -441,22 +451,31 @@ impl Pool { (hash, validity) } - /// get a reference to the underlying validated pool. + /// Get a reference to the underlying validated pool. pub fn validated_pool(&self) -> &ValidatedPool { &self.validated_pool } + + /// Clears the recently pruned transactions in validated pool. + pub fn clear_recently_pruned(&mut self) { + self.validated_pool.pool.write().clear_recently_pruned(); + } } -impl Clone for Pool { - fn clone(&self) -> Self { - Self { validated_pool: self.validated_pool.clone() } +impl Pool { + /// Deep clones the pool. + /// + /// Must be called on purpose: it duplicates all the internal structures. + pub fn deep_clone(&self) -> Self { + let other: ValidatedPool = (*self.validated_pool).clone(); + Self { validated_pool: Arc::from(other) } } } #[cfg(test)] mod tests { use super::{super::base_pool::Limit, *}; - use crate::tests::{pool, uxt, TestApi, INVALID_NONCE}; + use crate::common::tests::{pool, uxt, TestApi, INVALID_NONCE}; use assert_matches::assert_matches; use codec::Encode; use futures::executor::block_on; @@ -475,22 +494,58 @@ mod tests { let (pool, api) = pool(); // when - let hash = block_on(pool.submit_one( - api.expect_hash_from_number(0), - SOURCE, - uxt(Transfer { - from: Alice.into(), - to: AccountId::from_h256(H256::from_low_u64_be(2)), - amount: 5, - nonce: 0, - }), - )) + let hash = block_on( + pool.submit_one( + &api.expect_hash_and_number(0), + SOURCE, + uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: 0, + }) + .into(), + ), + ) .unwrap(); // then assert_eq!(pool.validated_pool().ready().map(|v| v.hash).collect::>(), vec![hash]); } + #[test] + fn submit_at_preserves_order() { + sp_tracing::try_init_simple(); + // given + let (pool, api) = pool(); + + let txs = (0..10) + .map(|i| { + uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(i)), + amount: 5, + nonce: i, + }) + .into() + }) + .collect::>(); + + 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)); + log::debug!("--> {hashes:#?}"); + + // then + hashes.into_iter().zip(initial_hashes.into_iter()).for_each( + |(result_hash, initial_hash)| { + assert_eq!(result_hash.unwrap(), initial_hash); + }, + ); + } + #[test] fn should_reject_if_temporarily_banned() { // given @@ -504,7 +559,7 @@ mod tests { // when pool.validated_pool.ban(&Instant::now(), vec![pool.hash_of(&uxt)]); - let res = block_on(pool.submit_one(api.expect_hash_from_number(0), SOURCE, uxt)); + let res = block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.into())); assert_eq!(pool.validated_pool().status().ready, 0); assert_eq!(pool.validated_pool().status().future, 0); @@ -527,7 +582,7 @@ mod tests { let uxt = ExtrinsicBuilder::new_include_data(vec![42]).build(); // when - let res = block_on(pool.submit_one(api.expect_hash_from_number(0), SOURCE, uxt)); + let res = block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.into())); // then assert_matches!(res.unwrap_err(), error::Error::Unactionable); @@ -538,43 +593,52 @@ mod tests { let (stream, hash0, hash1) = { // given let (pool, api) = pool(); - let hash_of_block0 = api.expect_hash_from_number(0); + let han_of_block0 = api.expect_hash_and_number(0); let stream = pool.validated_pool().import_notification_stream(); // when - let hash0 = block_on(pool.submit_one( - hash_of_block0, - SOURCE, - uxt(Transfer { - from: Alice.into(), - to: AccountId::from_h256(H256::from_low_u64_be(2)), - amount: 5, - nonce: 0, - }), - )) + let hash0 = block_on( + pool.submit_one( + &han_of_block0, + SOURCE, + uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: 0, + }) + .into(), + ), + ) .unwrap(); - let hash1 = block_on(pool.submit_one( - hash_of_block0, - SOURCE, - uxt(Transfer { - from: Alice.into(), - to: AccountId::from_h256(H256::from_low_u64_be(2)), - amount: 5, - nonce: 1, - }), - )) + let hash1 = block_on( + pool.submit_one( + &han_of_block0, + SOURCE, + uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: 1, + }) + .into(), + ), + ) .unwrap(); // future doesn't count - let _hash = block_on(pool.submit_one( - hash_of_block0, - SOURCE, - uxt(Transfer { - from: Alice.into(), - to: AccountId::from_h256(H256::from_low_u64_be(2)), - amount: 5, - nonce: 3, - }), - )) + let _hash = block_on( + pool.submit_one( + &han_of_block0, + SOURCE, + uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: 3, + }) + .into(), + ), + ) .unwrap(); assert_eq!(pool.validated_pool().status().ready, 2); @@ -594,43 +658,52 @@ mod tests { fn should_clear_stale_transactions() { // given let (pool, api) = pool(); - let hash_of_block0 = api.expect_hash_from_number(0); - let hash1 = block_on(pool.submit_one( - hash_of_block0, - SOURCE, - uxt(Transfer { - from: Alice.into(), - to: AccountId::from_h256(H256::from_low_u64_be(2)), - amount: 5, - nonce: 0, - }), - )) + let han_of_block0 = api.expect_hash_and_number(0); + let hash1 = block_on( + pool.submit_one( + &han_of_block0, + SOURCE, + uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: 0, + }) + .into(), + ), + ) .unwrap(); - let hash2 = block_on(pool.submit_one( - hash_of_block0, - SOURCE, - uxt(Transfer { - from: Alice.into(), - to: AccountId::from_h256(H256::from_low_u64_be(2)), - amount: 5, - nonce: 1, - }), - )) + let hash2 = block_on( + pool.submit_one( + &han_of_block0, + SOURCE, + uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: 1, + }) + .into(), + ), + ) .unwrap(); - let hash3 = block_on(pool.submit_one( - hash_of_block0, - SOURCE, - uxt(Transfer { - from: Alice.into(), - to: AccountId::from_h256(H256::from_low_u64_be(2)), - amount: 5, - nonce: 3, - }), - )) + let hash3 = block_on( + pool.submit_one( + &han_of_block0, + SOURCE, + uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: 3, + }) + .into(), + ), + ) .unwrap(); // when - pool.validated_pool.clear_stale(&BlockId::Number(5)).unwrap(); + pool.validated_pool.clear_stale(&api.expect_hash_and_number(5)); // then assert_eq!(pool.validated_pool().ready().count(), 0); @@ -646,21 +719,23 @@ mod tests { fn should_ban_mined_transactions() { // given let (pool, api) = pool(); - let hash1 = block_on(pool.submit_one( - api.expect_hash_from_number(0), - SOURCE, - uxt(Transfer { - from: Alice.into(), - to: AccountId::from_h256(H256::from_low_u64_be(2)), - amount: 5, - nonce: 0, - }), - )) + let hash1 = block_on( + pool.submit_one( + &api.expect_hash_and_number(0), + SOURCE, + uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: 0, + }) + .into(), + ), + ) .unwrap(); // when - block_on(pool.prune_tags(api.expect_hash_from_number(1), vec![vec![0]], vec![hash1])) - .unwrap(); + block_on(pool.prune_tags(&api.expect_hash_and_number(1), vec![vec![0]], vec![hash1])); // then assert!(pool.validated_pool.is_banned(&hash1)); @@ -685,20 +760,24 @@ 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_from_number(0), SOURCE, xt)).unwrap(); + let hash1 = + block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, xt.into())).unwrap(); assert_eq!(pool.validated_pool().status().future, 1); // when - let hash2 = block_on(pool.submit_one( - api.expect_hash_from_number(0), - SOURCE, - uxt(Transfer { - from: Bob.into(), - to: AccountId::from_h256(H256::from_low_u64_be(2)), - amount: 5, - nonce: 10, - }), - )) + let hash2 = block_on( + pool.submit_one( + &api.expect_hash_and_number(0), + SOURCE, + uxt(Transfer { + from: Bob.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: 10, + }) + .into(), + ), + ) .unwrap(); // then @@ -718,16 +797,19 @@ mod tests { let pool = Pool::new(options, true.into(), api.clone()); // when - block_on(pool.submit_one( - api.expect_hash_from_number(0), - SOURCE, - uxt(Transfer { - from: Alice.into(), - to: AccountId::from_h256(H256::from_low_u64_be(2)), - amount: 5, - nonce: 1, - }), - )) + block_on( + pool.submit_one( + &api.expect_hash_and_number(0), + SOURCE, + uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: 1, + }) + .into(), + ), + ) .unwrap_err(); // then @@ -741,16 +823,19 @@ mod tests { let (pool, api) = pool(); // when - let err = block_on(pool.submit_one( - api.expect_hash_from_number(0), - SOURCE, - uxt(Transfer { - from: Alice.into(), - to: AccountId::from_h256(H256::from_low_u64_be(2)), - amount: 5, - nonce: INVALID_NONCE, - }), - )) + let err = block_on( + pool.submit_one( + &api.expect_hash_and_number(0), + SOURCE, + uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: INVALID_NONCE, + }) + .into(), + ), + ) .unwrap_err(); // then @@ -766,96 +851,113 @@ mod tests { fn should_trigger_ready_and_finalized() { // given let (pool, api) = pool(); - let watcher = block_on(pool.submit_and_watch( - api.expect_hash_from_number(0), - SOURCE, - uxt(Transfer { - from: Alice.into(), - to: AccountId::from_h256(H256::from_low_u64_be(2)), - amount: 5, - nonce: 0, - }), - )) + let watcher = block_on( + pool.submit_and_watch( + &api.expect_hash_and_number(0), + SOURCE, + uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: 0, + }) + .into(), + ), + ) .unwrap(); assert_eq!(pool.validated_pool().status().ready, 1); assert_eq!(pool.validated_pool().status().future, 0); - let hash_of_block2 = api.expect_hash_from_number(2); + let han_of_block2 = api.expect_hash_and_number(2); // when - block_on(pool.prune_tags(hash_of_block2, vec![vec![0u8]], vec![])).unwrap(); + block_on(pool.prune_tags(&han_of_block2, vec![vec![0u8]], vec![])); assert_eq!(pool.validated_pool().status().ready, 0); assert_eq!(pool.validated_pool().status().future, 0); // then let mut stream = futures::executor::block_on_stream(watcher.into_stream()); assert_eq!(stream.next(), Some(TransactionStatus::Ready)); - assert_eq!(stream.next(), Some(TransactionStatus::InBlock((hash_of_block2.into(), 0))),); + assert_eq!( + stream.next(), + Some(TransactionStatus::InBlock((han_of_block2.hash.into(), 0))), + ); } #[test] fn should_trigger_ready_and_finalized_when_pruning_via_hash() { // given let (pool, api) = pool(); - let watcher = block_on(pool.submit_and_watch( - api.expect_hash_from_number(0), - SOURCE, - uxt(Transfer { - from: Alice.into(), - to: AccountId::from_h256(H256::from_low_u64_be(2)), - amount: 5, - nonce: 0, - }), - )) + let watcher = block_on( + pool.submit_and_watch( + &api.expect_hash_and_number(0), + SOURCE, + uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: 0, + }) + .into(), + ), + ) .unwrap(); assert_eq!(pool.validated_pool().status().ready, 1); assert_eq!(pool.validated_pool().status().future, 0); - let hash_of_block2 = api.expect_hash_from_number(2); + let han_of_block2 = api.expect_hash_and_number(2); // when - block_on(pool.prune_tags(hash_of_block2, vec![vec![0u8]], vec![*watcher.hash()])) - .unwrap(); + block_on(pool.prune_tags(&han_of_block2, vec![vec![0u8]], vec![*watcher.hash()])); assert_eq!(pool.validated_pool().status().ready, 0); assert_eq!(pool.validated_pool().status().future, 0); // then let mut stream = futures::executor::block_on_stream(watcher.into_stream()); assert_eq!(stream.next(), Some(TransactionStatus::Ready)); - assert_eq!(stream.next(), Some(TransactionStatus::InBlock((hash_of_block2.into(), 0))),); + assert_eq!( + stream.next(), + Some(TransactionStatus::InBlock((han_of_block2.hash.into(), 0))), + ); } #[test] fn should_trigger_future_and_ready_after_promoted() { // given let (pool, api) = pool(); - let hash_of_block0 = api.expect_hash_from_number(0); - - let watcher = block_on(pool.submit_and_watch( - hash_of_block0, - SOURCE, - uxt(Transfer { - from: Alice.into(), - to: AccountId::from_h256(H256::from_low_u64_be(2)), - amount: 5, - nonce: 1, - }), - )) + let han_of_block0 = api.expect_hash_and_number(0); + + let watcher = block_on( + pool.submit_and_watch( + &han_of_block0, + SOURCE, + uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: 1, + }) + .into(), + ), + ) .unwrap(); assert_eq!(pool.validated_pool().status().ready, 0); assert_eq!(pool.validated_pool().status().future, 1); // when - block_on(pool.submit_one( - hash_of_block0, - SOURCE, - uxt(Transfer { - from: Alice.into(), - to: AccountId::from_h256(H256::from_low_u64_be(2)), - amount: 5, - nonce: 0, - }), - )) + block_on( + pool.submit_one( + &han_of_block0, + SOURCE, + uxt(Transfer { + from: Alice.into(), + to: AccountId::from_h256(H256::from_low_u64_be(2)), + amount: 5, + nonce: 0, + }) + .into(), + ), + ) .unwrap(); assert_eq!(pool.validated_pool().status().ready, 2); @@ -876,7 +978,7 @@ mod tests { nonce: 0, }); let watcher = - block_on(pool.submit_and_watch(api.expect_hash_from_number(0), SOURCE, uxt)) + block_on(pool.submit_and_watch(&api.expect_hash_and_number(0), SOURCE, uxt.into())) .unwrap(); assert_eq!(pool.validated_pool().status().ready, 1); @@ -901,7 +1003,7 @@ mod tests { nonce: 0, }); let watcher = - block_on(pool.submit_and_watch(api.expect_hash_from_number(0), SOURCE, uxt)) + block_on(pool.submit_and_watch(&api.expect_hash_and_number(0), SOURCE, uxt.into())) .unwrap(); assert_eq!(pool.validated_pool().status().ready, 1); @@ -934,7 +1036,7 @@ mod tests { nonce: 0, }); let watcher = - block_on(pool.submit_and_watch(api.expect_hash_from_number(0), SOURCE, xt)) + block_on(pool.submit_and_watch(&api.expect_hash_and_number(0), SOURCE, xt.into())) .unwrap(); assert_eq!(pool.validated_pool().status().ready, 1); @@ -945,7 +1047,7 @@ mod tests { amount: 4, nonce: 1, }); - block_on(pool.submit_one(api.expect_hash_from_number(1), SOURCE, xt)).unwrap(); + block_on(pool.submit_one(&api.expect_hash_and_number(1), SOURCE, xt.into())).unwrap(); assert_eq!(pool.validated_pool().status().ready, 1); // then @@ -968,7 +1070,8 @@ 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_one(api.expect_hash_from_number(0), SOURCE, xt)).unwrap(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, xt.into())) + .unwrap(); assert_eq!(pool.validated_pool().status().ready, 1); // then @@ -980,7 +1083,8 @@ mod tests { amount: 4, nonce: 1, }); - let result = block_on(pool.submit_one(api.expect_hash_from_number(1), SOURCE, xt)); + let result = + block_on(pool.submit_one(&api.expect_hash_and_number(1), SOURCE, xt.into())); assert!(matches!( result, Err(sc_transaction_pool_api::error::Error::ImmediatelyDropped) @@ -995,12 +1099,12 @@ mod tests { let api = Arc::new(TestApi::default()); let pool = Pool::new(options, true.into(), api.clone()); - let hash_of_block0 = api.expect_hash_from_number(0); + let han_of_block0 = api.expect_hash_and_number(0); // 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(hash_of_block0, SOURCE, xt)).unwrap(); + block_on(pool.submit_and_watch(&han_of_block0, SOURCE, xt.into())).unwrap(); assert_eq!(pool.validated_pool().status().ready, 1); // after validation `Transfer` will have priority set to 4 (validate_transaction @@ -1011,14 +1115,16 @@ mod tests { amount: 5, nonce: 0, }); - let watcher = block_on(pool.submit_and_watch(hash_of_block0, SOURCE, xt)).unwrap(); + let watcher = + block_on(pool.submit_and_watch(&han_of_block0, SOURCE, xt.into())).unwrap(); assert_eq!(pool.validated_pool().status().ready, 2); // when // after validation `Store` will have priority set to 9001 (validate_transaction // mock) let xt = ExtrinsicBuilder::new_indexed_call(Vec::new()).build(); - block_on(pool.submit_one(api.expect_hash_from_number(1), SOURCE, xt)).unwrap(); + block_on(pool.submit_one(&api.expect_hash_and_number(1), SOURCE, xt.into())) + .unwrap(); assert_eq!(pool.validated_pool().status().ready, 2); // then @@ -1038,7 +1144,7 @@ mod tests { let api = Arc::new(api); let pool = Arc::new(Pool::new(Default::default(), true.into(), api.clone())); - let hash_of_block0 = api.expect_hash_from_number(0); + let han_of_block0 = api.expect_hash_and_number(0); // when let xt = uxt(Transfer { @@ -1050,9 +1156,12 @@ mod tests { // This transaction should go to future, since we use `nonce: 1` let pool2 = pool.clone(); - std::thread::spawn(move || { - block_on(pool2.submit_one(hash_of_block0, SOURCE, xt)).unwrap(); - ready.send(()).unwrap(); + std::thread::spawn({ + let hash_of_block0 = han_of_block0.clone(); + move || { + block_on(pool2.submit_one(&hash_of_block0, SOURCE, xt.into())).unwrap(); + ready.send(()).unwrap(); + } }); // But now before the previous one is imported we import @@ -1065,13 +1174,12 @@ mod tests { }); // The tag the above transaction provides (TestApi is using just nonce as u8) let provides = vec![0_u8]; - block_on(pool.submit_one(hash_of_block0, SOURCE, xt)).unwrap(); + block_on(pool.submit_one(&han_of_block0, SOURCE, xt.into())).unwrap(); assert_eq!(pool.validated_pool().status().ready, 1); // Now block import happens before the second transaction is able to finish // verification. - block_on(pool.prune_tags(api.expect_hash_from_number(1), vec![provides], vec![])) - .unwrap(); + block_on(pool.prune_tags(&api.expect_hash_and_number(1), vec![provides], vec![])); assert_eq!(pool.validated_pool().status().ready, 0); // so when we release the verification of the previous one it will have diff --git a/substrate/client/transaction-pool/src/graph/ready.rs b/substrate/client/transaction-pool/src/graph/ready.rs index b4a5d9e3ba71..860bcff0bace 100644 --- a/substrate/client/transaction-pool/src/graph/ready.rs +++ b/substrate/client/transaction-pool/src/graph/ready.rs @@ -24,7 +24,7 @@ use std::{ }; use crate::LOG_TARGET; -use log::{debug, trace}; +use log::trace; use sc_transaction_pool_api::error; use serde::Serialize; use sp_runtime::{traits::Member, transaction_validity::TransactionTag as Tag}; @@ -84,7 +84,7 @@ pub struct ReadyTx { /// How many required tags are provided inherently /// /// Some transactions might be already pruned from the queue, - /// so when we compute ready set we may consider this transactions ready earlier. + /// so when we compute ready set we may consider these transactions ready earlier. pub requires_offset: usize, } @@ -106,7 +106,7 @@ qed "#; /// Validated transactions that are block ready with all their dependencies met. -#[derive(Debug)] +#[derive(Clone, Debug)] pub struct ReadyTransactions { /// Next free insertion id (used to indicate when a transaction was inserted into the pool). insertion_id: u64, @@ -521,9 +521,9 @@ impl BestIterator { /// When invoked on a fully drained iterator it has no effect either. pub fn report_invalid(&mut self, tx: &Arc>) { if let Some(to_report) = self.all.get(&tx.hash) { - debug!( + trace!( target: LOG_TARGET, - "[{:?}] Reported as invalid. Will skip sub-chains while iterating.", + "[{:?}] best-iterator: Reported as invalid. Will skip sub-chains while iterating.", to_report.transaction.transaction.hash ); for hash in &to_report.unlocks { @@ -544,7 +544,7 @@ impl Iterator for BestIterator { // Check if the transaction was marked invalid. if self.invalid.contains(hash) { - debug!( + trace!( target: LOG_TARGET, "[{:?}] Skipping invalid child transaction while iterating.", hash, ); @@ -703,7 +703,7 @@ mod tests { tx6.requires = vec![tx5.provides[0].clone()]; tx6.provides = vec![]; let tx7 = Transaction { - data: vec![7], + data: vec![7].into(), bytes: 1, hash: 7, priority: 1, diff --git a/substrate/client/transaction-pool/src/graph/tracked_map.rs b/substrate/client/transaction-pool/src/graph/tracked_map.rs index 44c2c738ab1b..9e92dffc9f96 100644 --- a/substrate/client/transaction-pool/src/graph/tracked_map.rs +++ b/substrate/client/transaction-pool/src/graph/tracked_map.rs @@ -46,6 +46,20 @@ impl Default for TrackedMap { } } +impl Clone for TrackedMap +where + K: Clone, + V: Clone, +{ + fn clone(&self) -> Self { + Self { + index: Arc::from(RwLock::from(self.index.read().clone())), + bytes: self.bytes.load(AtomicOrdering::Relaxed).into(), + length: self.length.load(AtomicOrdering::Relaxed).into(), + } + } +} + impl TrackedMap { /// Current tracked length of the content. pub fn len(&self) -> usize { diff --git a/substrate/client/transaction-pool/src/graph/validated_pool.rs b/substrate/client/transaction-pool/src/graph/validated_pool.rs index 3d7cfeb46b04..d7f55198a40a 100644 --- a/substrate/client/transaction-pool/src/graph/validated_pool.rs +++ b/substrate/client/transaction-pool/src/graph/validated_pool.rs @@ -22,13 +22,13 @@ use std::{ sync::Arc, }; -use crate::LOG_TARGET; +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 sp_blockchain::HashAndNumber; use sp_runtime::{ - generic::BlockId, traits::{self, SaturatedConversion}, transaction_validity::{TransactionSource, TransactionTag as Tag, ValidTransaction}, }; @@ -86,17 +86,18 @@ pub type ValidatedTransactionFor = ValidatedTransaction, ExtrinsicFor, ::Error>; /// A closure that returns true if the local node is a validator that can author blocks. -pub struct IsValidator(Box bool + Send + Sync>); +#[derive(Clone)] +pub struct IsValidator(Arc bool + Send + Sync>>); impl From for IsValidator { fn from(is_validator: bool) -> Self { - Self(Box::new(move || is_validator)) + Self(Arc::new(Box::new(move || is_validator))) } } impl From bool + Send + Sync>> for IsValidator { fn from(is_validator: Box bool + Send + Sync>) -> Self { - Self(is_validator) + Self(Arc::new(is_validator)) } } @@ -111,6 +112,20 @@ pub struct ValidatedPool { rotator: PoolRotator>, } +impl Clone for ValidatedPool { + fn clone(&self) -> Self { + Self { + api: self.api.clone(), + is_validator: self.is_validator.clone(), + options: self.options.clone(), + listener: Default::default(), + pool: RwLock::from(self.pool.read().clone()), + import_notification_sinks: Default::default(), + rotator: PoolRotator::default(), + } + } +} + impl ValidatedPool { /// Create a new transaction pool. pub fn new(options: Options, is_validator: IsValidator, api: Arc) -> Self { @@ -187,6 +202,7 @@ impl ValidatedPool { fn submit_one(&self, tx: ValidatedTransactionFor) -> Result, B::Error> { match tx { ValidatedTransaction::Valid(tx) => { + log::trace!(target: LOG_TARGET, "[{:?}] ValidatedPool::submit_one", tx.hash); if !tx.propagate && !(self.is_validator.0)() { return Err(error::Error::Unactionable.into()) } @@ -216,10 +232,12 @@ impl ValidatedPool { Ok(*imported.hash()) }, ValidatedTransaction::Invalid(hash, err) => { + log::trace!(target: LOG_TARGET, "[{:?}] ValidatedPool::submit_one invalid: {:?}", hash, err); self.rotator.ban(&Instant::now(), std::iter::once(hash)); Err(err) }, ValidatedTransaction::Unknown(hash, err) => { + log::trace!(target: LOG_TARGET, "[{:?}] ValidatedPool::submit_one unknown {:?}", hash, err); self.listener.write().invalid(&hash); Err(err) }, @@ -231,7 +249,6 @@ impl ValidatedPool { let ready_limit = &self.options.ready; let future_limit = &self.options.future; - log::debug!(target: LOG_TARGET, "Pool Status: {:?}", status); if ready_limit.is_exceeded(status.ready, status.ready_bytes) || future_limit.is_exceeded(status.future, status.future_bytes) { @@ -257,13 +274,13 @@ impl ValidatedPool { removed }; if !removed.is_empty() { - log::debug!(target: LOG_TARGET, "Enforcing limits: {} dropped", removed.len()); + log::trace!(target: LOG_TARGET, "Enforcing limits: {} dropped", removed.len()); } // run notifications let mut listener = self.listener.write(); for h in &removed { - listener.dropped(h, None); + listener.dropped(h, None, true); } removed @@ -280,7 +297,7 @@ impl ValidatedPool { match tx { ValidatedTransaction::Valid(tx) => { let hash = self.api.hash_and_length(&tx.data).0; - let watcher = self.listener.write().create_watcher(hash); + let watcher = self.create_watcher(hash); self.submit(std::iter::once(ValidatedTransaction::Valid(tx))) .pop() .expect("One extrinsic passed; one result returned; qed") @@ -294,6 +311,19 @@ impl ValidatedPool { } } + /// Creates a new watcher for given extrinsic. + pub fn create_watcher( + &self, + tx_hash: ExtrinsicHash, + ) -> Watcher, ExtrinsicHash> { + self.listener.write().create_watcher(tx_hash) + } + + /// Provides a list of hashes for all watched transactions in the pool. + pub fn watched_transactions(&self) -> Vec> { + self.listener.read().watched_transactions().map(Clone::clone).collect() + } + /// Resubmits revalidated transactions back to the pool. /// /// Removes and then submits passed transactions and all dependent transactions. @@ -351,7 +381,7 @@ impl ValidatedPool { initial_statuses.insert(removed_hash, Status::Ready); txs_to_resubmit.push((removed_hash, tx_to_resubmit)); } - // make sure to remove the hash even if it's not present in the pool any more. + // make sure to remove the hash even if it's not present in the pool anymore. updated_transactions.remove(&hash); } @@ -423,7 +453,7 @@ impl ValidatedPool { match final_status { Status::Future => listener.future(&hash), Status::Ready => listener.ready(&hash, None), - Status::Dropped => listener.dropped(&hash, None), + Status::Dropped => listener.dropped(&hash, None, false), Status::Failed => listener.invalid(&hash), } } @@ -451,7 +481,7 @@ impl ValidatedPool { pub fn prune_tags( &self, tags: impl IntoIterator, - ) -> Result, ExtrinsicFor>, B::Error> { + ) -> PruneStatus, ExtrinsicFor> { // Perform tag-based pruning in the base pool let status = self.pool.write().prune_tags(tags); // Notify event listeners of all transactions @@ -462,21 +492,21 @@ impl ValidatedPool { fire_events(&mut *listener, promoted); } for f in &status.failed { - listener.dropped(f, None); + listener.dropped(f, None, false); } } - Ok(status) + status } /// Resubmit transactions that have been revalidated after prune_tags call. pub fn resubmit_pruned( &self, - at: &BlockId, + at: &HashAndNumber, known_imported_hashes: impl IntoIterator> + Clone, pruned_hashes: Vec>, pruned_xts: Vec>, - ) -> Result<(), B::Error> { + ) { debug_assert_eq!(pruned_hashes.len(), pruned_xts.len()); // Resubmit pruned transactions @@ -493,35 +523,29 @@ impl ValidatedPool { // Fire `pruned` notifications for collected hashes and make sure to include // `known_imported_hashes` since they were just imported as part of the block. let hashes = hashes.chain(known_imported_hashes.into_iter()); - self.fire_pruned(at, hashes)?; + self.fire_pruned(at, hashes); // perform regular cleanup of old transactions in the pool // and update temporary bans. - self.clear_stale(at)?; - Ok(()) + self.clear_stale(at); } /// Fire notifications for pruned transactions. pub fn fire_pruned( &self, - at: &BlockId, + at: &HashAndNumber, hashes: impl Iterator>, - ) -> Result<(), B::Error> { - let header_hash = self - .api - .block_id_to_hash(at)? - .ok_or_else(|| error::Error::InvalidBlockId(format!("{:?}", at)))?; + ) { let mut listener = self.listener.write(); let mut set = HashSet::with_capacity(hashes.size_hint().0); for h in hashes { // `hashes` has possibly duplicate hashes. // we'd like to send out the `InBlock` notification only once. if !set.contains(&h) { - listener.pruned(header_hash, &h); + listener.pruned(at.hash, &h); set.insert(h); } } - Ok(()) } /// Removes stale transactions from the pool. @@ -529,16 +553,13 @@ impl ValidatedPool { /// Stale transactions are transaction beyond their longevity period. /// Note this function does not remove transactions that are already included in the chain. /// See `prune_tags` if you want this. - pub fn clear_stale(&self, at: &BlockId) -> Result<(), B::Error> { - let block_number = self - .api - .block_id_to_number(at)? - .ok_or_else(|| error::Error::InvalidBlockId(format!("{:?}", at)))? - .saturated_into::(); + pub fn clear_stale(&self, at: &HashAndNumber) { + let HashAndNumber { number, .. } = *at; + let number = number.saturated_into::(); let now = Instant::now(); let to_remove = { self.ready() - .filter(|tx| self.rotator.ban_if_stale(&now, block_number, tx)) + .filter(|tx| self.rotator.ban_if_stale(&now, number, tx)) .map(|tx| tx.hash) .collect::>() }; @@ -546,7 +567,7 @@ impl ValidatedPool { let p = self.pool.read(); let mut hashes = Vec::new(); for tx in p.futures() { - if self.rotator.ban_if_stale(&now, block_number, tx) { + if self.rotator.ban_if_stale(&now, number, tx) { hashes.push(tx.hash); } } @@ -557,8 +578,6 @@ impl ValidatedPool { self.remove_invalid(&futures_to_remove); // clear banned transactions timeouts self.rotator.clear_timeouts(&now); - - Ok(()) } /// Get api reference. @@ -598,14 +617,15 @@ impl ValidatedPool { return vec![] } - log::debug!(target: LOG_TARGET, "Removing invalid transactions: {:?}", hashes); + log::trace!(target: LOG_TARGET, "Removing invalid transactions: {:?}", hashes.len()); // temporarily ban invalid transactions self.rotator.ban(&Instant::now(), hashes.iter().cloned()); let invalid = self.pool.write().remove_subtree(hashes); - log::debug!(target: LOG_TARGET, "Removed invalid transactions: {:?}", invalid); + log::trace!(target: LOG_TARGET, "Removed invalid transactions: {:?}", invalid.len()); + log_xt_trace!(target: LOG_TARGET, invalid.iter().map(|t| t.hash), "{:?} Removed invalid transaction"); let mut listener = self.listener.write(); for tx in &invalid { @@ -645,6 +665,12 @@ impl ValidatedPool { pub fn on_block_retracted(&self, block_hash: BlockHash) { self.listener.write().retracted(block_hash) } + + pub fn create_dropped_by_limits_stream( + &self, + ) -> super::listener::DroppedByLimitsStream, BlockHash> { + self.listener.write().create_dropped_by_limits_stream() + } } fn fire_events(listener: &mut Listener, imported: &base::Imported) @@ -656,7 +682,7 @@ where 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))); + removed.iter().for_each(|r| listener.dropped(&r.hash, Some(hash), false)); 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 fc440771d7bb..fb7cf99d4dc6 100644 --- a/substrate/client/transaction-pool/src/graph/watcher.rs +++ b/substrate/client/transaction-pool/src/graph/watcher.rs @@ -123,7 +123,7 @@ impl Sender { self.send(TransactionStatus::Broadcast(peers)) } - /// Returns true if the are no more listeners for this extrinsic or it was finalized. + /// Returns true if there are no more listeners for this extrinsic, or it was finalized. pub fn is_done(&self) -> bool { self.is_finalized || self.receivers.is_empty() } diff --git a/substrate/client/transaction-pool/src/lib.rs b/substrate/client/transaction-pool/src/lib.rs index 64b301e6bf36..888d25d3a0d2 100644 --- a/substrate/client/transaction-pool/src/lib.rs +++ b/substrate/client/transaction-pool/src/lib.rs @@ -22,776 +22,38 @@ #![warn(missing_docs)] #![warn(unused_extern_crates)] -mod api; -mod enactment_state; -pub mod error; +mod builder; +mod common; +mod fork_aware_txpool; mod graph; -mod metrics; -mod revalidation; -#[cfg(test)] -mod tests; - -pub use crate::api::FullChainApi; -use async_trait::async_trait; -use enactment_state::{EnactmentAction, EnactmentState}; -use futures::{ - channel::oneshot, - future::{self, ready}, - prelude::*, -}; -pub use graph::{ - base_pool::Limit as PoolLimit, ChainApi, Options, Pool, Transaction, ValidatedTransaction, -}; -use parking_lot::Mutex; -use std::{ - collections::{HashMap, HashSet}, - pin::Pin, - sync::Arc, -}; - -use graph::{ExtrinsicHash, IsValidator}; -use sc_transaction_pool_api::{ - error::Error as TxPoolError, ChainEvent, ImportNotificationStream, MaintainedTransactionPool, - PoolFuture, PoolStatus, ReadyTransactions, TransactionFor, TransactionPool, TransactionSource, - TransactionStatusStreamFor, TxHash, -}; -use sp_core::traits::SpawnEssentialNamed; -use sp_runtime::{ - generic::BlockId, - traits::{AtLeast32Bit, Block as BlockT, Extrinsic, Header as HeaderT, NumberFor, Zero}, -}; -use std::time::Instant; - -use crate::metrics::MetricsLink as PrometheusMetrics; -use prometheus_endpoint::Registry as PrometheusRegistry; - -use sp_blockchain::{HashAndNumber, TreeRoute}; - -pub(crate) const LOG_TARGET: &str = "txpool"; - -type BoxedReadyIterator = - Box>> + Send>; +mod single_state_txpool; +mod transaction_pool_wrapper; + +use common::{api, enactment_state}; +use std::{future::Future, pin::Pin, sync::Arc}; + +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}; +use single_state_txpool::prune_known_txs_for_block; +pub use single_state_txpool::{BasicPool, RevalidationType}; +pub use transaction_pool_wrapper::TransactionPoolWrapper; + +type BoxedReadyIterator = Box< + dyn sc_transaction_pool_api::ReadyTransactions< + Item = Arc>, + > + Send, +>; type ReadyIteratorFor = BoxedReadyIterator, graph::ExtrinsicFor>; type PolledIterator = Pin> + Send>>; -/// A transaction pool for a full node. -pub type FullPool = BasicPool, Block>; - -/// Basic implementation of transaction pool that can be customized by providing PoolApi. -pub struct BasicPool -where - Block: BlockT, - PoolApi: graph::ChainApi, -{ - pool: Arc>, - api: Arc, - revalidation_strategy: Arc>>>, - revalidation_queue: Arc>, - ready_poll: Arc, Block>>>, - metrics: PrometheusMetrics, - enactment_state: Arc>>, -} - -struct ReadyPoll { - updated_at: NumberFor, - pollers: Vec<(NumberFor, oneshot::Sender)>, -} - -impl Default for ReadyPoll { - fn default() -> Self { - Self { updated_at: NumberFor::::zero(), pollers: Default::default() } - } -} - -impl ReadyPoll { - fn new(best_block_number: NumberFor) -> Self { - Self { updated_at: best_block_number, pollers: Default::default() } - } - - fn trigger(&mut self, number: NumberFor, iterator_factory: impl Fn() -> T) { - self.updated_at = number; - - let mut idx = 0; - while idx < self.pollers.len() { - if self.pollers[idx].0 <= number { - let poller_sender = self.pollers.swap_remove(idx); - log::debug!(target: LOG_TARGET, "Sending ready signal at block {}", number); - let _ = poller_sender.1.send(iterator_factory()); - } else { - idx += 1; - } - } - } - - fn add(&mut self, number: NumberFor) -> oneshot::Receiver { - let (sender, receiver) = oneshot::channel(); - self.pollers.push((number, sender)); - receiver - } - - fn updated_at(&self) -> NumberFor { - self.updated_at - } -} - -/// Type of revalidation. -pub enum RevalidationType { - /// Light revalidation type. - /// - /// During maintenance, transaction pool makes periodic revalidation - /// of all transactions depending on number of blocks or time passed. - /// Also this kind of revalidation does not resubmit transactions from - /// retracted blocks, since it is too expensive. - Light, - - /// Full revalidation type. - /// - /// During maintenance, transaction pool revalidates some fixed amount of - /// transactions from the pool of valid transactions. - Full, -} - -impl BasicPool -where - Block: BlockT, - PoolApi: graph::ChainApi + 'static, -{ - /// Create new basic transaction pool with provided api, for tests. - pub fn new_test( - pool_api: Arc, - best_block_hash: Block::Hash, - finalized_hash: Block::Hash, - options: graph::Options, - ) -> (Self, Pin + Send>>) { - let pool = Arc::new(graph::Pool::new(options, true.into(), pool_api.clone())); - let (revalidation_queue, background_task) = revalidation::RevalidationQueue::new_background( - pool_api.clone(), - pool.clone(), - finalized_hash, - ); - ( - Self { - api: pool_api, - pool, - revalidation_queue: Arc::new(revalidation_queue), - revalidation_strategy: Arc::new(Mutex::new(RevalidationStrategy::Always)), - ready_poll: Default::default(), - metrics: Default::default(), - enactment_state: Arc::new(Mutex::new(EnactmentState::new( - best_block_hash, - finalized_hash, - ))), - }, - background_task, - ) - } - - /// Create new basic transaction pool with provided api and custom - /// revalidation type. - pub fn with_revalidation_type( - options: graph::Options, - is_validator: IsValidator, - pool_api: Arc, - prometheus: Option<&PrometheusRegistry>, - revalidation_type: RevalidationType, - spawner: impl SpawnEssentialNamed, - best_block_number: NumberFor, - best_block_hash: Block::Hash, - finalized_hash: Block::Hash, - ) -> Self { - let pool = Arc::new(graph::Pool::new(options, is_validator, pool_api.clone())); - let (revalidation_queue, background_task) = match revalidation_type { - RevalidationType::Light => - (revalidation::RevalidationQueue::new(pool_api.clone(), pool.clone()), None), - RevalidationType::Full => { - let (queue, background) = revalidation::RevalidationQueue::new_background( - pool_api.clone(), - pool.clone(), - finalized_hash, - ); - (queue, Some(background)) - }, - }; - - if let Some(background_task) = background_task { - spawner.spawn_essential("txpool-background", Some("transaction-pool"), background_task); - } - - Self { - api: pool_api, - pool, - revalidation_queue: Arc::new(revalidation_queue), - revalidation_strategy: Arc::new(Mutex::new(match revalidation_type { - RevalidationType::Light => - RevalidationStrategy::Light(RevalidationStatus::NotScheduled), - RevalidationType::Full => RevalidationStrategy::Always, - })), - ready_poll: Arc::new(Mutex::new(ReadyPoll::new(best_block_number))), - metrics: PrometheusMetrics::new(prometheus), - enactment_state: Arc::new(Mutex::new(EnactmentState::new( - best_block_hash, - finalized_hash, - ))), - } - } - - /// Gets shared reference to the underlying pool. - pub fn pool(&self) -> &Arc> { - &self.pool - } - - /// Get access to the underlying api - pub fn api(&self) -> &PoolApi { - &self.api - } -} - -impl TransactionPool for BasicPool -where - Block: BlockT, - PoolApi: 'static + graph::ChainApi, -{ - type Block = PoolApi::Block; - type Hash = graph::ExtrinsicHash; - type InPoolTransaction = graph::base_pool::Transaction, TransactionFor>; - type Error = PoolApi::Error; - - fn submit_at( - &self, - at: ::Hash, - source: TransactionSource, - xts: Vec>, - ) -> PoolFuture, Self::Error>>, Self::Error> { - let pool = self.pool.clone(); - - self.metrics - .report(|metrics| metrics.submitted_transactions.inc_by(xts.len() as u64)); - - async move { pool.submit_at(at, source, xts).await }.boxed() - } - - fn submit_one( - &self, - at: ::Hash, - source: TransactionSource, - xt: TransactionFor, - ) -> PoolFuture, Self::Error> { - let pool = self.pool.clone(); - - self.metrics.report(|metrics| metrics.submitted_transactions.inc()); - - async move { pool.submit_one(at, source, xt).await }.boxed() - } - - fn submit_and_watch( - &self, - at: ::Hash, - source: TransactionSource, - xt: TransactionFor, - ) -> PoolFuture>>, Self::Error> { - let pool = self.pool.clone(); - - self.metrics.report(|metrics| metrics.submitted_transactions.inc()); - - async move { - let watcher = pool.submit_and_watch(at, source, xt).await?; - - Ok(watcher.into_stream().boxed()) - } - .boxed() - } - - fn remove_invalid(&self, hashes: &[TxHash]) -> Vec> { - let removed = self.pool.validated_pool().remove_invalid(hashes); - self.metrics - .report(|metrics| metrics.validations_invalid.inc_by(removed.len() as u64)); - removed - } - - fn status(&self) -> PoolStatus { - self.pool.validated_pool().status() - } - - fn import_notification_stream(&self) -> ImportNotificationStream> { - self.pool.validated_pool().import_notification_stream() - } - - fn hash_of(&self, xt: &TransactionFor) -> TxHash { - self.pool.hash_of(xt) - } - - fn on_broadcasted(&self, propagations: HashMap, Vec>) { - self.pool.validated_pool().on_broadcasted(propagations) - } - - fn ready_transaction(&self, hash: &TxHash) -> Option> { - self.pool.validated_pool().ready_by_hash(hash) - } - - fn ready_at(&self, at: NumberFor) -> PolledIterator { - let status = self.status(); - // If there are no transactions in the pool, it is fine to return early. - // - // There could be transaction being added because of some re-org happening at the relevant - // block, but this is relative unlikely. - if status.ready == 0 && status.future == 0 { - return async { Box::new(std::iter::empty()) as Box<_> }.boxed() - } - - if self.ready_poll.lock().updated_at() >= at { - log::trace!(target: LOG_TARGET, "Transaction pool already processed block #{}", at); - let iterator: ReadyIteratorFor = Box::new(self.pool.validated_pool().ready()); - return async move { iterator }.boxed() - } - - self.ready_poll - .lock() - .add(at) - .map(|received| { - received.unwrap_or_else(|e| { - log::warn!("Error receiving pending set: {:?}", e); - Box::new(std::iter::empty()) - }) - }) - .boxed() - } - - fn ready(&self) -> ReadyIteratorFor { - Box::new(self.pool.validated_pool().ready()) - } - - fn futures(&self) -> Vec { - let pool = self.pool.validated_pool().pool.read(); - - pool.futures().cloned().collect::>() - } -} - -impl FullPool -where - Block: BlockT, - Client: sp_api::ProvideRuntimeApi - + sc_client_api::BlockBackend - + sc_client_api::blockchain::HeaderBackend - + sp_runtime::traits::BlockIdTo - + sc_client_api::ExecutorProvider - + sc_client_api::UsageProvider - + sp_blockchain::HeaderMetadata - + Send - + Sync - + 'static, - Client::Api: sp_transaction_pool::runtime_api::TaggedTransactionQueue, -{ - /// Create new basic transaction pool for a full node with the provided api. - pub fn new_full( - options: graph::Options, - is_validator: IsValidator, - prometheus: Option<&PrometheusRegistry>, - spawner: impl SpawnEssentialNamed, - client: Arc, - ) -> Arc { - let pool_api = Arc::new(FullChainApi::new(client.clone(), prometheus, &spawner)); - let pool = Arc::new(Self::with_revalidation_type( - options, - is_validator, - pool_api, - prometheus, - RevalidationType::Full, - spawner, - client.usage_info().chain.best_number, - client.usage_info().chain.best_hash, - client.usage_info().chain.finalized_hash, - )); - - pool - } -} - -impl sc_transaction_pool_api::LocalTransactionPool - for BasicPool, Block> -where - Block: BlockT, - Client: sp_api::ProvideRuntimeApi - + sc_client_api::BlockBackend - + sc_client_api::blockchain::HeaderBackend - + sp_runtime::traits::BlockIdTo - + sp_blockchain::HeaderMetadata, - Client: Send + Sync + 'static, - Client::Api: sp_transaction_pool::runtime_api::TaggedTransactionQueue, -{ - type Block = Block; - type Hash = graph::ExtrinsicHash>; - type Error = as graph::ChainApi>::Error; - - fn submit_local( - &self, - at: Block::Hash, - xt: sc_transaction_pool_api::LocalTransactionFor, - ) -> Result { - use sp_runtime::{ - traits::SaturatedConversion, transaction_validity::TransactionValidityError, - }; - - let validity = self - .api - .validate_transaction_blocking(at, TransactionSource::Local, xt.clone())? - .map_err(|e| { - Self::Error::Pool(match e { - TransactionValidityError::Invalid(i) => TxPoolError::InvalidTransaction(i), - TransactionValidityError::Unknown(u) => TxPoolError::UnknownTransaction(u), - }) - })?; - - let (hash, bytes) = self.pool.validated_pool().api().hash_and_length(&xt); - let block_number = self - .api - .block_id_to_number(&BlockId::hash(at))? - .ok_or_else(|| error::Error::BlockIdConversion(format!("{:?}", at)))?; - - let validated = ValidatedTransaction::valid_at( - block_number.saturated_into::(), - hash, - TransactionSource::Local, - xt, - bytes, - validity, - ); - - self.pool.validated_pool().submit(vec![validated]).remove(0) - } -} - -#[cfg_attr(test, derive(Debug))] -enum RevalidationStatus { - /// The revalidation has never been completed. - NotScheduled, - /// The revalidation is scheduled. - Scheduled(Option, Option), - /// The revalidation is in progress. - InProgress, -} - -enum RevalidationStrategy { - Always, - Light(RevalidationStatus), -} - -struct RevalidationAction { - revalidate: bool, - resubmit: bool, -} - -impl RevalidationStrategy { - pub fn clear(&mut self) { - if let Self::Light(status) = self { - status.clear() - } - } - - pub fn next( - &mut self, - block: N, - revalidate_time_period: Option, - revalidate_block_period: Option, - ) -> RevalidationAction { - match self { - Self::Light(status) => RevalidationAction { - revalidate: status.next_required( - block, - revalidate_time_period, - revalidate_block_period, - ), - resubmit: false, - }, - Self::Always => RevalidationAction { revalidate: true, resubmit: true }, - } - } -} - -impl RevalidationStatus { - /// Called when revalidation is completed. - pub fn clear(&mut self) { - *self = Self::NotScheduled; - } - - /// Returns true if revalidation is required. - pub fn next_required( - &mut self, - block: N, - revalidate_time_period: Option, - revalidate_block_period: Option, - ) -> bool { - match *self { - Self::NotScheduled => { - *self = Self::Scheduled( - revalidate_time_period.map(|period| Instant::now() + period), - revalidate_block_period.map(|period| block + period), - ); - false - }, - Self::Scheduled(revalidate_at_time, revalidate_at_block) => { - let is_required = - revalidate_at_time.map(|at| Instant::now() >= at).unwrap_or(false) || - revalidate_at_block.map(|at| block >= at).unwrap_or(false); - if is_required { - *self = Self::InProgress; - } - is_required - }, - Self::InProgress => false, - } - } -} - -/// Prune the known txs for the given block. -async fn prune_known_txs_for_block>( - block_hash: Block::Hash, - api: &Api, - pool: &graph::Pool, -) -> Vec> { - let extrinsics = api - .block_body(block_hash) - .await - .unwrap_or_else(|e| { - log::warn!("Prune known transactions: error request: {}", e); - None - }) - .unwrap_or_default(); - - let hashes = extrinsics.iter().map(|tx| pool.hash_of(tx)).collect::>(); - - log::trace!(target: LOG_TARGET, "Pruning transactions: {:?}", hashes); - - let header = match api.block_header(block_hash) { - Ok(Some(h)) => h, - Ok(None) => { - log::debug!(target: LOG_TARGET, "Could not find header for {:?}.", block_hash); - return hashes - }, - Err(e) => { - log::debug!(target: LOG_TARGET, "Error retrieving header for {:?}: {}", block_hash, e); - return hashes - }, - }; - - if let Err(e) = pool.prune(block_hash, *header.parent_hash(), &extrinsics).await { - log::error!("Cannot prune known in the pool: {}", e); - } - - hashes -} - -impl BasicPool -where - Block: BlockT, - PoolApi: 'static + graph::ChainApi, -{ - /// Handles enactment and retraction of blocks, prunes stale transactions - /// (that have already been enacted) and resubmits transactions that were - /// retracted. - async fn handle_enactment(&self, tree_route: TreeRoute) { - log::trace!(target: LOG_TARGET, "handle_enactment tree_route: {tree_route:?}"); - let pool = self.pool.clone(); - let api = self.api.clone(); - - let (hash, block_number) = match tree_route.last() { - Some(HashAndNumber { hash, number }) => (hash, number), - None => { - log::warn!( - target: LOG_TARGET, - "Skipping ChainEvent - no last block in tree route {:?}", - tree_route, - ); - return - }, - }; - - let next_action = self.revalidation_strategy.lock().next( - *block_number, - Some(std::time::Duration::from_secs(60)), - Some(20u32.into()), - ); - - // We keep track of everything we prune so that later we won't add - // transactions with those hashes from the retracted blocks. - let mut pruned_log = HashSet::>::new(); - - // If there is a tree route, we use this to prune known tx based on the enacted - // blocks. Before pruning enacted transactions, we inform the listeners about - // retracted blocks and their transactions. This order is important, because - // if we enact and retract the same transaction at the same time, we want to - // send first the retract and than the prune event. - for retracted in tree_route.retracted() { - // notify txs awaiting finality that it has been retracted - pool.validated_pool().on_block_retracted(retracted.hash); - } - - future::join_all( - tree_route - .enacted() - .iter() - .map(|h| prune_known_txs_for_block(h.hash, &*api, &*pool)), - ) - .await - .into_iter() - .for_each(|enacted_log| { - pruned_log.extend(enacted_log); - }); - - self.metrics - .report(|metrics| metrics.block_transactions_pruned.inc_by(pruned_log.len() as u64)); - - if next_action.resubmit { - let mut resubmit_transactions = Vec::new(); - - for retracted in tree_route.retracted() { - let hash = retracted.hash; - - let block_transactions = api - .block_body(hash) - .await - .unwrap_or_else(|e| { - log::warn!("Failed to fetch block body: {}", e); - None - }) - .unwrap_or_default() - .into_iter() - .filter(|tx| tx.is_signed().unwrap_or(true)); - - let mut resubmitted_to_report = 0; - - resubmit_transactions.extend(block_transactions.into_iter().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::debug!( - target: LOG_TARGET, - "[{:?}]: Resubmitting from retracted block {:?}", - tx_hash, - hash, - ); - } - !contains - })); - - self.metrics.report(|metrics| { - metrics.block_transactions_resubmitted.inc_by(resubmitted_to_report) - }); - } - - if let Err(e) = pool - .resubmit_at( - *hash, - // These transactions are coming from retracted blocks, we should - // simply consider them external. - TransactionSource::External, - resubmit_transactions, - ) - .await - { - log::debug!( - target: LOG_TARGET, - "[{:?}] Error re-submitting transactions: {}", - hash, - e, - ) - } - } - - let extra_pool = pool.clone(); - // After #5200 lands, this arguably might be moved to the - // handler of "all blocks notification". - self.ready_poll - .lock() - .trigger(*block_number, move || Box::new(extra_pool.validated_pool().ready())); - - if next_action.revalidate { - let hashes = pool.validated_pool().ready().map(|tx| tx.hash).collect(); - self.revalidation_queue.revalidate_later(*hash, hashes).await; - - self.revalidation_strategy.lock().clear(); - } - } -} - -#[async_trait] -impl MaintainedTransactionPool for BasicPool -where - Block: BlockT, - PoolApi: 'static + graph::ChainApi, -{ - async fn maintain(&self, event: ChainEvent) { - let prev_finalized_block = self.enactment_state.lock().recent_finalized_block(); - let compute_tree_route = |from, to| -> Result, String> { - match self.api.tree_route(from, to) { - Ok(tree_route) => Ok(tree_route), - Err(e) => - return Err(format!( - "Error occurred while computing tree_route from {from:?} to {to:?}: {e}" - )), - } - }; - let block_id_to_number = - |hash| self.api.block_id_to_number(&BlockId::Hash(hash)).map_err(|e| format!("{}", e)); - - let result = - self.enactment_state - .lock() - .update(&event, &compute_tree_route, &block_id_to_number); - - match result { - Err(msg) => { - log::debug!(target: LOG_TARGET, "{msg}"); - self.enactment_state.lock().force_update(&event); - }, - Ok(EnactmentAction::Skip) => return, - Ok(EnactmentAction::HandleFinalization) => {}, - Ok(EnactmentAction::HandleEnactment(tree_route)) => { - self.handle_enactment(tree_route).await; - }, - }; - - if let ChainEvent::Finalized { hash, tree_route } = event { - log::trace!( - target: LOG_TARGET, - "on-finalized enacted: {tree_route:?}, previously finalized: \ - {prev_finalized_block:?}", - ); - - for hash in tree_route.iter().chain(std::iter::once(&hash)) { - if let Err(e) = self.pool.validated_pool().on_block_finalized(*hash).await { - log::warn!( - target: LOG_TARGET, - "Error occurred while attempting to notify watchers about finalization {}: {}", - hash, e - ) - } - } - } - } -} - -/// Inform the transaction pool about imported and finalized blocks. -pub async fn notification_future(client: Arc, txpool: Arc) -where - Block: BlockT, - Client: sc_client_api::BlockchainEvents, - Pool: MaintainedTransactionPool, -{ - let import_stream = client - .import_notification_stream() - .filter_map(|n| ready(n.try_into().ok())) - .fuse(); - let finality_stream = client.finality_notification_stream().map(Into::into).fuse(); - - futures::stream::select(import_stream, finality_stream) - .for_each(|evt| txpool.maintain(evt)) - .await -} +/// Log target for transaction pool. +/// +/// It can be used by other components for logging functionality strictly related to txpool (e.g. +/// importing transaction). +pub const LOG_TARGET: &str = "txpool"; diff --git a/substrate/client/transaction-pool/src/single_state_txpool/metrics.rs b/substrate/client/transaction-pool/src/single_state_txpool/metrics.rs new file mode 100644 index 000000000000..28a0f66e7edc --- /dev/null +++ b/substrate/client/transaction-pool/src/single_state_txpool/metrics.rs @@ -0,0 +1,67 @@ +// 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 . + +//! Transaction pool Prometheus metrics for single-state transaction pool. + +use crate::common::metrics::{GenericMetricsLink, MetricsRegistrant}; +use prometheus_endpoint::{register, Counter, PrometheusError, Registry, U64}; + +pub type MetricsLink = GenericMetricsLink; + +/// Transaction pool Prometheus metrics. +pub struct Metrics { + pub submitted_transactions: Counter, + pub validations_invalid: Counter, + pub block_transactions_pruned: Counter, + pub block_transactions_resubmitted: Counter, +} + +impl MetricsRegistrant for Metrics { + fn register(registry: &Registry) -> Result, PrometheusError> { + Ok(Box::from(Self { + submitted_transactions: register( + Counter::new( + "substrate_sub_txpool_submitted_transactions", + "Total number of transactions submitted", + )?, + registry, + )?, + validations_invalid: register( + Counter::new( + "substrate_sub_txpool_validations_invalid", + "Total number of transactions that were removed from the pool as invalid", + )?, + registry, + )?, + block_transactions_pruned: register( + Counter::new( + "substrate_sub_txpool_block_transactions_pruned", + "Total number of transactions that was requested to be pruned by block events", + )?, + registry, + )?, + block_transactions_resubmitted: register( + Counter::new( + "substrate_sub_txpool_block_transactions_resubmitted", + "Total number of transactions that was requested to be resubmitted by block events", + )?, + registry, + )?, + })) + } +} diff --git a/substrate/client/transaction-pool/src/single_state_txpool/mod.rs b/substrate/client/transaction-pool/src/single_state_txpool/mod.rs new file mode 100644 index 000000000000..d7ebb8c01cec --- /dev/null +++ b/substrate/client/transaction-pool/src/single_state_txpool/mod.rs @@ -0,0 +1,26 @@ +// 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 . + +//! Substrate single state transaction pool implementation. + +mod metrics; +mod revalidation; +pub(crate) mod single_state_txpool; + +pub(crate) use single_state_txpool::prune_known_txs_for_block; +pub use single_state_txpool::{BasicPool, RevalidationType}; diff --git a/substrate/client/transaction-pool/src/revalidation.rs b/substrate/client/transaction-pool/src/single_state_txpool/revalidation.rs similarity index 91% rename from substrate/client/transaction-pool/src/revalidation.rs rename to substrate/client/transaction-pool/src/single_state_txpool/revalidation.rs index 488ab19d8eab..5ef726c9f7d3 100644 --- a/substrate/client/transaction-pool/src/revalidation.rs +++ b/substrate/client/transaction-pool/src/single_state_txpool/revalidation.rs @@ -24,10 +24,7 @@ use std::{ sync::Arc, }; -use crate::{ - graph::{BlockHash, ChainApi, ExtrinsicHash, Pool, ValidatedTransaction}, - LOG_TARGET, -}; +use crate::graph::{BlockHash, ChainApi, ExtrinsicHash, Pool, ValidatedTransaction}; use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; use sp_runtime::{ generic::BlockId, traits::SaturatedConversion, transaction_validity::TransactionValidityError, @@ -40,6 +37,8 @@ const BACKGROUND_REVALIDATION_INTERVAL: Duration = Duration::from_millis(200); const MIN_BACKGROUND_REVALIDATION_BATCH_SIZE: usize = 20; +const LOG_TARGET: &str = "txpool::revalidation"; + /// Payload from queue to worker. struct WorkerPayload { at: BlockHash, @@ -75,11 +74,11 @@ async fn batch_revalidate( let block_number = match api.block_id_to_number(&BlockId::Hash(at)) { Ok(Some(n)) => n, Ok(None) => { - log::debug!(target: LOG_TARGET, "revalidation skipped at block {at:?}, could not get block number."); + log::trace!(target: LOG_TARGET, "revalidation skipped at block {at:?}, could not get block number."); return }, Err(e) => { - log::debug!(target: LOG_TARGET, "revalidation skipped at block {at:?}: {e:?}."); + log::trace!(target: LOG_TARGET, "revalidation skipped at block {at:?}: {e:?}."); return }, }; @@ -98,7 +97,7 @@ async fn batch_revalidate( for (validation_result, ext_hash, ext) in validation_results { match validation_result { Ok(Err(TransactionValidityError::Invalid(err))) => { - log::debug!( + log::trace!( target: LOG_TARGET, "[{:?}]: Revalidation: invalid {:?}", ext_hash, @@ -130,7 +129,7 @@ async fn batch_revalidate( ); }, Err(validation_err) => { - log::debug!( + log::trace!( target: LOG_TARGET, "[{:?}]: Removing due to error during revalidation: {}", ext_hash, @@ -256,7 +255,7 @@ impl RevalidationWorker { batch_revalidate(this.pool.clone(), this.api.clone(), this.best_block, next_batch).await; if batch_len > 0 || this.len() > 0 { - log::debug!( + log::trace!( target: LOG_TARGET, "Revalidated {} transactions. Left in the queue for revalidation: {}.", batch_len, @@ -273,7 +272,7 @@ impl RevalidationWorker { this.push(worker_payload); if this.members.len() > 0 { - log::debug!( + log::trace!( target: LOG_TARGET, "Updated revalidation queue at {:?}. Transactions: {:?}", this.best_block, @@ -359,6 +358,10 @@ where log::warn!(target: LOG_TARGET, "Failed to update background worker: {:?}", e); } } else { + log::debug!( + target: LOG_TARGET, + "batch_revalidate direct call" + ); let pool = self.pool.clone(); let api = self.api.clone(); batch_revalidate(pool, api, at, transactions).await @@ -370,8 +373,8 @@ where mod tests { use super::*; use crate::{ + common::tests::{uxt, TestApi}, graph::Pool, - tests::{uxt, TestApi}, }; use futures::executor::block_on; use sc_transaction_pool_api::TransactionSource; @@ -391,13 +394,16 @@ mod tests { nonce: 0, }); - let hash_of_block0 = api.expect_hash_from_number(0); + let han_of_block0 = api.expect_hash_and_number(0); - let uxt_hash = - block_on(pool.submit_one(hash_of_block0, TransactionSource::External, uxt.clone())) - .expect("Should be valid"); + let uxt_hash = block_on(pool.submit_one( + &han_of_block0, + TransactionSource::External, + uxt.clone().into(), + )) + .expect("Should be valid"); - block_on(queue.revalidate_later(hash_of_block0, vec![uxt_hash])); + block_on(queue.revalidate_later(han_of_block0.hash, vec![uxt_hash])); // revalidated in sync offload 2nd time assert_eq!(api.validation_requests().len(), 2); @@ -424,21 +430,23 @@ mod tests { nonce: 1, }); - let hash_of_block0 = api.expect_hash_from_number(0); + 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(hash_of_block0, TransactionSource::External, vec![uxt0, uxt1])) - .expect("Should be valid") - .into_iter() - .map(|r| r.expect("Should be valid")) - .collect::>(); + 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::>(); assert_eq!(api.validation_requests().len(), 2); assert_eq!(pool.validated_pool().status().ready, 2); // revalidation works fine for block 0: - block_on(queue.revalidate_later(hash_of_block0, uxt_hashes.clone())); + block_on(queue.revalidate_later(han_of_block0.hash, uxt_hashes.clone())); assert_eq!(api.validation_requests().len(), 4); 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 new file mode 100644 index 000000000000..6b4feca44bf8 --- /dev/null +++ b/substrate/client/transaction-pool/src/single_state_txpool/single_state_txpool.rs @@ -0,0 +1,790 @@ +// 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 . + +//! Substrate transaction pool implementation. + +use super::{metrics::MetricsLink as PrometheusMetrics, revalidation}; +pub use crate::{ + api::FullChainApi, + graph::{ChainApi, ValidatedTransaction}, +}; +use crate::{ + common::{ + enactment_state::{EnactmentAction, EnactmentState}, + error, + log_xt::log_xt_trace, + }, + graph, + graph::{ExtrinsicHash, IsValidator}, + PolledIterator, ReadyIteratorFor, LOG_TARGET, +}; +use async_trait::async_trait; +use futures::{channel::oneshot, future, prelude::*, Future, FutureExt}; +use parking_lot::Mutex; +use prometheus_endpoint::Registry as PrometheusRegistry; +use sc_transaction_pool_api::{ + error::Error as TxPoolError, ChainEvent, ImportNotificationStream, MaintainedTransactionPool, + PoolFuture, PoolStatus, TransactionFor, TransactionPool, TransactionSource, + TransactionStatusStreamFor, TxHash, +}; +use sp_blockchain::{HashAndNumber, TreeRoute}; +use sp_core::traits::SpawnEssentialNamed; +use sp_runtime::{ + generic::BlockId, + traits::{AtLeast32Bit, Block as BlockT, Extrinsic, Header as HeaderT, NumberFor, Zero}, +}; +use std::{ + collections::{HashMap, HashSet}, + pin::Pin, + sync::Arc, + time::Instant, +}; +use tokio::select; + +/// Basic implementation of transaction pool that can be customized by providing PoolApi. +pub struct BasicPool +where + Block: BlockT, + PoolApi: graph::ChainApi, +{ + pool: Arc>, + api: Arc, + revalidation_strategy: Arc>>>, + revalidation_queue: Arc>, + ready_poll: Arc, Block>>>, + metrics: PrometheusMetrics, + enactment_state: Arc>>, +} + +struct ReadyPoll { + updated_at: NumberFor, + pollers: Vec<(NumberFor, oneshot::Sender)>, +} + +impl Default for ReadyPoll { + fn default() -> Self { + Self { updated_at: NumberFor::::zero(), pollers: Default::default() } + } +} + +impl ReadyPoll { + fn new(best_block_number: NumberFor) -> Self { + Self { updated_at: best_block_number, pollers: Default::default() } + } + + fn trigger(&mut self, number: NumberFor, iterator_factory: impl Fn() -> T) { + self.updated_at = number; + + let mut idx = 0; + while idx < self.pollers.len() { + if self.pollers[idx].0 <= number { + let poller_sender = self.pollers.swap_remove(idx); + log::trace!(target: LOG_TARGET, "Sending ready signal at block {}", number); + let _ = poller_sender.1.send(iterator_factory()); + } else { + idx += 1; + } + } + } + + fn add(&mut self, number: NumberFor) -> oneshot::Receiver { + let (sender, receiver) = oneshot::channel(); + self.pollers.push((number, sender)); + receiver + } + + fn updated_at(&self) -> NumberFor { + self.updated_at + } +} + +/// Type of revalidation. +pub enum RevalidationType { + /// Light revalidation type. + /// + /// During maintenance, transaction pool makes periodic revalidation + /// of all transactions depending on number of blocks or time passed. + /// Also this kind of revalidation does not resubmit transactions from + /// retracted blocks, since it is too expensive. + Light, + + /// Full revalidation type. + /// + /// During maintenance, transaction pool revalidates some fixed amount of + /// transactions from the pool of valid transactions. + Full, +} + +impl BasicPool +where + Block: BlockT, + PoolApi: graph::ChainApi + 'static, +{ + /// Create new basic transaction pool with provided api, for tests. + pub fn new_test( + pool_api: Arc, + best_block_hash: Block::Hash, + finalized_hash: Block::Hash, + options: graph::Options, + ) -> (Self, Pin + Send>>) { + let pool = Arc::new(graph::Pool::new(options, true.into(), pool_api.clone())); + let (revalidation_queue, background_task) = revalidation::RevalidationQueue::new_background( + pool_api.clone(), + pool.clone(), + finalized_hash, + ); + ( + Self { + api: pool_api, + pool, + revalidation_queue: Arc::new(revalidation_queue), + revalidation_strategy: Arc::new(Mutex::new(RevalidationStrategy::Always)), + ready_poll: Default::default(), + metrics: Default::default(), + enactment_state: Arc::new(Mutex::new(EnactmentState::new( + best_block_hash, + finalized_hash, + ))), + }, + background_task, + ) + } + + /// Create new basic transaction pool with provided api and custom + /// revalidation type. + pub fn with_revalidation_type( + options: graph::Options, + is_validator: IsValidator, + pool_api: Arc, + prometheus: Option<&PrometheusRegistry>, + revalidation_type: RevalidationType, + spawner: impl SpawnEssentialNamed, + best_block_number: NumberFor, + best_block_hash: Block::Hash, + finalized_hash: Block::Hash, + ) -> Self { + let pool = Arc::new(graph::Pool::new(options, is_validator, pool_api.clone())); + let (revalidation_queue, background_task) = match revalidation_type { + RevalidationType::Light => + (revalidation::RevalidationQueue::new(pool_api.clone(), pool.clone()), None), + RevalidationType::Full => { + let (queue, background) = revalidation::RevalidationQueue::new_background( + pool_api.clone(), + pool.clone(), + finalized_hash, + ); + (queue, Some(background)) + }, + }; + + if let Some(background_task) = background_task { + spawner.spawn_essential("txpool-background", Some("transaction-pool"), background_task); + } + + Self { + api: pool_api, + pool, + revalidation_queue: Arc::new(revalidation_queue), + revalidation_strategy: Arc::new(Mutex::new(match revalidation_type { + RevalidationType::Light => + RevalidationStrategy::Light(RevalidationStatus::NotScheduled), + RevalidationType::Full => RevalidationStrategy::Always, + })), + ready_poll: Arc::new(Mutex::new(ReadyPoll::new(best_block_number))), + metrics: PrometheusMetrics::new(prometheus), + enactment_state: Arc::new(Mutex::new(EnactmentState::new( + best_block_hash, + finalized_hash, + ))), + } + } + + /// Gets shared reference to the underlying pool. + pub fn pool(&self) -> &Arc> { + &self.pool + } + + /// Get access to the underlying api + pub fn api(&self) -> &PoolApi { + &self.api + } + + fn ready_at_with_timeout_internal( + &self, + at: Block::Hash, + timeout: std::time::Duration, + ) -> PolledIterator { + let timeout = futures_timer::Delay::new(timeout); + let ready_maintained = self.ready_at(at); + let ready_current = self.ready(); + + let ready = async { + select! { + ready = ready_maintained => ready, + _ = timeout => ready_current + } + }; + + Box::pin(ready) + } +} + +impl TransactionPool for BasicPool +where + Block: BlockT, + PoolApi: 'static + graph::ChainApi, +{ + type Block = PoolApi::Block; + type Hash = graph::ExtrinsicHash; + type InPoolTransaction = + graph::base_pool::Transaction, graph::ExtrinsicFor>; + type Error = PoolApi::Error; + + fn submit_at( + &self, + at: ::Hash, + source: TransactionSource, + xts: Vec>, + ) -> PoolFuture, Self::Error>>, Self::Error> { + let pool = self.pool.clone(); + let xts = xts.into_iter().map(Arc::from).collect::>(); + + self.metrics + .report(|metrics| metrics.submitted_transactions.inc_by(xts.len() as u64)); + + let number = self.api.resolve_block_number(at); + async move { + let at = HashAndNumber { hash: at, number: number? }; + Ok(pool.submit_at(&at, source, xts).await) + } + .boxed() + } + + fn submit_one( + &self, + at: ::Hash, + source: TransactionSource, + xt: TransactionFor, + ) -> PoolFuture, Self::Error> { + let pool = self.pool.clone(); + let xt = Arc::from(xt); + + self.metrics.report(|metrics| metrics.submitted_transactions.inc()); + + let number = self.api.resolve_block_number(at); + async move { + let at = HashAndNumber { hash: at, number: number? }; + pool.submit_one(&at, source, xt).await + } + .boxed() + } + + fn submit_and_watch( + &self, + at: ::Hash, + source: TransactionSource, + xt: TransactionFor, + ) -> PoolFuture>>, Self::Error> { + let pool = self.pool.clone(); + let xt = Arc::from(xt); + + self.metrics.report(|metrics| metrics.submitted_transactions.inc()); + + let number = self.api.resolve_block_number(at); + + async move { + let at = HashAndNumber { hash: at, number: number? }; + let watcher = pool.submit_and_watch(&at, source, xt).await?; + + Ok(watcher.into_stream().boxed()) + } + .boxed() + } + + fn remove_invalid(&self, hashes: &[TxHash]) -> Vec> { + let removed = self.pool.validated_pool().remove_invalid(hashes); + self.metrics + .report(|metrics| metrics.validations_invalid.inc_by(removed.len() as u64)); + removed + } + + fn status(&self) -> PoolStatus { + self.pool.validated_pool().status() + } + + fn import_notification_stream(&self) -> ImportNotificationStream> { + self.pool.validated_pool().import_notification_stream() + } + + fn hash_of(&self, xt: &TransactionFor) -> TxHash { + self.pool.hash_of(xt) + } + + fn on_broadcasted(&self, propagations: HashMap, Vec>) { + self.pool.validated_pool().on_broadcasted(propagations) + } + + fn ready_transaction(&self, hash: &TxHash) -> Option> { + self.pool.validated_pool().ready_by_hash(hash) + } + + fn ready_at(&self, at: ::Hash) -> PolledIterator { + let Ok(at) = self.api.resolve_block_number(at) else { + return async { Box::new(std::iter::empty()) as Box<_> }.boxed() + }; + + let status = self.status(); + // If there are no transactions in the pool, it is fine to return early. + // + // There could be transaction being added because of some re-org happening at the relevant + // block, but this is relative unlikely. + if status.ready == 0 && status.future == 0 { + return async { Box::new(std::iter::empty()) as Box<_> }.boxed() + } + + if self.ready_poll.lock().updated_at() >= at { + log::trace!(target: LOG_TARGET, "Transaction pool already processed block #{}", at); + let iterator: ReadyIteratorFor = Box::new(self.pool.validated_pool().ready()); + return async move { iterator }.boxed() + } + + self.ready_poll + .lock() + .add(at) + .map(|received| { + received.unwrap_or_else(|e| { + log::warn!(target: LOG_TARGET, "Error receiving pending set: {:?}", e); + Box::new(std::iter::empty()) + }) + }) + .boxed() + } + + fn ready(&self) -> ReadyIteratorFor { + Box::new(self.pool.validated_pool().ready()) + } + + fn futures(&self) -> Vec { + let pool = self.pool.validated_pool().pool.read(); + pool.futures().cloned().collect::>() + } + + fn ready_at_with_timeout( + &self, + at: ::Hash, + timeout: std::time::Duration, + ) -> PolledIterator { + self.ready_at_with_timeout_internal(at, timeout) + } +} + +impl BasicPool, Block> +where + Block: BlockT, + Client: sp_api::ProvideRuntimeApi + + sc_client_api::BlockBackend + + sc_client_api::blockchain::HeaderBackend + + sp_runtime::traits::BlockIdTo + + sc_client_api::ExecutorProvider + + sc_client_api::UsageProvider + + sp_blockchain::HeaderMetadata + + Send + + Sync + + 'static, + Client::Api: sp_transaction_pool::runtime_api::TaggedTransactionQueue, +{ + /// Create new basic transaction pool for a full node with the provided api. + pub fn new_full( + options: graph::Options, + is_validator: IsValidator, + prometheus: Option<&PrometheusRegistry>, + spawner: impl SpawnEssentialNamed, + client: Arc, + ) -> Self { + let pool_api = Arc::new(FullChainApi::new(client.clone(), prometheus, &spawner)); + let pool = Self::with_revalidation_type( + options, + is_validator, + pool_api, + prometheus, + RevalidationType::Full, + spawner, + client.usage_info().chain.best_number, + client.usage_info().chain.best_hash, + client.usage_info().chain.finalized_hash, + ); + + pool + } +} + +impl sc_transaction_pool_api::LocalTransactionPool + for BasicPool, Block> +where + Block: BlockT, + Client: sp_api::ProvideRuntimeApi + + sc_client_api::BlockBackend + + sc_client_api::blockchain::HeaderBackend + + sp_runtime::traits::BlockIdTo + + sp_blockchain::HeaderMetadata, + Client: Send + Sync + 'static, + Client::Api: sp_transaction_pool::runtime_api::TaggedTransactionQueue, +{ + type Block = Block; + type Hash = graph::ExtrinsicHash>; + type Error = as graph::ChainApi>::Error; + + fn submit_local( + &self, + at: Block::Hash, + xt: sc_transaction_pool_api::LocalTransactionFor, + ) -> Result { + use sp_runtime::{ + traits::SaturatedConversion, transaction_validity::TransactionValidityError, + }; + + let validity = self + .api + .validate_transaction_blocking(at, TransactionSource::Local, Arc::from(xt.clone()))? + .map_err(|e| { + Self::Error::Pool(match e { + TransactionValidityError::Invalid(i) => TxPoolError::InvalidTransaction(i), + TransactionValidityError::Unknown(u) => TxPoolError::UnknownTransaction(u), + }) + })?; + + let (hash, bytes) = self.pool.validated_pool().api().hash_and_length(&xt); + let block_number = self + .api + .block_id_to_number(&BlockId::hash(at))? + .ok_or_else(|| error::Error::BlockIdConversion(format!("{:?}", at)))?; + + let validated = ValidatedTransaction::valid_at( + block_number.saturated_into::(), + hash, + TransactionSource::Local, + Arc::from(xt), + bytes, + validity, + ); + + self.pool.validated_pool().submit(vec![validated]).remove(0) + } +} + +#[cfg_attr(test, derive(Debug))] +enum RevalidationStatus { + /// The revalidation has never been completed. + NotScheduled, + /// The revalidation is scheduled. + Scheduled(Option, Option), + /// The revalidation is in progress. + InProgress, +} + +enum RevalidationStrategy { + Always, + Light(RevalidationStatus), +} + +struct RevalidationAction { + revalidate: bool, + resubmit: bool, +} + +impl RevalidationStrategy { + pub fn clear(&mut self) { + if let Self::Light(status) = self { + status.clear() + } + } + + pub fn next( + &mut self, + block: N, + revalidate_time_period: Option, + revalidate_block_period: Option, + ) -> RevalidationAction { + match self { + Self::Light(status) => RevalidationAction { + revalidate: status.next_required( + block, + revalidate_time_period, + revalidate_block_period, + ), + resubmit: false, + }, + Self::Always => RevalidationAction { revalidate: true, resubmit: true }, + } + } +} + +impl RevalidationStatus { + /// Called when revalidation is completed. + pub fn clear(&mut self) { + *self = Self::NotScheduled; + } + + /// Returns true if revalidation is required. + pub fn next_required( + &mut self, + block: N, + revalidate_time_period: Option, + revalidate_block_period: Option, + ) -> bool { + match *self { + Self::NotScheduled => { + *self = Self::Scheduled( + revalidate_time_period.map(|period| Instant::now() + period), + revalidate_block_period.map(|period| block + period), + ); + false + }, + Self::Scheduled(revalidate_at_time, revalidate_at_block) => { + let is_required = + revalidate_at_time.map(|at| Instant::now() >= at).unwrap_or(false) || + revalidate_at_block.map(|at| block >= at).unwrap_or(false); + if is_required { + *self = Self::InProgress; + } + is_required + }, + Self::InProgress => false, + } + } +} + +/// Prune the known txs for the given block. +pub async fn prune_known_txs_for_block>( + at: &HashAndNumber, + api: &Api, + pool: &graph::Pool, +) -> Vec> { + let extrinsics = api + .block_body(at.hash) + .await + .unwrap_or_else(|e| { + log::warn!(target: LOG_TARGET, "Prune known transactions: error request: {}", e); + None + }) + .unwrap_or_default(); + + let hashes = extrinsics.iter().map(|tx| pool.hash_of(tx)).collect::>(); + + let header = match api.block_header(at.hash) { + Ok(Some(h)) => h, + Ok(None) => { + log::trace!(target: LOG_TARGET, "Could not find header for {:?}.", at.hash); + return hashes + }, + Err(e) => { + log::trace!(target: LOG_TARGET, "Error retrieving header for {:?}: {}", at.hash, e); + return hashes + }, + }; + + log_xt_trace!(target: LOG_TARGET, &hashes, "[{:?}] Pruning transaction."); + + pool.prune(at, *header.parent_hash(), &extrinsics).await; + hashes +} + +impl BasicPool +where + Block: BlockT, + PoolApi: 'static + graph::ChainApi, +{ + /// Handles enactment and retraction of blocks, prunes stale transactions + /// (that have already been enacted) and resubmits transactions that were + /// retracted. + async fn handle_enactment(&self, tree_route: TreeRoute) { + log::trace!(target: LOG_TARGET, "handle_enactment tree_route: {tree_route:?}"); + let pool = self.pool.clone(); + let api = self.api.clone(); + + let hash_and_number = match tree_route.last() { + Some(hash_and_number) => hash_and_number, + None => { + log::warn!( + target: LOG_TARGET, + "Skipping ChainEvent - no last block in tree route {:?}", + tree_route, + ); + return + }, + }; + + let next_action = self.revalidation_strategy.lock().next( + hash_and_number.number, + Some(std::time::Duration::from_secs(60)), + Some(20u32.into()), + ); + + // We keep track of everything we prune so that later we won't add + // transactions with those hashes from the retracted blocks. + let mut pruned_log = HashSet::>::new(); + + // If there is a tree route, we use this to prune known tx based on the enacted + // blocks. Before pruning enacted transactions, we inform the listeners about + // retracted blocks and their transactions. This order is important, because + // if we enact and retract the same transaction at the same time, we want to + // send first the retract and then the prune event. + for retracted in tree_route.retracted() { + // notify txs awaiting finality that it has been retracted + pool.validated_pool().on_block_retracted(retracted.hash); + } + + future::join_all( + tree_route.enacted().iter().map(|h| prune_known_txs_for_block(h, &*api, &*pool)), + ) + .await + .into_iter() + .for_each(|enacted_log| { + pruned_log.extend(enacted_log); + }); + + self.metrics + .report(|metrics| metrics.block_transactions_pruned.inc_by(pruned_log.len() as u64)); + + if next_action.resubmit { + let mut resubmit_transactions = Vec::new(); + + for retracted in tree_route.retracted() { + let hash = retracted.hash; + + let block_transactions = api + .block_body(hash) + .await + .unwrap_or_else(|e| { + log::warn!(target: LOG_TARGET, "Failed to fetch block body: {}", e); + None + }) + .unwrap_or_default() + .into_iter() + .filter(|tx| tx.is_signed().unwrap_or(true)); + + let mut resubmitted_to_report = 0; + + 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 + }), + ); + + self.metrics.report(|metrics| { + metrics.block_transactions_resubmitted.inc_by(resubmitted_to_report) + }); + } + + 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 extra_pool = pool.clone(); + // After #5200 lands, this arguably might be moved to the + // handler of "all blocks notification". + self.ready_poll + .lock() + .trigger(hash_and_number.number, move || Box::new(extra_pool.validated_pool().ready())); + + if next_action.revalidate { + let hashes = pool.validated_pool().ready().map(|tx| tx.hash).collect(); + self.revalidation_queue.revalidate_later(hash_and_number.hash, hashes).await; + + self.revalidation_strategy.lock().clear(); + } + } +} + +#[async_trait] +impl MaintainedTransactionPool for BasicPool +where + Block: BlockT, + PoolApi: 'static + graph::ChainApi, +{ + async fn maintain(&self, event: ChainEvent) { + let prev_finalized_block = self.enactment_state.lock().recent_finalized_block(); + let compute_tree_route = |from, to| -> Result, String> { + match self.api.tree_route(from, to) { + Ok(tree_route) => Ok(tree_route), + Err(e) => + return Err(format!( + "Error occurred while computing tree_route from {from:?} to {to:?}: {e}" + )), + } + }; + let block_id_to_number = + |hash| self.api.block_id_to_number(&BlockId::Hash(hash)).map_err(|e| format!("{}", e)); + + let result = + self.enactment_state + .lock() + .update(&event, &compute_tree_route, &block_id_to_number); + + match result { + Err(msg) => { + log::trace!(target: LOG_TARGET, "{msg}"); + self.enactment_state.lock().force_update(&event); + }, + Ok(EnactmentAction::Skip) => return, + Ok(EnactmentAction::HandleFinalization) => {}, + Ok(EnactmentAction::HandleEnactment(tree_route)) => { + self.handle_enactment(tree_route).await; + }, + }; + + if let ChainEvent::Finalized { hash, tree_route } = event { + log::trace!( + target: LOG_TARGET, + "on-finalized enacted: {tree_route:?}, previously finalized: \ + {prev_finalized_block:?}", + ); + + for hash in tree_route.iter().chain(std::iter::once(&hash)) { + if let Err(e) = self.pool.validated_pool().on_block_finalized(*hash).await { + log::warn!( + target: LOG_TARGET, + "Error occurred while attempting to notify watchers about finalization {}: {}", + hash, e + ) + } + } + } + } +} diff --git a/substrate/client/transaction-pool/src/transaction_pool_wrapper.rs b/substrate/client/transaction-pool/src/transaction_pool_wrapper.rs new file mode 100644 index 000000000000..4e1b53833b8f --- /dev/null +++ b/substrate/client/transaction-pool/src/transaction_pool_wrapper.rs @@ -0,0 +1,198 @@ +// 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 . + +//! Transaction pool wrapper. Provides a type for wrapping object providing actual implementation of +//! transaction pool. + +use crate::{ + builder::FullClientTransactionPool, + graph::{base_pool::Transaction, ExtrinsicFor, ExtrinsicHash}, + ChainApi, FullChainApi, +}; +use async_trait::async_trait; +use sc_transaction_pool_api::{ + ChainEvent, ImportNotificationStream, LocalTransactionFor, LocalTransactionPool, + MaintainedTransactionPool, PoolFuture, PoolStatus, ReadyTransactions, TransactionFor, + TransactionPool, TransactionSource, TransactionStatusStreamFor, TxHash, +}; +use sp_runtime::traits::Block as BlockT; +use std::{collections::HashMap, future::Future, pin::Pin, sync::Arc}; + +/// The wrapper for actual object providing implementation of TransactionPool. +/// +/// This wraps actual implementation of the TransactionPool, e.g. fork-aware or single-state. +pub struct TransactionPoolWrapper( + pub Box>, +) +where + Block: BlockT, + Client: sp_api::ProvideRuntimeApi + + sc_client_api::BlockBackend + + sc_client_api::blockchain::HeaderBackend + + sp_runtime::traits::BlockIdTo + + sp_blockchain::HeaderMetadata + + 'static, + Client::Api: sp_transaction_pool::runtime_api::TaggedTransactionQueue; + +impl TransactionPool for TransactionPoolWrapper +where + Block: BlockT, + Client: sp_api::ProvideRuntimeApi + + sc_client_api::BlockBackend + + sc_client_api::blockchain::HeaderBackend + + sp_runtime::traits::BlockIdTo + + sp_blockchain::HeaderMetadata + + 'static, + Client::Api: sp_transaction_pool::runtime_api::TaggedTransactionQueue, +{ + type Block = Block; + type Hash = ExtrinsicHash>; + type InPoolTransaction = Transaction< + ExtrinsicHash>, + ExtrinsicFor>, + >; + type Error = as ChainApi>::Error; + + fn submit_at( + &self, + at: ::Hash, + source: TransactionSource, + xts: Vec>, + ) -> PoolFuture, Self::Error>>, Self::Error> { + self.0.submit_at(at, source, xts) + } + + fn submit_one( + &self, + at: ::Hash, + source: TransactionSource, + xt: TransactionFor, + ) -> PoolFuture, Self::Error> { + self.0.submit_one(at, source, xt) + } + + fn submit_and_watch( + &self, + at: ::Hash, + source: TransactionSource, + xt: TransactionFor, + ) -> PoolFuture>>, Self::Error> { + self.0.submit_and_watch(at, source, xt) + } + + fn ready_at( + &self, + at: ::Hash, + ) -> Pin< + Box< + dyn Future< + Output = Box> + Send>, + > + Send, + >, + > { + self.0.ready_at(at) + } + + fn ready(&self) -> Box> + Send> { + self.0.ready() + } + + fn remove_invalid(&self, hashes: &[TxHash]) -> Vec> { + self.0.remove_invalid(hashes) + } + + fn futures(&self) -> Vec { + self.0.futures() + } + + fn status(&self) -> PoolStatus { + self.0.status() + } + + fn import_notification_stream(&self) -> ImportNotificationStream> { + self.0.import_notification_stream() + } + + fn on_broadcasted(&self, propagations: HashMap, Vec>) { + self.0.on_broadcasted(propagations) + } + + fn hash_of(&self, xt: &TransactionFor) -> TxHash { + self.0.hash_of(xt) + } + + fn ready_transaction(&self, hash: &TxHash) -> Option> { + self.0.ready_transaction(hash) + } + + fn ready_at_with_timeout( + &self, + at: ::Hash, + timeout: std::time::Duration, + ) -> Pin< + Box< + dyn Future< + Output = Box> + Send>, + > + Send + + '_, + >, + > { + self.0.ready_at_with_timeout(at, timeout) + } +} + +#[async_trait] +impl MaintainedTransactionPool for TransactionPoolWrapper +where + Block: BlockT, + Client: sp_api::ProvideRuntimeApi + + sc_client_api::BlockBackend + + sc_client_api::blockchain::HeaderBackend + + sp_runtime::traits::BlockIdTo + + sp_blockchain::HeaderMetadata + + 'static, + Client::Api: sp_transaction_pool::runtime_api::TaggedTransactionQueue, +{ + async fn maintain(&self, event: ChainEvent) { + self.0.maintain(event).await; + } +} + +impl LocalTransactionPool for TransactionPoolWrapper +where + Block: BlockT, + Client: sp_api::ProvideRuntimeApi + + sc_client_api::BlockBackend + + sc_client_api::blockchain::HeaderBackend + + sp_runtime::traits::BlockIdTo + + sp_blockchain::HeaderMetadata + + 'static, + Client::Api: sp_transaction_pool::runtime_api::TaggedTransactionQueue, +{ + type Block = Block; + type Hash = ExtrinsicHash>; + type Error = as ChainApi>::Error; + + fn submit_local( + &self, + at: ::Hash, + xt: LocalTransactionFor, + ) -> Result { + self.0.submit_local(at, xt) + } +} diff --git a/substrate/client/transaction-pool/tests/fatp.rs b/substrate/client/transaction-pool/tests/fatp.rs new file mode 100644 index 000000000000..9f343a9bd029 --- /dev/null +++ b/substrate/client/transaction-pool/tests/fatp.rs @@ -0,0 +1,2617 @@ +// 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 for fork-aware transaction pool. + +use fatp_common::{ + finalized_block_event, invalid_hash, new_best_block_event, pool, pool_with_api, + test_chain_with_forks, LOG_TARGET, SOURCE, +}; +use futures::{executor::block_on, task::Poll, FutureExt, StreamExt}; +use sc_transaction_pool::ChainApi; +use sc_transaction_pool_api::{ + error::{Error as TxPoolError, IntoPoolError}, + ChainEvent, MaintainedTransactionPool, TransactionPool, TransactionStatus, +}; +use sp_runtime::transaction_validity::InvalidTransaction; +use std::{sync::Arc, time::Duration}; +use substrate_test_runtime_client::AccountKeyring::*; +use substrate_test_runtime_transaction_pool::uxt; + +pub mod fatp_common; + +// Some ideas for tests: +// - view.ready iterator +// - stale transaction submission when there is single view only (expect error) +// - stale transaction submission when there are more views (expect ok if tx is ok for at least one +// view) +// - view count (e.g. same new block notified twice) +// - invalid with many views (different cases) +// +// review (from old pool) and maybe re-use: +// fn import_notification_to_pool_maintain_works() +// fn prune_tags_should_work() +// fn should_ban_invalid_transactions() +// fn should_correctly_prune_transactions_providing_more_than_one_tag() + +#[test] +fn fatp_no_view_future_and_ready_submit_one_works() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let header = api.push_block(1, vec![], true); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 202); + + let submissions = vec![ + pool.submit_one(header.hash(), SOURCE, xt0.clone()), + pool.submit_one(header.hash(), SOURCE, xt1.clone()), + ]; + + let results = block_on(futures::future::join_all(submissions)); + + assert!(results.iter().all(|r| { r.is_ok() })); +} + +#[test] +fn fatp_no_view_future_and_ready_submit_works() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let header = api.push_block(1, vec![], true); + + let xts0 = (200..205).map(|i| uxt(Alice, i)).collect::>(); + let xts1 = (205..210).map(|i| uxt(Alice, i)).collect::>(); + let xts2 = (215..220).map(|i| uxt(Alice, i)).collect::>(); + + let submissions = vec![ + pool.submit_at(header.hash(), SOURCE, xts0.clone()), + pool.submit_at(header.hash(), SOURCE, xts1.clone()), + pool.submit_at(header.hash(), SOURCE, xts2.clone()), + ]; + + let results = block_on(futures::future::join_all(submissions)); + + assert!(results.into_iter().flat_map(|x| x.unwrap()).all(|r| { r.is_ok() })); +} + +#[test] +fn fatp_no_view_submit_already_imported_reports_error() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let header = api.push_block(1, vec![], true); + + let xts0 = (215..220).map(|i| uxt(Alice, i)).collect::>(); + let xts1 = xts0.clone(); + + let submission_ok = pool.submit_at(header.hash(), SOURCE, xts0.clone()); + let results = block_on(submission_ok); + assert!(results.unwrap().into_iter().all(|r| r.is_ok())); + + let submission_failing = pool.submit_at(header.hash(), SOURCE, xts1.clone()); + let results = block_on(submission_failing); + + assert!(results + .unwrap() + .into_iter() + .all(|r| { matches!(r.unwrap_err().0, TxPoolError::AlreadyImported(_)) })); +} + +#[test] +fn fatp_one_view_future_and_ready_submit_one_works() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let header = api.push_block(1, vec![], true); + // let header01b = api.push_block(1, vec![], true); + + let event = new_best_block_event(&pool, None, header.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 202); + + let submissions = vec![ + pool.submit_one(header.hash(), SOURCE, xt0.clone()), + pool.submit_one(header.hash(), SOURCE, xt1.clone()), + ]; + + block_on(futures::future::join_all(submissions)); + + assert_pool_status!(header.hash(), &pool, 1, 1); +} + +#[test] +fn fatp_one_view_future_and_ready_submit_many_works() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let header = api.push_block(1, vec![], true); + // let header01b = api.push_block(1, vec![], true); + + let event = new_best_block_event(&pool, None, header.hash()); + block_on(pool.maintain(event)); + + let xts0 = (200..205).map(|i| uxt(Alice, i)).collect::>(); + let xts1 = (205..210).map(|i| uxt(Alice, i)).collect::>(); + let xts2 = (215..220).map(|i| uxt(Alice, i)).collect::>(); + + let submissions = vec![ + pool.submit_at(header.hash(), SOURCE, xts0.clone()), + pool.submit_at(header.hash(), SOURCE, xts1.clone()), + pool.submit_at(header.hash(), SOURCE, xts2.clone()), + ]; + + block_on(futures::future::join_all(submissions)); + + assert_pool_status!(header.hash(), &pool, 10, 5); +} + +#[test] +fn fatp_one_view_stale_submit_one_fails() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let header = api.push_block(1, vec![], true); + + let event = new_best_block_event(&pool, None, header.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 100); + let submissions = vec![pool.submit_one(invalid_hash(), SOURCE, xt0.clone())]; + let results = block_on(futures::future::join_all(submissions)); + + //xt0 should be stale + assert!(matches!( + &results[0].as_ref().unwrap_err().0, + TxPoolError::InvalidTransaction(InvalidTransaction::Stale,) + )); + + assert_pool_status!(header.hash(), &pool, 0, 0); +} + +#[test] +fn fatp_one_view_stale_submit_many_fails() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let header = api.push_block(1, vec![], true); + + let event = new_best_block_event(&pool, None, header.hash()); + block_on(pool.maintain(event)); + + let xts0 = (100..105).map(|i| uxt(Alice, i)).collect::>(); + let xts1 = (105..110).map(|i| uxt(Alice, i)).collect::>(); + let xts2 = (195..201).map(|i| uxt(Alice, i)).collect::>(); + + let submissions = vec![ + pool.submit_at(header.hash(), SOURCE, xts0.clone()), + pool.submit_at(header.hash(), SOURCE, xts1.clone()), + pool.submit_at(header.hash(), SOURCE, xts2.clone()), + ]; + + let results = block_on(futures::future::join_all(submissions)); + + //xts2 contains one ready transaction (nonce:200) + let mut results = results.into_iter().flat_map(|x| x.unwrap()).collect::>(); + log::debug!("{:#?}", results); + assert!(results.pop().unwrap().is_ok()); + assert!(results.into_iter().all(|r| { + matches!( + &r.as_ref().unwrap_err().0, + TxPoolError::InvalidTransaction(InvalidTransaction::Stale,) + ) + })); + + assert_pool_status!(header.hash(), &pool, 1, 0); +} + +#[test] +fn fatp_one_view_future_turns_to_ready_works() { + let (pool, api, _) = pool(); + + let header = api.push_block(1, vec![], true); + let at = header.hash(); + let event = new_best_block_event(&pool, None, at); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 201); + block_on(pool.submit_one(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + assert!(pool.ready().count() == 0); + assert_pool_status!(at, &pool, 0, 1); + + let xt1 = uxt(Alice, 200); + block_on(pool.submit_one(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let ready: Vec<_> = pool.ready().map(|v| (*v.data).clone()).collect(); + assert_eq!(ready, vec![xt1, xt0]); + assert_pool_status!(at, &pool, 2, 0); +} + +#[test] +fn fatp_one_view_ready_gets_pruned() { + let (pool, api, _) = pool(); + + let header = api.push_block(1, vec![], true); + let block1 = header.hash(); + let event = new_best_block_event(&pool, None, block1); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + block_on(pool.submit_one(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let pending: Vec<_> = pool.ready().map(|v| (*v.data).clone()).collect(); + assert_eq!(pending, vec![xt0.clone()]); + assert_eq!(pool.status_all()[&block1].ready, 1); + + let header = api.push_block(2, vec![xt0], true); + let block2 = header.hash(); + let event = new_best_block_event(&pool, Some(block1), block2); + block_on(pool.maintain(event)); + assert_pool_status!(block2, &pool, 0, 0); + assert!(pool.ready().count() == 0); +} + +#[test] +fn fatp_one_view_ready_turns_to_stale_works() { + let (pool, api, _) = pool(); + + let header = api.push_block(1, vec![], true); + let block1 = header.hash(); + let event = new_best_block_event(&pool, None, block1); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + block_on(pool.submit_one(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let pending: Vec<_> = pool.ready().map(|v| (*v.data).clone()).collect(); + assert_eq!(pending, vec![xt0.clone()]); + assert_eq!(pool.status_all()[&block1].ready, 1); + + let header = api.push_block(2, vec![], true); + let block2 = header.hash(); + //tricky: typically the block2 shall contain conflicting transaction for Alice. In this test we + //want to check revalidation, so we manually adjust nonce. + api.set_nonce(block2, Alice.into(), 201); + let event = new_best_block_event(&pool, Some(block1), block2); + //note: blocking revalidation (w/o background worker) which is used in this test will detect + // xt0 is stale + block_on(pool.maintain(event)); + //todo: should it work at all? (it requires better revalidation: mempool keeping validated txs) + // assert_pool_status!(block2, &pool, 0, 0); + // assert!(pool.ready(block2).unwrap().count() == 0); +} + +#[test] +fn fatp_two_views_future_and_ready_submit_one() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let genesis = api.genesis_hash(); + let header01a = api.push_block(1, vec![], true); + let header01b = api.push_block(1, vec![], true); + + let event = new_best_block_event(&pool, None, header01a.hash()); + block_on(pool.maintain(event)); + + let event = new_best_block_event(&pool, None, header01b.hash()); + block_on(pool.maintain(event)); + + api.set_nonce(header01b.hash(), Alice.into(), 202); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 202); + + let submissions = vec![ + pool.submit_one(genesis, SOURCE, xt0.clone()), + pool.submit_one(genesis, SOURCE, xt1.clone()), + ]; + + block_on(futures::future::join_all(submissions)); + + assert_pool_status!(header01a.hash(), &pool, 1, 1); + assert_pool_status!(header01b.hash(), &pool, 1, 0); +} + +#[test] +fn fatp_two_views_future_and_ready_submit_many() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let header01a = api.push_block(1, vec![], true); + let header01b = api.push_block(1, vec![], true); + + let event = new_best_block_event(&pool, None, header01a.hash()); + block_on(pool.maintain(event)); + + let event = new_best_block_event(&pool, None, header01b.hash()); + block_on(pool.maintain(event)); + + api.set_nonce(header01b.hash(), Alice.into(), 215); + + let xts0 = (200..205).map(|i| uxt(Alice, i)).collect::>(); + let xts1 = (205..210).map(|i| uxt(Alice, i)).collect::>(); + let xts2 = (215..220).map(|i| uxt(Alice, i)).collect::>(); + + let submissions = vec![ + pool.submit_at(invalid_hash(), SOURCE, xts0.clone()), + pool.submit_at(invalid_hash(), SOURCE, xts1.clone()), + pool.submit_at(invalid_hash(), SOURCE, xts2.clone()), + ]; + + block_on(futures::future::join_all(submissions)); + + log::debug!(target:LOG_TARGET, "stats: {:#?}", pool.status_all()); + + assert_pool_status!(header01a.hash(), &pool, 10, 5); + assert_pool_status!(header01b.hash(), &pool, 5, 0); +} + +#[test] +fn fatp_two_views_submit_many_variations() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let xt0 = uxt(Alice, 206); + let xt1 = uxt(Alice, 206); + + let result = block_on(pool.submit_one(invalid_hash(), SOURCE, xt1.clone())); + assert!(result.is_ok()); + + let header01a = api.push_block(1, vec![xt0.clone()], true); + let header01b = api.push_block(1, vec![xt0.clone()], true); + + api.set_nonce(header01a.hash(), Alice.into(), 201); + api.set_nonce(header01b.hash(), Alice.into(), 202); + + let event = new_best_block_event(&pool, None, header01a.hash()); + block_on(pool.maintain(event)); + + let event = new_best_block_event(&pool, None, header01b.hash()); + block_on(pool.maintain(event)); + + let mut xts = (199..204).map(|i| uxt(Alice, i)).collect::>(); + xts.push(xt0); + xts.push(xt1); + + let results = block_on(pool.submit_at(invalid_hash(), SOURCE, xts.clone())).unwrap(); + + log::debug!(target:LOG_TARGET, "res: {:#?}", results); + log::debug!(target:LOG_TARGET, "stats: {:#?}", pool.status_all()); + + (0..2).for_each(|i| { + assert!(matches!( + results[i].as_ref().unwrap_err().0, + TxPoolError::InvalidTransaction(InvalidTransaction::Stale,) + )); + }); + //note: tx at 2 is valid at header01a and invalid at header01b + (2..5).for_each(|i| { + assert_eq!(*results[i].as_ref().unwrap(), api.hash_and_length(&xts[i]).0); + }); + //xt0 at index 5 (transaction from the imported block, gets banned when pruned) + assert!(matches!(results[5].as_ref().unwrap_err().0, TxPoolError::TemporarilyBanned)); + //xt1 at index 6 + assert!(matches!(results[6].as_ref().unwrap_err().0, TxPoolError::AlreadyImported(_))); +} + +#[test] +fn fatp_linear_progress() { + sp_tracing::try_init_simple(); + + let (api, forks) = test_chain_with_forks::chain(None); + let (pool, _) = pool_with_api(api.clone()); + + let f11 = forks[1][1].hash(); + let f13 = forks[1][3].hash(); + + let event = new_best_block_event(&pool, None, f11); + block_on(pool.maintain(event)); + + let xt0 = uxt(Bob, 203); + let submissions = vec![pool.submit_one(invalid_hash(), SOURCE, xt0.clone())]; + + block_on(futures::future::join_all(submissions)); + + let event = new_best_block_event(&pool, Some(f11), f13); + log::debug!(target:LOG_TARGET, "event: {:#?}", event); + block_on(pool.maintain(event)); + + //note: we only keep tip of the fork + assert_eq!(pool.active_views_count(), 1); + assert_pool_status!(f13, &pool, 1, 0); +} + +#[test] +fn fatp_linear_old_ready_becoming_stale() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + // Our initial transactions + let xts = vec![uxt(Alice, 300), uxt(Alice, 301), uxt(Alice, 302)]; + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + xts.into_iter().for_each(|xt| { + block_on(pool.submit_one(invalid_hash(), SOURCE, xt)).unwrap(); + }); + assert_eq!(pool.status_all()[&header01.hash()].ready, 0); + assert_eq!(pool.status_all()[&header01.hash()].future, 3); + + // Import enough blocks to make our transactions stale (longevity is 64) + let mut prev_header = header01; + for n in 2..66 { + let header = api.push_block(n, vec![], true); + let event = new_best_block_event(&pool, Some(prev_header.hash()), header.hash()); + block_on(pool.maintain(event)); + + if n == 65 { + assert_eq!(pool.status_all()[&header.hash()].ready, 0); + assert_eq!(pool.status_all()[&header.hash()].future, 0); + } else { + assert_eq!(pool.status_all()[&header.hash()].ready, 0); + assert_eq!(pool.status_all()[&header.hash()].future, 3); + } + prev_header = header; + } +} + +#[test] +fn fatp_fork_reorg() { + sp_tracing::try_init_simple(); + + let (api, forks) = test_chain_with_forks::chain(None); + let (pool, _) = pool_with_api(api.clone()); + + let f03 = forks[0][3].hash(); + let f13 = forks[1][3].hash(); + + let event = new_best_block_event(&pool, None, f03); + block_on(pool.maintain(event)); + + let xt0 = uxt(Bob, 203); + let xt1 = uxt(Bob, 204); + let xt2 = uxt(Alice, 203); + let submissions = vec![ + pool.submit_one(invalid_hash(), SOURCE, xt0.clone()), + pool.submit_one(invalid_hash(), SOURCE, xt1.clone()), + pool.submit_one(invalid_hash(), SOURCE, xt2.clone()), + ]; + + block_on(futures::future::join_all(submissions)); + + let event = new_best_block_event(&pool, Some(f03), f13); + log::debug!(target:LOG_TARGET, "event: {:#?}", event); + block_on(pool.maintain(event)); + + assert_pool_status!(f03, &pool, 1, 2); + assert_pool_status!(f13, &pool, 6, 0); + + //check if ready for block[1][3] contains resubmitted transactions + let mut expected = forks[0] + .iter() + .take(4) + .flat_map(|h| block_on(api.block_body(h.hash())).unwrap().unwrap()) + .collect::>(); + expected.extend_from_slice(&[xt0, xt1, xt2]); + + let ready_f13 = pool.ready().collect::>(); + expected.iter().for_each(|e| { + assert!(ready_f13.iter().any(|v| *v.data == *e)); + }); + assert_eq!(expected.len(), ready_f13.len()); +} + +#[test] +fn fatp_fork_do_resubmit_same_tx() { + let xt = uxt(Alice, 200); + + let (pool, api, _) = pool(); + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + + block_on(pool.submit_one(api.expect_hash_from_number(0), SOURCE, xt.clone())).unwrap(); + assert_eq!(pool.status_all()[&header01.hash()].ready, 1); + + let header02a = api.push_block(1, vec![xt.clone()], true); + let header02b = api.push_block(1, vec![xt], true); + + let event = new_best_block_event(&pool, Some(header02a.hash()), header02b.hash()); + api.set_nonce(header02a.hash(), Alice.into(), 201); + block_on(pool.maintain(event)); + assert_eq!(pool.status_all()[&header02b.hash()].ready, 0); + + let event = new_best_block_event(&pool, Some(api.genesis_hash()), header02b.hash()); + api.set_nonce(header02b.hash(), Alice.into(), 201); + block_on(pool.maintain(event)); + + assert_eq!(pool.status_all()[&header02b.hash()].ready, 0); +} + +#[test] +fn fatp_fork_stale_rejected() { + sp_tracing::try_init_simple(); + + // note: there are no xts in blocks on fork 0! + let (api, forks) = test_chain_with_forks::chain(Some(&|f, b| match (f, b) { + (0, _) => false, + _ => true, + })); + let (pool, _) = pool_with_api(api.clone()); + + let f03 = forks[0][3].hash(); + let f13 = forks[1][3].hash(); + + // n:201 n:202 n:203 <-- alice nonce + // F01 - F02 - F03 <-- xt2 is stale + // / + // F00 + // \ + // F11[t0] - F12[t1] - F13[t2] + // n:201 n:202 n:203 <-- bob nonce + // + // t0 = uxt(Bob,200) + // t1 = uxt(Bob,201) + // t2 = uxt(Bob,201) + // xt0 = uxt(Bob, 203) + // xt1 = uxt(Bob, 204) + // xt2 = uxt(Alice, 201); + + let event = new_best_block_event(&pool, None, f03); + block_on(pool.maintain(event)); + + let xt0 = uxt(Bob, 203); + let xt1 = uxt(Bob, 204); + let xt2 = uxt(Alice, 201); + let submissions = vec![ + pool.submit_one(invalid_hash(), SOURCE, xt0.clone()), + pool.submit_one(invalid_hash(), SOURCE, xt1.clone()), + pool.submit_one(invalid_hash(), SOURCE, xt2.clone()), + ]; + let submission_results = block_on(futures::future::join_all(submissions)); + let futures_f03 = pool.futures(); + + //xt2 should be stale + assert!(matches!( + &submission_results[2].as_ref().unwrap_err().0, + TxPoolError::InvalidTransaction(InvalidTransaction::Stale,) + )); + + let event = new_best_block_event(&pool, Some(f03), f13); + log::debug!(target:LOG_TARGET, "event: {:#?}", event); + block_on(pool.maintain(event)); + + assert_pool_status!(f03, &pool, 0, 2); + + //xt2 was removed from the pool, it is not becoming future: + //note: theoretically we could keep xt2 in the pool, even if it was reported as stale. But it + //seems to be an unnecessary complication. + assert_pool_status!(f13, &pool, 2, 0); + + let futures_f13 = pool.futures(); + let ready_f13 = pool.ready().collect::>(); + assert!(futures_f13.iter().next().is_none()); + assert!(futures_f03.iter().any(|v| *v.data == xt0)); + assert!(futures_f03.iter().any(|v| *v.data == xt1)); + assert!(ready_f13.iter().any(|v| *v.data == xt0)); + assert!(ready_f13.iter().any(|v| *v.data == xt1)); +} + +#[test] +fn fatp_fork_no_xts_ready_switch_to_future() { + //this scenario w/o xts is not likely to happen, but similar thing (xt changing from ready to + //future) could occur e.g. when runtime was updated on fork1. + sp_tracing::try_init_simple(); + + // note: there are no xts in blocks! + let (api, forks) = test_chain_with_forks::chain(Some(&|_, _| false)); + let (pool, _) = pool_with_api(api.clone()); + + let f03 = forks[0][3].hash(); + let f12 = forks[1][2].hash(); + + let event = new_best_block_event(&pool, None, f03); + block_on(pool.maintain(event)); + + // xt0 is ready on f03, but future on f12, f13 + let xt0 = uxt(Alice, 203); + let submissions = vec![pool.submit_one(invalid_hash(), SOURCE, xt0.clone())]; + block_on(futures::future::join_all(submissions)); + + let event = new_best_block_event(&pool, Some(f03), f12); + block_on(pool.maintain(event)); + + assert_pool_status!(f03, &pool, 1, 0); + // f12 was not updated - xt0 is still ready there + // (todo: can we do better? shall we revalidate all future xts?) + assert_pool_status!(f12, &pool, 1, 0); + + //xt0 becomes future, and this may only happen after view revalidation (which happens on + //finalization). So trigger it. + let event = finalized_block_event(&pool, api.genesis_hash(), f12); + block_on(pool.maintain(event)); + + // f03 still dangling + assert_eq!(pool.active_views_count(), 2); + + // wait 10 blocks for revalidation and 1 extra for applying revalidation results + let mut prev_header = forks[1][2].clone(); + log::debug!("====> {:?}", prev_header); + for _ in 3..=12 { + let header = api.push_block_with_parent(prev_header.hash(), vec![], true); + let event = finalized_block_event(&pool, prev_header.hash(), header.hash()); + block_on(pool.maintain(event)); + prev_header = header; + } + + assert_pool_status!(prev_header.hash(), &pool, 0, 1); +} + +#[test] +fn fatp_ready_at_does_not_trigger() { + sp_tracing::try_init_simple(); + + let (api, forks) = test_chain_with_forks::chain(None); + let (pool, _) = pool_with_api(api.clone()); + + let f03 = forks[0][3].hash(); + let f13 = forks[1][3].hash(); + + assert!(pool.ready_at(f03).now_or_never().is_none()); + assert!(pool.ready_at(f13).now_or_never().is_none()); +} + +#[test] +fn fatp_ready_at_does_not_trigger_after_submit() { + sp_tracing::try_init_simple(); + + let (api, forks) = test_chain_with_forks::chain(None); + let (pool, _) = pool_with_api(api.clone()); + + let xt0 = uxt(Alice, 200); + let _ = block_on(pool.submit_one(invalid_hash(), SOURCE, xt0)); + + let f03 = forks[0][3].hash(); + let f13 = forks[1][3].hash(); + + assert!(pool.ready_at(f03).now_or_never().is_none()); + assert!(pool.ready_at(f13).now_or_never().is_none()); +} + +#[test] +fn fatp_ready_at_triggered_by_maintain() { + //this scenario w/o xts is not likely to happen, but similar thing (xt changing from ready to + //future) could occur e.g. when runtime was updated on fork1. + sp_tracing::try_init_simple(); + let (api, forks) = test_chain_with_forks::chain(Some(&|_, _| false)); + let (pool, _) = pool_with_api(api.clone()); + + let f03 = forks[0][3].hash(); + let f13 = forks[1][3].hash(); + + assert!(pool.ready_at(f03).now_or_never().is_none()); + + let event = new_best_block_event(&pool, None, f03); + block_on(pool.maintain(event)); + + assert!(pool.ready_at(f03).now_or_never().is_some()); + + let xt0 = uxt(Alice, 203); + let submissions = vec![pool.submit_one(invalid_hash(), SOURCE, xt0.clone())]; + block_on(futures::future::join_all(submissions)); + + let event = new_best_block_event(&pool, Some(f03), f13); + log::debug!(target:LOG_TARGET, "event: {:#?}", event); + assert!(pool.ready_at(f13).now_or_never().is_none()); + block_on(pool.maintain(event)); + assert!(pool.ready_at(f03).now_or_never().is_some()); + assert!(pool.ready_at(f13).now_or_never().is_some()); +} + +#[test] +fn fatp_ready_at_triggered_by_maintain2() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let header01 = api.push_block(1, vec![], true); + + let xt0 = uxt(Alice, 200); + block_on(pool.submit_one(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + + // let (pool, api, _guard) = maintained_pool(); + // let header = api.push_block(1, vec![], true); + // + // let xt1 = uxt(Alice, 209); + // + // block_on(pool.submit_one(api.expect_hash_from_number(1), SOURCE, xt1.clone())) + // .expect("1. Imported"); + + let noop_waker = futures::task::noop_waker(); + let mut context = futures::task::Context::from_waker(&noop_waker); + + let mut ready_set_future = pool.ready_at(header01.hash()); + if ready_set_future.poll_unpin(&mut context).is_ready() { + panic!("Ready set should not be ready before block update!"); + } + + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + // block_on(pool.maintain(block_event(header))); + + match ready_set_future.poll_unpin(&mut context) { + Poll::Pending => { + panic!("Ready set should become ready after block update!"); + }, + Poll::Ready(iterator) => { + let data = iterator.collect::>(); + assert_eq!(data.len(), 1); + }, + } +} + +#[test] +fn fatp_linear_progress_finalization() { + sp_tracing::try_init_simple(); + + let (api, forks) = test_chain_with_forks::chain(None); + let (pool, _) = pool_with_api(api.clone()); + + let f00 = forks[0][0].hash(); + let f12 = forks[1][2].hash(); + let f14 = forks[1][4].hash(); + + let event = new_best_block_event(&pool, None, f00); + block_on(pool.maintain(event)); + + let xt0 = uxt(Bob, 204); + let submissions = vec![pool.submit_one(invalid_hash(), SOURCE, xt0.clone())]; + block_on(futures::future::join_all(submissions)); + + let event = new_best_block_event(&pool, Some(f00), f12); + block_on(pool.maintain(event)); + assert_pool_status!(f12, &pool, 0, 1); + assert_eq!(pool.active_views_count(), 1); + + log::debug!(target:LOG_TARGET, "stats: {:#?}", pool.status_all()); + + let event = ChainEvent::Finalized { hash: f14, tree_route: Arc::from(vec![]) }; + block_on(pool.maintain(event)); + + log::debug!(target:LOG_TARGET, "stats: {:#?}", pool.status_all()); + + assert_eq!(pool.active_views_count(), 1); + assert_pool_status!(f14, &pool, 1, 0); +} + +#[test] +fn fatp_fork_finalization_removes_stale_views() { + sp_tracing::try_init_simple(); + + let (api, forks) = test_chain_with_forks::chain(None); + let (pool, _) = pool_with_api(api.clone()); + + let f00 = forks[0][0].hash(); + let f12 = forks[1][2].hash(); + let f14 = forks[1][4].hash(); + let f02 = forks[0][2].hash(); + let f03 = forks[0][3].hash(); + let f04 = forks[0][4].hash(); + + let xt0 = uxt(Bob, 203); + let submissions = vec![pool.submit_one(invalid_hash(), SOURCE, xt0.clone())]; + block_on(futures::future::join_all(submissions)); + + let event = new_best_block_event(&pool, Some(f00), f12); + block_on(pool.maintain(event)); + let event = new_best_block_event(&pool, Some(f00), f14); + block_on(pool.maintain(event)); + let event = new_best_block_event(&pool, Some(f00), f02); + block_on(pool.maintain(event)); + + //only views at the tips of the forks are kept + assert_eq!(pool.active_views_count(), 2); + + log::debug!(target:LOG_TARGET, "stats: {:#?}", pool.status_all()); + + let event = ChainEvent::Finalized { hash: f03, tree_route: Arc::from(vec![]) }; + block_on(pool.maintain(event)); + log::debug!(target:LOG_TARGET, "stats: {:#?}", pool.status_all()); + // note: currently the pruning views only cleans views with block number less than finalized + // block. views with higher number on other forks are not cleaned (will be done in next round). + assert_eq!(pool.active_views_count(), 2); + + let event = ChainEvent::Finalized { hash: f04, tree_route: Arc::from(vec![]) }; + block_on(pool.maintain(event)); + assert_eq!(pool.active_views_count(), 1); +} + +#[test] +fn fatp_watcher_invalid_fails_on_submission() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + 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, 150); + api.add_invalid(&xt0); + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())); + let xt0_watcher = xt0_watcher.map(|_| ()); + + assert_pool_status!(header01.hash(), &pool, 0, 0); + assert!(matches!( + xt0_watcher.unwrap_err().into_pool_error(), + Ok(TxPoolError::InvalidTransaction(InvalidTransaction::Stale)) + )); +} + +#[test] +fn fatp_watcher_invalid_single_revalidation() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, Some(api.genesis_hash()), header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + + api.add_invalid(&xt0); + + let header02 = api.push_block_with_parent(header01.hash(), vec![], true); + let event = finalized_block_event(&pool, header01.hash(), header02.hash()); + block_on(pool.maintain(event)); + + // wait 10 blocks for revalidation + let mut prev_header = header02; + for n in 3..=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 xt0_events = futures::executor::block_on_stream(xt0_watcher).collect::>(); + log::debug!("xt0_events: {:#?}", xt0_events); + assert_eq!(xt0_events, vec![TransactionStatus::Ready, TransactionStatus::Invalid]); +} + +#[test] +fn fatp_watcher_invalid_single_revalidation2() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let xt0 = uxt(Alice, 200); + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + assert_eq!(pool.mempool_len(), (0, 1)); + api.add_invalid(&xt0); + + 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_events = futures::executor::block_on_stream(xt0_watcher).collect::>(); + log::debug!("xt0_events: {:#?}", xt0_events); + assert_eq!(xt0_events, vec![TransactionStatus::Invalid]); + assert_eq!(pool.mempool_len(), (0, 0)); +} + +#[test] +fn fatp_watcher_invalid_single_revalidation3() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let xt0 = uxt(Alice, 150); + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + assert_eq!(pool.mempool_len(), (0, 1)); + + let header01 = api.push_block(1, vec![], true); + let event = finalized_block_event(&pool, api.genesis_hash(), header01.hash()); + block_on(pool.maintain(event)); + + // wait 10 blocks for revalidation + let mut prev_header = header01; + for n in 2..=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 xt0_events = futures::executor::block_on_stream(xt0_watcher).collect::>(); + log::debug!("xt0_events: {:#?}", xt0_events); + assert_eq!(xt0_events, vec![TransactionStatus::Invalid]); + assert_eq!(pool.mempool_len(), (0, 0)); +} + +#[test] +fn fatp_watcher_future() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + 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, 202); + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 0, 1); + + let header02 = api.push_block(2, vec![], true); + let event = ChainEvent::Finalized { + hash: header02.hash(), + tree_route: Arc::from(vec![header01.hash()]), + }; + block_on(pool.maintain(event)); + + assert_pool_status!(header02.hash(), &pool, 0, 1); + + let xt0_events = block_on(xt0_watcher.take(1).collect::>()); + assert_eq!(xt0_events, vec![TransactionStatus::Future]); +} + +#[test] +fn fatp_watcher_ready() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + 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 xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 1, 0); + + let header02 = api.push_block(2, vec![], true); + let event = ChainEvent::Finalized { + hash: header02.hash(), + tree_route: Arc::from(vec![header01.hash()]), + }; + block_on(pool.maintain(event)); + + assert_pool_status!(header02.hash(), &pool, 1, 0); + + let xt0_events = block_on(xt0_watcher.take(1).collect::>()); + assert_eq!(xt0_events, vec![TransactionStatus::Ready]); +} + +#[test] +fn fatp_watcher_finalized() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + 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 xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 1, 0); + + let header02 = api.push_block(2, vec![xt0], true); + let event = ChainEvent::Finalized { + hash: header02.hash(), + tree_route: Arc::from(vec![header01.hash()]), + }; + block_on(pool.maintain(event)); + + assert_pool_status!(header02.hash(), &pool, 0, 0); + + let xt0_events = block_on(xt0_watcher.collect::>()); + assert_eq!( + xt0_events, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header02.hash(), 0)), + TransactionStatus::Finalized((header02.hash(), 0)), + ] + ); +} + +#[test] +fn fatp_watcher_in_block() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + 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 xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 1, 0); + + let header02 = api.push_block(2, vec![xt0], true); + + let event = new_best_block_event(&pool, Some(header01.hash()), header02.hash()); + block_on(pool.maintain(event)); + let xt0_events = block_on(xt0_watcher.take(2).collect::>()); + assert_eq!( + xt0_events, + vec![TransactionStatus::Ready, TransactionStatus::InBlock((header02.hash(), 0)),] + ); +} + +#[test] +fn fatp_watcher_future_and_finalized() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + 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, 202); + + let submissions = vec![ + pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone()), + ]; + + let mut submissions = block_on(futures::future::join_all(submissions)); + let xt1_watcher = submissions.remove(1).unwrap(); + let xt0_watcher = submissions.remove(0).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 1, 1); + + let header02 = api.push_block(2, vec![xt0], true); + let event = ChainEvent::Finalized { + hash: header02.hash(), + tree_route: Arc::from(vec![header01.hash()]), + }; + // let event = new_best_block_event(&pool, Some(header01.hash()), header02.hash()); + block_on(pool.maintain(event)); + + assert_pool_status!(header02.hash(), &pool, 0, 1); + + let xt1_status = block_on(xt1_watcher.take(1).collect::>()); + assert_eq!(xt1_status, vec![TransactionStatus::Future]); + let xt0_status = block_on(xt0_watcher.collect::>()); + assert_eq!( + xt0_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header02.hash(), 0)), + TransactionStatus::Finalized((header02.hash(), 0)), + ] + ); +} + +#[test] +fn fatp_watcher_two_finalized_in_different_block() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + api.set_nonce(api.genesis_hash(), Bob.into(), 200); + api.set_nonce(api.genesis_hash(), Dave.into(), 200); + + 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(Bob, 200); + let xt3 = uxt(Dave, 200); + + let submissions = vec![ + pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone()), + ]; + let mut submissions = block_on(futures::future::join_all(submissions)); + let xt2_watcher = submissions.remove(2).unwrap(); + let xt1_watcher = submissions.remove(1).unwrap(); + let xt0_watcher = submissions.remove(0).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 3, 0); + + let header02 = api.push_block(2, vec![xt3.clone(), xt2.clone(), xt0.clone()], true); + api.set_nonce(header02.hash(), Alice.into(), 201); + //note: no maintain for block02 (!) + + let header03 = api.push_block(3, vec![xt1.clone()], true); + block_on(pool.maintain(finalized_block_event(&pool, header01.hash(), header03.hash()))); + + assert_pool_status!(header03.hash(), &pool, 0, 0); + + let xt1_status = futures::executor::block_on_stream(xt1_watcher).collect::>(); + + assert_eq!( + xt1_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header03.hash(), 0)), + TransactionStatus::Finalized((header03.hash(), 0)) + ] + ); + + let xt0_status = futures::executor::block_on_stream(xt0_watcher).collect::>(); + + log::debug!("xt0_status: {:#?}", xt0_status); + + assert_eq!( + xt0_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header02.hash(), 2)), + TransactionStatus::Finalized((header02.hash(), 2)) + ] + ); + + let xt2_status = futures::executor::block_on_stream(xt2_watcher).collect::>(); + log::debug!("xt2_status: {:#?}", xt2_status); + + assert_eq!( + xt2_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header02.hash(), 1)), + TransactionStatus::Finalized((header02.hash(), 1)) + ] + ); +} + +#[test] +fn fatp_no_view_pool_watcher_two_finalized_in_different_block() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + api.set_nonce(api.genesis_hash(), Bob.into(), 200); + api.set_nonce(api.genesis_hash(), Dave.into(), 200); + + let header01 = api.push_block(1, vec![], true); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 201); + let xt2 = uxt(Bob, 200); + let xt3 = uxt(Dave, 200); + + let submissions = vec![ + pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone()), + ]; + let mut submissions = block_on(futures::future::join_all(submissions)); + let xt2_watcher = submissions.remove(2).unwrap(); + let xt1_watcher = submissions.remove(1).unwrap(); + let xt0_watcher = submissions.remove(0).unwrap(); + + let header02 = api.push_block(2, vec![xt3.clone(), xt2.clone(), xt0.clone()], true); + api.set_nonce(header02.hash(), Alice.into(), 201); + api.set_nonce(header02.hash(), Bob.into(), 201); + api.set_nonce(header02.hash(), Dave.into(), 201); + //note: no maintain for block02 (!) + + let header03 = api.push_block(3, vec![xt1.clone()], true); + api.set_nonce(header03.hash(), Alice.into(), 202); + block_on(pool.maintain(finalized_block_event(&pool, header01.hash(), header03.hash()))); + + assert_pool_status!(header03.hash(), &pool, 0, 0); + + let xt1_status = futures::executor::block_on_stream(xt1_watcher).collect::>(); + + log::debug!("xt1_status: {:#?}", xt1_status); + + assert_eq!( + xt1_status, + vec![ + TransactionStatus::InBlock((header03.hash(), 0)), + TransactionStatus::Finalized((header03.hash(), 0)) + ] + ); + + let xt0_status = futures::executor::block_on_stream(xt0_watcher).collect::>(); + + log::debug!("xt0_status: {:#?}", xt0_status); + + assert_eq!( + xt0_status, + vec![ + TransactionStatus::InBlock((header02.hash(), 2)), + TransactionStatus::Finalized((header02.hash(), 2)) + ] + ); + + let xt2_status = futures::executor::block_on_stream(xt2_watcher).collect::>(); + log::debug!("xt2_status: {:#?}", xt2_status); + + assert_eq!( + xt2_status, + vec![ + TransactionStatus::InBlock((header02.hash(), 1)), + TransactionStatus::Finalized((header02.hash(), 1)) + ] + ); +} + +#[test] +fn fatp_watcher_in_block_across_many_blocks() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + 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 xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let _ = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + assert_pool_status!(header01.hash(), &pool, 2, 0); + + let header02 = api.push_block(2, vec![], true); + let event = new_best_block_event(&pool, Some(header01.hash()), header02.hash()); + block_on(pool.maintain(event)); + + let _ = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + //note 1: transaction is not submitted to views that are not at the tip of the fork + assert_eq!(pool.active_views_count(), 1); + assert_eq!(pool.inactive_views_count(), 1); + assert_pool_status!(header02.hash(), &pool, 3, 0); + + let header03 = api.push_block(3, vec![xt0.clone()], true); + let event = new_best_block_event(&pool, Some(header02.hash()), header03.hash()); + block_on(pool.maintain(event)); + + assert_pool_status!(header03.hash(), &pool, 2, 0); + + let xt0_status = futures::executor::block_on_stream(xt0_watcher).take(2).collect::>(); + + log::debug!("xt0_status: {:#?}", xt0_status); + assert_eq!( + xt0_status, + vec![TransactionStatus::Ready, TransactionStatus::InBlock((header03.hash(), 0)),] + ); +} + +#[test] +fn fatp_watcher_in_block_across_many_blocks2() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + 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 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); + + let header02 = api.push_block(2, vec![], true); + let event = new_best_block_event(&pool, Some(header01.hash()), header02.hash()); + block_on(pool.maintain(event)); + + let _ = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + //note 1: transaction is not submitted to views that are not at the tip of the fork + assert_eq!(pool.active_views_count(), 1); + assert_eq!(pool.inactive_views_count(), 1); + assert_pool_status!(header02.hash(), &pool, 3, 0); + + let header03 = api.push_block(3, vec![xt0.clone()], true); + let header04 = api.push_block(4, vec![xt1.clone()], true); + let event = new_best_block_event(&pool, Some(header02.hash()), header04.hash()); + block_on(pool.maintain(event)); + + assert_pool_status!(header04.hash(), &pool, 1, 0); + + let xt0_status = futures::executor::block_on_stream(xt0_watcher).take(2).collect::>(); + let xt1_status = futures::executor::block_on_stream(xt1_watcher).take(2).collect::>(); + + log::debug!("xt0_status: {:#?}", xt0_status); + log::debug!("xt1_status: {:#?}", xt1_status); + assert_eq!( + xt0_status, + vec![TransactionStatus::Ready, TransactionStatus::InBlock((header03.hash(), 0)),] + ); + assert_eq!( + xt1_status, + vec![TransactionStatus::Ready, TransactionStatus::InBlock((header04.hash(), 0)),] + ); +} + +#[test] +fn fatp_watcher_dropping_listener_should_work() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + 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); + + // intentionally drop the listener - nothing should panic. + let _ = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + assert_pool_status!(header01.hash(), &pool, 1, 0); + + let header02 = api.push_block(2, vec![], true); + let event = new_best_block_event(&pool, Some(header01.hash()), header02.hash()); + block_on(pool.maintain(event)); +} + +#[test] +fn fatp_watcher_fork_retract_and_finalize() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + 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 xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + assert_pool_status!(header01.hash(), &pool, 1, 0); + + let header02a = api.push_block_with_parent(header01.hash(), vec![xt0.clone()], true); + let event = new_best_block_event(&pool, Some(header01.hash()), header02a.hash()); + block_on(pool.maintain(event)); + assert_pool_status!(header02a.hash(), &pool, 0, 0); + + let header02b = api.push_block_with_parent(header01.hash(), vec![xt0.clone()], true); + let event = ChainEvent::Finalized { + hash: header02b.hash(), + tree_route: Arc::from(vec![header01.hash()]), + }; + block_on(pool.maintain(event)); + assert_pool_status!(header02b.hash(), &pool, 0, 0); + let xt0_status = futures::executor::block_on_stream(xt0_watcher).collect::>(); + + log::debug!("xt0_status: {:#?}", xt0_status); + + assert_eq!( + xt0_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header02a.hash(), 0)), + TransactionStatus::InBlock((header02b.hash(), 0)), + TransactionStatus::Finalized((header02b.hash(), 0)), + ] + ); +} + +#[test] +fn fatp_retract_all_forks() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + api.set_nonce(api.genesis_hash(), Bob.into(), 200); + let genesis = api.genesis_hash(); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 200); + + let header02a = api.push_block_with_parent(genesis, vec![xt0.clone()], true); + let event = new_best_block_event(&pool, Some(genesis), header02a.hash()); + block_on(pool.maintain(event)); + assert_pool_status!(header02a.hash(), &pool, 0, 0); + + let header02b = api.push_block_with_parent(genesis, vec![xt1.clone()], true); + let event = new_best_block_event(&pool, Some(header02a.hash()), header02b.hash()); + block_on(pool.maintain(event)); + assert_pool_status!(header02b.hash(), &pool, 1, 0); + + let header02c = api.push_block_with_parent(genesis, vec![], true); + let event = + ChainEvent::Finalized { hash: header02c.hash(), tree_route: Arc::from(vec![genesis]) }; + block_on(pool.maintain(event)); + assert_pool_status!(header02c.hash(), &pool, 2, 0); +} + +#[test] +fn fatp_watcher_finalizing_forks() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + api.set_nonce(api.genesis_hash(), Bob.into(), 200); + api.set_nonce(api.genesis_hash(), Charlie.into(), 200); + api.set_nonce(api.genesis_hash(), Dave.into(), 200); + api.set_nonce(api.genesis_hash(), Eve.into(), 200); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 200); + let xt2 = uxt(Charlie, 200); + let xt3 = uxt(Dave, 200); + let xt4 = uxt(Eve, 200); + + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let header01 = api.push_block(1, vec![xt0.clone()], true); + block_on(pool.maintain(new_best_block_event(&pool, None, header01.hash()))); + block_on(pool.maintain(finalized_block_event(&pool, api.genesis_hash(), header01.hash()))); + + let xt1_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let header02a = api.push_block_with_parent(header01.hash(), vec![xt1.clone()], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header02a.hash()))); + + let xt2_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + let header03a = api.push_block_with_parent(header02a.hash(), vec![xt2.clone()], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header02a.hash()), header03a.hash()))); + + let xt3_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt3.clone())).unwrap(); + let header02b = api.push_block_with_parent(header01.hash(), vec![xt3.clone()], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header02b.hash()))); + block_on(pool.maintain(finalized_block_event(&pool, header01.hash(), header02b.hash()))); + + let xt4_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt4.clone())).unwrap(); + let header03b = api.push_block_with_parent(header02b.hash(), vec![xt4.clone()], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header02b.hash()), header03b.hash()))); + + let header04b = + api.push_block_with_parent(header03b.hash(), vec![xt1.clone(), xt2.clone()], true); + block_on(pool.maintain(new_best_block_event(&pool, Some(header03b.hash()), header04b.hash()))); + block_on(pool.maintain(finalized_block_event(&pool, header02b.hash(), header04b.hash()))); + + //======================= + + let xt0_status = futures::executor::block_on_stream(xt0_watcher).collect::>(); + let xt1_status = futures::executor::block_on_stream(xt1_watcher).collect::>(); + let xt2_status = futures::executor::block_on_stream(xt2_watcher).collect::>(); + let xt3_status = futures::executor::block_on_stream(xt3_watcher).collect::>(); + let xt4_status = futures::executor::block_on_stream(xt4_watcher).collect::>(); + + assert_eq!( + xt0_status, + vec![ + TransactionStatus::InBlock((header01.hash(), 0)), + TransactionStatus::Finalized((header01.hash(), 0)), + ] + ); + + assert_eq!( + xt1_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header02a.hash(), 0)), + TransactionStatus::InBlock((header04b.hash(), 0)), + TransactionStatus::Finalized((header04b.hash(), 0)), + ] + ); + assert_eq!( + xt2_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header03a.hash(), 0)), + TransactionStatus::InBlock((header04b.hash(), 1)), + TransactionStatus::Finalized((header04b.hash(), 1)), + ] + ); + assert_eq!( + xt3_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header02b.hash(), 0)), + TransactionStatus::Finalized((header02b.hash(), 0)), + ] + ); + assert_eq!( + xt4_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header03b.hash(), 0)), + TransactionStatus::Finalized((header03b.hash(), 0)), + ] + ); +} + +#[test] +fn fatp_watcher_best_block_after_finalized() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + let header01 = api.push_block(1, vec![], true); + let event = finalized_block_event(&pool, api.genesis_hash(), header01.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + + // todo: shall we submit to finalized views? (if it is at the tip of the fork then yes?) + // assert_pool_status!(header01.hash(), &pool, 1, 0); + + let header02 = api.push_block(2, vec![xt0.clone()], true); + + let event = finalized_block_event(&pool, header01.hash(), header02.hash()); + block_on(pool.maintain(event)); + let event = new_best_block_event(&pool, Some(header01.hash()), header02.hash()); + block_on(pool.maintain(event)); + + let xt0_events = block_on(xt0_watcher.collect::>()); + assert_eq!( + xt0_events, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header02.hash(), 0)), + TransactionStatus::Finalized((header02.hash(), 0)), + ] + ); +} + +#[test] +fn fatp_watcher_best_block_after_finalized2() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let xt0 = uxt(Alice, 200); + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + + let header01 = api.push_block(1, vec![xt0.clone()], true); + + let event = finalized_block_event(&pool, api.genesis_hash(), header01.hash()); + block_on(pool.maintain(event)); + let event = new_best_block_event(&pool, Some(api.genesis_hash()), header01.hash()); + block_on(pool.maintain(event)); + + let xt0_events = block_on(xt0_watcher.collect::>()); + assert_eq!( + xt0_events, + vec![ + TransactionStatus::InBlock((header01.hash(), 0)), + TransactionStatus::Finalized((header01.hash(), 0)), + ] + ); +} + +#[test] +fn fatp_watcher_switching_fork_multiple_times_works() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + api.set_nonce(api.genesis_hash(), Bob.into(), 200); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 200); + + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let header01a = api.push_block(1, vec![xt0.clone()], true); + + let xt1_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let header01b = api.push_block(1, vec![xt0.clone(), xt1.clone()], true); + + //note: finalized block here must be header01b. + //It is because of how the order in which MultiViewListener is processing tx events and view + //events. tx events from single view are processed first, then view commands are handled. If + //finalization happens in first view reported then no events from others views will be + //processed. + + block_on(pool.maintain(new_best_block_event(&pool, None, header01a.hash()))); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01a.hash()), header01b.hash()))); + block_on(pool.maintain(new_best_block_event(&pool, Some(header01b.hash()), header01a.hash()))); + block_on(pool.maintain(finalized_block_event(&pool, api.genesis_hash(), header01b.hash()))); + + let xt0_status = futures::executor::block_on_stream(xt0_watcher).collect::>(); + let xt1_status = futures::executor::block_on_stream(xt1_watcher).take(2).collect::>(); + + log::debug!("xt0_status: {:#?}", xt0_status); + log::debug!("xt1_status: {:#?}", xt1_status); + + assert_eq!( + xt0_status, + vec![ + TransactionStatus::InBlock((header01a.hash(), 0)), + TransactionStatus::InBlock((header01b.hash(), 0)), + TransactionStatus::Finalized((header01b.hash(), 0)), + ] + ); + + assert_eq!( + xt1_status, + vec![TransactionStatus::Ready, TransactionStatus::InBlock((header01b.hash(), 1)),] + ); +} + +#[test] +fn fatp_watcher_two_blocks_delayed_finalization_works() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + api.set_nonce(api.genesis_hash(), Bob.into(), 200); + api.set_nonce(api.genesis_hash(), Charlie.into(), 200); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 200); + let xt2 = uxt(Charlie, 200); + + let header01 = api.push_block(1, vec![], true); + + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let header02 = api.push_block_with_parent(header01.hash(), vec![xt0.clone()], true); + + let xt1_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let header03 = api.push_block_with_parent(header02.hash(), vec![xt1.clone()], true); + + let xt2_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + let header04 = api.push_block_with_parent(header03.hash(), vec![xt2.clone()], true); + + block_on(pool.maintain(finalized_block_event(&pool, api.genesis_hash(), header01.hash()))); + block_on(pool.maintain(new_best_block_event(&pool, None, header04.hash()))); + block_on(pool.maintain(finalized_block_event(&pool, header01.hash(), header03.hash()))); + block_on(pool.maintain(finalized_block_event(&pool, header03.hash(), header04.hash()))); + + let xt0_status = futures::executor::block_on_stream(xt0_watcher).collect::>(); + let xt1_status = futures::executor::block_on_stream(xt1_watcher).collect::>(); + let xt2_status = futures::executor::block_on_stream(xt2_watcher).collect::>(); + + //todo: double events. + //view for header04 reported InBlock for all xts. + //Then finalization comes for header03. We need to create a view to sent finalization events. + //But in_block are also sent because of pruning - normal process during view creation. + // + //Do not know what solution should be in this case? + // - just jeep two events, + // - block pruning somehow (seems like excessive additional logic not really needed) + // - build view from recent best block? (retracting instead of enacting?) + // - de-dup events in listener (implemented) + + assert_eq!( + xt0_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header02.hash(), 0)), + TransactionStatus::Finalized((header02.hash(), 0)), + ] + ); + assert_eq!( + xt1_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header03.hash(), 0)), + TransactionStatus::Finalized((header03.hash(), 0)), + ] + ); + assert_eq!( + xt2_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header04.hash(), 0)), + TransactionStatus::Finalized((header04.hash(), 0)), + ] + ); +} + +#[test] +fn fatp_watcher_delayed_finalization_does_not_retract() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + api.set_nonce(api.genesis_hash(), Bob.into(), 200); + api.set_nonce(api.genesis_hash(), Charlie.into(), 200); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 200); + + let header01 = api.push_block(1, vec![], true); + + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let header02 = api.push_block_with_parent(header01.hash(), vec![xt0.clone()], true); + + let xt1_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let header03 = api.push_block_with_parent(header02.hash(), vec![xt1.clone()], true); + + block_on(pool.maintain(new_best_block_event(&pool, None, header02.hash()))); + block_on(pool.maintain(new_best_block_event(&pool, Some(header02.hash()), header03.hash()))); + + block_on(pool.maintain(finalized_block_event(&pool, api.genesis_hash(), header02.hash()))); + block_on(pool.maintain(finalized_block_event(&pool, header02.hash(), header03.hash()))); + + let xt0_status = futures::executor::block_on_stream(xt0_watcher).collect::>(); + let xt1_status = futures::executor::block_on_stream(xt1_watcher).collect::>(); + + assert_eq!( + xt0_status, + vec![ + TransactionStatus::InBlock((header02.hash(), 0)), + TransactionStatus::Finalized((header02.hash(), 0)), + ] + ); + assert_eq!( + xt1_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header03.hash(), 0)), + TransactionStatus::Finalized((header03.hash(), 0)), + ] + ); +} + +#[test] +fn fatp_watcher_best_block_after_finalization_does_not_retract() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + api.set_nonce(api.genesis_hash(), Bob.into(), 200); + api.set_nonce(api.genesis_hash(), Charlie.into(), 200); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 200); + + let header01 = api.push_block(1, vec![], true); + + let xt0_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + let header02 = api.push_block_with_parent(header01.hash(), vec![xt0.clone()], true); + + let xt1_watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let header03 = api.push_block_with_parent(header02.hash(), vec![xt1.clone()], true); + + block_on(pool.maintain(finalized_block_event(&pool, api.genesis_hash(), header01.hash()))); + block_on(pool.maintain(finalized_block_event(&pool, header01.hash(), header03.hash()))); + block_on(pool.maintain(new_best_block_event(&pool, Some(api.genesis_hash()), header02.hash()))); + + let xt0_status = futures::executor::block_on_stream(xt0_watcher).collect::>(); + let xt1_status = futures::executor::block_on_stream(xt1_watcher).collect::>(); + + log::debug!("xt0_status: {:#?}", xt0_status); + log::debug!("xt1_status: {:#?}", xt1_status); + + assert_eq!( + xt0_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header02.hash(), 0)), + TransactionStatus::Finalized((header02.hash(), 0)), + ] + ); + assert_eq!( + xt1_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header03.hash(), 0)), + TransactionStatus::Finalized((header03.hash(), 0)), + ] + ); +} + +#[test] +fn fatp_watcher_invalid_many_revalidation() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + 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, 201); + let xt2 = uxt(Alice, 202); + let xt3 = uxt(Alice, 203); + let xt4 = uxt(Alice, 204); + + let submissions = vec![ + pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt3.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt4.clone()), + ]; + + let submissions = block_on(futures::future::join_all(submissions)); + assert_eq!(pool.status_all()[&header01.hash()].ready, 5); + + let mut watchers = submissions.into_iter().map(Result::unwrap).collect::>(); + let xt4_watcher = watchers.remove(4); + let xt3_watcher = watchers.remove(3); + let xt2_watcher = watchers.remove(2); + let xt1_watcher = watchers.remove(1); + let xt0_watcher = watchers.remove(0); + + api.add_invalid(&xt3); + api.add_invalid(&xt4); + + let header02 = api.push_block(2, vec![], true); + block_on(pool.maintain(finalized_block_event(&pool, header01.hash(), header02.hash()))); + + //todo: shall revalidation check finalized (fork's tip) view? + assert_eq!(pool.status_all()[&header02.hash()].ready, 5); + + let header03 = api.push_block(3, vec![xt0.clone(), xt1.clone(), xt2.clone()], true); + block_on(pool.maintain(finalized_block_event(&pool, header02.hash(), header03.hash()))); + + // wait 10 blocks for revalidation + let mut prev_header = header03.clone(); + for n in 4..=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 xt0_events = futures::executor::block_on_stream(xt0_watcher).collect::>(); + let xt1_events = futures::executor::block_on_stream(xt1_watcher).collect::>(); + let xt2_events = futures::executor::block_on_stream(xt2_watcher).collect::>(); + let xt3_events = futures::executor::block_on_stream(xt3_watcher).collect::>(); + let xt4_events = futures::executor::block_on_stream(xt4_watcher).collect::>(); + + log::debug!("xt0_events: {:#?}", xt0_events); + log::debug!("xt1_events: {:#?}", xt1_events); + log::debug!("xt2_events: {:#?}", xt2_events); + log::debug!("xt3_events: {:#?}", xt3_events); + log::debug!("xt4_events: {:#?}", xt4_events); + + assert_eq!( + xt0_events, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header03.hash(), 0)), + TransactionStatus::Finalized((header03.hash(), 0)) + ], + ); + assert_eq!( + xt1_events, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header03.hash(), 1)), + TransactionStatus::Finalized((header03.hash(), 1)) + ], + ); + assert_eq!( + xt2_events, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header03.hash(), 2)), + TransactionStatus::Finalized((header03.hash(), 2)) + ], + ); + assert_eq!(xt3_events, vec![TransactionStatus::Ready, TransactionStatus::Invalid],); + assert_eq!(xt4_events, vec![TransactionStatus::Ready, TransactionStatus::Invalid],); +} + +#[test] +fn should_not_retain_invalid_hashes_from_retracted() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + let xt = uxt(Alice, 200); + + let header01 = api.push_block(1, vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, None, header01.hash()))); + let watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt.clone())).unwrap(); + + let header02a = api.push_block_with_parent(header01.hash(), vec![xt.clone()], true); + + block_on(pool.maintain(new_best_block_event(&pool, Some(header01.hash()), header02a.hash()))); + assert_eq!(pool.status_all()[&header02a.hash()].ready, 0); + + api.add_invalid(&xt); + let header02b = api.push_block_with_parent(header01.hash(), vec![], true); + block_on(pool.maintain(finalized_block_event(&pool, api.genesis_hash(), header02b.hash()))); + + // wait 10 blocks for revalidation + let mut prev_header = header02b.clone(); + for _ in 3..=11 { + let header = api.push_block_with_parent(prev_header.hash(), vec![], true); + let event = finalized_block_event(&pool, prev_header.hash(), header.hash()); + block_on(pool.maintain(event)); + prev_header = header; + } + + assert_eq!( + futures::executor::block_on_stream(watcher).collect::>(), + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header02a.hash(), 0)), + TransactionStatus::Invalid + ], + ); + + //todo: shall revalidation check finalized (fork's tip) view? + assert_eq!(pool.status_all()[&prev_header.hash()].ready, 0); +} + +#[test] +fn should_revalidate_during_maintenance() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + let xt1 = uxt(Alice, 200); + let xt2 = uxt(Alice, 201); + + let header01 = api.push_block(1, vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, None, header01.hash()))); + + block_on(pool.submit_one(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let watcher = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + assert_eq!(pool.status_all()[&header01.hash()].ready, 2); + assert_eq!(api.validation_requests().len(), 2); + + let header02 = api.push_block(2, vec![xt1.clone()], true); + api.add_invalid(&xt2); + block_on(pool.maintain(finalized_block_event(&pool, api.genesis_hash(), header02.hash()))); + + //todo: shall revalidation check finalized (fork's tip) view? + assert_eq!(pool.status_all()[&header02.hash()].ready, 1); + + // wait 10 blocks for revalidation + let mut prev_header = header02.clone(); + for _ in 3..=11 { + let header = api.push_block_with_parent(prev_header.hash(), vec![], true); + let event = finalized_block_event(&pool, prev_header.hash(), header.hash()); + block_on(pool.maintain(event)); + prev_header = header; + } + + assert_eq!( + futures::executor::block_on_stream(watcher).collect::>(), + vec![TransactionStatus::Ready, TransactionStatus::Invalid], + ); +} + +#[test] +fn fatp_transactions_purging_stale_on_finalization_works() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let xt1 = uxt(Alice, 200); + let xt2 = uxt(Alice, 201); + let xt3 = uxt(Alice, 202); + + let header01 = api.push_block(1, vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, None, header01.hash()))); + + let watcher1 = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let watcher2 = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + block_on(pool.submit_one(invalid_hash(), SOURCE, xt3.clone())).unwrap(); + + assert_eq!(api.validation_requests().len(), 3); + assert_eq!(pool.status_all()[&header01.hash()].ready, 3); + assert_eq!(pool.mempool_len(), (1, 2)); + + let header02 = api.push_block(2, vec![xt1.clone(), xt2.clone(), xt3.clone()], true); + api.set_nonce(header02.hash(), Alice.into(), 203); + block_on(pool.maintain(finalized_block_event(&pool, header01.hash(), header02.hash()))); + + assert_eq!(pool.status_all()[&header02.hash()].ready, 0); + assert_eq!(pool.mempool_len(), (0, 0)); + + let xt1_events = futures::executor::block_on_stream(watcher1).collect::>(); + let xt2_events = futures::executor::block_on_stream(watcher2).collect::>(); + assert_eq!( + xt1_events, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header02.hash(), 0)), + TransactionStatus::Finalized((header02.hash(), 0)) + ], + ); + assert_eq!( + xt2_events, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header02.hash(), 1)), + TransactionStatus::Finalized((header02.hash(), 1)) + ], + ); +} + +#[test] +fn fatp_transactions_purging_invalid_on_finalization_works() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let xt1 = uxt(Alice, 200); + let xt2 = uxt(Alice, 201); + let xt3 = uxt(Alice, 202); + + let header01 = api.push_block(1, vec![], true); + block_on(pool.maintain(new_best_block_event(&pool, None, header01.hash()))); + + let watcher1 = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let watcher2 = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + block_on(pool.submit_one(invalid_hash(), SOURCE, xt3.clone())).unwrap(); + + assert_eq!(api.validation_requests().len(), 3); + assert_eq!(pool.status_all()[&header01.hash()].ready, 3); + assert_eq!(pool.mempool_len(), (1, 2)); + + let header02 = api.push_block(2, vec![], true); + api.add_invalid(&xt1); + api.add_invalid(&xt2); + api.add_invalid(&xt3); + block_on(pool.maintain(finalized_block_event(&pool, header01.hash(), header02.hash()))); + + // wait 10 blocks for revalidation + let mut prev_header = header02; + for n in 3..=13 { + 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; + } + + //todo: should it work at all? (it requires better revalidation: mempool keeping validated txs) + //additionally it also requires revalidation of finalized view. + // assert_eq!(pool.status_all()[&header02.hash()].ready, 0); + assert_eq!(pool.mempool_len(), (0, 0)); + + let xt1_events = futures::executor::block_on_stream(watcher1).collect::>(); + let xt2_events = futures::executor::block_on_stream(watcher2).collect::>(); + assert_eq!(xt1_events, vec![TransactionStatus::Ready, TransactionStatus::Invalid]); + assert_eq!(xt2_events, vec![TransactionStatus::Ready, TransactionStatus::Invalid]); +} + +#[test] +fn import_sink_works() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let genesis = api.genesis_hash(); + let header01a = api.push_block(1, vec![], true); + let header01b = api.push_block(1, vec![], true); + + let import_stream = pool.import_notification_stream(); + + let event = new_best_block_event(&pool, None, header01a.hash()); + block_on(pool.maintain(event)); + + let event = new_best_block_event(&pool, None, header01b.hash()); + block_on(pool.maintain(event)); + + api.set_nonce(header01b.hash(), Alice.into(), 202); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 202); + + let submissions = vec![ + pool.submit_one(genesis, SOURCE, xt0.clone()), + pool.submit_one(genesis, SOURCE, xt1.clone()), + ]; + + block_on(futures::future::join_all(submissions)); + + assert_pool_status!(header01a.hash(), &pool, 1, 1); + assert_pool_status!(header01b.hash(), &pool, 1, 0); + + let import_events = + futures::executor::block_on_stream(import_stream).take(2).collect::>(); + + let expected_import_events = vec![api.hash_and_length(&xt0).0, api.hash_and_length(&xt1).0]; + assert!(import_events.iter().all(|v| expected_import_events.contains(v))); +} + +#[test] +fn import_sink_works2() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let genesis = api.genesis_hash(); + let header01a = api.push_block(1, vec![], true); + let header01b = api.push_block(1, vec![], true); + + let import_stream = pool.import_notification_stream(); + + let event = new_best_block_event(&pool, None, header01a.hash()); + block_on(pool.maintain(event)); + + let event = new_best_block_event(&pool, None, header01b.hash()); + block_on(pool.maintain(event)); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 202); + + let submissions = vec![ + pool.submit_one(genesis, SOURCE, xt0.clone()), + pool.submit_one(genesis, SOURCE, xt1.clone()), + ]; + + block_on(futures::future::join_all(submissions)); + + assert_pool_status!(header01a.hash(), &pool, 1, 1); + assert_pool_status!(header01b.hash(), &pool, 1, 1); + + let import_events = + futures::executor::block_on_stream(import_stream).take(1).collect::>(); + + let expected_import_events = vec![api.hash_and_length(&xt0).0]; + assert_eq!(import_events, expected_import_events); +} + +#[test] +fn import_sink_works3() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let import_stream = pool.import_notification_stream(); + let genesis = api.genesis_hash(); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 202); + + let submissions = vec![ + pool.submit_one(genesis, SOURCE, xt0.clone()), + pool.submit_one(genesis, SOURCE, xt1.clone()), + ]; + + let x = block_on(futures::future::join_all(submissions)); + + let header01a = api.push_block(1, vec![], true); + let header01b = api.push_block(1, vec![], true); + + let event = new_best_block_event(&pool, None, header01a.hash()); + block_on(pool.maintain(event)); + + let event = new_best_block_event(&pool, None, header01b.hash()); + block_on(pool.maintain(event)); + + assert_pool_status!(header01a.hash(), &pool, 1, 1); + assert_pool_status!(header01b.hash(), &pool, 1, 1); + + log::debug!("xxx {x:#?}"); + + let import_events = + futures::executor::block_on_stream(import_stream).take(1).collect::>(); + + let expected_import_events = vec![api.hash_and_length(&xt0).0]; + assert_eq!(import_events, expected_import_events); +} + +#[test] +fn fatp_avoid_stuck_transaction() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 201); + let xt2 = uxt(Alice, 202); + let xt3 = uxt(Alice, 203); + let xt4 = uxt(Alice, 204); + let xt4i = uxt(Alice, 204); + let xt4i_watcher = + block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt4i.clone())).unwrap(); + + assert_eq!(pool.mempool_len(), (0, 1)); + + let header01 = api.push_block(1, vec![xt0], true); + api.set_nonce(header01.hash(), Alice.into(), 201); + let header02 = api.push_block(2, vec![xt1], true); + api.set_nonce(header02.hash(), Alice.into(), 202); + let header03 = api.push_block(3, vec![xt2], true); + api.set_nonce(header03.hash(), Alice.into(), 203); + + let header04 = api.push_block(4, vec![], true); + api.set_nonce(header04.hash(), Alice.into(), 203); + + let header05 = api.push_block(5, vec![], true); + api.set_nonce(header05.hash(), Alice.into(), 203); + + let event = new_best_block_event(&pool, None, header05.hash()); + block_on(pool.maintain(event)); + + let event = finalized_block_event(&pool, api.genesis_hash(), header03.hash()); + block_on(pool.maintain(event)); + + assert_pool_status!(header05.hash(), &pool, 0, 1); + + let header06 = api.push_block(6, vec![xt3, xt4], true); + api.set_nonce(header06.hash(), Alice.into(), 205); + let event = new_best_block_event(&pool, None, header06.hash()); + block_on(pool.maintain(event)); + + 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 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!(pool.mempool_len(), (0, 0)); +} + +#[test] +fn fatp_future_is_pruned_by_conflicting_tags() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 201); + let xt2 = uxt(Alice, 202); + let xt2i = uxt(Alice, 202); + log::debug!("xt0: {:#?}", api.hash_and_length(&xt0).0); + log::debug!("xt1: {:#?}", api.hash_and_length(&xt1).0); + log::debug!("xt2: {:#?}", api.hash_and_length(&xt2).0); + log::debug!("xt2i: {:#?}", api.hash_and_length(&xt2i).0); + let _ = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2i.clone())).unwrap(); + + assert_eq!(pool.mempool_len(), (0, 1)); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + assert_pool_status!(header01.hash(), &pool, 0, 1); + + let header02 = api.push_block(2, vec![xt0, xt1, xt2], true); + api.set_nonce(header02.hash(), Alice.into(), 203); + + let event = new_best_block_event(&pool, None, header02.hash()); + block_on(pool.maintain(event)); + + assert_pool_status!(header02.hash(), &pool, 0, 0); +} + +#[test] +fn fatp_dangling_ready_gets_revalidated() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let xt2 = uxt(Alice, 202); + log::debug!("xt2: {:#?}", api.hash_and_length(&xt2).0); + + let header01 = api.push_block(1, vec![], true); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + assert_pool_status!(header01.hash(), &pool, 0, 0); + + let header02a = api.push_block_with_parent(header01.hash(), vec![], true); + api.set_nonce(header02a.hash(), Alice.into(), 202); + let event = new_best_block_event(&pool, Some(header01.hash()), header02a.hash()); + block_on(pool.maintain(event)); + + // send xt2 - it will become ready on block 02a. + let _ = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + assert_pool_status!(header02a.hash(), &pool, 1, 0); + assert_eq!(pool.mempool_len(), (0, 1)); + + //xt2 is still ready: view was just cloned (revalidation executed in background) + let header02b = api.push_block_with_parent(header01.hash(), vec![], true); + let event = new_best_block_event(&pool, Some(header02a.hash()), header02b.hash()); + block_on(pool.maintain(event)); + assert_pool_status!(header02b.hash(), &pool, 1, 0); + + //xt2 is now future - view revalidation worked. + let header03b = api.push_block_with_parent(header02b.hash(), vec![], true); + let event = new_best_block_event(&pool, Some(header02b.hash()), header03b.hash()); + block_on(pool.maintain(event)); + assert_pool_status!(header03b.hash(), &pool, 0, 1); +} + +#[test] +fn fatp_ready_txs_are_provided_in_valid_order() { + // this test checks if recently_pruned tags are cleared for views cloned from retracted path + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 201); + let xt2 = uxt(Alice, 202); + log::debug!("xt0: {:#?}", api.hash_and_length(&xt0).0); + log::debug!("xt1: {:#?}", api.hash_and_length(&xt1).0); + log::debug!("xt2: {:#?}", api.hash_and_length(&xt2).0); + + let _ = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + let _ = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone())).unwrap(); + let _ = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone())).unwrap(); + + let header01 = api.push_block(1, vec![xt0], true); + api.set_nonce(header01.hash(), Alice.into(), 201); + let event = new_best_block_event(&pool, None, header01.hash()); + block_on(pool.maintain(event)); + assert_pool_status!(header01.hash(), &pool, 2, 0); + + let header02a = + api.push_block_with_parent(header01.hash(), vec![xt1.clone(), xt2.clone()], true); + api.set_nonce(header02a.hash(), Alice.into(), 203); + let event = new_best_block_event(&pool, Some(header01.hash()), header02a.hash()); + block_on(pool.maintain(event)); + assert_pool_status!(header02a.hash(), &pool, 0, 0); + + let header02b = api.push_block_with_parent(header01.hash(), vec![], true); + api.set_nonce(header02b.hash(), Alice.into(), 201); + let event = new_best_block_event(&pool, Some(header02a.hash()), header02b.hash()); + block_on(pool.maintain(event)); + assert_pool_status!(header02b.hash(), &pool, 2, 0); + assert_ready_iterator!(header02b.hash(), pool, [xt1, xt2]); +} + +//todo: add test: check len of filter after finalization (!) +//todo: broadcasted test? + +#[test] +fn fatp_ready_light_empty_on_unmaintained_fork() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + api.set_nonce(api.genesis_hash(), Bob.into(), 200); + let genesis = api.genesis_hash(); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 200); + + let header01a = api.push_block_with_parent(genesis, vec![xt0.clone()], true); + let event = new_best_block_event(&pool, Some(genesis), header01a.hash()); + block_on(pool.maintain(event)); + assert_pool_status!(header01a.hash(), &pool, 0, 0); + + let header01b = api.push_block_with_parent(genesis, vec![xt1.clone()], true); + + let mut ready_iterator = pool.ready_at_light(header01b.hash()).now_or_never().unwrap(); + assert!(ready_iterator.next().is_none()); +} + +#[test] +fn fatp_ready_light_misc_scenarios_works() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + api.set_nonce(api.genesis_hash(), Bob.into(), 200); + api.set_nonce(api.genesis_hash(), Charlie.into(), 200); + let genesis = api.genesis_hash(); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 200); + let xt2 = uxt(Charlie, 200); + + //fork A + let header01a = api.push_block_with_parent(genesis, vec![xt0.clone()], true); + let event = new_best_block_event(&pool, Some(genesis), header01a.hash()); + block_on(pool.maintain(event)); + assert_pool_status!(header01a.hash(), &pool, 0, 0); + + //fork B + let header01b = api.push_block_with_parent(genesis, vec![xt1.clone()], true); + let event = new_best_block_event(&pool, Some(header01a.hash()), header01b.hash()); + block_on(pool.maintain(event)); + assert_pool_status!(header01b.hash(), &pool, 1, 0); + + //new block at fork B + let header02b = api.push_block_with_parent(header01b.hash(), vec![xt1.clone()], true); + + // test 1: + //ready light returns just txs from view @header01b (which contains retracted xt0) + let mut ready_iterator = pool.ready_at_light(header02b.hash()).now_or_never().unwrap(); + let ready01 = ready_iterator.next(); + assert_eq!(ready01.unwrap().hash, api.hash_and_length(&xt0).0); + assert!(ready_iterator.next().is_none()); + + // test 2: + // submit new transaction to all views + block_on(pool.submit_one(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + + //new block at fork A, not yet notified to pool + let header02a = api.push_block_with_parent(header01a.hash(), vec![], true); + + //ready light returns just txs from view @header01a (which contains newly submitted xt2) + let mut ready_iterator = pool.ready_at_light(header02a.hash()).now_or_never().unwrap(); + let ready01 = ready_iterator.next(); + assert_eq!(ready01.unwrap().hash, api.hash_and_length(&xt2).0); + assert!(ready_iterator.next().is_none()); + + //test 3: + let mut ready_iterator = pool.ready_at_light(header02b.hash()).now_or_never().unwrap(); + let ready01 = ready_iterator.next(); + assert_eq!(ready01.unwrap().hash, api.hash_and_length(&xt0).0); + let ready02 = ready_iterator.next(); + assert_eq!(ready02.unwrap().hash, api.hash_and_length(&xt2).0); + assert!(ready_iterator.next().is_none()); + + //test 4: + //new block at fork B, not yet notified to pool + let header03b = + api.push_block_with_parent(header02b.hash(), vec![xt0.clone(), xt2.clone()], true); + //ready light @header03b will be empty: as new block contains xt0/xt2 + let mut ready_iterator = pool.ready_at_light(header03b.hash()).now_or_never().unwrap(); + assert!(ready_iterator.next().is_none()); +} + +#[test] +fn fatp_ready_light_long_fork_works() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + api.set_nonce(api.genesis_hash(), Bob.into(), 200); + api.set_nonce(api.genesis_hash(), Charlie.into(), 200); + api.set_nonce(api.genesis_hash(), Dave.into(), 200); + api.set_nonce(api.genesis_hash(), Eve.into(), 200); + + let genesis = api.genesis_hash(); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 200); + let xt2 = uxt(Charlie, 200); + let xt3 = uxt(Dave, 200); + let xt4 = uxt(Eve, 200); + + let submissions = vec![pool.submit_at( + genesis, + SOURCE, + vec![xt0.clone(), xt1.clone(), xt2.clone(), xt3.clone(), xt4.clone()], + )]; + let results = block_on(futures::future::join_all(submissions)); + assert!(results.iter().all(Result::is_ok)); + + let header01 = api.push_block_with_parent(genesis, vec![xt0.clone()], true); + let event = new_best_block_event(&pool, Some(genesis), header01.hash()); + block_on(pool.maintain(event)); + + let header02 = api.push_block_with_parent(header01.hash(), vec![xt1.clone()], true); + let header03 = api.push_block_with_parent(header02.hash(), vec![xt2.clone()], true); + let header04 = api.push_block_with_parent(header03.hash(), vec![xt3.clone()], true); + + let mut ready_iterator = pool.ready_at_light(header04.hash()).now_or_never().unwrap(); + let ready01 = ready_iterator.next(); + assert_eq!(ready01.unwrap().hash, api.hash_and_length(&xt4).0); + assert!(ready_iterator.next().is_none()); +} + +#[test] +fn fatp_ready_light_long_fork_retracted_works() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + api.set_nonce(api.genesis_hash(), Bob.into(), 200); + api.set_nonce(api.genesis_hash(), Charlie.into(), 200); + api.set_nonce(api.genesis_hash(), Dave.into(), 200); + api.set_nonce(api.genesis_hash(), Eve.into(), 200); + + let genesis = api.genesis_hash(); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 200); + let xt2 = uxt(Charlie, 200); + let xt3 = uxt(Dave, 200); + let xt4 = uxt(Eve, 200); + + let submissions = vec![pool.submit_at( + genesis, + SOURCE, + vec![xt0.clone(), xt1.clone(), xt2.clone(), xt3.clone()], + )]; + let results = block_on(futures::future::join_all(submissions)); + assert!(results.iter().all(|r| { r.is_ok() })); + + let header01a = api.push_block_with_parent(genesis, vec![xt4.clone()], true); + let event = new_best_block_event(&pool, Some(genesis), header01a.hash()); + block_on(pool.maintain(event)); + + let header01b = api.push_block_with_parent(genesis, vec![xt0.clone()], true); + let header02b = api.push_block_with_parent(header01b.hash(), vec![xt1.clone()], true); + let header03b = api.push_block_with_parent(header02b.hash(), vec![xt2.clone()], true); + + let mut ready_iterator = pool.ready_at_light(header03b.hash()).now_or_never().unwrap(); + assert!(ready_iterator.next().is_none()); + + let event = new_best_block_event(&pool, Some(header01a.hash()), header01b.hash()); + block_on(pool.maintain(event)); + + let mut ready_iterator = pool.ready_at_light(header03b.hash()).now_or_never().unwrap(); + let ready01 = ready_iterator.next(); + assert_eq!(ready01.unwrap().hash, api.hash_and_length(&xt3).0); + let ready02 = ready_iterator.next(); + assert_eq!(ready02.unwrap().hash, api.hash_and_length(&xt4).0); + assert!(ready_iterator.next().is_none()); +} + +#[test] +fn fatp_ready_at_with_timeout_works_for_misc_scenarios() { + sp_tracing::try_init_simple(); + + let (pool, api, _) = pool(); + api.set_nonce(api.genesis_hash(), Bob.into(), 200); + api.set_nonce(api.genesis_hash(), Charlie.into(), 200); + let genesis = api.genesis_hash(); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Bob, 200); + + let header01a = api.push_block_with_parent(genesis, vec![xt0.clone()], true); + let event = new_best_block_event(&pool, Some(genesis), header01a.hash()); + block_on(pool.maintain(event)); + assert_pool_status!(header01a.hash(), &pool, 0, 0); + + let header01b = api.push_block_with_parent(genesis, vec![xt1.clone()], true); + + let mut ready_at_future = + pool.ready_at_with_timeout(header01b.hash(), Duration::from_secs(36000)); + + let noop_waker = futures::task::noop_waker(); + let mut context = futures::task::Context::from_waker(&noop_waker); + + if ready_at_future.poll_unpin(&mut context).is_ready() { + panic!("Ready set should not be ready before maintenance on block update!"); + } + + let event = new_best_block_event(&pool, Some(header01a.hash()), header01b.hash()); + block_on(pool.maintain(event)); + + // ready should now be triggered: + let mut ready_at = ready_at_future.now_or_never().unwrap(); + assert_eq!(ready_at.next().unwrap().hash, api.hash_and_length(&xt0).0); + assert!(ready_at.next().is_none()); + + let header02a = api.push_block_with_parent(header01a.hash(), vec![], true); + let xt2 = uxt(Charlie, 200); + block_on(pool.submit_one(invalid_hash(), SOURCE, xt2.clone())).unwrap(); + + // ready light should now be triggered: + let mut ready_at2 = block_on(pool.ready_at_with_timeout(header02a.hash(), Duration::ZERO)); + assert_eq!(ready_at2.next().unwrap().hash, api.hash_and_length(&xt2).0); + assert!(ready_at2.next().is_none()); +} diff --git a/substrate/client/transaction-pool/tests/fatp_common/mod.rs b/substrate/client/transaction-pool/tests/fatp_common/mod.rs new file mode 100644 index 000000000000..63af729b8b73 --- /dev/null +++ b/substrate/client/transaction-pool/tests/fatp_common/mod.rs @@ -0,0 +1,285 @@ +// 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 for fork-aware transaction pool. + +use sc_transaction_pool::{ChainApi, PoolLimit}; +use sc_transaction_pool_api::ChainEvent; +use sp_runtime::transaction_validity::TransactionSource; +use std::sync::Arc; +use substrate_test_runtime_client::{ + runtime::{Block, Hash, Header}, + AccountKeyring::*, +}; +use substrate_test_runtime_transaction_pool::{uxt, TestApi}; +pub const LOG_TARGET: &str = "txpool"; + +use sc_transaction_pool::ForkAwareTxPool; + +pub fn invalid_hash() -> Hash { + Default::default() +} + +pub fn new_best_block_event( + pool: &ForkAwareTxPool, + from: Option, + to: Hash, +) -> ChainEvent { + ChainEvent::NewBestBlock { + hash: to, + tree_route: from.map(|from| { + // note: real tree route in NewBestBlock event does not contain 'to' block. + Arc::from( + pool.api() + .tree_route(from, pool.api().block_header(to).unwrap().unwrap().parent_hash) + .expect("Tree route exists"), + ) + }), + } +} + +pub fn finalized_block_event( + pool: &ForkAwareTxPool, + from: Hash, + to: Hash, +) -> ChainEvent { + let t = pool.api().tree_route(from, to).expect("Tree route exists"); + + let e = t.enacted().iter().map(|h| h.hash).collect::>(); + ChainEvent::Finalized { hash: to, tree_route: Arc::from(&e[0..e.len() - 1]) } +} + +pub struct TestPoolBuilder { + api: Option>, + use_default_limits: bool, + ready_limits: sc_transaction_pool::PoolLimit, + future_limits: sc_transaction_pool::PoolLimit, + mempool_max_transactions_count: usize, +} + +impl Default for TestPoolBuilder { + fn default() -> Self { + Self { + api: None, + use_default_limits: true, + ready_limits: PoolLimit { count: 8192, total_bytes: 20 * 1024 * 1024 }, + future_limits: PoolLimit { count: 512, total_bytes: 1 * 1024 * 1024 }, + mempool_max_transactions_count: usize::MAX, + } + } +} + +impl TestPoolBuilder { + pub fn new() -> Self { + Self::default() + } + + pub fn with_api(mut self, api: Arc) -> Self { + self.api = Some(api); + self + } + + pub fn with_mempool_count_limit(mut self, mempool_count_limit: usize) -> Self { + self.mempool_max_transactions_count = mempool_count_limit; + self.use_default_limits = false; + self + } + + pub fn with_ready_count(mut self, ready_count: usize) -> Self { + self.ready_limits.count = ready_count; + self.use_default_limits = false; + self + } + + pub fn with_ready_bytes_size(mut self, ready_bytes_size: usize) -> Self { + self.ready_limits.total_bytes = ready_bytes_size; + self.use_default_limits = false; + self + } + + pub fn with_future_count(mut self, future_count: usize) -> Self { + self.future_limits.count = future_count; + self.use_default_limits = false; + self + } + + pub fn with_future_bytes_size(mut self, future_bytes_size: usize) -> Self { + self.future_limits.total_bytes = future_bytes_size; + self.use_default_limits = false; + self + } + + pub fn build( + self, + ) -> (ForkAwareTxPool, Arc, futures::executor::ThreadPool) { + let api = self + .api + .unwrap_or(Arc::from(TestApi::with_alice_nonce(200).enable_stale_check())); + + let genesis_hash = api + .chain() + .read() + .block_by_number + .get(&0) + .map(|blocks| blocks[0].0.header.hash()) + .expect("there is block 0. qed"); + + let (pool, txpool_task) = if self.use_default_limits { + ForkAwareTxPool::new_test(api.clone(), genesis_hash, genesis_hash) + } else { + ForkAwareTxPool::new_test_with_limits( + api.clone(), + genesis_hash, + genesis_hash, + self.ready_limits, + self.future_limits, + self.mempool_max_transactions_count, + ) + }; + + let thread_pool = futures::executor::ThreadPool::new().unwrap(); + thread_pool.spawn_ok(txpool_task); + + (pool, api, thread_pool) + } +} + +pub fn pool_with_api( + test_api: Arc, +) -> (ForkAwareTxPool, futures::executor::ThreadPool) { + let builder = TestPoolBuilder::new(); + let (pool, _, threadpool) = builder.with_api(test_api).build(); + (pool, threadpool) +} + +pub fn pool() -> (ForkAwareTxPool, Arc, futures::executor::ThreadPool) { + let builder = TestPoolBuilder::new(); + builder.build() +} + +#[macro_export] +macro_rules! assert_pool_status { + ($hash:expr, $pool:expr, $ready:expr, $future:expr) => { + { + log::debug!(target:LOG_TARGET, "stats: {:#?}", $pool.status_all()); + let status = &$pool.status_all()[&$hash]; + assert_eq!(status.ready, $ready, "ready"); + assert_eq!(status.future, $future, "future"); + } + } +} + +#[macro_export] +macro_rules! assert_ready_iterator { + ($hash:expr, $pool:expr, [$( $xt:expr ),+]) => {{ + let ready_iterator = $pool.ready_at($hash).now_or_never().unwrap(); + let expected = vec![ $($pool.api().hash_and_length(&$xt).0),+]; + let output: Vec<_> = ready_iterator.collect(); + log::debug!(target:LOG_TARGET, "expected: {:#?}", expected); + log::debug!(target:LOG_TARGET, "output: {:#?}", output); + assert_eq!(expected.len(), output.len()); + assert!( + output.iter().zip(expected.iter()).all(|(o,e)| { + o.hash == *e + }) + ); + }}; +} + +pub const SOURCE: TransactionSource = TransactionSource::External; + +#[cfg(test)] +pub mod test_chain_with_forks { + use super::*; + + pub fn chain( + include_xts: Option<&dyn Fn(usize, usize) -> bool>, + ) -> (Arc, Vec>) { + // Fork layout: + // + // (fork 0) + // F01 - F02 - F03 - F04 - F05 | Alice nonce increasing, alice's txs + // / + // F00 + // \ (fork 1) + // F11 - F12 - F13 - F14 - F15 | Bob nonce increasing, Bob's txs + // + // + // e.g. F03 contains uxt(Alice, 202), nonces: Alice = 203, Bob = 200 + // F12 contains uxt(Bob, 201), nonces: Alice = 200, Bob = 202 + + let api = Arc::from(TestApi::empty().enable_stale_check()); + + let genesis = api.genesis_hash(); + + let mut forks = vec![Vec::with_capacity(6), Vec::with_capacity(6)]; + let accounts = vec![Alice, Bob]; + accounts.iter().for_each(|a| api.set_nonce(genesis, (*a).into(), 200)); + + for fork in 0..2 { + let account = accounts[fork]; + forks[fork].push(api.block_header(genesis).unwrap().unwrap()); + let mut parent = genesis; + for block in 1..6 { + let xts = if include_xts.map_or(true, |v| v(fork, block)) { + log::debug!("{},{} -> add", fork, block); + vec![uxt(account, (200 + block - 1) as u64)] + } else { + log::debug!("{},{} -> skip", fork, block); + vec![] + }; + let header = api.push_block_with_parent(parent, xts, true); + parent = header.hash(); + api.set_nonce(header.hash(), account.into(), (200 + block) as u64); + forks[fork].push(header); + } + } + + (api, forks) + } + + pub fn print_block(api: Arc, hash: Hash) { + let accounts = vec![Alice.into(), Bob.into()]; + let header = api.block_header(hash).unwrap().unwrap(); + + let nonces = accounts + .iter() + .map(|a| api.chain().read().nonces.get(&hash).unwrap().get(a).map(Clone::clone)) + .collect::>(); + log::debug!( + "number: {:?} hash: {:?}, parent: {:?}, nonces:{:?}", + header.number, + header.hash(), + header.parent_hash, + nonces + ); + } + + #[test] + fn test_chain_works() { + sp_tracing::try_init_simple(); + let (api, f) = chain(None); + log::debug!("forks: {f:#?}"); + f[0].iter().for_each(|h| print_block(api.clone(), h.hash())); + f[1].iter().for_each(|h| print_block(api.clone(), h.hash())); + let tr = api.tree_route(f[0][5].hash(), f[1][5].hash()).unwrap(); + log::debug!("{:#?}", tr); + log::debug!("e:{:#?}", tr.enacted()); + log::debug!("r:{:#?}", tr.retracted()); + } +} diff --git a/substrate/client/transaction-pool/tests/fatp_limits.rs b/substrate/client/transaction-pool/tests/fatp_limits.rs new file mode 100644 index 000000000000..6fd5f93ed070 --- /dev/null +++ b/substrate/client/transaction-pool/tests/fatp_limits.rs @@ -0,0 +1,353 @@ +// 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 limits for fork-aware transaction pool. + +pub mod fatp_common; +use fatp_common::{ + finalized_block_event, 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_limits_no_views_mempool_count() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(2).build(); + + let header = api.push_block(1, vec![], true); + + let xt0 = uxt(Alice, 200); + let xt1 = uxt(Alice, 201); + let xt2 = uxt(Alice, 202); + + let submissions = vec![ + pool.submit_one(header.hash(), SOURCE, xt0.clone()), + pool.submit_one(header.hash(), SOURCE, xt1.clone()), + pool.submit_one(header.hash(), SOURCE, xt2.clone()), + ]; + + let results = block_on(futures::future::join_all(submissions)); + let mut results = results.iter(); + + assert!(results.next().unwrap().is_ok()); + assert!(results.next().unwrap().is_ok()); + assert!(matches!( + results.next().unwrap().as_ref().unwrap_err().0, + TxPoolError::ImmediatelyDropped + )); +} + +#[test] +fn fatp_limits_ready_count_works() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(3).with_ready_count(2).build(); + api.set_nonce(api.genesis_hash(), Bob.into(), 200); + api.set_nonce(api.genesis_hash(), Charlie.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)); + + //note: we need Charlie to be first as the oldest is removed. + //For 3x alice, all tree would be removed. + //(alice,bob,charlie would work too) + let xt0 = uxt(Charlie, 500); + let xt1 = uxt(Alice, 200); + let xt2 = uxt(Alice, 201); + + let submissions = vec![ + pool.submit_one(header01.hash(), SOURCE, xt0.clone()), + pool.submit_one(header01.hash(), SOURCE, xt1.clone()), + pool.submit_one(header01.hash(), SOURCE, xt2.clone()), + ]; + + 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); + assert_ready_iterator!(header01.hash(), pool, [xt1, xt2]); + + //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, 3); + //charlie was resubmitted from mmepool into the view: + assert_pool_status!(header02b.hash(), &pool, 1, 0); + assert_ready_iterator!(header02b.hash(), pool, [xt0]); + + //branch with alice/charlie transactions shall also work: + let header02a = api.push_block(2, vec![xt0.clone(), xt1.clone()], true); + let event = new_best_block_event(&pool, Some(header02b.hash()), header02a.hash()); + block_on(pool.maintain(event)); + assert_eq!(pool.mempool_len().0, 3); + assert_pool_status!(header02a.hash(), &pool, 1, 0); + assert_ready_iterator!(header02a.hash(), pool, [xt2]); +} + +#[test] +fn fatp_limits_future_count_works() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(3).with_future_count(2).build(); + api.set_nonce(api.genesis_hash(), Bob.into(), 200); + api.set_nonce(api.genesis_hash(), Charlie.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(Charlie, 501); + let xt2 = uxt(Alice, 201); + let xt3 = uxt(Alice, 202); + + let submissions = vec![ + pool.submit_one(header01.hash(), SOURCE, xt1.clone()), + pool.submit_one(header01.hash(), SOURCE, xt2.clone()), + pool.submit_one(header01.hash(), SOURCE, xt3.clone()), + ]; + + let results = block_on(futures::future::join_all(submissions)); + assert!(results.iter().all(Result::is_ok)); + //charlie was not included into view due to limits: + assert_pool_status!(header01.hash(), &pool, 0, 2); + + let header02 = api.push_block(2, vec![xt0], true); + api.set_nonce(header02.hash(), Alice.into(), 201); //redundant + let event = new_best_block_event(&pool, Some(header01.hash()), header02.hash()); + block_on(pool.maintain(event)); + + //charlie was resubmitted from mmepool into the view: + assert_pool_status!(header02.hash(), &pool, 2, 1); + assert_eq!(pool.mempool_len().0, 3); +} + +#[test] +fn fatp_limits_watcher_mempool_prevents_dropping() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(3).with_ready_count(2).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(Charlie, 400); + let xt1 = uxt(Bob, 300); + let xt2 = uxt(Alice, 200); + + let submissions = vec![ + pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone()), + ]; + let mut submissions = block_on(futures::future::join_all(submissions)); + let xt2_watcher = submissions.remove(2).unwrap(); + let xt1_watcher = submissions.remove(1).unwrap(); + let xt0_watcher = submissions.remove(0).unwrap(); + + assert_pool_status!(header01.hash(), &pool, 2, 0); + + let xt0_status = futures::executor::block_on_stream(xt0_watcher).take(1).collect::>(); + + log::debug!("xt0_status: {:#?}", xt0_status); + + assert_eq!(xt0_status, vec![TransactionStatus::Ready]); + let xt1_status = futures::executor::block_on_stream(xt1_watcher).take(1).collect::>(); + + assert_eq!(xt1_status, vec![TransactionStatus::Ready]); + + let xt2_status = futures::executor::block_on_stream(xt2_watcher).take(1).collect::>(); + log::debug!("xt2_status: {:#?}", xt2_status); + + assert_eq!(xt2_status, vec![TransactionStatus::Ready]); +} + +#[test] +fn fatp_limits_watcher_non_intial_view_drops_transaction() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.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(Dave, 500); + let xt1 = uxt(Charlie, 400); + let xt2 = uxt(Bob, 300); + + let submissions = vec![ + pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone()), + ]; + let mut submissions = block_on(futures::future::join_all(submissions)); + let xt2_watcher = submissions.remove(2).unwrap(); + let xt1_watcher = submissions.remove(1).unwrap(); + let xt0_watcher = submissions.remove(0).unwrap(); + + assert_ready_iterator!(header01.hash(), pool, [xt1, xt2]); + + let header02 = api.push_block_with_parent(header01.hash(), vec![], true); + block_on(pool.maintain(finalized_block_event(&pool, api.genesis_hash(), header02.hash()))); + assert_pool_status!(header02.hash(), &pool, 2, 0); + assert_ready_iterator!(header02.hash(), pool, [xt2, xt0]); + + let xt0_status = futures::executor::block_on_stream(xt0_watcher).take(1).collect::>(); + assert_eq!(xt0_status, vec![TransactionStatus::Ready]); + + let xt1_status = futures::executor::block_on_stream(xt1_watcher).take(2).collect::>(); + assert_eq!(xt1_status, vec![TransactionStatus::Ready, TransactionStatus::Dropped]); + + let xt2_status = futures::executor::block_on_stream(xt2_watcher).take(1).collect::>(); + assert_eq!(xt2_status, vec![TransactionStatus::Ready]); +} + +#[test] +fn fatp_limits_watcher_finalized_transaction_frees_ready_space() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.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(Dave, 500); + let xt1 = uxt(Charlie, 400); + let xt2 = uxt(Bob, 300); + + let submissions = vec![ + pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone()), + ]; + let mut submissions = block_on(futures::future::join_all(submissions)); + let xt2_watcher = submissions.remove(2).unwrap(); + let xt1_watcher = submissions.remove(1).unwrap(); + let xt0_watcher = submissions.remove(0).unwrap(); + assert_ready_iterator!(header01.hash(), pool, [xt1, xt2]); + + let header02 = api.push_block_with_parent(header01.hash(), vec![xt0.clone()], true); + block_on(pool.maintain(finalized_block_event(&pool, api.genesis_hash(), header02.hash()))); + assert_pool_status!(header02.hash(), &pool, 2, 0); + assert_ready_iterator!(header02.hash(), pool, [xt1, xt2]); + + let xt0_status = futures::executor::block_on_stream(xt0_watcher).take(3).collect::>(); + assert_eq!( + xt0_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header02.hash(), 0)), + TransactionStatus::Finalized((header02.hash(), 0)) + ] + ); + + let xt1_status = futures::executor::block_on_stream(xt1_watcher).take(1).collect::>(); + assert_eq!(xt1_status, vec![TransactionStatus::Ready]); + + let xt2_status = futures::executor::block_on_stream(xt2_watcher).take(1).collect::>(); + assert_eq!(xt2_status, vec![TransactionStatus::Ready]); +} + +#[test] +fn fatp_limits_watcher_view_can_drop_transcation() { + sp_tracing::try_init_simple(); + + let builder = TestPoolBuilder::new(); + let (pool, api, _) = builder.with_mempool_count_limit(3).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(Dave, 500); + let xt1 = uxt(Charlie, 400); + let xt2 = uxt(Bob, 300); + let xt3 = uxt(Alice, 200); + + let submissions = vec![ + pool.submit_and_watch(invalid_hash(), SOURCE, xt0.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt1.clone()), + pool.submit_and_watch(invalid_hash(), SOURCE, xt2.clone()), + ]; + let mut submissions = block_on(futures::future::join_all(submissions)); + let xt2_watcher = submissions.remove(2).unwrap(); + let xt1_watcher = submissions.remove(1).unwrap(); + let xt0_watcher = submissions.remove(0).unwrap(); + + assert_ready_iterator!(header01.hash(), pool, [xt1, xt2]); + + let header02 = api.push_block_with_parent(header01.hash(), vec![xt0.clone()], true); + block_on(pool.maintain(finalized_block_event(&pool, api.genesis_hash(), header02.hash()))); + + let submission = block_on(pool.submit_and_watch(invalid_hash(), SOURCE, xt3.clone())); + let xt3_watcher = submission.unwrap(); + + assert_pool_status!(header02.hash(), pool, 2, 0); + assert_ready_iterator!(header02.hash(), pool, [xt2, xt3]); + + let xt0_status = futures::executor::block_on_stream(xt0_watcher).take(3).collect::>(); + assert_eq!( + xt0_status, + vec![ + TransactionStatus::Ready, + TransactionStatus::InBlock((header02.hash(), 0)), + TransactionStatus::Finalized((header02.hash(), 0)) + ] + ); + + let xt1_status = futures::executor::block_on_stream(xt1_watcher).take(2).collect::>(); + assert_eq!(xt1_status, vec![TransactionStatus::Ready, TransactionStatus::Dropped]); + + let xt2_status = futures::executor::block_on_stream(xt2_watcher).take(1).collect::>(); + assert_eq!(xt2_status, vec![TransactionStatus::Ready]); + + let xt3_status = futures::executor::block_on_stream(xt3_watcher).take(1).collect::>(); + assert_eq!(xt3_status, vec![TransactionStatus::Ready]); +} diff --git a/substrate/client/transaction-pool/tests/pool.rs b/substrate/client/transaction-pool/tests/pool.rs index 6d70b6ce67ec..ed0fd7d4e655 100644 --- a/substrate/client/transaction-pool/tests/pool.rs +++ b/substrate/client/transaction-pool/tests/pool.rs @@ -85,12 +85,13 @@ const SOURCE: TransactionSource = TransactionSource::External; #[test] fn submission_should_work() { let (pool, api) = pool(); - block_on(pool.submit_one(api.expect_hash_from_number(0), SOURCE, uxt(Alice, 209))).unwrap(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 209).into())) + .unwrap(); let pending: Vec<_> = pool .validated_pool() .ready() - .map(|a| TransferData::try_from(&a.data).unwrap().nonce) + .map(|a| TransferData::try_from(&*a.data).unwrap().nonce) .collect(); assert_eq!(pending, vec![209]); } @@ -98,13 +99,15 @@ fn submission_should_work() { #[test] fn multiple_submission_should_work() { let (pool, api) = pool(); - block_on(pool.submit_one(api.expect_hash_from_number(0), SOURCE, uxt(Alice, 209))).unwrap(); - block_on(pool.submit_one(api.expect_hash_from_number(0), SOURCE, uxt(Alice, 210))).unwrap(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 209).into())) + .unwrap(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 210).into())) + .unwrap(); let pending: Vec<_> = pool .validated_pool() .ready() - .map(|a| TransferData::try_from(&a.data).unwrap().nonce) + .map(|a| TransferData::try_from(&*a.data).unwrap().nonce) .collect(); assert_eq!(pending, vec![209, 210]); } @@ -113,12 +116,14 @@ 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_from_number(0), SOURCE, uxt(Alice, 208))).unwrap(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 208).into())) + .unwrap(); + log::debug!("-> {:?}", pool.validated_pool().status()); let pending: Vec<_> = pool .validated_pool() .ready() - .map(|a| TransferData::try_from(&a.data).unwrap().nonce) + .map(|a| TransferData::try_from(&*a.data).unwrap().nonce) .collect(); assert_eq!(pending, Vec::::new()); } @@ -127,19 +132,21 @@ fn early_nonce_should_be_culled() { fn late_nonce_should_be_queued() { let (pool, api) = pool(); - block_on(pool.submit_one(api.expect_hash_from_number(0), SOURCE, uxt(Alice, 210))).unwrap(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 210).into())) + .unwrap(); let pending: Vec<_> = pool .validated_pool() .ready() - .map(|a| TransferData::try_from(&a.data).unwrap().nonce) + .map(|a| TransferData::try_from(&*a.data).unwrap().nonce) .collect(); assert_eq!(pending, Vec::::new()); - block_on(pool.submit_one(api.expect_hash_from_number(0), SOURCE, uxt(Alice, 209))).unwrap(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 209).into())) + .unwrap(); let pending: Vec<_> = pool .validated_pool() .ready() - .map(|a| TransferData::try_from(&a.data).unwrap().nonce) + .map(|a| TransferData::try_from(&*a.data).unwrap().nonce) .collect(); assert_eq!(pending, vec![209, 210]); } @@ -148,24 +155,25 @@ 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_from_number(0), SOURCE, uxt(Alice, 209))).unwrap(); - block_on(pool.submit_one(api.expect_hash_from_number(0), SOURCE, uxt(Alice, 210))).unwrap(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 209).into())) + .unwrap(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt(Alice, 210).into())) + .unwrap(); let pending: Vec<_> = pool .validated_pool() .ready() - .map(|a| TransferData::try_from(&a.data).unwrap().nonce) + .map(|a| TransferData::try_from(&*a.data).unwrap().nonce) .collect(); assert_eq!(pending, vec![209, 210]); pool.validated_pool().api().push_block(1, Vec::new(), true); - block_on(pool.prune_tags(api.expect_hash_from_number(1), vec![vec![209]], vec![hash209])) - .expect("Prune tags"); + block_on(pool.prune_tags(&api.expect_hash_and_number(1), vec![vec![209]], vec![hash209])); let pending: Vec<_> = pool .validated_pool() .ready() - .map(|a| TransferData::try_from(&a.data).unwrap().nonce) + .map(|a| TransferData::try_from(&*a.data).unwrap().nonce) .collect(); assert_eq!(pending, vec![210]); } @@ -173,22 +181,22 @@ fn prune_tags_should_work() { #[test] fn should_ban_invalid_transactions() { let (pool, api) = pool(); - let uxt = uxt(Alice, 209); + let uxt = Arc::from(uxt(Alice, 209)); let hash = - block_on(pool.submit_one(api.expect_hash_from_number(0), SOURCE, uxt.clone())).unwrap(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.clone())).unwrap(); pool.validated_pool().remove_invalid(&[hash]); - block_on(pool.submit_one(api.expect_hash_from_number(0), SOURCE, uxt.clone())).unwrap_err(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.clone())).unwrap_err(); // when let pending: Vec<_> = pool .validated_pool() .ready() - .map(|a| TransferData::try_from(&a.data).unwrap().nonce) + .map(|a| TransferData::try_from(&*a.data).unwrap().nonce) .collect(); assert_eq!(pending, Vec::::new()); // then - block_on(pool.submit_one(api.expect_hash_from_number(0), SOURCE, uxt.clone())).unwrap_err(); + block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, uxt.clone())).unwrap_err(); } #[test] @@ -209,47 +217,56 @@ fn only_prune_on_new_best() { #[test] fn should_correctly_prune_transactions_providing_more_than_one_tag() { + sp_tracing::try_init_simple(); let api = Arc::new(TestApi::with_alice_nonce(209)); api.set_valid_modifier(Box::new(|v: &mut ValidTransaction| { v.provides.push(vec![155]); })); let pool = Pool::new(Default::default(), true.into(), api.clone()); - let xt = uxt(Alice, 209); - block_on(pool.submit_one(api.expect_hash_from_number(0), SOURCE, xt.clone())) + let xt0 = Arc::from(uxt(Alice, 209)); + block_on(pool.submit_one(&api.expect_hash_and_number(0), SOURCE, xt0.clone())) .expect("1. Imported"); assert_eq!(pool.validated_pool().status().ready, 1); + assert_eq!(api.validation_requests().len(), 1); // remove the transaction that just got imported. api.increment_nonce(Alice.into()); api.push_block(1, Vec::new(), true); - block_on(pool.prune_tags(api.expect_hash_from_number(1), vec![vec![209]], vec![])) - .expect("1. Pruned"); + block_on(pool.prune_tags(&api.expect_hash_and_number(1), vec![vec![209]], vec![])); + assert_eq!(api.validation_requests().len(), 2); assert_eq!(pool.validated_pool().status().ready, 0); - // it's re-imported to future + // it's re-imported to future, API does not support stale - xt0 becomes future assert_eq!(pool.validated_pool().status().future, 1); // so now let's insert another transaction that also provides the 155 api.increment_nonce(Alice.into()); api.push_block(2, Vec::new(), true); - let xt = uxt(Alice, 211); - block_on(pool.submit_one(api.expect_hash_from_number(2), SOURCE, xt.clone())) + let xt1 = uxt(Alice, 211); + block_on(pool.submit_one(&api.expect_hash_and_number(2), SOURCE, xt1.clone().into())) .expect("2. Imported"); + assert_eq!(api.validation_requests().len(), 3); assert_eq!(pool.validated_pool().status().ready, 1); assert_eq!(pool.validated_pool().status().future, 1); let pending: Vec<_> = pool .validated_pool() .ready() - .map(|a| TransferData::try_from(&a.data).unwrap().nonce) + .map(|a| TransferData::try_from(&*a.data).unwrap().nonce) .collect(); assert_eq!(pending, vec![211]); // prune it and make sure the pool is empty api.increment_nonce(Alice.into()); api.push_block(3, Vec::new(), true); - block_on(pool.prune_tags(api.expect_hash_from_number(3), vec![vec![155]], vec![])) - .expect("2. Pruned"); + block_on(pool.prune_tags(&api.expect_hash_and_number(3), vec![vec![155]], vec![])); + assert_eq!(api.validation_requests().len(), 4); + //xt0 was future, it failed (bc of 155 tag conflict) and was removed assert_eq!(pool.validated_pool().status().ready, 0); - assert_eq!(pool.validated_pool().status().future, 2); + //xt1 was ready, it was pruned (bc of 155 tag conflict) but was revalidated and resubmitted + // (API does not know about 155). + assert_eq!(pool.validated_pool().status().future, 1); + + let pending: Vec<_> = pool.validated_pool().futures().iter().map(|(hash, _)| *hash).collect(); + assert_eq!(pending[0], api.hash_and_length(&xt1).0); } fn block_event(header: Header) -> ChainEvent { @@ -297,7 +314,7 @@ fn should_revalidate_during_maintenance() { .expect("1. Imported"); let watcher = block_on(pool.submit_and_watch(api.expect_hash_from_number(0), SOURCE, xt2.clone())) - .expect("2. Imported"); + .expect("import"); //todo assert_eq!(pool.status().ready, 2); assert_eq!(api.validation_requests().len(), 2); @@ -929,14 +946,16 @@ fn ready_set_should_not_resolve_before_block_update() { let xt1 = uxt(Alice, 209); block_on(pool.submit_one(api.expect_hash_from_number(0), SOURCE, xt1.clone())) .expect("1. Imported"); + let hash_of_1 = api.push_block_with_parent(api.genesis_hash(), vec![], true).hash(); - assert!(pool.ready_at(1).now_or_never().is_none()); + assert!(pool.ready_at(hash_of_1).now_or_never().is_none()); } #[test] fn ready_set_should_resolve_after_block_update() { let (pool, api, _guard) = maintained_pool(); let header = api.push_block(1, vec![], true); + let hash_of_1 = header.hash(); let xt1 = uxt(Alice, 209); @@ -944,13 +963,14 @@ fn ready_set_should_resolve_after_block_update() { .expect("1. Imported"); block_on(pool.maintain(block_event(header))); - assert!(pool.ready_at(1).now_or_never().is_some()); + assert!(pool.ready_at(hash_of_1).now_or_never().is_some()); } #[test] fn ready_set_should_eventually_resolve_when_block_update_arrives() { let (pool, api, _guard) = maintained_pool(); let header = api.push_block(1, vec![], true); + let hash_of_1 = header.hash(); let xt1 = uxt(Alice, 209); @@ -960,7 +980,7 @@ fn ready_set_should_eventually_resolve_when_block_update_arrives() { let noop_waker = futures::task::noop_waker(); let mut context = futures::task::Context::from_waker(&noop_waker); - let mut ready_set_future = pool.ready_at(1); + let mut ready_set_future = pool.ready_at(hash_of_1); if ready_set_future.poll_unpin(&mut context).is_ready() { panic!("Ready set should not be ready before block update!"); } @@ -1052,9 +1072,9 @@ fn stale_transactions_are_pruned() { // Our initial transactions let xts = vec![ - Transfer { from: Alice.into(), to: Bob.into(), nonce: 1, amount: 1 }, - Transfer { from: Alice.into(), to: Bob.into(), nonce: 2, amount: 1 }, - Transfer { from: Alice.into(), to: Bob.into(), nonce: 3, amount: 1 }, + Transfer { from: Alice.into(), to: Bob.into(), nonce: 10, amount: 1 }, + Transfer { from: Alice.into(), to: Bob.into(), nonce: 11, amount: 1 }, + Transfer { from: Alice.into(), to: Bob.into(), nonce: 12, amount: 1 }, ]; let (pool, api, _guard) = maintained_pool(); @@ -1086,6 +1106,7 @@ fn stale_transactions_are_pruned() { block_on(pool.maintain(block_event(header))); // The imported transactions have a different hash and should not evict our initial // transactions. + log::debug!("-> {:?}", pool.status()); assert_eq!(pool.status().future, 3); // Import enough blocks to make our transactions stale diff --git a/substrate/primitives/runtime/src/transaction_validity.rs b/substrate/primitives/runtime/src/transaction_validity.rs index ffff94e17461..2d800e29b8bb 100644 --- a/substrate/primitives/runtime/src/transaction_validity.rs +++ b/substrate/primitives/runtime/src/transaction_validity.rs @@ -226,7 +226,7 @@ impl From for TransactionValidity { /// Depending on the source we might apply different validation schemes. /// For instance we can disallow specific kinds of transactions if they were not produced /// by our local node (for instance off-chain workers). -#[derive(Copy, Clone, PartialEq, Eq, Encode, Decode, RuntimeDebug, TypeInfo)] +#[derive(Copy, Clone, PartialEq, Eq, Encode, Decode, RuntimeDebug, TypeInfo, Hash)] pub enum TransactionSource { /// Transaction is already included in block. /// diff --git a/substrate/test-utils/runtime/transaction-pool/Cargo.toml b/substrate/test-utils/runtime/transaction-pool/Cargo.toml index b5dc034fed13..3cdaea642263 100644 --- a/substrate/test-utils/runtime/transaction-pool/Cargo.toml +++ b/substrate/test-utils/runtime/transaction-pool/Cargo.toml @@ -17,6 +17,7 @@ targets = ["x86_64-unknown-linux-gnu"] [dependencies] codec = { workspace = true, default-features = true } futures = { workspace = true } +log = { workspace = true } parking_lot = { workspace = true, default-features = true } thiserror = { workspace = true } sc-transaction-pool = { workspace = true, default-features = true } diff --git a/substrate/test-utils/runtime/transaction-pool/src/lib.rs b/substrate/test-utils/runtime/transaction-pool/src/lib.rs index 5202e6e65154..2d19dbfb6d49 100644 --- a/substrate/test-utils/runtime/transaction-pool/src/lib.rs +++ b/substrate/test-utils/runtime/transaction-pool/src/lib.rs @@ -23,7 +23,7 @@ use codec::Encode; use futures::future::ready; use parking_lot::RwLock; use sc_transaction_pool::ChainApi; -use sp_blockchain::{CachedHeaderMetadata, TreeRoute}; +use sp_blockchain::{CachedHeaderMetadata, HashAndNumber, TreeRoute}; use sp_runtime::{ generic::{self, BlockId}, traits::{ @@ -34,7 +34,10 @@ use sp_runtime::{ ValidTransaction, }, }; -use std::collections::{BTreeMap, HashMap, HashSet}; +use std::{ + collections::{BTreeMap, HashMap, HashSet}, + sync::Arc, +}; use substrate_test_runtime_client::{ runtime::{ AccountId, Block, BlockNumber, Extrinsic, ExtrinsicBuilder, Hash, Header, Nonce, Transfer, @@ -46,7 +49,7 @@ use substrate_test_runtime_client::{ /// Error type used by [`TestApi`]. #[derive(Debug, thiserror::Error)] #[error(transparent)] -pub struct Error(#[from] sc_transaction_pool_api::error::Error); +pub struct Error(#[from] pub sc_transaction_pool_api::error::Error); impl sc_transaction_pool_api::error::IntoPoolError for Error { fn into_pool_error(self) -> Result { @@ -79,7 +82,7 @@ impl From for IsBestBlock { pub struct ChainState { pub block_by_number: BTreeMap>, pub block_by_hash: HashMap, - pub nonces: HashMap, + pub nonces: HashMap>, pub invalid_hashes: HashSet, pub priorities: HashMap, } @@ -89,14 +92,22 @@ pub struct TestApi { valid_modifier: RwLock>, chain: RwLock, validation_requests: RwLock>, + enable_stale_check: bool, } impl TestApi { /// Test Api with Alice nonce set initially. pub fn with_alice_nonce(nonce: u64) -> Self { let api = Self::empty(); + assert_eq!(api.chain.read().block_by_hash.len(), 1); + assert_eq!(api.chain.read().nonces.len(), 1); - api.chain.write().nonces.insert(Alice.into(), nonce); + api.chain + .write() + .nonces + .values_mut() + .nth(0) + .map(|h| h.insert(Alice.into(), nonce)); api } @@ -107,14 +118,23 @@ impl TestApi { valid_modifier: RwLock::new(Box::new(|_| {})), chain: Default::default(), validation_requests: RwLock::new(Default::default()), + enable_stale_check: false, }; // Push genesis block api.push_block(0, Vec::new(), true); + let hash0 = *api.chain.read().block_by_hash.keys().nth(0).unwrap(); + api.chain.write().nonces.insert(hash0, Default::default()); + api } + pub fn enable_stale_check(mut self) -> Self { + self.enable_stale_check = true; + self + } + /// Set hook on modify valid result of transaction. pub fn set_valid_modifier(&self, modifier: Box) { *self.valid_modifier.write() = modifier; @@ -184,6 +204,24 @@ impl TestApi { let mut chain = self.chain.write(); chain.block_by_hash.insert(hash, block.clone()); + if *block_number > 0 { + // copy nonces to new block + let prev_nonces = chain + .nonces + .get(block.header.parent_hash()) + .expect("there shall be nonces for parent block") + .clone(); + chain.nonces.insert(hash, prev_nonces); + } + + log::info!( + "add_block: {:?} {:?} {:?} nonces:{:#?}", + block_number, + hash, + block.header.parent_hash(), + chain.nonces + ); + if is_best_block { chain .block_by_number @@ -241,10 +279,33 @@ impl TestApi { &self.chain } + /// Set nonce in the inner state for given block. + pub fn set_nonce(&self, at: Hash, account: AccountId, nonce: u64) { + let mut chain = self.chain.write(); + chain.nonces.entry(at).and_modify(|h| { + h.insert(account, nonce); + }); + + log::debug!("set_nonce: {:?} nonces:{:#?}", at, chain.nonces); + } + + /// Increment nonce in the inner state for given block. + pub fn increment_nonce_at_block(&self, at: Hash, account: AccountId) { + let mut chain = self.chain.write(); + chain.nonces.entry(at).and_modify(|h| { + h.entry(account).and_modify(|n| *n += 1).or_insert(1); + }); + + log::debug!("increment_nonce_at_block: {:?} nonces:{:#?}", at, chain.nonces); + } + /// Increment nonce in the inner state. pub fn increment_nonce(&self, account: AccountId) { let mut chain = self.chain.write(); - chain.nonces.entry(account).and_modify(|n| *n += 1).or_insert(1); + // if no particular block was given, then update nonce everywhere + chain.nonces.values_mut().for_each(|h| { + h.entry(account).and_modify(|n| *n += 1).or_insert(1); + }) } /// Calculate a tree route between the two given blocks. @@ -260,6 +321,26 @@ impl TestApi { pub fn expect_hash_from_number(&self, n: BlockNumber) -> Hash { self.block_id_to_hash(&BlockId::Number(n)).unwrap().unwrap() } + + /// Helper function for getting genesis hash + pub fn genesis_hash(&self) -> Hash { + self.expect_hash_from_number(0) + } + + pub fn expect_hash_and_number(&self, n: BlockNumber) -> HashAndNumber { + HashAndNumber { hash: self.expect_hash_from_number(n), number: n } + } +} + +trait TagFrom { + fn tag_from(&self) -> u8; +} + +impl TagFrom for AccountId { + fn tag_from(&self) -> u8 { + let f = AccountKeyring::iter().enumerate().find(|k| AccountId::from(k.1) == *self); + u8::try_from(f.unwrap().0).unwrap() + } } impl ChainApi for TestApi { @@ -272,9 +353,11 @@ impl ChainApi for TestApi { &self, at: ::Hash, _source: TransactionSource, - uxt: ::Extrinsic, + uxt: Arc<::Extrinsic>, ) -> Self::ValidationFuture { + let uxt = (*uxt).clone(); self.validation_requests.write().push(uxt.clone()); + let block_number; match self.block_id_to_number(&BlockId::Hash(at)) { Ok(Some(number)) => { @@ -285,6 +368,7 @@ impl ChainApi for TestApi { .get(&number) .map(|blocks| blocks.iter().any(|b| b.1.is_best())) .unwrap_or(false); + block_number = Some(number); // If there is no best block, we don't know based on which block we should validate // the transaction. (This is not required for this test function, but in real @@ -303,10 +387,44 @@ impl ChainApi for TestApi { } let (requires, provides) = if let Ok(transfer) = TransferData::try_from(&uxt) { - let chain_nonce = self.chain.read().nonces.get(&transfer.from).cloned().unwrap_or(0); - let requires = - if chain_nonce == transfer.nonce { vec![] } else { vec![vec![chain_nonce as u8]] }; - let provides = vec![vec![transfer.nonce as u8]]; + let chain_nonce = self + .chain + .read() + .nonces + .get(&at) + .expect("nonces must be there for every block") + .get(&transfer.from) + .cloned() + .unwrap_or(0); + let requires = if chain_nonce == transfer.nonce { + vec![] + } else { + if self.enable_stale_check { + vec![vec![transfer.from.tag_from(), (transfer.nonce - 1) as u8]] + } else { + vec![vec![(transfer.nonce - 1) as u8]] + } + }; + let provides = if self.enable_stale_check { + vec![vec![transfer.from.tag_from(), transfer.nonce as u8]] + } else { + vec![vec![transfer.nonce as u8]] + }; + + log::info!( + "test_api::validate_transaction: h:{:?} n:{:?} cn:{:?} tn:{:?} r:{:?} p:{:?}", + at, + block_number, + chain_nonce, + transfer.nonce, + requires, + provides, + ); + + if self.enable_stale_check && transfer.nonce < chain_nonce { + log::info!("test_api::validate_transaction: invalid_transaction(stale)...."); + return ready(Ok(Err(TransactionValidityError::Invalid(InvalidTransaction::Stale)))) + } (requires, provides) } else { @@ -314,6 +432,7 @@ impl ChainApi for TestApi { }; if self.chain.read().invalid_hashes.contains(&self.hash_and_length(&uxt).0) { + log::info!("test_api::validate_transaction: invalid_transaction...."); return ready(Ok(Err(TransactionValidityError::Invalid(InvalidTransaction::Custom(0))))) } diff --git a/substrate/utils/frame/rpc/system/src/lib.rs b/substrate/utils/frame/rpc/system/src/lib.rs index 9fcaa53a35d8..824c871a3562 100644 --- a/substrate/utils/frame/rpc/system/src/lib.rs +++ b/substrate/utils/frame/rpc/system/src/lib.rs @@ -245,8 +245,13 @@ mod tests { // given let client = Arc::new(substrate_test_runtime_client::new()); let spawner = sp_core::testing::TaskExecutor::new(); - let pool = - BasicPool::new_full(Default::default(), true.into(), None, spawner, client.clone()); + let pool = Arc::from(BasicPool::new_full( + Default::default(), + true.into(), + None, + spawner, + client.clone(), + )); let source = sp_runtime::transaction_validity::TransactionSource::External; let new_transaction = |nonce: u64| { @@ -281,8 +286,13 @@ mod tests { // given let client = Arc::new(substrate_test_runtime_client::new()); let spawner = sp_core::testing::TaskExecutor::new(); - let pool = - BasicPool::new_full(Default::default(), true.into(), None, spawner, client.clone()); + let pool = Arc::from(BasicPool::new_full( + Default::default(), + true.into(), + None, + spawner, + client.clone(), + )); let accounts = System::new(client, pool); @@ -300,8 +310,13 @@ mod tests { // given let client = Arc::new(substrate_test_runtime_client::new()); let spawner = sp_core::testing::TaskExecutor::new(); - let pool = - BasicPool::new_full(Default::default(), true.into(), None, spawner, client.clone()); + let pool = Arc::from(BasicPool::new_full( + Default::default(), + true.into(), + None, + spawner, + client.clone(), + )); let accounts = System::new(client, pool); @@ -331,8 +346,13 @@ mod tests { // given let client = Arc::new(substrate_test_runtime_client::new()); let spawner = sp_core::testing::TaskExecutor::new(); - let pool = - BasicPool::new_full(Default::default(), true.into(), None, spawner, client.clone()); + let pool = Arc::from(BasicPool::new_full( + Default::default(), + true.into(), + None, + spawner, + client.clone(), + )); let accounts = System::new(client, pool); diff --git a/substrate/utils/prometheus/src/lib.rs b/substrate/utils/prometheus/src/lib.rs index 460640bcd8e4..35597cad03d8 100644 --- a/substrate/utils/prometheus/src/lib.rs +++ b/substrate/utils/prometheus/src/lib.rs @@ -27,8 +27,8 @@ pub use prometheus::{ AtomicF64 as F64, AtomicI64 as I64, AtomicU64 as U64, GenericCounter as Counter, GenericCounterVec as CounterVec, GenericGauge as Gauge, GenericGaugeVec as GaugeVec, }, - exponential_buckets, Error as PrometheusError, Histogram, HistogramOpts, HistogramVec, Opts, - Registry, + exponential_buckets, histogram_opts, linear_buckets, Error as PrometheusError, Histogram, + HistogramOpts, HistogramVec, Opts, Registry, }; pub use sourced::{MetricSource, SourcedCounter, SourcedGauge, SourcedMetric}; diff --git a/templates/minimal/node/src/service.rs b/templates/minimal/node/src/service.rs index 08cd345f1e3e..6ba6959202c4 100644 --- a/templates/minimal/node/src/service.rs +++ b/templates/minimal/node/src/service.rs @@ -46,7 +46,7 @@ pub type Service = sc_service::PartialComponents< FullBackend, FullSelectChain, sc_consensus::DefaultImportQueue, - sc_transaction_pool::FullPool, + sc_transaction_pool::TransactionPoolHandle, Option, >; @@ -79,12 +79,15 @@ pub fn new_partial(config: &Configuration) -> Result { let select_chain = sc_consensus::LongestChain::new(backend.clone()); - let transaction_pool = sc_transaction_pool::BasicPool::new_full( - config.transaction_pool.clone(), - config.role.is_authority().into(), - config.prometheus_registry(), - task_manager.spawn_essential_handle(), - client.clone(), + let transaction_pool = Arc::from( + sc_transaction_pool::Builder::new( + task_manager.spawn_essential_handle(), + client.clone(), + config.role.is_authority().into(), + ) + .with_options(config.transaction_pool.clone()) + .with_prometheus(config.prometheus_registry()) + .build(), ); let import_queue = sc_consensus_manual_seal::import_queue( diff --git a/templates/parachain/node/src/service.rs b/templates/parachain/node/src/service.rs index 6729b9d7ef4f..dd7dff2ebf16 100644 --- a/templates/parachain/node/src/service.rs +++ b/templates/parachain/node/src/service.rs @@ -57,7 +57,7 @@ pub type Service = PartialComponents< ParachainBackend, (), sc_consensus::DefaultImportQueue, - sc_transaction_pool::FullPool, + sc_transaction_pool::TransactionPoolHandle, (ParachainBlockImport, Option, Option), >; @@ -107,12 +107,15 @@ pub fn new_partial(config: &Configuration) -> Result telemetry }); - let transaction_pool = sc_transaction_pool::BasicPool::new_full( - config.transaction_pool.clone(), - config.role.is_authority().into(), - config.prometheus_registry(), - task_manager.spawn_essential_handle(), - client.clone(), + let transaction_pool = Arc::from( + sc_transaction_pool::Builder::new( + task_manager.spawn_essential_handle(), + client.clone(), + config.role.is_authority().into(), + ) + .with_options(config.transaction_pool.clone()) + .with_prometheus(config.prometheus_registry()) + .build(), ); let block_import = ParachainBlockImport::new(client.clone(), backend.clone()); @@ -173,7 +176,7 @@ fn start_consensus( telemetry: Option, task_manager: &TaskManager, relay_chain_interface: Arc, - transaction_pool: Arc>, + transaction_pool: Arc>, keystore: KeystorePtr, relay_chain_slot_duration: Duration, para_id: ParaId, diff --git a/templates/solochain/node/src/service.rs b/templates/solochain/node/src/service.rs index 2de543235ec8..4192128b6724 100644 --- a/templates/solochain/node/src/service.rs +++ b/templates/solochain/node/src/service.rs @@ -31,7 +31,7 @@ pub type Service = sc_service::PartialComponents< FullBackend, FullSelectChain, sc_consensus::DefaultImportQueue, - sc_transaction_pool::FullPool, + sc_transaction_pool::TransactionPoolHandle, ( sc_consensus_grandpa::GrandpaBlockImport, sc_consensus_grandpa::LinkHalf, @@ -67,12 +67,15 @@ pub fn new_partial(config: &Configuration) -> Result { let select_chain = sc_consensus::LongestChain::new(backend.clone()); - let transaction_pool = sc_transaction_pool::BasicPool::new_full( - config.transaction_pool.clone(), - config.role.is_authority().into(), - config.prometheus_registry(), - task_manager.spawn_essential_handle(), - client.clone(), + let transaction_pool = Arc::from( + sc_transaction_pool::Builder::new( + task_manager.spawn_essential_handle(), + client.clone(), + config.role.is_authority().into(), + ) + .with_options(config.transaction_pool.clone()) + .with_prometheus(config.prometheus_registry()) + .build(), ); let (grandpa_block_import, grandpa_link) = sc_consensus_grandpa::block_import(