From 5178a3fbb4d9b629b68337229e45695ec8eb6046 Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Fri, 12 Jun 2020 15:00:32 +0200 Subject: [PATCH 01/25] WIP --- Cargo.lock | 37 ---- Cargo.toml | 2 +- bin/node-template/node/src/service.rs | 28 +-- bin/node/cli/Cargo.toml | 2 +- bin/node/cli/src/browser.rs | 4 +- bin/node/cli/src/chain_spec.rs | 6 +- bin/node/cli/src/service.rs | 42 ++--- client/cli/src/runner.rs | 14 +- client/network/src/service/out_events.rs | 13 +- client/service/src/builder.rs | 53 +++--- client/service/src/lib.rs | 231 ++++++++--------------- client/service/src/task_manager.rs | 42 ++++- utils/browser/src/lib.rs | 8 +- 13 files changed, 207 insertions(+), 275 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 83c98c65c6cb0..abf336f68cdb6 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3332,7 +3332,6 @@ dependencies = [ "sc-offchain", "sc-rpc", "sc-service", - "sc-service-test", "sc-telemetry", "sc-tracing", "sc-transaction-pool", @@ -6667,42 +6666,6 @@ dependencies = [ "wasm-timer", ] -[[package]] -name = "sc-service-test" -version = "2.0.0-rc3" -dependencies = [ - "env_logger 0.7.1", - "fdlimit", - "futures 0.1.29", - "futures 0.3.4", - "hex-literal", - "log", - "parity-scale-codec", - "parking_lot 0.10.2", - "sc-block-builder", - "sc-client-api", - "sc-client-db", - "sc-executor", - "sc-light", - "sc-network", - "sc-service", - "sp-api", - "sp-blockchain", - "sp-consensus", - "sp-core", - "sp-externalities", - "sp-panic-handler", - "sp-runtime", - "sp-state-machine", - "sp-storage", - "sp-transaction-pool", - "sp-trie", - "substrate-test-runtime", - "substrate-test-runtime-client", - "tempfile", - "tokio 0.1.22", -] - [[package]] name = "sc-state-db" version = "0.8.0-rc3" diff --git a/Cargo.toml b/Cargo.toml index d3004fcadca5a..6752687375b45 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -51,7 +51,7 @@ members = [ "client/rpc", "client/rpc-api", "client/service", - "client/service/test", + #"client/service/test", "client/state-db", "client/telemetry", "client/transaction-pool", diff --git a/bin/node-template/node/src/service.rs b/bin/node-template/node/src/service.rs index e8578ab5b52de..072987119a3df 100644 --- a/bin/node-template/node/src/service.rs +++ b/bin/node-template/node/src/service.rs @@ -5,7 +5,7 @@ use std::time::Duration; use sc_client_api::ExecutorProvider; use sc_consensus::LongestChain; use node_template_runtime::{self, opaque::Block, RuntimeApi}; -use sc_service::{error::{Error as ServiceError}, AbstractService, Configuration, ServiceBuilder}; +use sc_service::{error::{Error as ServiceError}, AbstractService, Configuration, ServiceBuilder, TaskManager}; use sp_inherents::InherentDataProviders; use sc_executor::native_executor_instance; pub use sc_executor::NativeExecutor; @@ -93,7 +93,7 @@ macro_rules! new_full_start { } /// Builds a new service for a full client. -pub fn new_full(config: Configuration) -> Result { +pub fn new_full(config: Configuration) -> Result<(impl AbstractService, TaskManager), ServiceError> { let role = config.role.clone(); let force_authoring = config.force_authoring; let name = config.network.node_name.clone(); @@ -105,7 +105,7 @@ pub fn new_full(config: Configuration) -> Result>; @@ -115,12 +115,11 @@ pub fn new_full(config: Configuration) -> Result Result( sc_consensus_aura::slot_duration(&*client)?, - client, + client.clone(), select_chain, block_import, proposer, service.network(), inherent_data_providers.clone(), force_authoring, - service.keystore(), + keystore.clone(), can_author_with, )?; // the AURA authoring task is considered essential, i.e. if it // fails we take down the service with it. - service.spawn_essential_task("aura", aura); + task_manager.spawn_essential("aura", aura); } // if the node isn't actively participating in consensus then it doesn't // need a keystore, regardless of which protocol we use below. let keystore = if role.is_authority() { - Some(service.keystore() as sp_core::traits::BareCryptoStorePtr) + Some(keystore.clone() as sp_core::traits::BareCryptoStorePtr) } else { None }; @@ -184,23 +183,23 @@ pub fn new_full(config: Configuration) -> Result Result { +pub fn new_light(config: Configuration) -> Result<(impl AbstractService, TaskManager), ServiceError> { let inherent_data_providers = InherentDataProviders::new(); ServiceBuilder::new_light::(config)? @@ -265,4 +264,5 @@ pub fn new_light(config: Configuration) -> Result, log_level: String) -> Result ChainSpec { pub(crate) mod tests { use super::*; use crate::service::{new_full, new_light}; - use sc_service_test; + //use sc_service_test; use sp_runtime::BuildStorage; fn local_testnet_genesis_instant_single() -> GenesisConfig { @@ -428,11 +428,11 @@ pub(crate) mod tests { #[test] #[ignore] fn test_connectivity() { - sc_service_test::connectivity( + /*sc_service_test::connectivity( integration_test_config_with_two_authorities(), |config| new_full(config), |config| new_light(config), - ); + );*/ } #[test] diff --git a/bin/node/cli/src/service.rs b/bin/node/cli/src/service.rs index e087186d5485a..3b78bcf6ac6e5 100644 --- a/bin/node/cli/src/service.rs +++ b/bin/node/cli/src/service.rs @@ -30,6 +30,7 @@ use node_primitives::Block; use node_runtime::RuntimeApi; use sc_service::{ AbstractService, ServiceBuilder, config::Configuration, error::{Error as ServiceError}, + TaskManager, }; use sp_inherents::InherentDataProviders; use sc_consensus::LongestChain; @@ -173,7 +174,7 @@ macro_rules! new_full { let (builder, mut import_setup, inherent_data_providers, mut rpc_setup) = new_full_start!($config); - let service = builder + let (service, client, transaction_pool, task_manager, keystore) = builder .with_finality_proof_provider(|client, backend| { // GenesisAuthoritySetProvider is implemented for StorageAndProofProvider let provider = client as Arc>; @@ -191,12 +192,11 @@ macro_rules! new_full { if let sc_service::config::Role::Authority { .. } = &role { let proposer = sc_basic_authorship::ProposerFactory::new( - service.client(), - service.transaction_pool(), + client.clone(), + transaction_pool, service.prometheus_registry().as_ref(), ); - let client = service.client(); let select_chain = service.select_chain() .ok_or(sc_service::Error::SelectChainRequired)?; @@ -204,8 +204,8 @@ macro_rules! new_full { sp_consensus::CanAuthorWithNativeVersion::new(client.executor().clone()); let babe_config = sc_consensus_babe::BabeParams { - keystore: service.keystore(), - client, + keystore: keystore.clone(), + client: client.clone(), select_chain, env: proposer, block_import, @@ -217,7 +217,7 @@ macro_rules! new_full { }; let babe = sc_consensus_babe::start_babe(babe_config)?; - service.spawn_essential_task("babe-proposer", babe); + task_manager.spawn_essential("babe-proposer", babe); } // Spawn authority discovery module. @@ -226,7 +226,7 @@ macro_rules! new_full { sc_service::config::Role::Authority { ref sentry_nodes } => ( sentry_nodes.clone(), sc_authority_discovery::Role::Authority ( - service.keystore(), + keystore.clone(), ), ), sc_service::config::Role::Sentry {..} => ( @@ -242,7 +242,7 @@ macro_rules! new_full { _ => None, }}).boxed(); let authority_discovery = sc_authority_discovery::AuthorityDiscovery::new( - service.client(), + client.clone(), network, sentries, dht_event_stream, @@ -250,13 +250,13 @@ macro_rules! new_full { service.prometheus_registry(), ); - service.spawn_task("authority-discovery", authority_discovery); + task_manager.spawn("authority-discovery", authority_discovery); } // if the node isn't actively participating in consensus then it doesn't // need a keystore, regardless of which protocol we use below. let keystore = if role.is_authority() { - Some(service.keystore() as BareCryptoStorePtr) + Some(keystore.clone() as BareCryptoStorePtr) } else { None }; @@ -292,19 +292,19 @@ macro_rules! new_full { // the GRANDPA voter task is considered infallible, i.e. // if it fails we take down the service with it. - service.spawn_essential_task( + task_manager.spawn_essential( "grandpa-voter", grandpa::run_grandpa_voter(grandpa_config)? ); } else { grandpa::setup_disabled_grandpa( - service.client(), + client, &inherent_data_providers, service.network(), )?; } - Ok((service, inherent_data_providers)) + Ok((service, inherent_data_providers, task_manager)) }}; ($config:expr) => {{ new_full!($config, |_, _| {}) @@ -313,17 +313,17 @@ macro_rules! new_full { /// Builds a new service for a full client. pub fn new_full(config: Configuration) --> Result +-> Result<(impl AbstractService, TaskManager), ServiceError> { - new_full!(config).map(|(service, _)| service) + new_full!(config).map(|(service, _, task_manager)| (service, task_manager)) } /// Builds a new service for a light client. pub fn new_light(config: Configuration) --> Result { +-> Result<(impl AbstractService, TaskManager), ServiceError> { let inherent_data_providers = InherentDataProviders::new(); - let service = ServiceBuilder::new_light::(config)? + let (service, _, _, task_manager, _) = ServiceBuilder::new_light::(config)? .with_select_chain(|_config, backend| { Ok(LongestChain::new(backend.clone())) })? @@ -407,7 +407,7 @@ pub fn new_light(config: Configuration) })? .build_light()?; - Ok(service) + Ok((service, task_manager)) } #[cfg(test)] @@ -442,7 +442,7 @@ mod tests { type AccountPublic = ::Signer; - #[test] + /*#[test] // It is "ignored", but the node-cli ignored tests are running on the CI. // This can be run locally with `cargo test --release -p node-cli test_sync -- --ignored`. #[ignore] @@ -631,5 +631,5 @@ mod tests { "//Bob".into(), ], ) - } + }*/ } diff --git a/client/cli/src/runner.rs b/client/cli/src/runner.rs index b068af0166817..32d3dd3ff8145 100644 --- a/client/cli/src/runner.rs +++ b/client/cli/src/runner.rs @@ -25,7 +25,7 @@ use futures::pin_mut; use futures::select; use futures::{future, future::FutureExt, Future}; use log::info; -use sc_service::{AbstractService, Configuration, Role, ServiceBuilderCommand, TaskType}; +use sc_service::{AbstractService, Configuration, Role, ServiceBuilderCommand, TaskType, TaskManager}; use sp_runtime::traits::{Block as BlockT, Header as HeaderT}; use sp_utils::metrics::{TOKIO_THREADS_ALIVE, TOKIO_THREADS_TOTAL}; use sp_version::RuntimeVersion; @@ -179,8 +179,8 @@ impl Runner { /// the node's configuration. pub fn run_node( self, - new_light: impl FnOnce(Configuration) -> sc_service::error::Result, - new_full: impl FnOnce(Configuration) -> sc_service::error::Result, + new_light: impl FnOnce(Configuration) -> sc_service::error::Result<(SL, TaskManager)>, + new_full: impl FnOnce(Configuration) -> sc_service::error::Result<(SF, TaskManager)>, runtime_version: RuntimeVersion, ) -> Result<()> where @@ -198,7 +198,7 @@ impl Runner { /// the node's configuration uses a "light" role. pub fn run_full_node( self, - new_full: impl FnOnce(Configuration) -> sc_service::error::Result, + new_full: impl FnOnce(Configuration) -> sc_service::error::Result<(S, TaskManager)>, runtime_version: RuntimeVersion, ) -> Result<()> where @@ -217,7 +217,7 @@ impl Runner { /// the node's configuration uses a "full" role. pub fn run_light_node( self, - new_light: impl FnOnce(Configuration) -> sc_service::error::Result, + new_light: impl FnOnce(Configuration) -> sc_service::error::Result<(S, TaskManager)>, runtime_version: RuntimeVersion, ) -> Result<()> where @@ -261,10 +261,10 @@ impl Runner { fn run_service_until_exit(mut self, service_builder: F) -> Result<()> where - F: FnOnce(Configuration) -> std::result::Result, + F: FnOnce(Configuration) -> std::result::Result<(T, TaskManager), sc_service::error::Error>, T: AbstractService + Unpin, { - let service = service_builder(self.config)?; + let (service, _task_manager) = service_builder(self.config)?; // we eagerly drop the service so that the internal exit future is fired, // but we need to keep holding a reference to the global telemetry guard diff --git a/client/network/src/service/out_events.rs b/client/network/src/service/out_events.rs index 4a631601a669e..e77d2c43f0db1 100644 --- a/client/network/src/service/out_events.rs +++ b/client/network/src/service/out_events.rs @@ -35,7 +35,7 @@ use crate::Event; use super::maybe_utf8_bytes_to_string; -use futures::{prelude::*, channel::mpsc, ready}; +use futures::{prelude::*, channel::mpsc, ready, stream::FusedStream}; use parking_lot::Mutex; use prometheus_endpoint::{register, CounterVec, GaugeVec, Opts, PrometheusError, Registry, U64}; use std::{ @@ -120,7 +120,16 @@ impl fmt::Debug for Receiver { impl Drop for Receiver { fn drop(&mut self) { // Empty the list to properly decrease the metrics. - while let Some(Some(_)) = self.next().now_or_never() {} + loop { + if self.inner.is_terminated() { + return; + } + + match self.next().now_or_never() { + Some(Some(_)) => {}, + _ => return, + } + } } } diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index 813fe50cce513..3c1b0eed4b728 100644 --- a/client/service/src/builder.rs +++ b/client/service/src/builder.rs @@ -28,7 +28,7 @@ use sc_client_api::{ ExecutorProvider, CallExecutor, ForkBlocks, BadBlocks, CloneableSpawn, UsageProvider, backend::RemoteBackend, }; -use sp_utils::mpsc::{tracing_unbounded, TracingUnboundedSender}; +use sp_utils::mpsc::{tracing_unbounded, TracingUnboundedSender, TracingUnboundedReceiver}; use sc_chain_spec::get_extension; use sp_consensus::{ block_validation::{BlockAnnounceValidator, DefaultBlockAnnounceValidator}, @@ -103,6 +103,7 @@ pub struct ServiceBuilder, block_announce_validator_builder: Option) -> Box + Send> + Send>>, informant_prefix: String, + essential_failed_rx: TracingUnboundedReceiver<()>, } /// A utility trait for building an RPC extension given a `DenyUnsafe` instance. @@ -205,6 +206,7 @@ type TFullParts = ( Arc>, Arc>, TaskManager, + TracingUnboundedReceiver<()>, ); /// Creates a new full client for the given config. @@ -231,9 +233,12 @@ fn new_full_parts( KeystoreConfig::InMemory => Keystore::new_in_memory(), }; + // A side-channel for essential tasks to communicate shutdown. + let (essential_failed_tx, essential_failed_rx) = tracing_unbounded("mpsc_essential_tasks"); + let task_manager = { let registry = config.prometheus_config.as_ref().map(|cfg| &cfg.registry); - TaskManager::new(config.task_executor.clone(), registry)? + TaskManager::new(config.task_executor.clone(), registry, essential_failed_tx)? }; let executor = NativeExecutor::::new( @@ -281,7 +286,7 @@ fn new_full_parts( )? }; - Ok((client, backend, keystore, task_manager)) + Ok((client, backend, keystore, task_manager, essential_failed_rx)) } @@ -347,7 +352,7 @@ impl ServiceBuilder<(), (), (), (), (), (), (), (), (), (), ()> { (), TFullBackend, >, Error> { - let (client, backend, keystore, task_manager) = new_full_parts(&config)?; + let (client, backend, keystore, task_manager, essential_failed_rx) = new_full_parts(&config)?; let client = Arc::new(client); @@ -357,6 +362,7 @@ impl ServiceBuilder<(), (), (), (), (), (), (), (), (), (), ()> { backend, keystore, task_manager, + essential_failed_rx, fetcher: None, select_chain: None, import_queue: (), @@ -387,9 +393,12 @@ impl ServiceBuilder<(), (), (), (), (), (), (), (), (), (), ()> { (), TLightBackend, >, Error> { + // A side-channel for essential tasks to communicate shutdown. + let (essential_failed_tx, essential_failed_rx) = tracing_unbounded("mpsc_essential_tasks"); + let task_manager = { let registry = config.prometheus_config.as_ref().map(|cfg| &cfg.registry); - TaskManager::new(config.task_executor.clone(), registry)? + TaskManager::new(config.task_executor.clone(), registry, essential_failed_tx)? }; let keystore = match &config.keystore { @@ -441,6 +450,7 @@ impl ServiceBuilder<(), (), (), (), (), (), (), (), (), (), ()> { backend, task_manager, keystore, + essential_failed_rx, fetcher: Some(fetcher.clone()), select_chain: None, import_queue: (), @@ -547,6 +557,7 @@ impl block_announce_validator_builder: self.block_announce_validator_builder, informant_prefix: self.informant_prefix, marker: self.marker, + essential_failed_rx: self.essential_failed_rx, }) } @@ -593,6 +604,7 @@ impl block_announce_validator_builder: self.block_announce_validator_builder, informant_prefix: self.informant_prefix, marker: self.marker, + essential_failed_rx: self.essential_failed_rx, }) } @@ -632,6 +644,7 @@ impl block_announce_validator_builder: self.block_announce_validator_builder, informant_prefix: self.informant_prefix, marker: self.marker, + essential_failed_rx: self.essential_failed_rx, }) } @@ -699,6 +712,7 @@ impl block_announce_validator_builder: self.block_announce_validator_builder, informant_prefix: self.informant_prefix, marker: self.marker, + essential_failed_rx: self.essential_failed_rx, }) } @@ -756,6 +770,7 @@ impl block_announce_validator_builder: self.block_announce_validator_builder, informant_prefix: self.informant_prefix, marker: self.marker, + essential_failed_rx: self.essential_failed_rx, }) } @@ -794,6 +809,7 @@ impl block_announce_validator_builder: self.block_announce_validator_builder, informant_prefix: self.informant_prefix, marker: self.marker, + essential_failed_rx: self.essential_failed_rx, }) } @@ -840,6 +856,7 @@ impl block_announce_validator_builder: Some(Box::new(block_announce_validator_builder)), informant_prefix: self.informant_prefix, marker: self.marker, + essential_failed_rx: self.essential_failed_rx, }) } @@ -959,7 +976,7 @@ ServiceBuilder< Ok(self) } - fn build_common(self) -> Result Result<(Service< TBl, Client, TSc, @@ -971,7 +988,7 @@ ServiceBuilder< TBackend::OffchainStorage, TBl >, - >, Error> + >, Arc>, Arc, TaskManager, Arc>), Error> where TExec: CallExecutor, { let ServiceBuilder { @@ -991,6 +1008,7 @@ ServiceBuilder< remote_backend, block_announce_validator_builder, informant_prefix, + essential_failed_rx, } = self; sp_session::generate_initial_session_keys( @@ -999,9 +1017,6 @@ ServiceBuilder< config.dev_key_seed.clone().map(|s| vec![s]).unwrap_or_default(), )?; - // A side-channel for essential tasks to communicate shutdown. - let (essential_failed_tx, essential_failed_rx) = tracing_unbounded("mpsc_essential_tasks"); - let import_queue = Box::new(import_queue); let chain_info = client.chain_info(); let chain_spec = &config.chain_spec; @@ -1396,29 +1411,25 @@ ServiceBuilder< ); spawn_handle.spawn("informant", informant_future); - Ok(Service { - client, - task_manager, + Ok((Service { network, network_status_sinks, select_chain, - transaction_pool, - essential_failed_tx, essential_failed_rx, rpc_handlers, _rpc: rpc, _telemetry: telemetry, _offchain_workers: offchain_workers, _telemetry_on_connect_sinks: telemetry_connection_sinks.clone(), - keystore, marker: PhantomData::, prometheus_registry: config.prometheus_config.map(|config| config.registry), _base_path: config.base_path.map(Arc::new), - }) + _phantom: PhantomData + }, client, transaction_pool, task_manager, keystore)) } /// Builds the light service. - pub fn build_light(self) -> Result Result<(Service< TBl, Client, TSc, @@ -1430,7 +1441,7 @@ ServiceBuilder< TBackend::OffchainStorage, TBl >, - >, Error> + >, Arc>, Arc, TaskManager, Arc>), Error> where TExec: CallExecutor, { self.build_common() @@ -1473,7 +1484,7 @@ ServiceBuilder< { /// Builds the full service. - pub fn build_full(self) -> Result Result<(Service< TBl, Client, TSc, @@ -1485,7 +1496,7 @@ ServiceBuilder< TBackend::OffchainStorage, TBl >, - >, Error> + >, Arc>, Arc, TaskManager, Arc>), Error> where TExec: CallExecutor, { // make transaction pool available for off-chain runtime calls. diff --git a/client/service/src/lib.rs b/client/service/src/lib.rs index fc0567e268260..5a4841d23e3f8 100644 --- a/client/service/src/lib.rs +++ b/client/service/src/lib.rs @@ -46,12 +46,7 @@ use std::task::{Poll, Context}; use parking_lot::Mutex; use client::Client; -use futures::{ - Future, FutureExt, Stream, StreamExt, - compat::*, - sink::SinkExt, - task::{Spawn, FutureObj, SpawnError}, -}; +use futures::{Future, FutureExt, Stream, StreamExt, stream::FusedStream, compat::*}; use sc_network::{NetworkService, NetworkStatus, network_state::NetworkState, PeerId}; use log::{log, warn, debug, error, Level}; use codec::{Encode, Decode}; @@ -84,7 +79,7 @@ pub use sc_network::config::{ }; pub use sc_tracing::TracingReceiver; pub use task_manager::SpawnTaskHandle; -use task_manager::TaskManager; +pub use task_manager::TaskManager; use sp_blockchain::{HeaderBackend, HeaderMetadata}; use sp_api::{ApiExt, ConstructRuntimeApi, ApiErrorExt}; use sc_client_api::{ @@ -106,17 +101,11 @@ impl MallocSizeOfWasm for T {} /// Substrate service. pub struct Service { - client: Arc, - task_manager: TaskManager, select_chain: Option, network: Arc, // Sinks to propagate network status updates. // For each element, every time the `Interval` fires we push an element on the sender. network_status_sinks: Arc>>, - transaction_pool: Arc, - // Send a signal when a spawned essential task has concluded. The next time - // the service future is polled it should complete with an error. - essential_failed_tx: TracingUnboundedSender<()>, // A receiver for spawned essential-tasks concluding. essential_failed_rx: TracingUnboundedReceiver<()>, rpc_handlers: sc_rpc_server::RpcHandler, @@ -124,18 +113,18 @@ pub struct Service { _telemetry: Option, _telemetry_on_connect_sinks: Arc>>>, _offchain_workers: Option>, - keystore: sc_keystore::KeyStorePtr, marker: PhantomData, prometheus_registry: Option, // The base path is kept here because it can be a temporary directory which will be deleted // when dropped _base_path: Option>, + _phantom: PhantomData<(TCl, TTxPool)> } impl Unpin for Service {} /// Abstraction over a Substrate service. -pub trait AbstractService: Future> + Send + Unpin + Spawn + 'static { +pub trait AbstractService: Future> + Send + Unpin + 'static { /// Type of block of this chain. type Block: BlockT; /// Backend storage for the client. @@ -160,26 +149,6 @@ pub trait AbstractService: Future> + Send + Unpin + S /// return a shared instance of Telemetry (if enabled) fn telemetry(&self) -> Option; - /// Spawns a task in the background that runs the future passed as parameter. - /// - /// Information about this task will be reported to Prometheus. - /// - /// The task name is a `&'static str` as opposed to a `String`. The reason for that is that - /// in order to avoid memory consumption issues with the Prometheus metrics, the set of - /// possible task names has to be bounded. - fn spawn_task(&self, name: &'static str, task: impl Future + Send + 'static); - - /// Spawns a task in the background that runs the future passed as - /// parameter. The given task is considered essential, i.e. if it errors we - /// trigger a service exit. - fn spawn_essential_task(&self, name: &'static str, task: impl Future + Send + 'static); - - /// Returns a handle for spawning tasks. - fn spawn_task_handle(&self) -> SpawnTaskHandle; - - /// Returns the keystore that stores keys. - fn keystore(&self) -> sc_keystore::KeyStorePtr; - /// Starts an RPC query. /// /// The query is passed as a string and must be a JSON text similar to what an HTTP client @@ -191,9 +160,6 @@ pub trait AbstractService: Future> + Send + Unpin + S /// send back spontaneous events. fn rpc_query(&self, mem: &RpcSession, request: &str) -> Pin> + Send>>; - /// Get shared client instance. - fn client(&self) -> Arc; - /// Get clone of select chain. fn select_chain(&self) -> Option; @@ -204,13 +170,6 @@ pub trait AbstractService: Future> + Send + Unpin + S /// Returns a receiver that periodically receives a status of the network. fn network_status(&self, interval: Duration) -> TracingUnboundedReceiver<(NetworkStatus, NetworkState)>; - /// Get shared transaction pool instance. - fn transaction_pool(&self) -> Arc; - - /// Get a handle to a future that will resolve on exit. - #[deprecated(note = "Use `spawn_task`/`spawn_essential_task` instead, those functions will attach on_exit signal.")] - fn on_exit(&self) -> ::exit_future::Exit; - /// Get the prometheus metrics registry, if available. fn prometheus_registry(&self) -> Option; @@ -253,30 +212,6 @@ where self._telemetry.clone() } - fn keystore(&self) -> sc_keystore::KeyStorePtr { - self.keystore.clone() - } - - fn spawn_task(&self, name: &'static str, task: impl Future + Send + 'static) { - self.task_manager.spawn(name, task) - } - - fn spawn_essential_task(&self, name: &'static str, task: impl Future + Send + 'static) { - let mut essential_failed = self.essential_failed_tx.clone(); - let essential_task = std::panic::AssertUnwindSafe(task) - .catch_unwind() - .map(move |_| { - error!("Essential task `{}` failed. Shutting down service.", name); - let _ = essential_failed.send(()); - }); - - let _ = self.spawn_task(name, essential_task); - } - - fn spawn_task_handle(&self) -> SpawnTaskHandle { - self.task_manager.spawn_handle() - } - fn rpc_query(&self, mem: &RpcSession, request: &str) -> Pin> + Send>> { Box::pin( self.rpc_handlers.handle_request(request, mem.metadata.clone()) @@ -285,10 +220,6 @@ where ) } - fn client(&self) -> Arc { - self.client.clone() - } - fn select_chain(&self) -> Option { self.select_chain.clone() } @@ -305,14 +236,6 @@ where stream } - fn transaction_pool(&self) -> Arc { - self.transaction_pool.clone() - } - - fn on_exit(&self) -> exit_future::Exit { - self.task_manager.on_exit() - } - fn prometheus_registry(&self) -> Option { self.prometheus_registry.clone() } @@ -344,18 +267,6 @@ impl Future for } } -impl Spawn for - Service -{ - fn spawn_obj( - &self, - future: FutureObj<'static, ()> - ) -> Result<(), SpawnError> { - self.task_manager.spawn_handle().spawn("unnamed", future); - Ok(()) - } -} - /// Builds a never-ending future that continuously polls the network. /// /// The `status_sink` contain a list of senders to send a periodic network status to. @@ -395,71 +306,81 @@ fn build_network_future< } // Poll the RPC requests and answer them. - while let Poll::Ready(Some(request)) = Pin::new(&mut rpc_rx).poll_next(cx) { - match request { - sc_rpc::system::Request::Health(sender) => { - let _ = sender.send(sc_rpc::system::Health { - peers: network.peers_debug_info().len(), - is_syncing: network.service().is_major_syncing(), - should_have_peers, - }); - }, - sc_rpc::system::Request::LocalPeerId(sender) => { - let _ = sender.send(network.local_peer_id().to_base58()); - }, - sc_rpc::system::Request::LocalListenAddresses(sender) => { - let peer_id = network.local_peer_id().clone().into(); - let p2p_proto_suffix = sc_network::multiaddr::Protocol::P2p(peer_id); - let addresses = network.listen_addresses() - .map(|addr| addr.clone().with(p2p_proto_suffix.clone()).to_string()) - .collect(); - let _ = sender.send(addresses); - }, - sc_rpc::system::Request::Peers(sender) => { - let _ = sender.send(network.peers_debug_info().into_iter().map(|(peer_id, p)| - sc_rpc::system::PeerInfo { - peer_id: peer_id.to_base58(), - roles: format!("{:?}", p.roles), - protocol_version: p.protocol_version, - best_hash: p.best_hash, - best_number: p.best_number, + loop { + if rpc_rx.is_terminated() { + return Poll::Ready(()); + } + + match Pin::new(&mut rpc_rx).poll_next(cx) { + Poll::Ready(Some(request)) => { + match request { + sc_rpc::system::Request::Health(sender) => { + let _ = sender.send(sc_rpc::system::Health { + peers: network.peers_debug_info().len(), + is_syncing: network.service().is_major_syncing(), + should_have_peers, + }); + }, + sc_rpc::system::Request::LocalPeerId(sender) => { + let _ = sender.send(network.local_peer_id().to_base58()); + }, + sc_rpc::system::Request::LocalListenAddresses(sender) => { + let peer_id = network.local_peer_id().clone().into(); + let p2p_proto_suffix = sc_network::multiaddr::Protocol::P2p(peer_id); + let addresses = network.listen_addresses() + .map(|addr| addr.clone().with(p2p_proto_suffix.clone()).to_string()) + .collect(); + let _ = sender.send(addresses); + }, + sc_rpc::system::Request::Peers(sender) => { + let _ = sender.send(network.peers_debug_info().into_iter().map(|(peer_id, p)| + sc_rpc::system::PeerInfo { + peer_id: peer_id.to_base58(), + roles: format!("{:?}", p.roles), + protocol_version: p.protocol_version, + best_hash: p.best_hash, + best_number: p.best_number, + } + ).collect()); } - ).collect()); - } - sc_rpc::system::Request::NetworkState(sender) => { - if let Some(network_state) = serde_json::to_value(&network.network_state()).ok() { - let _ = sender.send(network_state); - } - } - sc_rpc::system::Request::NetworkAddReservedPeer(peer_addr, sender) => { - let x = network.add_reserved_peer(peer_addr) - .map_err(sc_rpc::system::error::Error::MalformattedPeerArg); - let _ = sender.send(x); - } - sc_rpc::system::Request::NetworkRemoveReservedPeer(peer_id, sender) => { - let _ = match peer_id.parse::() { - Ok(peer_id) => { - network.remove_reserved_peer(peer_id); - sender.send(Ok(())) + sc_rpc::system::Request::NetworkState(sender) => { + if let Some(network_state) = serde_json::to_value(&network.network_state()).ok() { + let _ = sender.send(network_state); + } + } + sc_rpc::system::Request::NetworkAddReservedPeer(peer_addr, sender) => { + let x = network.add_reserved_peer(peer_addr) + .map_err(sc_rpc::system::error::Error::MalformattedPeerArg); + let _ = sender.send(x); + } + sc_rpc::system::Request::NetworkRemoveReservedPeer(peer_id, sender) => { + let _ = match peer_id.parse::() { + Ok(peer_id) => { + network.remove_reserved_peer(peer_id); + sender.send(Ok(())) + } + Err(e) => sender.send(Err(sc_rpc::system::error::Error::MalformattedPeerArg( + e.to_string(), + ))), + }; + } + sc_rpc::system::Request::NodeRoles(sender) => { + use sc_rpc::system::NodeRole; + + let node_role = match role { + Role::Authority { .. } => NodeRole::Authority, + Role::Light => NodeRole::LightClient, + Role::Full => NodeRole::Full, + Role::Sentry { .. } => NodeRole::Sentry, + }; + + let _ = sender.send(vec![node_role]); } - Err(e) => sender.send(Err(sc_rpc::system::error::Error::MalformattedPeerArg( - e.to_string(), - ))), - }; - } - sc_rpc::system::Request::NodeRoles(sender) => { - use sc_rpc::system::NodeRole; - - let node_role = match role { - Role::Authority { .. } => NodeRole::Authority, - Role::Light => NodeRole::LightClient, - Role::Full => NodeRole::Full, - Role::Sentry { .. } => NodeRole::Sentry, }; - - let _ = sender.send(vec![node_role]); - } - }; + }, + Poll::Ready(None) => return Poll::Ready(()), + Poll::Pending => break, + } } // Interval report for the external API. diff --git a/client/service/src/task_manager.rs b/client/service/src/task_manager.rs index 553ca9c326d8b..450e45e803042 100644 --- a/client/service/src/task_manager.rs +++ b/client/service/src/task_manager.rs @@ -15,12 +15,13 @@ use std::{panic, pin::Pin, result::Result, sync::Arc}; use exit_future::Signal; -use log::debug; +use log::{debug, error}; use futures::{ Future, FutureExt, future::{select, Either, BoxFuture}, compat::*, task::{Spawn, FutureObj, SpawnError}, + sink::SinkExt, }; use prometheus_endpoint::{ exponential_buckets, register, @@ -29,6 +30,7 @@ use prometheus_endpoint::{ }; use sc_client_api::CloneableSpawn; use crate::config::TaskType; +use sp_utils::mpsc::TracingUnboundedSender; mod prometheus_future; @@ -41,6 +43,7 @@ pub struct SpawnTaskHandle { on_exit: exit_future::Exit, executor: ServiceTaskExecutor, metrics: Option, + essential_failed_tx: TracingUnboundedSender<()>, } impl SpawnTaskHandle { @@ -61,6 +64,21 @@ impl SpawnTaskHandle { self.spawn_inner(name, task, TaskType::Blocking) } + /// Spawns a task in the background that runs the future passed as + /// parameter. The given task is considered essential, i.e. if it errors we + /// trigger a service exit. + pub fn spawn_essential(&self, name: &'static str, task: impl Future + Send + 'static) { + let mut essential_failed = self.essential_failed_tx.clone(); + let essential_task = std::panic::AssertUnwindSafe(task) + .catch_unwind() + .map(move |_| { + error!("Essential task `{}` failed. Shutting down service.", name); + let _ = essential_failed.send(()); + }); + + self.spawn(name, essential_task) + } + /// Helper function that implements the spawning logic. See `spawn` and `spawn_blocking`. fn spawn_inner( &self, @@ -156,6 +174,9 @@ pub struct TaskManager { executor: ServiceTaskExecutor, /// Prometheus metric where to report the polling times. metrics: Option, + /// Send a signal when a spawned essential task has concluded. The next time + /// the service future is polled it should complete with an error. + essential_failed_tx: TracingUnboundedSender<()>, } impl TaskManager { @@ -163,7 +184,8 @@ impl TaskManager { /// service tasks. pub(super) fn new( executor: ServiceTaskExecutor, - prometheus_registry: Option<&Registry> + prometheus_registry: Option<&Registry>, + essential_failed_tx: TracingUnboundedSender<()>, ) -> Result { let (signal, on_exit) = exit_future::signal(); @@ -174,13 +196,21 @@ impl TaskManager { signal: Some(signal), executor, metrics, + essential_failed_tx, }) } + /// Spawns a task in the background that runs the future passed as + /// parameter. The given task is considered essential, i.e. if it errors we + /// trigger a service exit. + pub fn spawn_essential(&self, name: &'static str, task: impl Future + Send + 'static) { + self.spawn_handle().spawn_essential(name, task) + } + /// Spawn background/async task, which will be aware on exit signal. /// /// See also the documentation of [`SpawnTaskHandler::spawn`]. - pub(super) fn spawn(&self, name: &'static str, task: impl Future + Send + 'static) { + pub fn spawn(&self, name: &'static str, task: impl Future + Send + 'static) { self.spawn_handle().spawn(name, task) } @@ -189,13 +219,9 @@ impl TaskManager { on_exit: self.on_exit.clone(), executor: self.executor.clone(), metrics: self.metrics.clone(), + essential_failed_tx: self.essential_failed_tx.clone(), } } - - /// Clone on exit signal. - pub(super) fn on_exit(&self) -> exit_future::Exit { - self.on_exit.clone() - } } impl Drop for TaskManager { diff --git a/utils/browser/src/lib.rs b/utils/browser/src/lib.rs index 04af2ceb58136..8b26d7063f1b8 100644 --- a/utils/browser/src/lib.rs +++ b/utils/browser/src/lib.rs @@ -22,7 +22,7 @@ use sc_network::config::TransportConfig; use sc_service::{ AbstractService, RpcSession, Role, Configuration, config::{DatabaseConfig, KeystoreConfig, NetworkConfiguration}, - GenericChainSpec, RuntimeGenesis + GenericChainSpec, RuntimeGenesis, TaskManager, }; use wasm_bindgen::prelude::*; use futures::{prelude::*, channel::{oneshot, mpsc}, future::{poll_fn, ok}, compat::*}; @@ -107,6 +107,7 @@ where #[wasm_bindgen] pub struct Client { rpc_send_tx: mpsc::UnboundedSender, + _task_manager: TaskManager } struct RpcMessage { @@ -116,14 +117,14 @@ struct RpcMessage { } /// Create a Client object that connects to a service. -pub fn start_client(mut service: impl AbstractService) -> Client { +pub fn start_client(mut service: impl AbstractService, task_manager: TaskManager) -> Client { // We dispatch a background task responsible for processing the service. // // The main action performed by the code below consists in polling the service with // `service.poll()`. // The rest consists in handling RPC requests. let (rpc_send_tx, mut rpc_send_rx) = mpsc::unbounded::(); - wasm_bindgen_futures::spawn_local(poll_fn(move |cx| { + wasm_bindgen_futures::spawn_local(poll_fn(move |cx| { loop { match Pin::new(&mut rpc_send_rx).poll_next(cx) { Poll::Ready(Some(message)) => { @@ -144,6 +145,7 @@ pub fn start_client(mut service: impl AbstractService) -> Client { Client { rpc_send_tx, + _task_manager: task_manager } } From 9bb01291cf0e5e2bed65f08ee95332dcf6936968 Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Mon, 15 Jun 2020 11:03:49 +0200 Subject: [PATCH 02/25] Making progress --- Cargo.lock | 1 + bin/node-template/node/Cargo.toml | 1 + bin/node-template/node/src/service.rs | 25 +++-- bin/node/cli/src/browser.rs | 4 +- bin/node/cli/src/service.rs | 42 +++++--- client/cli/src/runner.rs | 69 ++++++------- client/service/src/builder.rs | 131 ++++++++++++------------ client/service/src/lib.rs | 137 ++++++-------------------- client/service/src/task_manager.rs | 33 ++++++- utils/browser/src/lib.rs | 10 +- 10 files changed, 204 insertions(+), 249 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index abf336f68cdb6..9f12ef3686e4b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3545,6 +3545,7 @@ dependencies = [ "sc-finality-grandpa", "sc-network", "sc-service", + "sc-telemetry", "sc-transaction-pool", "sp-consensus", "sp-consensus-aura", diff --git a/bin/node-template/node/Cargo.toml b/bin/node-template/node/Cargo.toml index 88cdc6d608ec1..de218a1fd5cac 100644 --- a/bin/node-template/node/Cargo.toml +++ b/bin/node-template/node/Cargo.toml @@ -38,6 +38,7 @@ sp-finality-grandpa = { version = "2.0.0-rc3", path = "../../../primitives/final sc-client-api = { version = "2.0.0-rc3", path = "../../../client/api" } sp-runtime = { version = "2.0.0-rc3", path = "../../../primitives/runtime" } sc-basic-authorship = { path = "../../../client/basic-authorship", version = "0.8.0-rc3"} +sc-telemetry = { version = "2.0.0-rc3", path = "../../../client/telemetry" } node-template-runtime = { version = "2.0.0-rc3", path = "../runtime" } diff --git a/bin/node-template/node/src/service.rs b/bin/node-template/node/src/service.rs index 072987119a3df..a2f11f68fb77c 100644 --- a/bin/node-template/node/src/service.rs +++ b/bin/node-template/node/src/service.rs @@ -5,7 +5,10 @@ use std::time::Duration; use sc_client_api::ExecutorProvider; use sc_consensus::LongestChain; use node_template_runtime::{self, opaque::Block, RuntimeApi}; -use sc_service::{error::{Error as ServiceError}, AbstractService, Configuration, ServiceBuilder, TaskManager}; +use sc_service::{ + error::{Error as ServiceError}, + AbstractService, Configuration, ServiceBuilder, TaskManager, BasePath, +}; use sp_inherents::InherentDataProviders; use sc_executor::native_executor_instance; pub use sc_executor::NativeExecutor; @@ -93,7 +96,9 @@ macro_rules! new_full_start { } /// Builds a new service for a full client. -pub fn new_full(config: Configuration) -> Result<(impl AbstractService, TaskManager), ServiceError> { +pub fn new_full(config: Configuration) +-> Result<(impl AbstractService, TaskManager, Option, Option>), ServiceError> +{ let role = config.role.clone(); let force_authoring = config.force_authoring; let name = config.network.node_name.clone(); @@ -105,7 +110,7 @@ pub fn new_full(config: Configuration) -> Result<(impl AbstractService, TaskMana import_setup.take() .expect("Link Half and Block Import are present for Full Services or setup failed before. qed"); - let (service, client, transaction_pool, task_manager, keystore) = builder + let (service, client, transaction_pool, task_manager, keystore, network, telemetry, base_path, _) = builder .with_finality_proof_provider(|client, backend| { // GenesisAuthoritySetProvider is implemented for StorageAndProofProvider let provider = client as Arc>; @@ -132,7 +137,7 @@ pub fn new_full(config: Configuration) -> Result<(impl AbstractService, TaskMana select_chain, block_import, proposer, - service.network(), + network.clone(), inherent_data_providers.clone(), force_authoring, keystore.clone(), @@ -173,7 +178,7 @@ pub fn new_full(config: Configuration) -> Result<(impl AbstractService, TaskMana let grandpa_config = sc_finality_grandpa::GrandpaParams { config: grandpa_config, link: grandpa_link, - network: service.network(), + network: network.clone(), inherent_data_providers: inherent_data_providers.clone(), telemetry_on_connect: Some(service.telemetry_on_connect_stream()), voting_rule: sc_finality_grandpa::VotingRulesBuilder::default().build(), @@ -191,15 +196,15 @@ pub fn new_full(config: Configuration) -> Result<(impl AbstractService, TaskMana sc_finality_grandpa::setup_disabled_grandpa( client, &inherent_data_providers, - service.network(), + network.clone(), )?; } - Ok((service, task_manager)) + Ok((service, task_manager, telemetry, base_path)) } /// Builds a new service for a light client. -pub fn new_light(config: Configuration) -> Result<(impl AbstractService, TaskManager), ServiceError> { +pub fn new_light(config: Configuration) -> Result<(impl AbstractService, TaskManager, Option, Option>), ServiceError> { let inherent_data_providers = InherentDataProviders::new(); ServiceBuilder::new_light::(config)? @@ -264,5 +269,7 @@ pub fn new_light(config: Configuration) -> Result<(impl AbstractService, TaskMan Ok(Arc::new(GrandpaFinalityProofProvider::new(backend, provider)) as _) })? .build_light() - .map(|(service, _, _, task_manager, _)| (service, task_manager)) + .map(|(service, _, _, task_manager, _, _, telemetry, base_path, _)| { + (service, task_manager, telemetry, base_path) + }) } diff --git a/bin/node/cli/src/browser.rs b/bin/node/cli/src/browser.rs index b8418026b302f..b33e5dea59382 100644 --- a/bin/node/cli/src/browser.rs +++ b/bin/node/cli/src/browser.rs @@ -52,8 +52,8 @@ async fn start_inner(chain_spec: Option, log_level: String) -> Result>; @@ -209,7 +209,7 @@ macro_rules! new_full { select_chain, env: proposer, block_import, - sync_oracle: service.network(), + sync_oracle: network.clone(), inherent_data_providers: inherent_data_providers.clone(), force_authoring, babe_link, @@ -236,14 +236,13 @@ macro_rules! new_full { _ => unreachable!("Due to outer matches! constraint; qed.") }; - let network = service.network(); let dht_event_stream = network.event_stream("authority-discovery").filter_map(|e| async move { match e { Event::Dht(e) => Some(e), _ => None, }}).boxed(); let authority_discovery = sc_authority_discovery::AuthorityDiscovery::new( client.clone(), - network, + network.clone(), sentries, dht_event_stream, authority_discovery_role, @@ -282,7 +281,7 @@ macro_rules! new_full { let grandpa_config = grandpa::GrandpaParams { config, link: grandpa_link, - network: service.network(), + network: network.clone(), inherent_data_providers: inherent_data_providers.clone(), telemetry_on_connect: Some(service.telemetry_on_connect_stream()), voting_rule: grandpa::VotingRulesBuilder::default().build(), @@ -300,11 +299,11 @@ macro_rules! new_full { grandpa::setup_disabled_grandpa( client, &inherent_data_providers, - service.network(), + network.clone(), )?; } - Ok((service, inherent_data_providers, task_manager)) + Ok((service, inherent_data_providers, task_manager, telemetry, base_path, rpc_handlers)) }}; ($config:expr) => {{ new_full!($config, |_, _| {}) @@ -313,17 +312,18 @@ macro_rules! new_full { /// Builds a new service for a full client. pub fn new_full(config: Configuration) --> Result<(impl AbstractService, TaskManager), ServiceError> +-> Result<(impl AbstractService, TaskManager, Option, Option>), ServiceError> { - new_full!(config).map(|(service, _, task_manager)| (service, task_manager)) + new_full!(config).map(|(service, _, task_manager, telemetry, base_path, _)| { + (service, task_manager, telemetry, base_path) + }) } -/// Builds a new service for a light client. -pub fn new_light(config: Configuration) --> Result<(impl AbstractService, TaskManager), ServiceError> { +fn new_light_base(config: Configuration) +-> Result<(impl AbstractService, TaskManager, Option, Option>, RpcHandlers), ServiceError> { let inherent_data_providers = InherentDataProviders::new(); - let (service, _, _, task_manager, _) = ServiceBuilder::new_light::(config)? + let (service, _, _, task_manager, _, _, telemetry, base_path, rpc_handlers) = ServiceBuilder::new_light::(config)? .with_select_chain(|_config, backend| { Ok(LongestChain::new(backend.clone())) })? @@ -407,7 +407,19 @@ pub fn new_light(config: Configuration) })? .build_light()?; - Ok((service, task_manager)) + Ok((service, task_manager, telemetry, base_path, rpc_handlers)) +} + +/// Builds a new service for a light client. +pub fn new_light(config: Configuration) +-> Result<(impl AbstractService, TaskManager, Option, Option>), ServiceError> { + new_light_base(config).map(|(service, task_manager, telemetry, base_path, _)| (service, task_manager, telemetry, base_path)) +} + +/// Builds a new service for a browser light client. +pub fn new_light_browser(config: Configuration) +-> Result<(impl AbstractService, TaskManager, Option, Option>, RpcHandlers), ServiceError> { + new_light_base(config) } #[cfg(test)] diff --git a/client/cli/src/runner.rs b/client/cli/src/runner.rs index 32d3dd3ff8145..e31ed1ea5809c 100644 --- a/client/cli/src/runner.rs +++ b/client/cli/src/runner.rs @@ -25,12 +25,17 @@ use futures::pin_mut; use futures::select; use futures::{future, future::FutureExt, Future}; use log::info; -use sc_service::{AbstractService, Configuration, Role, ServiceBuilderCommand, TaskType, TaskManager}; +use sc_service::{ + AbstractService, Configuration, Role, ServiceBuilderCommand, TaskType, TaskManager, + BasePath, +}; use sp_runtime::traits::{Block as BlockT, Header as HeaderT}; use sp_utils::metrics::{TOKIO_THREADS_ALIVE, TOKIO_THREADS_TOTAL}; use sp_version::RuntimeVersion; use std::{fmt::Debug, marker::PhantomData, str::FromStr, sync::Arc}; +type Initialised = (T, TaskManager, Option, Option>); + #[cfg(target_family = "unix")] async fn main(func: F) -> std::result::Result<(), Box> where @@ -177,16 +182,12 @@ impl Runner { /// A helper function that runs an `AbstractService` with tokio and stops if the process /// receives the signal `SIGTERM` or `SIGINT`. It can run a full or a light node depending on /// the node's configuration. - pub fn run_node( + pub fn run_node( self, - new_light: impl FnOnce(Configuration) -> sc_service::error::Result<(SL, TaskManager)>, - new_full: impl FnOnce(Configuration) -> sc_service::error::Result<(SF, TaskManager)>, + new_light: impl FnOnce(Configuration) -> sc_service::error::Result>, + new_full: impl FnOnce(Configuration) -> sc_service::error::Result>, runtime_version: RuntimeVersion, - ) -> Result<()> - where - SL: AbstractService + Unpin, - SF: AbstractService + Unpin, - { + ) -> Result<()> { match self.config.role { Role::Light => self.run_light_node(new_light, runtime_version), _ => self.run_full_node(new_full, runtime_version), @@ -196,14 +197,11 @@ impl Runner { /// A helper function that runs an `AbstractService` with tokio and stops if the process /// receives the signal `SIGTERM` or `SIGINT`. It can only run a "full" node and will fail if /// the node's configuration uses a "light" role. - pub fn run_full_node( + pub fn run_full_node( self, - new_full: impl FnOnce(Configuration) -> sc_service::error::Result<(S, TaskManager)>, + new_full: impl FnOnce(Configuration) -> sc_service::error::Result>, runtime_version: RuntimeVersion, - ) -> Result<()> - where - S: AbstractService + Unpin, - { + ) -> Result<()> { if matches!(self.config.role, Role::Light) { return Err("Light node has been requested but this is not implemented".into()); } @@ -215,14 +213,11 @@ impl Runner { /// A helper function that runs an `AbstractService` with tokio and stops if the process /// receives the signal `SIGTERM` or `SIGINT`. It can only run a "light" node and will fail if /// the node's configuration uses a "full" role. - pub fn run_light_node( + pub fn run_light_node( self, - new_light: impl FnOnce(Configuration) -> sc_service::error::Result<(S, TaskManager)>, + new_light: impl FnOnce(Configuration) -> sc_service::error::Result>, runtime_version: RuntimeVersion, - ) -> Result<()> - where - S: AbstractService + Unpin, - { + ) -> Result<()> { if !matches!(self.config.role, Role::Light) { return Err("Full node has been requested but this is not implemented".into()); } @@ -259,24 +254,24 @@ impl Runner { } } - fn run_service_until_exit(mut self, service_builder: F) -> Result<()> - where - F: FnOnce(Configuration) -> std::result::Result<(T, TaskManager), sc_service::error::Error>, - T: AbstractService + Unpin, - { - let (service, _task_manager) = service_builder(self.config)?; - - // we eagerly drop the service so that the internal exit future is fired, - // but we need to keep holding a reference to the global telemetry guard - // and drop the runtime first. - let _telemetry = service.telemetry(); - - // we hold a reference to the base path so if the base path is a temporary directory it will - // not be deleted before the tokio runtime finish to clean up - let _base_path = service.base_path(); + fn run_service_until_exit( + mut self, + initialise: impl FnOnce(Configuration) -> sc_service::error::Result>, + ) -> Result<()> { + let ( + _service, + task_manager, + // we eagerly drop the service so that the internal exit future is fired, + // but we need to keep holding a reference to the global telemetry guard + // and drop the runtime first. + _telemetry, + // we hold a reference to the base path so if the base path is a temporary directory it will + // not be deleted before the tokio runtime finish to clean up + _base_path, + ) = initialise(self.config)?; { - let f = service.fuse(); + let f = task_manager.fuse(); self.tokio_runtime .block_on(main(f)) .map_err(|e| e.to_string())?; diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index 3c1b0eed4b728..7d6d8f8677273 100644 --- a/client/service/src/builder.rs +++ b/client/service/src/builder.rs @@ -21,7 +21,7 @@ use crate::{ start_rpc_servers, build_network_future, TransactionPoolAdapter, TaskManager, SpawnTaskHandle, status_sinks, metrics::MetricsService, client::{light, Client, ClientConfig}, - config::{Configuration, KeystoreConfig, PrometheusConfig, OffchainWorkerConfig}, + config::{Configuration, KeystoreConfig, PrometheusConfig, OffchainWorkerConfig, BasePath}, }; use sc_client_api::{ self, BlockchainEvents, light::RemoteBlockchain, execution_extensions::ExtensionsFactory, @@ -103,7 +103,6 @@ pub struct ServiceBuilder, block_announce_validator_builder: Option) -> Box + Send> + Send>>, informant_prefix: String, - essential_failed_rx: TracingUnboundedReceiver<()>, } /// A utility trait for building an RPC extension given a `DenyUnsafe` instance. @@ -206,7 +205,6 @@ type TFullParts = ( Arc>, Arc>, TaskManager, - TracingUnboundedReceiver<()>, ); /// Creates a new full client for the given config. @@ -233,12 +231,9 @@ fn new_full_parts( KeystoreConfig::InMemory => Keystore::new_in_memory(), }; - // A side-channel for essential tasks to communicate shutdown. - let (essential_failed_tx, essential_failed_rx) = tracing_unbounded("mpsc_essential_tasks"); - let task_manager = { let registry = config.prometheus_config.as_ref().map(|cfg| &cfg.registry); - TaskManager::new(config.task_executor.clone(), registry, essential_failed_tx)? + TaskManager::new(config.task_executor.clone(), registry)? }; let executor = NativeExecutor::::new( @@ -286,7 +281,7 @@ fn new_full_parts( )? }; - Ok((client, backend, keystore, task_manager, essential_failed_rx)) + Ok((client, backend, keystore, task_manager)) } @@ -352,7 +347,7 @@ impl ServiceBuilder<(), (), (), (), (), (), (), (), (), (), ()> { (), TFullBackend, >, Error> { - let (client, backend, keystore, task_manager, essential_failed_rx) = new_full_parts(&config)?; + let (client, backend, keystore, task_manager) = new_full_parts(&config)?; let client = Arc::new(client); @@ -362,7 +357,6 @@ impl ServiceBuilder<(), (), (), (), (), (), (), (), (), (), ()> { backend, keystore, task_manager, - essential_failed_rx, fetcher: None, select_chain: None, import_queue: (), @@ -393,12 +387,9 @@ impl ServiceBuilder<(), (), (), (), (), (), (), (), (), (), ()> { (), TLightBackend, >, Error> { - // A side-channel for essential tasks to communicate shutdown. - let (essential_failed_tx, essential_failed_rx) = tracing_unbounded("mpsc_essential_tasks"); - let task_manager = { let registry = config.prometheus_config.as_ref().map(|cfg| &cfg.registry); - TaskManager::new(config.task_executor.clone(), registry, essential_failed_tx)? + TaskManager::new(config.task_executor.clone(), registry)? }; let keystore = match &config.keystore { @@ -450,7 +441,6 @@ impl ServiceBuilder<(), (), (), (), (), (), (), (), (), (), ()> { backend, task_manager, keystore, - essential_failed_rx, fetcher: Some(fetcher.clone()), select_chain: None, import_queue: (), @@ -557,7 +547,6 @@ impl block_announce_validator_builder: self.block_announce_validator_builder, informant_prefix: self.informant_prefix, marker: self.marker, - essential_failed_rx: self.essential_failed_rx, }) } @@ -604,7 +593,6 @@ impl block_announce_validator_builder: self.block_announce_validator_builder, informant_prefix: self.informant_prefix, marker: self.marker, - essential_failed_rx: self.essential_failed_rx, }) } @@ -644,7 +632,6 @@ impl block_announce_validator_builder: self.block_announce_validator_builder, informant_prefix: self.informant_prefix, marker: self.marker, - essential_failed_rx: self.essential_failed_rx, }) } @@ -712,7 +699,6 @@ impl block_announce_validator_builder: self.block_announce_validator_builder, informant_prefix: self.informant_prefix, marker: self.marker, - essential_failed_rx: self.essential_failed_rx, }) } @@ -770,7 +756,6 @@ impl block_announce_validator_builder: self.block_announce_validator_builder, informant_prefix: self.informant_prefix, marker: self.marker, - essential_failed_rx: self.essential_failed_rx, }) } @@ -809,7 +794,6 @@ impl block_announce_validator_builder: self.block_announce_validator_builder, informant_prefix: self.informant_prefix, marker: self.marker, - essential_failed_rx: self.essential_failed_rx, }) } @@ -856,7 +840,6 @@ impl block_announce_validator_builder: Some(Box::new(block_announce_validator_builder)), informant_prefix: self.informant_prefix, marker: self.marker, - essential_failed_rx: self.essential_failed_rx, }) } @@ -976,19 +959,26 @@ ServiceBuilder< Ok(self) } - fn build_common(self) -> Result<(Service< - TBl, - Client, - TSc, - NetworkStatus, - NetworkService::Hash>, - TExPool, - sc_offchain::OffchainWorkers< - Client, - TBackend::OffchainStorage, - TBl + fn build_common(self) -> Result<( + Service< + TBl, + TSc, + NetworkStatus, + sc_offchain::OffchainWorkers< + Client, + TBackend::OffchainStorage, + TBl + >, >, - >, Arc>, Arc, TaskManager, Arc>), Error> + Arc>, + Arc, + TaskManager, + Arc>, + Arc::Hash>>, + Option, + Option>, + crate::RpcHandlers, + ), Error> where TExec: CallExecutor, { let ServiceBuilder { @@ -1008,7 +998,6 @@ ServiceBuilder< remote_backend, block_announce_validator_builder, informant_prefix, - essential_failed_rx, } = self; sp_session::generate_initial_session_keys( @@ -1316,7 +1305,7 @@ ServiceBuilder< }; let rpc = start_rpc_servers(&config, gen_handler)?; // This is used internally, so don't restrict access to unsafe RPC - let rpc_handlers = gen_handler(sc_rpc::DenyUnsafe::No); + let rpc_handlers = crate::RpcHandlers(gen_handler(sc_rpc::DenyUnsafe::No)); // The network worker is responsible for gathering all network messages and processing // them. This is quite a heavy task, and at the time of the writing of this comment it @@ -1412,36 +1401,37 @@ ServiceBuilder< spawn_handle.spawn("informant", informant_future); Ok((Service { - network, network_status_sinks, select_chain, - essential_failed_rx, - rpc_handlers, _rpc: rpc, - _telemetry: telemetry, _offchain_workers: offchain_workers, _telemetry_on_connect_sinks: telemetry_connection_sinks.clone(), marker: PhantomData::, prometheus_registry: config.prometheus_config.map(|config| config.registry), - _base_path: config.base_path.map(Arc::new), - _phantom: PhantomData - }, client, transaction_pool, task_manager, keystore)) + }, client, transaction_pool, task_manager, keystore, network, telemetry, config.base_path.map(Arc::new), rpc_handlers)) } /// Builds the light service. - pub fn build_light(self) -> Result<(Service< - TBl, - Client, - TSc, - NetworkStatus, - NetworkService::Hash>, - TExPool, - sc_offchain::OffchainWorkers< - Client, - TBackend::OffchainStorage, - TBl + pub fn build_light(self) -> Result<( + Service< + TBl, + TSc, + NetworkStatus, + sc_offchain::OffchainWorkers< + Client, + TBackend::OffchainStorage, + TBl + >, >, - >, Arc>, Arc, TaskManager, Arc>), Error> + Arc>, + Arc, + TaskManager, + Arc>, + Arc::Hash>>, + Option, + Option>, + crate::RpcHandlers, + ), Error> where TExec: CallExecutor, { self.build_common() @@ -1484,19 +1474,26 @@ ServiceBuilder< { /// Builds the full service. - pub fn build_full(self) -> Result<(Service< - TBl, - Client, - TSc, - NetworkStatus, - NetworkService::Hash>, - TExPool, - sc_offchain::OffchainWorkers< - Client, - TBackend::OffchainStorage, - TBl + pub fn build_full(self) -> Result<( + Service< + TBl, + TSc, + NetworkStatus, + sc_offchain::OffchainWorkers< + Client, + TBackend::OffchainStorage, + TBl + >, >, - >, Arc>, Arc, TaskManager, Arc>), Error> + Arc>, + Arc, + TaskManager, + Arc>, + Arc::Hash>>, + Option, + Option>, + crate::RpcHandlers, + ), Error> where TExec: CallExecutor, { // make transaction pool available for off-chain runtime calls. diff --git a/client/service/src/lib.rs b/client/service/src/lib.rs index 5a4841d23e3f8..c63fcde5e65a2 100644 --- a/client/service/src/lib.rs +++ b/client/service/src/lib.rs @@ -47,7 +47,7 @@ use parking_lot::Mutex; use client::Client; use futures::{Future, FutureExt, Stream, StreamExt, stream::FusedStream, compat::*}; -use sc_network::{NetworkService, NetworkStatus, network_state::NetworkState, PeerId}; +use sc_network::{NetworkStatus, network_state::NetworkState, PeerId}; use log::{log, warn, debug, error, Level}; use codec::{Encode, Decode}; use sp_runtime::generic::BlockId; @@ -99,108 +99,71 @@ impl MallocSizeOfWasm for T {} #[cfg(target_os = "unknown")] impl MallocSizeOfWasm for T {} +pub struct RpcHandlers(sc_rpc_server::RpcHandler); + +impl RpcHandlers { + /// Starts an RPC query. + /// + /// The query is passed as a string and must be a JSON text similar to what an HTTP client + /// would for example send. + /// + /// Returns a `Future` that contains the optional response. + /// + /// If the request subscribes you to events, the `Sender` in the `RpcSession` object is used to + /// send back spontaneous events. + pub fn rpc_query(&self, mem: &RpcSession, request: &str) -> Pin> + Send>> { + self.0.handle_request(request, mem.metadata.clone()) + .compat() + .map(|res| res.expect("this should never fail")) + .boxed() + } +} + /// Substrate service. -pub struct Service { +pub struct Service { select_chain: Option, - network: Arc, // Sinks to propagate network status updates. // For each element, every time the `Interval` fires we push an element on the sender. network_status_sinks: Arc>>, - // A receiver for spawned essential-tasks concluding. - essential_failed_rx: TracingUnboundedReceiver<()>, - rpc_handlers: sc_rpc_server::RpcHandler, _rpc: Box, - _telemetry: Option, _telemetry_on_connect_sinks: Arc>>>, _offchain_workers: Option>, marker: PhantomData, prometheus_registry: Option, - // The base path is kept here because it can be a temporary directory which will be deleted - // when dropped - _base_path: Option>, - _phantom: PhantomData<(TCl, TTxPool)> } -impl Unpin for Service {} +impl Unpin for Service {} /// Abstraction over a Substrate service. -pub trait AbstractService: Future> + Send + Unpin + 'static { +pub trait AbstractService: Send + Unpin + 'static { /// Type of block of this chain. type Block: BlockT; - /// Backend storage for the client. - type Backend: 'static + BackendT; - /// How to execute calls towards the runtime. - type CallExecutor: 'static + CallExecutor + Send + Sync + Clone; - /// API that the runtime provides. - type RuntimeApi: Send + Sync; /// Chain selection algorithm. - type SelectChain: sp_consensus::SelectChain; - /// Transaction pool. - type TransactionPool: TransactionPool + MallocSizeOfWasm; - /// The generic Client type, the bounds here are the ones specifically required by - /// internal crates like sc_informant. - type Client: - HeaderMetadata + UsageProvider - + BlockchainEvents + HeaderBackend + Send + Sync; - + type SelectChain: sp_consensus::SelectChain + Sized; + /// Get event stream for telemetry connection established events. fn telemetry_on_connect_stream(&self) -> TracingUnboundedReceiver<()>; - /// return a shared instance of Telemetry (if enabled) - fn telemetry(&self) -> Option; - - /// Starts an RPC query. - /// - /// The query is passed as a string and must be a JSON text similar to what an HTTP client - /// would for example send. - /// - /// Returns a `Future` that contains the optional response. - /// - /// If the request subscribes you to events, the `Sender` in the `RpcSession` object is used to - /// send back spontaneous events. - fn rpc_query(&self, mem: &RpcSession, request: &str) -> Pin> + Send>>; /// Get clone of select chain. fn select_chain(&self) -> Option; - /// Get shared network instance. - fn network(&self) - -> Arc::Hash>>; - /// Returns a receiver that periodically receives a status of the network. fn network_status(&self, interval: Duration) -> TracingUnboundedReceiver<(NetworkStatus, NetworkState)>; /// Get the prometheus metrics registry, if available. fn prometheus_registry(&self) -> Option; - - /// Get a clone of the base_path - fn base_path(&self) -> Option>; } -impl AbstractService for - Service, TSc, NetworkStatus, - NetworkService, TExPool, TOc> +impl AbstractService for + Service, TOc> where TBl: BlockT, - TBackend: 'static + BackendT, - TExec: 'static + CallExecutor + Send + Sync + Clone, - TRtApi: 'static + Send + Sync + ConstructRuntimeApi>, - >>::RuntimeApi: - sp_api::Core - + ApiExt - + ApiErrorExt - + BlockBuilder, TSc: sp_consensus::SelectChain + 'static + Clone + Send + Unpin, - TExPool: 'static + TransactionPool + MallocSizeOfWasm, TOc: 'static + Send + Sync, { type Block = TBl; - type Backend = TBackend; - type CallExecutor = TExec; - type RuntimeApi = TRtApi; type SelectChain = TSc; - type TransactionPool = TExPool; - type Client = Client; fn telemetry_on_connect_stream(&self) -> TracingUnboundedReceiver<()> { let (sink, stream) = tracing_unbounded("mpsc_telemetry_on_connect"); @@ -208,28 +171,10 @@ where stream } - fn telemetry(&self) -> Option { - self._telemetry.clone() - } - - fn rpc_query(&self, mem: &RpcSession, request: &str) -> Pin> + Send>> { - Box::pin( - self.rpc_handlers.handle_request(request, mem.metadata.clone()) - .compat() - .map(|res| res.expect("this should never fail")) - ) - } - fn select_chain(&self) -> Option { self.select_chain.clone() } - fn network(&self) - -> Arc::Hash>> - { - self.network.clone() - } - fn network_status(&self, interval: Duration) -> TracingUnboundedReceiver<(NetworkStatus, NetworkState)> { let (sink, stream) = tracing_unbounded("mpsc_network_status"); self.network_status_sinks.lock().push(interval, sink); @@ -239,32 +184,6 @@ where fn prometheus_registry(&self) -> Option { self.prometheus_registry.clone() } - - fn base_path(&self) -> Option> { - self._base_path.clone() - } -} - -impl Future for - Service -{ - type Output = Result<(), Error>; - - fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { - let this = Pin::into_inner(self); - - match Pin::new(&mut this.essential_failed_rx).poll_next(cx) { - Poll::Pending => {}, - Poll::Ready(_) => { - // Ready(None) should not be possible since we hold a live - // sender. - return Poll::Ready(Err(Error::Other("Essential task failed.".into()))); - } - } - - // The service future never ends. - Poll::Pending - } } /// Builds a never-ending future that continuously polls the network. diff --git a/client/service/src/task_manager.rs b/client/service/src/task_manager.rs index 450e45e803042..03f9719956f09 100644 --- a/client/service/src/task_manager.rs +++ b/client/service/src/task_manager.rs @@ -13,11 +13,11 @@ //! Substrate service tasks management module. -use std::{panic, pin::Pin, result::Result, sync::Arc}; +use std::{panic, pin::Pin, result::Result, sync::Arc, task::{Poll, Context}}; use exit_future::Signal; use log::{debug, error}; use futures::{ - Future, FutureExt, + Future, FutureExt, Stream, future::{select, Either, BoxFuture}, compat::*, task::{Spawn, FutureObj, SpawnError}, @@ -30,7 +30,7 @@ use prometheus_endpoint::{ }; use sc_client_api::CloneableSpawn; use crate::config::TaskType; -use sp_utils::mpsc::TracingUnboundedSender; +use sp_utils::mpsc::{TracingUnboundedSender, TracingUnboundedReceiver, tracing_unbounded}; mod prometheus_future; @@ -177,6 +177,8 @@ pub struct TaskManager { /// Send a signal when a spawned essential task has concluded. The next time /// the service future is polled it should complete with an error. essential_failed_tx: TracingUnboundedSender<()>, + /// A receiver for spawned essential-tasks concluding. + essential_failed_rx: TracingUnboundedReceiver<()>, } impl TaskManager { @@ -185,9 +187,10 @@ impl TaskManager { pub(super) fn new( executor: ServiceTaskExecutor, prometheus_registry: Option<&Registry>, - essential_failed_tx: TracingUnboundedSender<()>, ) -> Result { let (signal, on_exit) = exit_future::signal(); + // A side-channel for essential tasks to communicate shutdown. + let (essential_failed_tx, essential_failed_rx) = tracing_unbounded("mpsc_essential_tasks"); let metrics = prometheus_registry.map(Metrics::register).transpose()?; @@ -197,6 +200,7 @@ impl TaskManager { executor, metrics, essential_failed_tx, + essential_failed_rx, }) } @@ -224,6 +228,27 @@ impl TaskManager { } } +impl Future for TaskManager { + type Output = Result<(), crate::Error>; + + fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { + let this = Pin::into_inner(self); + + match Pin::new(&mut this.essential_failed_rx).poll_next(cx) { + Poll::Pending => {}, + Poll::Ready(_) => { + // Ready(None) should not be possible since we hold a live + // sender. + return Poll::Ready(Err(crate::Error::Other("Essential task failed.".into()))); + } + } + + // The task manager future never ends. + Poll::Pending + } +} + + impl Drop for TaskManager { fn drop(&mut self) { debug!(target: "service", "Tasks manager shutdown"); diff --git a/utils/browser/src/lib.rs b/utils/browser/src/lib.rs index 8b26d7063f1b8..131be258ec5c2 100644 --- a/utils/browser/src/lib.rs +++ b/utils/browser/src/lib.rs @@ -22,7 +22,7 @@ use sc_network::config::TransportConfig; use sc_service::{ AbstractService, RpcSession, Role, Configuration, config::{DatabaseConfig, KeystoreConfig, NetworkConfiguration}, - GenericChainSpec, RuntimeGenesis, TaskManager, + GenericChainSpec, RuntimeGenesis, TaskManager, RpcHandlers, }; use wasm_bindgen::prelude::*; use futures::{prelude::*, channel::{oneshot, mpsc}, future::{poll_fn, ok}, compat::*}; @@ -107,7 +107,6 @@ where #[wasm_bindgen] pub struct Client { rpc_send_tx: mpsc::UnboundedSender, - _task_manager: TaskManager } struct RpcMessage { @@ -117,7 +116,7 @@ struct RpcMessage { } /// Create a Client object that connects to a service. -pub fn start_client(mut service: impl AbstractService, task_manager: TaskManager) -> Client { +pub fn start_client(mut task_manager: TaskManager, rpc_handlers: RpcHandlers) -> Client { // We dispatch a background task responsible for processing the service. // // The main action performed by the code below consists in polling the service with @@ -128,7 +127,7 @@ pub fn start_client(mut service: impl AbstractService, task_manager: TaskManager loop { match Pin::new(&mut rpc_send_rx).poll_next(cx) { Poll::Ready(Some(message)) => { - let fut = service + let fut = rpc_handlers .rpc_query(&message.session, &message.rpc_json) .boxed(); let _ = message.send_back.send(fut); @@ -138,14 +137,13 @@ pub fn start_client(mut service: impl AbstractService, task_manager: TaskManager } } - Pin::new(&mut service) + Pin::new(&mut task_manager) .poll(cx) .map(drop) })); Client { rpc_send_tx, - _task_manager: task_manager } } From 91767ac310a67b3605b77a01392b2a201d9da245 Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Mon, 15 Jun 2020 13:17:35 +0200 Subject: [PATCH 03/25] Almost ready --- Cargo.lock | 1 - bin/node-template/node/Cargo.toml | 1 - bin/node-template/node/src/service.rs | 30 ++++--- bin/node/cli/src/browser.rs | 3 +- bin/node/cli/src/chain_spec.rs | 7 +- bin/node/cli/src/service.rs | 59 ++++++------ client/cli/src/runner.rs | 43 +++++---- client/service/src/builder.rs | 95 +++++--------------- client/service/src/lib.rs | 124 +++++++++++++------------- client/service/src/task_manager.rs | 1 - utils/browser/src/lib.rs | 2 +- 11 files changed, 172 insertions(+), 194 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 9f12ef3686e4b..abf336f68cdb6 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3545,7 +3545,6 @@ dependencies = [ "sc-finality-grandpa", "sc-network", "sc-service", - "sc-telemetry", "sc-transaction-pool", "sp-consensus", "sp-consensus-aura", diff --git a/bin/node-template/node/Cargo.toml b/bin/node-template/node/Cargo.toml index de218a1fd5cac..88cdc6d608ec1 100644 --- a/bin/node-template/node/Cargo.toml +++ b/bin/node-template/node/Cargo.toml @@ -38,7 +38,6 @@ sp-finality-grandpa = { version = "2.0.0-rc3", path = "../../../primitives/final sc-client-api = { version = "2.0.0-rc3", path = "../../../client/api" } sp-runtime = { version = "2.0.0-rc3", path = "../../../primitives/runtime" } sc-basic-authorship = { path = "../../../client/basic-authorship", version = "0.8.0-rc3"} -sc-telemetry = { version = "2.0.0-rc3", path = "../../../client/telemetry" } node-template-runtime = { version = "2.0.0-rc3", path = "../runtime" } diff --git a/bin/node-template/node/src/service.rs b/bin/node-template/node/src/service.rs index a2f11f68fb77c..ef5766dc35bdd 100644 --- a/bin/node-template/node/src/service.rs +++ b/bin/node-template/node/src/service.rs @@ -7,7 +7,7 @@ use sc_consensus::LongestChain; use node_template_runtime::{self, opaque::Block, RuntimeApi}; use sc_service::{ error::{Error as ServiceError}, - AbstractService, Configuration, ServiceBuilder, TaskManager, BasePath, + Configuration, ServiceBuilder, ChainComponents, }; use sp_inherents::InherentDataProviders; use sc_executor::native_executor_instance; @@ -16,6 +16,7 @@ use sp_consensus_aura::sr25519::{AuthorityPair as AuraPair}; use sc_finality_grandpa::{ FinalityProofProvider as GrandpaFinalityProofProvider, StorageAndProofProvider, SharedVoterState, }; +use sc_cli::KeepAliveChainComponents; // Our native executor instance. native_executor_instance!( @@ -97,7 +98,7 @@ macro_rules! new_full_start { /// Builds a new service for a full client. pub fn new_full(config: Configuration) --> Result<(impl AbstractService, TaskManager, Option, Option>), ServiceError> +-> Result { let role = config.role.clone(); let force_authoring = config.force_authoring; @@ -110,7 +111,10 @@ pub fn new_full(config: Configuration) import_setup.take() .expect("Link Half and Block Import are present for Full Services or setup failed before. qed"); - let (service, client, transaction_pool, task_manager, keystore, network, telemetry, base_path, _) = builder + let ChainComponents { + client, transaction_pool, task_manager, keystore, network, telemetry, base_path, + select_chain, prometheus_registry, telemetry_on_connect_sinks, rpc, .. + } = builder .with_finality_proof_provider(|client, backend| { // GenesisAuthoritySetProvider is implemented for StorageAndProofProvider let provider = client as Arc>; @@ -122,10 +126,10 @@ pub fn new_full(config: Configuration) let proposer = sc_basic_authorship::ProposerFactory::new( client.clone(), transaction_pool, - service.prometheus_registry().as_ref(), + prometheus_registry.as_ref(), ); - let select_chain = service.select_chain() + let select_chain = select_chain .ok_or(ServiceError::SelectChainRequired)?; let can_author_with = @@ -180,9 +184,9 @@ pub fn new_full(config: Configuration) link: grandpa_link, network: network.clone(), inherent_data_providers: inherent_data_providers.clone(), - telemetry_on_connect: Some(service.telemetry_on_connect_stream()), + telemetry_on_connect: Some(telemetry_on_connect_sinks.on_connect_stream()), voting_rule: sc_finality_grandpa::VotingRulesBuilder::default().build(), - prometheus_registry: service.prometheus_registry(), + prometheus_registry: prometheus_registry.clone(), shared_voter_state: SharedVoterState::empty(), }; @@ -200,11 +204,13 @@ pub fn new_full(config: Configuration) )?; } - Ok((service, task_manager, telemetry, base_path)) + Ok(KeepAliveChainComponents { + task_manager, telemetry, base_path, rpc, + }) } /// Builds a new service for a light client. -pub fn new_light(config: Configuration) -> Result<(impl AbstractService, TaskManager, Option, Option>), ServiceError> { +pub fn new_light(config: Configuration) -> Result { let inherent_data_providers = InherentDataProviders::new(); ServiceBuilder::new_light::(config)? @@ -269,7 +275,9 @@ pub fn new_light(config: Configuration) -> Result<(impl AbstractService, TaskMan Ok(Arc::new(GrandpaFinalityProofProvider::new(backend, provider)) as _) })? .build_light() - .map(|(service, _, _, task_manager, _, _, telemetry, base_path, _)| { - (service, task_manager, telemetry, base_path) + .map(|ChainComponents { task_manager, telemetry, base_path, rpc, .. }| { + KeepAliveChainComponents { + task_manager, telemetry, base_path, rpc, + } }) } diff --git a/bin/node/cli/src/browser.rs b/bin/node/cli/src/browser.rs index b33e5dea59382..554d7abc16441 100644 --- a/bin/node/cli/src/browser.rs +++ b/bin/node/cli/src/browser.rs @@ -23,6 +23,7 @@ use browser_utils::{ Client, browser_configuration, set_console_error_panic_hook, init_console_log, }; +use sc_cli::KeepAliveChainComponents; use std::str::FromStr; /// Starts the client. @@ -52,7 +53,7 @@ async fn start_inner(chain_spec: Option, log_level: String) -> Result ChainSpec { pub(crate) mod tests { use super::*; use crate::service::{new_full, new_light}; - //use sc_service_test; use sp_runtime::BuildStorage; fn local_testnet_genesis_instant_single() -> GenesisConfig { @@ -425,15 +424,17 @@ pub(crate) mod tests { ) } + /* #[test] #[ignore] fn test_connectivity() { - /*sc_service_test::connectivity( + sc_service_test::connectivity( integration_test_config_with_two_authorities(), |config| new_full(config), |config| new_light(config), - );*/ + ); } + */ #[test] fn test_create_development_chain_spec() { diff --git a/bin/node/cli/src/service.rs b/bin/node/cli/src/service.rs index 83e465a9b8fb5..d6d6b630c803a 100644 --- a/bin/node/cli/src/service.rs +++ b/bin/node/cli/src/service.rs @@ -29,11 +29,12 @@ use node_executor; use node_primitives::Block; use node_runtime::RuntimeApi; use sc_service::{ - AbstractService, ServiceBuilder, config::Configuration, error::{Error as ServiceError}, - TaskManager, BasePath, RpcHandlers, + ServiceBuilder, config::Configuration, error::{Error as ServiceError}, + RpcHandlers, ChainComponents, }; use sp_inherents::InherentDataProviders; use sc_consensus::LongestChain; +use sc_cli::KeepAliveChainComponents; /// Starts a `ServiceBuilder` for a full service. /// @@ -174,7 +175,10 @@ macro_rules! new_full { let (builder, mut import_setup, inherent_data_providers, mut rpc_setup) = new_full_start!($config); - let (service, client, transaction_pool, task_manager, keystore, network, telemetry, base_path, rpc_handlers) = builder + let ChainComponents { + client, transaction_pool, task_manager, keystore, network, telemetry, base_path, + select_chain, prometheus_registry, rpc, telemetry_on_connect_sinks, .. + } = builder .with_finality_proof_provider(|client, backend| { // GenesisAuthoritySetProvider is implemented for StorageAndProofProvider let provider = client as Arc>; @@ -194,10 +198,10 @@ macro_rules! new_full { let proposer = sc_basic_authorship::ProposerFactory::new( client.clone(), transaction_pool, - service.prometheus_registry().as_ref(), + prometheus_registry.as_ref(), ); - let select_chain = service.select_chain() + let select_chain = select_chain .ok_or(sc_service::Error::SelectChainRequired)?; let can_author_with = @@ -246,7 +250,7 @@ macro_rules! new_full { sentries, dht_event_stream, authority_discovery_role, - service.prometheus_registry(), + prometheus_registry.clone(), ); task_manager.spawn("authority-discovery", authority_discovery); @@ -283,9 +287,9 @@ macro_rules! new_full { link: grandpa_link, network: network.clone(), inherent_data_providers: inherent_data_providers.clone(), - telemetry_on_connect: Some(service.telemetry_on_connect_stream()), + telemetry_on_connect: Some(telemetry_on_connect_sinks.on_connect_stream()), voting_rule: grandpa::VotingRulesBuilder::default().build(), - prometheus_registry: service.prometheus_registry(), + prometheus_registry: prometheus_registry.clone(), shared_voter_state, }; @@ -303,7 +307,9 @@ macro_rules! new_full { )?; } - Ok((service, inherent_data_providers, task_manager, telemetry, base_path, rpc_handlers)) + Ok(KeepAliveChainComponents { + task_manager, telemetry, base_path, rpc, + }) }}; ($config:expr) => {{ new_full!($config, |_, _| {}) @@ -312,18 +318,17 @@ macro_rules! new_full { /// Builds a new service for a full client. pub fn new_full(config: Configuration) --> Result<(impl AbstractService, TaskManager, Option, Option>), ServiceError> -{ - new_full!(config).map(|(service, _, task_manager, telemetry, base_path, _)| { - (service, task_manager, telemetry, base_path) - }) +-> Result { + new_full!(config) } fn new_light_base(config: Configuration) --> Result<(impl AbstractService, TaskManager, Option, Option>, RpcHandlers), ServiceError> { +-> Result<(KeepAliveChainComponents, RpcHandlers), ServiceError> { let inherent_data_providers = InherentDataProviders::new(); - let (service, _, _, task_manager, _, _, telemetry, base_path, rpc_handlers) = ServiceBuilder::new_light::(config)? + let ChainComponents { + task_manager, telemetry, base_path, rpc_handlers, rpc, .. + } = ServiceBuilder::new_light::(config)? .with_select_chain(|_config, backend| { Ok(LongestChain::new(backend.clone())) })? @@ -407,21 +412,25 @@ fn new_light_base(config: Configuration) })? .build_light()?; - Ok((service, task_manager, telemetry, base_path, rpc_handlers)) + Ok(( + KeepAliveChainComponents { + task_manager, telemetry, base_path, rpc, + }, + rpc_handlers, + )) } /// Builds a new service for a light client. -pub fn new_light(config: Configuration) --> Result<(impl AbstractService, TaskManager, Option, Option>), ServiceError> { - new_light_base(config).map(|(service, task_manager, telemetry, base_path, _)| (service, task_manager, telemetry, base_path)) +pub fn new_light(config: Configuration) -> Result { + new_light_base(config).map(|(components, _)| components) } /// Builds a new service for a browser light client. -pub fn new_light_browser(config: Configuration) --> Result<(impl AbstractService, TaskManager, Option, Option>, RpcHandlers), ServiceError> { +pub fn new_light_browser(config: Configuration) -> Result<(KeepAliveChainComponents, RpcHandlers), ServiceError> { new_light_base(config) } +/* #[cfg(test)] mod tests { use std::{sync::Arc, borrow::Cow, any::Any}; @@ -447,14 +456,13 @@ mod tests { use sp_timestamp; use sp_finality_tracker; use sp_keyring::AccountKeyring; - use sc_service::AbstractService; use crate::service::{new_full, new_light}; use sp_runtime::traits::IdentifyAccount; use sp_transaction_pool::{MaintainedTransactionPool, ChainEvent}; type AccountPublic = ::Signer; - /*#[test] + #[test] // It is "ignored", but the node-cli ignored tests are running on the CI. // This can be run locally with `cargo test --release -p node-cli test_sync -- --ignored`. #[ignore] @@ -643,5 +651,6 @@ mod tests { "//Bob".into(), ], ) - }*/ + } } +*/ diff --git a/client/cli/src/runner.rs b/client/cli/src/runner.rs index e31ed1ea5809c..21d9f3723ada7 100644 --- a/client/cli/src/runner.rs +++ b/client/cli/src/runner.rs @@ -26,7 +26,7 @@ use futures::select; use futures::{future, future::FutureExt, Future}; use log::info; use sc_service::{ - AbstractService, Configuration, Role, ServiceBuilderCommand, TaskType, TaskManager, + Configuration, Role, ServiceBuilderCommand, TaskType, TaskManager, BasePath, }; use sp_runtime::traits::{Block as BlockT, Header as HeaderT}; @@ -34,7 +34,17 @@ use sp_utils::metrics::{TOKIO_THREADS_ALIVE, TOKIO_THREADS_TOTAL}; use sp_version::RuntimeVersion; use std::{fmt::Debug, marker::PhantomData, str::FromStr, sync::Arc}; -type Initialised = (T, TaskManager, Option, Option>); +/// The components of the chain that we need to keep alive until the node quits. +pub struct KeepAliveChainComponents { + /// The chain task manager. + pub task_manager: TaskManager, + /// A shared instance of Telemetry (if enabled). + pub telemetry: Option, + /// The base path. + pub base_path: Option>, + /// A RPC instance. + pub rpc: Box, +} #[cfg(target_family = "unix")] async fn main(func: F) -> std::result::Result<(), Box> @@ -182,10 +192,10 @@ impl Runner { /// A helper function that runs an `AbstractService` with tokio and stops if the process /// receives the signal `SIGTERM` or `SIGINT`. It can run a full or a light node depending on /// the node's configuration. - pub fn run_node( + pub fn run_node( self, - new_light: impl FnOnce(Configuration) -> sc_service::error::Result>, - new_full: impl FnOnce(Configuration) -> sc_service::error::Result>, + new_light: impl FnOnce(Configuration) -> sc_service::error::Result, + new_full: impl FnOnce(Configuration) -> sc_service::error::Result, runtime_version: RuntimeVersion, ) -> Result<()> { match self.config.role { @@ -197,9 +207,9 @@ impl Runner { /// A helper function that runs an `AbstractService` with tokio and stops if the process /// receives the signal `SIGTERM` or `SIGINT`. It can only run a "full" node and will fail if /// the node's configuration uses a "light" role. - pub fn run_full_node( + pub fn run_full_node( self, - new_full: impl FnOnce(Configuration) -> sc_service::error::Result>, + new_full: impl FnOnce(Configuration) -> sc_service::error::Result, runtime_version: RuntimeVersion, ) -> Result<()> { if matches!(self.config.role, Role::Light) { @@ -213,9 +223,9 @@ impl Runner { /// A helper function that runs an `AbstractService` with tokio and stops if the process /// receives the signal `SIGTERM` or `SIGINT`. It can only run a "light" node and will fail if /// the node's configuration uses a "full" role. - pub fn run_light_node( + pub fn run_light_node( self, - new_light: impl FnOnce(Configuration) -> sc_service::error::Result>, + new_light: impl FnOnce(Configuration) -> sc_service::error::Result, runtime_version: RuntimeVersion, ) -> Result<()> { if !matches!(self.config.role, Role::Light) { @@ -254,21 +264,22 @@ impl Runner { } } - fn run_service_until_exit( + fn run_service_until_exit( mut self, - initialise: impl FnOnce(Configuration) -> sc_service::error::Result>, + initialise: impl FnOnce(Configuration) -> sc_service::error::Result, ) -> Result<()> { - let ( - _service, + let KeepAliveChainComponents { task_manager, // we eagerly drop the service so that the internal exit future is fired, // but we need to keep holding a reference to the global telemetry guard // and drop the runtime first. - _telemetry, + telemetry: _telemetry, // we hold a reference to the base path so if the base path is a temporary directory it will // not be deleted before the tokio runtime finish to clean up - _base_path, - ) = initialise(self.config)?; + base_path: _base_path, + rpc: _rpc, + .. + } = initialise(self.config)?; { let f = task_manager.fuse(); diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index 7d6d8f8677273..a04f962b59c97 100644 --- a/client/service/src/builder.rs +++ b/client/service/src/builder.rs @@ -17,18 +17,18 @@ // along with this program. If not, see . use crate::{ - Service, NetworkStatus, NetworkState, error::Error, DEFAULT_PROTOCOL_ID, MallocSizeOfWasm, + NetworkStatus, NetworkState, error::Error, DEFAULT_PROTOCOL_ID, MallocSizeOfWasm, start_rpc_servers, build_network_future, TransactionPoolAdapter, TaskManager, SpawnTaskHandle, status_sinks, metrics::MetricsService, client::{light, Client, ClientConfig}, - config::{Configuration, KeystoreConfig, PrometheusConfig, OffchainWorkerConfig, BasePath}, + config::{Configuration, KeystoreConfig, PrometheusConfig, OffchainWorkerConfig}, }; use sc_client_api::{ self, BlockchainEvents, light::RemoteBlockchain, execution_extensions::ExtensionsFactory, ExecutorProvider, CallExecutor, ForkBlocks, BadBlocks, CloneableSpawn, UsageProvider, backend::RemoteBackend, }; -use sp_utils::mpsc::{tracing_unbounded, TracingUnboundedSender, TracingUnboundedReceiver}; +use sp_utils::mpsc::{tracing_unbounded, TracingUnboundedSender}; use sc_chain_spec::get_extension; use sp_consensus::{ block_validation::{BlockAnnounceValidator, DefaultBlockAnnounceValidator}, @@ -42,7 +42,7 @@ use jsonrpc_pubsub::manager::SubscriptionManager; use sc_keystore::Store as Keystore; use log::{info, warn, error}; use sc_network::config::{Role, FinalityProofProvider, OnDemand, BoxFinalityProofRequestBuilder}; -use sc_network::{NetworkService, NetworkStateInfo}; +use sc_network::NetworkStateInfo; use parking_lot::{Mutex, RwLock}; use sp_runtime::generic::BlockId; use sp_runtime::traits::{ @@ -64,6 +64,7 @@ use sp_core::traits::CodeExecutor; use sp_runtime::BuildStorage; use sc_client_api::execution_extensions::ExecutionExtensions; use sp_core::storage::Storage; +use crate::{ChainComponents, TelemetryOnConnectSinks, RpcHandlers, NetworkStatusSinks}; pub type BackgroundTask = Pin + Send>>; @@ -959,26 +960,7 @@ ServiceBuilder< Ok(self) } - fn build_common(self) -> Result<( - Service< - TBl, - TSc, - NetworkStatus, - sc_offchain::OffchainWorkers< - Client, - TBackend::OffchainStorage, - TBl - >, - >, - Arc>, - Arc, - TaskManager, - Arc>, - Arc::Hash>>, - Option, - Option>, - crate::RpcHandlers, - ), Error> + fn build_common(self) -> Result, Error> where TExec: CallExecutor, { let ServiceBuilder { @@ -1305,7 +1287,7 @@ ServiceBuilder< }; let rpc = start_rpc_servers(&config, gen_handler)?; // This is used internally, so don't restrict access to unsafe RPC - let rpc_handlers = crate::RpcHandlers(gen_handler(sc_rpc::DenyUnsafe::No)); + let rpc_handlers = RpcHandlers(gen_handler(sc_rpc::DenyUnsafe::No)); // The network worker is responsible for gathering all network messages and processing // them. This is quite a heavy task, and at the time of the writing of this comment it @@ -1400,38 +1382,26 @@ ServiceBuilder< ); spawn_handle.spawn("informant", informant_future); - Ok((Service { - network_status_sinks, + Ok(ChainComponents { + client, + task_manager, + network, select_chain, - _rpc: rpc, - _offchain_workers: offchain_workers, - _telemetry_on_connect_sinks: telemetry_connection_sinks.clone(), - marker: PhantomData::, + transaction_pool, + rpc_handlers, + rpc, + telemetry, + keystore, + offchain_workers, + telemetry_on_connect_sinks: TelemetryOnConnectSinks(telemetry_connection_sinks), + base_path: config.base_path.map(Arc::new), + network_status_sinks: NetworkStatusSinks::new(network_status_sinks), prometheus_registry: config.prometheus_config.map(|config| config.registry), - }, client, transaction_pool, task_manager, keystore, network, telemetry, config.base_path.map(Arc::new), rpc_handlers)) + }) } /// Builds the light service. - pub fn build_light(self) -> Result<( - Service< - TBl, - TSc, - NetworkStatus, - sc_offchain::OffchainWorkers< - Client, - TBackend::OffchainStorage, - TBl - >, - >, - Arc>, - Arc, - TaskManager, - Arc>, - Arc::Hash>>, - Option, - Option>, - crate::RpcHandlers, - ), Error> + pub fn build_light(self) -> Result, Error> where TExec: CallExecutor, { self.build_common() @@ -1474,26 +1444,7 @@ ServiceBuilder< { /// Builds the full service. - pub fn build_full(self) -> Result<( - Service< - TBl, - TSc, - NetworkStatus, - sc_offchain::OffchainWorkers< - Client, - TBackend::OffchainStorage, - TBl - >, - >, - Arc>, - Arc, - TaskManager, - Arc>, - Arc::Hash>>, - Option, - Option>, - crate::RpcHandlers, - ), Error> + pub fn build_full(self) -> Result, Error> where TExec: CallExecutor, { // make transaction pool available for off-chain runtime calls. diff --git a/client/service/src/lib.rs b/client/service/src/lib.rs index c63fcde5e65a2..10f0f79b927a1 100644 --- a/client/service/src/lib.rs +++ b/client/service/src/lib.rs @@ -37,12 +37,11 @@ mod status_sinks; mod task_manager; use std::{io, pin::Pin}; -use std::marker::PhantomData; use std::net::SocketAddr; use std::collections::HashMap; use std::time::Duration; use wasm_timer::Instant; -use std::task::{Poll, Context}; +use std::task::Poll; use parking_lot::Mutex; use client::Client; @@ -80,12 +79,7 @@ pub use sc_network::config::{ pub use sc_tracing::TracingReceiver; pub use task_manager::SpawnTaskHandle; pub use task_manager::TaskManager; -use sp_blockchain::{HeaderBackend, HeaderMetadata}; -use sp_api::{ApiExt, ConstructRuntimeApi, ApiErrorExt}; -use sc_client_api::{ - Backend as BackendT, BlockchainEvents, CallExecutor, UsageProvider, -}; -use sp_block_builder::BlockBuilder; +use sc_client_api::BlockchainEvents; const DEFAULT_PROTOCOL_ID: &str = "sup"; @@ -99,6 +93,7 @@ impl MallocSizeOfWasm for T {} #[cfg(target_os = "unknown")] impl MallocSizeOfWasm for T {} +/// RPC handlers that can perform RPC queries. pub struct RpcHandlers(sc_rpc_server::RpcHandler); impl RpcHandlers { @@ -119,71 +114,76 @@ impl RpcHandlers { } } -/// Substrate service. -pub struct Service { - select_chain: Option, - // Sinks to propagate network status updates. - // For each element, every time the `Interval` fires we push an element on the sender. - network_status_sinks: Arc>>, - _rpc: Box, - _telemetry_on_connect_sinks: Arc>>>, - _offchain_workers: Option>, - marker: PhantomData, - prometheus_registry: Option, +/// Sinks to propagate network status updates. +/// For each element, every time the `Interval` fires we push an element on the sender. +pub struct NetworkStatusSinks { + inner: Arc, NetworkState)>>>, } -impl Unpin for Service {} - -/// Abstraction over a Substrate service. -pub trait AbstractService: Send + Unpin + 'static { - /// Type of block of this chain. - type Block: BlockT; - /// Chain selection algorithm. - type SelectChain: sp_consensus::SelectChain + Sized; - - /// Get event stream for telemetry connection established events. - fn telemetry_on_connect_stream(&self) -> TracingUnboundedReceiver<()>; - - - /// Get clone of select chain. - fn select_chain(&self) -> Option; +impl NetworkStatusSinks { + fn new(sinks: Arc, NetworkState)>>>) -> Self { + Self { + inner: sinks, + } + } /// Returns a receiver that periodically receives a status of the network. - fn network_status(&self, interval: Duration) -> TracingUnboundedReceiver<(NetworkStatus, NetworkState)>; - - /// Get the prometheus metrics registry, if available. - fn prometheus_registry(&self) -> Option; -} - -impl AbstractService for - Service, TOc> -where - TBl: BlockT, - TSc: sp_consensus::SelectChain + 'static + Clone + Send + Unpin, - TOc: 'static + Send + Sync, -{ - type Block = TBl; - type SelectChain = TSc; - - fn telemetry_on_connect_stream(&self) -> TracingUnboundedReceiver<()> { - let (sink, stream) = tracing_unbounded("mpsc_telemetry_on_connect"); - self._telemetry_on_connect_sinks.lock().push(sink); + pub fn network_status(&self, interval: Duration) -> TracingUnboundedReceiver<(NetworkStatus, NetworkState)> { + let (sink, stream) = tracing_unbounded("mpsc_network_status"); + self.inner.lock().push(interval, sink); stream } +} - fn select_chain(&self) -> Option { - self.select_chain.clone() - } +/// Sinks to propagate telemetry connection established events. +pub struct TelemetryOnConnectSinks(pub Arc>>>); - fn network_status(&self, interval: Duration) -> TracingUnboundedReceiver<(NetworkStatus, NetworkState)> { - let (sink, stream) = tracing_unbounded("mpsc_network_status"); - self.network_status_sinks.lock().push(interval, sink); +impl TelemetryOnConnectSinks { + /// Get event stream for telemetry connection established events. + pub fn on_connect_stream(&self) -> TracingUnboundedReceiver<()> { + let (sink, stream) =tracing_unbounded("mpsc_telemetry_on_connect"); + self.0.lock().push(sink); stream } +} - fn prometheus_registry(&self) -> Option { - self.prometheus_registry.clone() - } +/// The individual components of the chain, built by the service builder. You are encouraged to +/// deconstruct this into its fields. +pub struct ChainComponents< + TBl: BlockT, TBackend: sc_client_api::backend::Backend, TExec, TRtApi, TSc, TExPool, +> { + /// A blockchain client. + pub client: Arc>, + /// A shared transaction pool instance. + pub transaction_pool: Arc, + /// The chain task manager. + pub task_manager: TaskManager, + /// A keystore that stores keys. + pub keystore: sc_keystore::KeyStorePtr, + /// A shared network instance. + pub network: Arc::Hash>>, + /// A shared instance of Telemetry (if enabled). + pub telemetry: Option, + /// The base path. + pub base_path: Option>, + /// RPC handlers that can perform RPC queries. + pub rpc_handlers: crate::RpcHandlers, + /// A shared instance of the chain selection algorithm. + pub select_chain: Option, + /// Sinks to propagate network status updates. + pub network_status_sinks: NetworkStatusSinks, + /// A prometheus metrics registry, (if enabled). + pub prometheus_registry: Option, + /// A RPC instance. + pub rpc: Box, + /// Shared Telemetry connection sinks, + pub telemetry_on_connect_sinks: TelemetryOnConnectSinks, + /// A shared offchain workers instance. + pub offchain_workers: Option, + TBackend::OffchainStorage, + TBl + >>>, } /// Builds a never-ending future that continuously polls the network. diff --git a/client/service/src/task_manager.rs b/client/service/src/task_manager.rs index 03f9719956f09..0111106bdca3b 100644 --- a/client/service/src/task_manager.rs +++ b/client/service/src/task_manager.rs @@ -248,7 +248,6 @@ impl Future for TaskManager { } } - impl Drop for TaskManager { fn drop(&mut self) { debug!(target: "service", "Tasks manager shutdown"); diff --git a/utils/browser/src/lib.rs b/utils/browser/src/lib.rs index 131be258ec5c2..006447d680139 100644 --- a/utils/browser/src/lib.rs +++ b/utils/browser/src/lib.rs @@ -20,7 +20,7 @@ use log::{debug, info}; use std::sync::Arc; use sc_network::config::TransportConfig; use sc_service::{ - AbstractService, RpcSession, Role, Configuration, + RpcSession, Role, Configuration, config::{DatabaseConfig, KeystoreConfig, NetworkConfiguration}, GenericChainSpec, RuntimeGenesis, TaskManager, RpcHandlers, }; From 3de5fcef2ef2c6731fbc540a4bcd252724889ddc Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Tue, 16 Jun 2020 14:43:36 +0200 Subject: [PATCH 04/25] Get service tests compiling --- Cargo.lock | 37 ++++ Cargo.toml | 2 +- bin/node/cli/Cargo.toml | 2 +- bin/node/cli/src/browser.rs | 2 +- bin/node/cli/src/chain_spec.rs | 15 +- bin/node/cli/src/service.rs | 360 ++++++++++++++++++--------------- client/cli/src/runner.rs | 28 +-- client/service/src/lib.rs | 8 + client/service/test/src/lib.rs | 181 +++++++++++------ 9 files changed, 371 insertions(+), 264 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index abf336f68cdb6..83c98c65c6cb0 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3332,6 +3332,7 @@ dependencies = [ "sc-offchain", "sc-rpc", "sc-service", + "sc-service-test", "sc-telemetry", "sc-tracing", "sc-transaction-pool", @@ -6666,6 +6667,42 @@ dependencies = [ "wasm-timer", ] +[[package]] +name = "sc-service-test" +version = "2.0.0-rc3" +dependencies = [ + "env_logger 0.7.1", + "fdlimit", + "futures 0.1.29", + "futures 0.3.4", + "hex-literal", + "log", + "parity-scale-codec", + "parking_lot 0.10.2", + "sc-block-builder", + "sc-client-api", + "sc-client-db", + "sc-executor", + "sc-light", + "sc-network", + "sc-service", + "sp-api", + "sp-blockchain", + "sp-consensus", + "sp-core", + "sp-externalities", + "sp-panic-handler", + "sp-runtime", + "sp-state-machine", + "sp-storage", + "sp-transaction-pool", + "sp-trie", + "substrate-test-runtime", + "substrate-test-runtime-client", + "tempfile", + "tokio 0.1.22", +] + [[package]] name = "sc-state-db" version = "0.8.0-rc3" diff --git a/Cargo.toml b/Cargo.toml index 6752687375b45..d3004fcadca5a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -51,7 +51,7 @@ members = [ "client/rpc", "client/rpc-api", "client/service", - #"client/service/test", + "client/service/test", "client/state-db", "client/telemetry", "client/transaction-pool", diff --git a/bin/node/cli/Cargo.toml b/bin/node/cli/Cargo.toml index dd10cd65bddde..74edf2f257bfa 100644 --- a/bin/node/cli/Cargo.toml +++ b/bin/node/cli/Cargo.toml @@ -115,7 +115,7 @@ sc-keystore = { version = "2.0.0-rc3", path = "../../../client/keystore" } sc-consensus = { version = "0.8.0-rc3", path = "../../../client/consensus/common" } sc-consensus-babe = { version = "0.8.0-rc3", features = ["test-helpers"], path = "../../../client/consensus/babe" } sc-consensus-epochs = { version = "0.8.0-rc3", path = "../../../client/consensus/epochs" } -#sc-service-test = { version = "2.0.0-rc3", path = "../../../client/service/test" } +sc-service-test = { version = "2.0.0-rc3", path = "../../../client/service/test" } futures = "0.3.4" tempfile = "3.1.0" assert_cmd = "1.0" diff --git a/bin/node/cli/src/browser.rs b/bin/node/cli/src/browser.rs index 554d7abc16441..ec0a5e8fcc206 100644 --- a/bin/node/cli/src/browser.rs +++ b/bin/node/cli/src/browser.rs @@ -23,7 +23,7 @@ use browser_utils::{ Client, browser_configuration, set_console_error_panic_hook, init_console_log, }; -use sc_cli::KeepAliveChainComponents; +use sc_service::KeepAliveChainComponents; use std::str::FromStr; /// Starts the client. diff --git a/bin/node/cli/src/chain_spec.rs b/bin/node/cli/src/chain_spec.rs index c74a526b08742..8617024255f28 100644 --- a/bin/node/cli/src/chain_spec.rs +++ b/bin/node/cli/src/chain_spec.rs @@ -380,7 +380,8 @@ pub fn local_testnet_config() -> ChainSpec { #[cfg(test)] pub(crate) mod tests { use super::*; - use crate::service::{new_full, new_light}; + use crate::service::{new_full_base, new_light_base}; + use sc_service_test; use sp_runtime::BuildStorage; fn local_testnet_genesis_instant_single() -> GenesisConfig { @@ -424,17 +425,21 @@ pub(crate) mod tests { ) } - /* #[test] #[ignore] fn test_connectivity() { sc_service_test::connectivity( integration_test_config_with_two_authorities(), - |config| new_full(config), - |config| new_light(config), + |config| { + let (keep_alive, _, client, network, transaction_pool) = new_full_base(config,|_, _| ())?; + Ok(sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool)) + }, + |config| { + let (keep_alive, _, client, network, transaction_pool) = new_light_base(config)?; + Ok(sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool)) + } ); } - */ #[test] fn test_create_development_chain_spec() { diff --git a/bin/node/cli/src/service.rs b/bin/node/cli/src/service.rs index d6d6b630c803a..1089119a33da4 100644 --- a/bin/node/cli/src/service.rs +++ b/bin/node/cli/src/service.rs @@ -30,11 +30,15 @@ use node_primitives::Block; use node_runtime::RuntimeApi; use sc_service::{ ServiceBuilder, config::Configuration, error::{Error as ServiceError}, - RpcHandlers, ChainComponents, + RpcHandlers, ChainComponents, KeepAliveChainComponents, }; use sp_inherents::InherentDataProviders; use sc_consensus::LongestChain; -use sc_cli::KeepAliveChainComponents; +use sc_network::{Event, NetworkService}; +use sp_runtime::traits::Block as BlockT; +use futures::prelude::*; +use sc_client_api::ExecutorProvider; +use sp_core::traits::BareCryptoStorePtr; /// Starts a `ServiceBuilder` for a full service. /// @@ -149,186 +153,196 @@ macro_rules! new_full_start { }} } +type FullClient = sc_service::TFullClient; +type FullBackend = sc_service::TFullBackend; +type GrandpaBlockImport = grandpa::GrandpaBlockImport>; +type BabeBlockImport = sc_consensus_babe::BabeBlockImport; + /// Creates a full service from the configuration. -/// -/// We need to use a macro because the test suit doesn't work with an opaque service. It expects -/// concrete types instead. -macro_rules! new_full { - ($config:expr, $with_startup_data: expr) => {{ - use futures::prelude::*; - use sc_network::Event; - use sc_client_api::ExecutorProvider; - use sp_core::traits::BareCryptoStorePtr; - - let ( - role, - force_authoring, - name, - disable_grandpa, - ) = ( - $config.role.clone(), - $config.force_authoring, - $config.network.node_name.clone(), - $config.disable_grandpa, - ); +pub fn new_full_base( + config: Configuration, + with_startup_data: impl FnOnce(&BabeBlockImport, &sc_consensus_babe::BabeLink) +) -> Result<( + KeepAliveChainComponents, + InherentDataProviders, + Arc, Arc::Hash>>, + Arc, Block>> +), ServiceError> { + let ( + role, + force_authoring, + name, + disable_grandpa, + ) = ( + config.role.clone(), + config.force_authoring, + config.network.node_name.clone(), + config.disable_grandpa, + ); + + let (builder, mut import_setup, inherent_data_providers, mut rpc_setup) = + new_full_start!(config); - let (builder, mut import_setup, inherent_data_providers, mut rpc_setup) = - new_full_start!($config); - - let ChainComponents { - client, transaction_pool, task_manager, keystore, network, telemetry, base_path, - select_chain, prometheus_registry, rpc, telemetry_on_connect_sinks, .. - } = builder - .with_finality_proof_provider(|client, backend| { - // GenesisAuthoritySetProvider is implemented for StorageAndProofProvider - let provider = client as Arc>; - Ok(Arc::new(grandpa::FinalityProofProvider::new(backend, provider)) as _) - })? - .build_full()?; + let ChainComponents { + client, transaction_pool, task_manager, keystore, network, telemetry, base_path, + select_chain, prometheus_registry, rpc, telemetry_on_connect_sinks, .. + } = builder + .with_finality_proof_provider(|client, backend| { + // GenesisAuthoritySetProvider is implemented for StorageAndProofProvider + let provider = client as Arc>; + Ok(Arc::new(grandpa::FinalityProofProvider::new(backend, provider)) as _) + })? + .build_full()?; - let (block_import, grandpa_link, babe_link) = import_setup.take() - .expect("Link Half and Block Import are present for Full Services or setup failed before. qed"); + let (block_import, grandpa_link, babe_link) = import_setup.take() + .expect("Link Half and Block Import are present for Full Services or setup failed before. qed"); - let shared_voter_state = rpc_setup.take() - .expect("The SharedVoterState is present for Full Services or setup failed before. qed"); + let shared_voter_state = rpc_setup.take() + .expect("The SharedVoterState is present for Full Services or setup failed before. qed"); - ($with_startup_data)(&block_import, &babe_link); + (with_startup_data)(&block_import, &babe_link); - if let sc_service::config::Role::Authority { .. } = &role { - let proposer = sc_basic_authorship::ProposerFactory::new( - client.clone(), - transaction_pool, - prometheus_registry.as_ref(), - ); + if let sc_service::config::Role::Authority { .. } = &role { + let proposer = sc_basic_authorship::ProposerFactory::new( + client.clone(), + transaction_pool.clone(), + prometheus_registry.as_ref(), + ); - let select_chain = select_chain - .ok_or(sc_service::Error::SelectChainRequired)?; - - let can_author_with = - sp_consensus::CanAuthorWithNativeVersion::new(client.executor().clone()); - - let babe_config = sc_consensus_babe::BabeParams { - keystore: keystore.clone(), - client: client.clone(), - select_chain, - env: proposer, - block_import, - sync_oracle: network.clone(), - inherent_data_providers: inherent_data_providers.clone(), - force_authoring, - babe_link, - can_author_with, - }; + let select_chain = select_chain + .ok_or(sc_service::Error::SelectChainRequired)?; - let babe = sc_consensus_babe::start_babe(babe_config)?; - task_manager.spawn_essential("babe-proposer", babe); - } - - // Spawn authority discovery module. - if matches!(role, sc_service::config::Role::Authority{..} | sc_service::config::Role::Sentry {..}) { - let (sentries, authority_discovery_role) = match role { - sc_service::config::Role::Authority { ref sentry_nodes } => ( - sentry_nodes.clone(), - sc_authority_discovery::Role::Authority ( - keystore.clone(), - ), - ), - sc_service::config::Role::Sentry {..} => ( - vec![], - sc_authority_discovery::Role::Sentry, - ), - _ => unreachable!("Due to outer matches! constraint; qed.") - }; + let can_author_with = + sp_consensus::CanAuthorWithNativeVersion::new(client.executor().clone()); - let dht_event_stream = network.event_stream("authority-discovery").filter_map(|e| async move { match e { - Event::Dht(e) => Some(e), - _ => None, - }}).boxed(); - let authority_discovery = sc_authority_discovery::AuthorityDiscovery::new( - client.clone(), - network.clone(), - sentries, - dht_event_stream, - authority_discovery_role, - prometheus_registry.clone(), - ); + let babe_config = sc_consensus_babe::BabeParams { + keystore: keystore.clone(), + client: client.clone(), + select_chain, + env: proposer, + block_import, + sync_oracle: network.clone(), + inherent_data_providers: inherent_data_providers.clone(), + force_authoring, + babe_link, + can_author_with, + }; - task_manager.spawn("authority-discovery", authority_discovery); - } + let babe = sc_consensus_babe::start_babe(babe_config)?; + task_manager.spawn_essential("babe-proposer", babe); + } - // if the node isn't actively participating in consensus then it doesn't - // need a keystore, regardless of which protocol we use below. - let keystore = if role.is_authority() { - Some(keystore.clone() as BareCryptoStorePtr) - } else { - None + // Spawn authority discovery module. + if matches!(role, sc_service::config::Role::Authority{..} | sc_service::config::Role::Sentry {..}) { + let (sentries, authority_discovery_role) = match role { + sc_service::config::Role::Authority { ref sentry_nodes } => ( + sentry_nodes.clone(), + sc_authority_discovery::Role::Authority ( + keystore.clone(), + ), + ), + sc_service::config::Role::Sentry {..} => ( + vec![], + sc_authority_discovery::Role::Sentry, + ), + _ => unreachable!("Due to outer matches! constraint; qed.") }; - let config = grandpa::Config { - // FIXME #1578 make this available through chainspec - gossip_duration: std::time::Duration::from_millis(333), - justification_period: 512, - name: Some(name), - observer_enabled: false, - keystore, - is_authority: role.is_network_authority(), - }; + let dht_event_stream = network.event_stream("authority-discovery").filter_map(|e| async move { match e { + Event::Dht(e) => Some(e), + _ => None, + }}).boxed(); + let authority_discovery = sc_authority_discovery::AuthorityDiscovery::new( + client.clone(), + network.clone(), + sentries, + dht_event_stream, + authority_discovery_role, + prometheus_registry.clone(), + ); - let enable_grandpa = !disable_grandpa; - if enable_grandpa { - // start the full GRANDPA voter - // NOTE: non-authorities could run the GRANDPA observer protocol, but at - // this point the full voter should provide better guarantees of block - // and vote data availability than the observer. The observer has not - // been tested extensively yet and having most nodes in a network run it - // could lead to finality stalls. - let grandpa_config = grandpa::GrandpaParams { - config, - link: grandpa_link, - network: network.clone(), - inherent_data_providers: inherent_data_providers.clone(), - telemetry_on_connect: Some(telemetry_on_connect_sinks.on_connect_stream()), - voting_rule: grandpa::VotingRulesBuilder::default().build(), - prometheus_registry: prometheus_registry.clone(), - shared_voter_state, - }; + task_manager.spawn("authority-discovery", authority_discovery); + } - // the GRANDPA voter task is considered infallible, i.e. - // if it fails we take down the service with it. - task_manager.spawn_essential( - "grandpa-voter", - grandpa::run_grandpa_voter(grandpa_config)? - ); - } else { - grandpa::setup_disabled_grandpa( - client, - &inherent_data_providers, - network.clone(), - )?; - } + // if the node isn't actively participating in consensus then it doesn't + // need a keystore, regardless of which protocol we use below. + let keystore = if role.is_authority() { + Some(keystore.clone() as BareCryptoStorePtr) + } else { + None + }; - Ok(KeepAliveChainComponents { - task_manager, telemetry, base_path, rpc, - }) - }}; - ($config:expr) => {{ - new_full!($config, |_, _| {}) - }} + let config = grandpa::Config { + // FIXME #1578 make this available through chainspec + gossip_duration: std::time::Duration::from_millis(333), + justification_period: 512, + name: Some(name), + observer_enabled: false, + keystore, + is_authority: role.is_network_authority(), + }; + + let enable_grandpa = !disable_grandpa; + if enable_grandpa { + // start the full GRANDPA voter + // NOTE: non-authorities could run the GRANDPA observer protocol, but at + // this point the full voter should provide better guarantees of block + // and vote data availability than the observer. The observer has not + // been tested extensively yet and having most nodes in a network run it + // could lead to finality stalls. + let grandpa_config = grandpa::GrandpaParams { + config, + link: grandpa_link, + network: network.clone(), + inherent_data_providers: inherent_data_providers.clone(), + telemetry_on_connect: Some(telemetry_on_connect_sinks.on_connect_stream()), + voting_rule: grandpa::VotingRulesBuilder::default().build(), + prometheus_registry: prometheus_registry.clone(), + shared_voter_state, + }; + + // the GRANDPA voter task is considered infallible, i.e. + // if it fails we take down the service with it. + task_manager.spawn_essential( + "grandpa-voter", + grandpa::run_grandpa_voter(grandpa_config)? + ); + } else { + grandpa::setup_disabled_grandpa( + client.clone(), + &inherent_data_providers, + network.clone(), + )?; + } + + Ok(( + KeepAliveChainComponents { + task_manager, other: Box::new((telemetry, base_path, rpc)), + }, + inherent_data_providers, + client, network, transaction_pool + )) } /// Builds a new service for a full client. pub fn new_full(config: Configuration) -> Result { - new_full!(config) + new_full_base(config, |_, _| ()).map(|(keep_alive, _, _, _, _)| keep_alive) } -fn new_light_base(config: Configuration) --> Result<(KeepAliveChainComponents, RpcHandlers), ServiceError> { +type LightClient = sc_service::TLightClient; +type LightFetcher = sc_network::config::OnDemand; + +pub fn new_light_base(config: Configuration) -> Result<( + KeepAliveChainComponents, RpcHandlers, Arc, + Arc::Hash>>, + Arc, Block>> +), ServiceError> { let inherent_data_providers = InherentDataProviders::new(); let ChainComponents { - task_manager, telemetry, base_path, rpc_handlers, rpc, .. - } = ServiceBuilder::new_light::(config)? + task_manager, telemetry, base_path, rpc_handlers, rpc, client, network, transaction_pool, .. + } = ServiceBuilder::new_light::(config)? .with_select_chain(|_config, backend| { Ok(LongestChain::new(backend.clone())) })? @@ -414,23 +428,23 @@ fn new_light_base(config: Configuration) Ok(( KeepAliveChainComponents { - task_manager, telemetry, base_path, rpc, + task_manager, other: Box::new((telemetry, base_path, rpc)), }, rpc_handlers, + client, network, transaction_pool )) } /// Builds a new service for a light client. pub fn new_light(config: Configuration) -> Result { - new_light_base(config).map(|(components, _)| components) + new_light_base(config).map(|(components, _, _, _, _)| components) } /// Builds a new service for a browser light client. pub fn new_light_browser(config: Configuration) -> Result<(KeepAliveChainComponents, RpcHandlers), ServiceError> { - new_light_base(config) + new_light_base(config).map(|(components, rpc_handlers, _, _, _)| (components, rpc_handlers)) } -/* #[cfg(test)] mod tests { use std::{sync::Arc, borrow::Cow, any::Any}; @@ -456,7 +470,8 @@ mod tests { use sp_timestamp; use sp_finality_tracker; use sp_keyring::AccountKeyring; - use crate::service::{new_full, new_light}; + use sc_service_test::TestNetNode; + use crate::service::{new_full_base, new_light_base}; use sp_runtime::traits::IdentifyAccount; use sp_transaction_pool::{MaintainedTransactionPool, ChainEvent}; @@ -487,14 +502,20 @@ mod tests { chain_spec, |config| { let mut setup_handles = None; - new_full!(config, | + let (keep_alive, inherent_data_providers, client, network, transaction_pool) = new_full_base(config, | block_import: &sc_consensus_babe::BabeBlockImport, babe_link: &sc_consensus_babe::BabeLink, | { setup_handles = Some((block_import.clone(), babe_link.clone())); - }).map(move |(node, x)| (node, (x, setup_handles.unwrap()))) + })?; + + let node = sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool); + Ok((node, (inherent_data_providers, setup_handles.unwrap()))) + }, + |config| { + let (keep_alive, _, client, network, transaction_pool) = new_light_base(config)?; + Ok(sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool)) }, - |config| new_light(config), |service, &mut (ref inherent_data_providers, (ref mut block_import, ref babe_link))| { let mut inherent_data = inherent_data_providers .create_inherent_data() @@ -644,8 +665,14 @@ mod tests { fn test_consensus() { sc_service_test::consensus( crate::chain_spec::tests::integration_test_config_with_two_authorities(), - |config| new_full(config), - |config| new_light(config), + |config| { + let (keep_alive, _, client, network, transaction_pool) = new_full_base(config, |_, _| ())?; + Ok(sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool)) + }, + |config| { + let (keep_alive, _, client, network, transaction_pool) = new_light_base(config)?; + Ok(sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool)) + }, vec![ "//Alice".into(), "//Bob".into(), @@ -653,4 +680,3 @@ mod tests { ) } } -*/ diff --git a/client/cli/src/runner.rs b/client/cli/src/runner.rs index 21d9f3723ada7..463d36d37aaa7 100644 --- a/client/cli/src/runner.rs +++ b/client/cli/src/runner.rs @@ -25,27 +25,12 @@ use futures::pin_mut; use futures::select; use futures::{future, future::FutureExt, Future}; use log::info; -use sc_service::{ - Configuration, Role, ServiceBuilderCommand, TaskType, TaskManager, - BasePath, -}; +use sc_service::{Configuration, Role, ServiceBuilderCommand, TaskType, KeepAliveChainComponents}; use sp_runtime::traits::{Block as BlockT, Header as HeaderT}; use sp_utils::metrics::{TOKIO_THREADS_ALIVE, TOKIO_THREADS_TOTAL}; use sp_version::RuntimeVersion; use std::{fmt::Debug, marker::PhantomData, str::FromStr, sync::Arc}; -/// The components of the chain that we need to keep alive until the node quits. -pub struct KeepAliveChainComponents { - /// The chain task manager. - pub task_manager: TaskManager, - /// A shared instance of Telemetry (if enabled). - pub telemetry: Option, - /// The base path. - pub base_path: Option>, - /// A RPC instance. - pub rpc: Box, -} - #[cfg(target_family = "unix")] async fn main(func: F) -> std::result::Result<(), Box> where @@ -270,15 +255,8 @@ impl Runner { ) -> Result<()> { let KeepAliveChainComponents { task_manager, - // we eagerly drop the service so that the internal exit future is fired, - // but we need to keep holding a reference to the global telemetry guard - // and drop the runtime first. - telemetry: _telemetry, - // we hold a reference to the base path so if the base path is a temporary directory it will - // not be deleted before the tokio runtime finish to clean up - base_path: _base_path, - rpc: _rpc, - .. + // We need to keep a hold of these until the tasks are finished. + other: _other, } = initialise(self.config)?; { diff --git a/client/service/src/lib.rs b/client/service/src/lib.rs index 10f0f79b927a1..930783010d61e 100644 --- a/client/service/src/lib.rs +++ b/client/service/src/lib.rs @@ -186,6 +186,14 @@ pub struct ChainComponents< >>>, } +/// The components of the chain that we need to keep alive until the node quits. +pub struct KeepAliveChainComponents { + /// The chain task manager. + pub task_manager: TaskManager, + /// Everything else. + pub other: Box, +} + /// Builds a never-ending future that continuously polls the network. /// /// The `status_sink` contain a list of senders to send a periodic network status to. diff --git a/client/service/test/src/lib.rs b/client/service/test/src/lib.rs index 613b0d71ce933..1d8ab250389fb 100644 --- a/client/service/test/src/lib.rs +++ b/client/service/test/src/lib.rs @@ -19,7 +19,7 @@ //! Service integration test utils. use std::iter; -use std::sync::{Arc, Mutex, MutexGuard}; +use std::sync::{Arc}; use std::net::Ipv4Addr; use std::pin::Pin; use std::time::Duration; @@ -30,7 +30,7 @@ use tempfile::TempDir; use tokio::{runtime::Runtime, prelude::FutureExt}; use tokio::timer::Interval; use sc_service::{ - AbstractService, + KeepAliveChainComponents, GenericChainSpec, ChainSpecExtension, Configuration, @@ -39,12 +39,16 @@ use sc_service::{ Role, Error, TaskType, + client::Client, }; use sp_blockchain::HeaderBackend; use sc_network::{multiaddr, Multiaddr}; use sc_network::config::{NetworkConfiguration, TransportConfig}; use sp_runtime::{generic::BlockId, traits::Block as BlockT}; use sp_transaction_pool::TransactionPool; +use sc_client_api::{Backend, CallExecutor}; +use parking_lot::Mutex; + #[cfg(test)] mod client; @@ -54,47 +58,98 @@ const MAX_WAIT_TIME: Duration = Duration::from_secs(60 * 3); struct TestNet { runtime: Runtime, - authority_nodes: Vec<(usize, SyncService, U, Multiaddr)>, - full_nodes: Vec<(usize, SyncService, U, Multiaddr)>, - light_nodes: Vec<(usize, SyncService, Multiaddr)>, + authority_nodes: Vec<(usize, F, U, Multiaddr)>, + full_nodes: Vec<(usize, F, U, Multiaddr)>, + light_nodes: Vec<(usize, L, Multiaddr)>, chain_spec: GenericChainSpec, base_port: u16, nodes: usize, } -/// Wraps around an `Arc` and implements `Future`. -pub struct SyncService(Arc>); +pub trait TestNetNode: Clone + Future + Send + 'static { + type Block: BlockT; + type Backend: Backend; + type Executor: CallExecutor + Send + Sync; + type RuntimeApi: Send + Sync; + type TransactionPool: TransactionPool; -impl SyncService { - pub fn get(&self) -> MutexGuard { - self.0.lock().unwrap() - } + fn client(&self) -> Arc>; + fn transaction_pool(&self) -> Arc; + fn network(&self) -> Arc::Hash>>; } -impl Clone for SyncService { - fn clone(&self) -> Self { - Self(self.0.clone()) +pub struct TestNetComponents { + keep_alive: Arc>, + client: Arc>, + transaction_pool: Arc, + network: Arc::Hash>>, +} + +impl +TestNetComponents { + pub fn new( + keep_alive: KeepAliveChainComponents, + client: Arc>, + network: Arc::Hash>>, + transaction_pool: Arc, + ) -> Self { + Self { + client, transaction_pool, network, + keep_alive: Arc::new(Mutex::new(keep_alive)), + } } } -impl From for SyncService { - fn from(service: T) -> Self { - SyncService(Arc::new(Mutex::new(service))) + +impl Clone for +TestNetComponents { + fn clone(&self) -> Self { + Self { + keep_alive: self.keep_alive.clone(), + client: self.client.clone(), + transaction_pool: self.transaction_pool.clone(), + network: self.network.clone(), + } } } -impl> + Unpin> Future for SyncService { +impl Future for TestNetComponents { type Item = (); type Error = sc_service::Error; fn poll(&mut self) -> Poll { - let mut f = self.0.lock().unwrap(); - futures::compat::Compat::new(&mut *f).poll() + futures::compat::Compat::new(&mut self.keep_alive.lock().task_manager).poll() + } +} + +impl TestNetNode for +TestNetComponents + where + TBl: BlockT, + TBackend: sc_client_api::Backend + Send + Sync + 'static, + TExec: CallExecutor + Send + Sync + 'static, + TRtApi: Send + Sync + 'static, + TExPool: TransactionPool + Send + Sync + 'static, +{ + type Block = TBl; + type Backend = TBackend; + type Executor = TExec; + type RuntimeApi = TRtApi; + type TransactionPool = TExPool; + + fn client(&self) -> Arc> { + self.client.clone() + } + fn transaction_pool(&self) -> Arc { + self.transaction_pool.clone() + } + fn network(&self) -> Arc::Hash>> { + self.network.clone() } } impl TestNet -where F: Send + 'static, L: Send +'static, U: Clone + Send + 'static +where F: Clone + Send + 'static, L: Clone + Send +'static, U: Clone + Send + 'static { pub fn run_until_all_full( &mut self, @@ -102,8 +157,8 @@ where F: Send + 'static, L: Send +'static, U: Clone + Send + 'static light_predicate: LP, ) where - FP: Send + Fn(usize, &SyncService) -> bool + 'static, - LP: Send + Fn(usize, &SyncService) -> bool + 'static, + FP: Send + Fn(usize, &F) -> bool + 'static, + LP: Send + Fn(usize, &L) -> bool + 'static, { let full_nodes = self.full_nodes.clone(); let light_nodes = self.light_nodes.clone(); @@ -215,8 +270,8 @@ fn node_config TestNet where - F: AbstractService, - L: AbstractService, + F: TestNetNode, + L: TestNetNode, E: ChainSpecExtension + Clone + 'static + Send, G: RuntimeGenesis + 'static, { @@ -272,10 +327,9 @@ impl TestNet where ); let addr = node_config.network.listen_addresses.iter().next().unwrap().clone(); let (service, user_data) = authority(node_config).expect("Error creating test node service"); - let service = SyncService::from(service); executor.spawn(service.clone().map_err(|_| ())); - let addr = addr.with(multiaddr::Protocol::P2p(service.get().network().local_peer_id().clone().into())); + let addr = addr.with(multiaddr::Protocol::P2p(service.network().local_peer_id().clone().into())); self.authority_nodes.push((self.nodes, service, user_data, addr)); self.nodes += 1; } @@ -288,10 +342,9 @@ impl TestNet where let node_config = node_config(self.nodes, &self.chain_spec, Role::Full, task_executor, None, self.base_port, &temp); let addr = node_config.network.listen_addresses.iter().next().unwrap().clone(); let (service, user_data) = full(node_config).expect("Error creating test node service"); - let service = SyncService::from(service); executor.spawn(service.clone().map_err(|_| ())); - let addr = addr.with(multiaddr::Protocol::P2p(service.get().network().local_peer_id().clone().into())); + let addr = addr.with(multiaddr::Protocol::P2p(service.network().local_peer_id().clone().into())); self.full_nodes.push((self.nodes, service, user_data, addr)); self.nodes += 1; } @@ -303,10 +356,10 @@ impl TestNet where }; let node_config = node_config(self.nodes, &self.chain_spec, Role::Light, task_executor, None, self.base_port, &temp); let addr = node_config.network.listen_addresses.iter().next().unwrap().clone(); - let service = SyncService::from(light(node_config).expect("Error creating test node service")); + let service = light(node_config).expect("Error creating test node service"); executor.spawn(service.clone().map_err(|_| ())); - let addr = addr.with(multiaddr::Protocol::P2p(service.get().network().local_peer_id().clone().into())); + let addr = addr.with(multiaddr::Protocol::P2p(service.network().local_peer_id().clone().into())); self.light_nodes.push((self.nodes, service, addr)); self.nodes += 1; } @@ -325,9 +378,9 @@ pub fn connectivity( E: ChainSpecExtension + Clone + 'static + Send, G: RuntimeGenesis + 'static, Fb: Fn(Configuration) -> Result, - F: AbstractService, + F: TestNetNode, Lb: Fn(Configuration) -> Result, - L: AbstractService, + L: TestNetNode, { const NUM_FULL_NODES: usize = 5; const NUM_LIGHT_NODES: usize = 5; @@ -351,18 +404,18 @@ pub fn connectivity( info!("Checking star topology"); let first_address = network.full_nodes[0].3.clone(); for (_, service, _, _) in network.full_nodes.iter().skip(1) { - service.get().network().add_reserved_peer(first_address.to_string()) + service.network().add_reserved_peer(first_address.to_string()) .expect("Error adding reserved peer"); } for (_, service, _) in network.light_nodes.iter() { - service.get().network().add_reserved_peer(first_address.to_string()) + service.network().add_reserved_peer(first_address.to_string()) .expect("Error adding reserved peer"); } network.run_until_all_full( - move |_index, service| service.get().network().num_connected() + move |_index, service| service.network().num_connected() == expected_full_connections, - move |_index, service| service.get().network().num_connected() + move |_index, service| service.network().num_connected() == expected_light_connections, ); @@ -392,23 +445,23 @@ pub fn connectivity( for i in 0..max_nodes { if i != 0 { if let Some((_, service, _, node_id)) = network.full_nodes.get(i) { - service.get().network().add_reserved_peer(address.to_string()) + service.network().add_reserved_peer(address.to_string()) .expect("Error adding reserved peer"); address = node_id.clone(); } } if let Some((_, service, node_id)) = network.light_nodes.get(i) { - service.get().network().add_reserved_peer(address.to_string()) + service.network().add_reserved_peer(address.to_string()) .expect("Error adding reserved peer"); address = node_id.clone(); } } network.run_until_all_full( - move |_index, service| service.get().network().num_connected() + move |_index, service| service.network().num_connected() == expected_full_connections, - move |_index, service| service.get().network().num_connected() + move |_index, service| service.network().num_connected() == expected_light_connections, ); } @@ -424,9 +477,9 @@ pub fn sync( mut extrinsic_factory: ExF ) where Fb: Fn(Configuration) -> Result<(F, U), Error>, - F: AbstractService, + F: TestNetNode, Lb: Fn(Configuration) -> Result, - L: AbstractService, + L: TestNetNode, B: FnMut(&F, &mut U), ExF: FnMut(&F, &U) -> ::Extrinsic, U: Clone + Send + 'static, @@ -456,39 +509,39 @@ pub fn sync( info!("Generating #{}", i + 1); } - make_block_and_import(&first_service.get(), first_user_data); + make_block_and_import(&first_service, first_user_data); } - (network.full_nodes[0].1).0.lock().unwrap().network().update_chain(); + network.full_nodes[0].1.network().update_chain(); network.full_nodes[0].3.clone() }; info!("Running sync"); for (_, service, _, _) in network.full_nodes.iter().skip(1) { - service.get().network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); + service.network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); } for (_, service, _) in network.light_nodes.iter() { - service.get().network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); + service.network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); } network.run_until_all_full( |_index, service| - service.get().client().info().best_number == (NUM_BLOCKS as u32).into(), + service.client().info().best_number == (NUM_BLOCKS as u32).into(), |_index, service| - service.get().client().info().best_number == (NUM_BLOCKS as u32).into(), + service.client().info().best_number == (NUM_BLOCKS as u32).into(), ); info!("Checking extrinsic propagation"); let first_service = network.full_nodes[0].1.clone(); let first_user_data = &network.full_nodes[0].2; - let best_block = BlockId::number(first_service.get().client().info().best_number); - let extrinsic = extrinsic_factory(&first_service.get(), first_user_data); + let best_block = BlockId::number(first_service.client().info().best_number); + let extrinsic = extrinsic_factory(&first_service, first_user_data); let source = sp_transaction_pool::TransactionSource::External; futures::executor::block_on( - first_service.get().transaction_pool().submit_one(&best_block, source, extrinsic) + first_service.transaction_pool().submit_one(&best_block, source, extrinsic) ).expect("failed to submit extrinsic"); network.run_until_all_full( - |_index, service| service.get().transaction_pool().ready().count() == 1, + |_index, service| service.transaction_pool().ready().count() == 1, |_index, _service| true, ); } @@ -500,9 +553,9 @@ pub fn consensus( authorities: impl IntoIterator ) where Fb: Fn(Configuration) -> Result, - F: AbstractService, + F: TestNetNode, Lb: Fn(Configuration) -> Result, - L: AbstractService, + L: TestNetNode, E: ChainSpecExtension + Clone + 'static + Send, G: RuntimeGenesis + 'static, { @@ -522,19 +575,19 @@ pub fn consensus( info!("Checking consensus"); let first_address = network.authority_nodes[0].3.clone(); for (_, service, _, _) in network.full_nodes.iter() { - service.get().network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); + service.network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); } for (_, service, _) in network.light_nodes.iter() { - service.get().network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); + service.network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); } for (_, service, _, _) in network.authority_nodes.iter().skip(1) { - service.get().network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); + service.network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); } network.run_until_all_full( |_index, service| - service.get().client().info().finalized_number >= (NUM_BLOCKS as u32 / 2).into(), + service.client().info().finalized_number >= (NUM_BLOCKS as u32 / 2).into(), |_index, service| - service.get().client().info().best_number >= (NUM_BLOCKS as u32 / 2).into(), + service.client().info().best_number >= (NUM_BLOCKS as u32 / 2).into(), ); info!("Adding more peers"); @@ -547,15 +600,15 @@ pub fn consensus( (0..0).map(|_| (String::new(), { |cfg| full_builder(cfg).map(|s| (s, ())) })), ); for (_, service, _, _) in network.full_nodes.iter() { - service.get().network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); + service.network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); } for (_, service, _) in network.light_nodes.iter() { - service.get().network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); + service.network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); } network.run_until_all_full( |_index, service| - service.get().client().info().finalized_number >= (NUM_BLOCKS as u32).into(), + service.client().info().finalized_number >= (NUM_BLOCKS as u32).into(), |_index, service| - service.get().client().info().best_number >= (NUM_BLOCKS as u32).into(), + service.client().info().best_number >= (NUM_BLOCKS as u32).into(), ); } From 937ad166fc9422f74636a8b312be3640f58ab1ee Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Tue, 16 Jun 2020 14:56:25 +0200 Subject: [PATCH 05/25] Fix node screenshot --- bin/node-template/node/src/service.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/bin/node-template/node/src/service.rs b/bin/node-template/node/src/service.rs index ef5766dc35bdd..35371edcb88f3 100644 --- a/bin/node-template/node/src/service.rs +++ b/bin/node-template/node/src/service.rs @@ -6,8 +6,8 @@ use sc_client_api::ExecutorProvider; use sc_consensus::LongestChain; use node_template_runtime::{self, opaque::Block, RuntimeApi}; use sc_service::{ - error::{Error as ServiceError}, - Configuration, ServiceBuilder, ChainComponents, + error::{Error as ServiceError}, Configuration, ServiceBuilder, ChainComponents, + KeepAliveChainComponents, }; use sp_inherents::InherentDataProviders; use sc_executor::native_executor_instance; @@ -16,7 +16,6 @@ use sp_consensus_aura::sr25519::{AuthorityPair as AuraPair}; use sc_finality_grandpa::{ FinalityProofProvider as GrandpaFinalityProofProvider, StorageAndProofProvider, SharedVoterState, }; -use sc_cli::KeepAliveChainComponents; // Our native executor instance. native_executor_instance!( @@ -205,7 +204,7 @@ pub fn new_full(config: Configuration) } Ok(KeepAliveChainComponents { - task_manager, telemetry, base_path, rpc, + task_manager, other: Box::new((telemetry, base_path, rpc)), }) } @@ -277,7 +276,7 @@ pub fn new_light(config: Configuration) -> Result Date: Thu, 18 Jun 2020 11:39:02 +0200 Subject: [PATCH 06/25] Line widths --- bin/node/cli/src/browser.rs | 4 ++-- bin/node/cli/src/service.rs | 4 +++- client/service/src/lib.rs | 22 ++++++++++++---------- 3 files changed, 17 insertions(+), 13 deletions(-) diff --git a/bin/node/cli/src/browser.rs b/bin/node/cli/src/browser.rs index ec0a5e8fcc206..7115e665c11a3 100644 --- a/bin/node/cli/src/browser.rs +++ b/bin/node/cli/src/browser.rs @@ -53,8 +53,8 @@ async fn start_inner(chain_spec: Option, log_level: String) -> Result; type FullBackend = sc_service::TFullBackend; -type GrandpaBlockImport = grandpa::GrandpaBlockImport>; +type GrandpaBlockImport = grandpa::GrandpaBlockImport< + FullBackend, Block, FullClient, sc_consensus::LongestChain +>; type BabeBlockImport = sc_consensus_babe::BabeBlockImport; /// Creates a full service from the configuration. diff --git a/client/service/src/lib.rs b/client/service/src/lib.rs index 667309adaa695..9894815a515c3 100644 --- a/client/service/src/lib.rs +++ b/client/service/src/lib.rs @@ -105,7 +105,8 @@ impl RpcHandlers { /// /// If the request subscribes you to events, the `Sender` in the `RpcSession` object is used to /// send back spontaneous events. - pub fn rpc_query(&self, mem: &RpcSession, request: &str) -> Pin> + Send>> { + pub fn rpc_query(&self, mem: &RpcSession, request: &str) + -> Pin> + Send>> { self.0.handle_request(request, mem.metadata.clone()) .compat() .map(|res| res.expect("this should never fail")) @@ -115,21 +116,22 @@ impl RpcHandlers { /// Sinks to propagate network status updates. /// For each element, every time the `Interval` fires we push an element on the sender. -pub struct NetworkStatusSinks { - inner: Arc, NetworkState)>>>, -} +pub struct NetworkStatusSinks( + Arc, NetworkState)>>>, +); impl NetworkStatusSinks { - fn new(sinks: Arc, NetworkState)>>>) -> Self { - Self { - inner: sinks, - } + fn new( + sinks: Arc, NetworkState)>>> + ) -> Self { + Self(sinks) } /// Returns a receiver that periodically receives a status of the network. - pub fn network_status(&self, interval: Duration) -> TracingUnboundedReceiver<(NetworkStatus, NetworkState)> { + pub fn network_status(&self, interval: Duration) + -> TracingUnboundedReceiver<(NetworkStatus, NetworkState)> { let (sink, stream) = tracing_unbounded("mpsc_network_status"); - self.inner.lock().push(interval, sink); + self.0.lock().push(interval, sink); stream } } From 5bcf9f63988f32fa64c2b81bb65be746c9221609 Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Thu, 18 Jun 2020 15:47:19 +0200 Subject: [PATCH 07/25] Fix node cli tests --- bin/node/cli/src/chain_spec.rs | 8 +- bin/node/cli/src/service.rs | 31 ++--- client/network/src/service/out_events.rs | 13 +-- client/service/src/builder.rs | 2 +- client/service/src/lib.rs | 138 +++++++++++------------ client/service/test/src/lib.rs | 38 +++++-- utils/browser/src/lib.rs | 4 +- 7 files changed, 116 insertions(+), 118 deletions(-) diff --git a/bin/node/cli/src/chain_spec.rs b/bin/node/cli/src/chain_spec.rs index 8617024255f28..384b866c2956a 100644 --- a/bin/node/cli/src/chain_spec.rs +++ b/bin/node/cli/src/chain_spec.rs @@ -431,12 +431,12 @@ pub(crate) mod tests { sc_service_test::connectivity( integration_test_config_with_two_authorities(), |config| { - let (keep_alive, _, client, network, transaction_pool) = new_full_base(config,|_, _| ())?; - Ok(sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool)) + let (keep_alive, _, rpc_handlers, client, network, transaction_pool) = new_full_base(config,|_, _| ())?; + Ok(sc_service_test::TestNetComponents::new(keep_alive, rpc_handlers, client, network, transaction_pool)) }, |config| { - let (keep_alive, _, client, network, transaction_pool) = new_light_base(config)?; - Ok(sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool)) + let (keep_alive, rpc_handlers, client, network, transaction_pool) = new_light_base(config)?; + Ok(sc_service_test::TestNetComponents::new(keep_alive, rpc_handlers, client, network, transaction_pool)) } ); } diff --git a/bin/node/cli/src/service.rs b/bin/node/cli/src/service.rs index 29b0d75398bdc..3e95bd3e5f9a8 100644 --- a/bin/node/cli/src/service.rs +++ b/bin/node/cli/src/service.rs @@ -167,7 +167,7 @@ pub fn new_full_base( ) -> Result<( KeepAliveChainComponents, InherentDataProviders, - Arc, Arc::Hash>>, + RpcHandlers, Arc, Arc::Hash>>, Arc, Block>> ), ServiceError> { let ( @@ -187,7 +187,7 @@ pub fn new_full_base( let ChainComponents { client, transaction_pool, task_manager, keystore, network, telemetry, base_path, - select_chain, prometheus_registry, rpc, telemetry_on_connect_sinks, .. + select_chain, prometheus_registry, rpc, telemetry_on_connect_sinks, rpc_handlers, .. } = builder .with_finality_proof_provider(|client, backend| { // GenesisAuthoritySetProvider is implemented for StorageAndProofProvider @@ -319,17 +319,19 @@ pub fn new_full_base( Ok(( KeepAliveChainComponents { - task_manager, other: Box::new((telemetry, base_path, rpc)), + task_manager, other: Box::new(( + telemetry, base_path, rpc, + )) }, inherent_data_providers, - client, network, transaction_pool + rpc_handlers, client, network, transaction_pool )) } /// Builds a new service for a full client. pub fn new_full(config: Configuration) -> Result { - new_full_base(config, |_, _| ()).map(|(keep_alive, _, _, _, _)| keep_alive) + new_full_base(config, |_, _| ()).map(|(keep_alive, _, _, _, _, _)| keep_alive) } type LightClient = sc_service::TLightClient; @@ -432,8 +434,7 @@ pub fn new_light_base(config: Configuration) -> Result<( KeepAliveChainComponents { task_manager, other: Box::new((telemetry, base_path, rpc)), }, - rpc_handlers, - client, network, transaction_pool + rpc_handlers, client, network, transaction_pool )) } @@ -504,19 +505,19 @@ mod tests { chain_spec, |config| { let mut setup_handles = None; - let (keep_alive, inherent_data_providers, client, network, transaction_pool) = new_full_base(config, | + let (keep_alive, inherent_data_providers, rpc_handlers, client, network, transaction_pool) = new_full_base(config, | block_import: &sc_consensus_babe::BabeBlockImport, babe_link: &sc_consensus_babe::BabeLink, | { setup_handles = Some((block_import.clone(), babe_link.clone())); })?; - let node = sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool); + let node = sc_service_test::TestNetComponents::new(keep_alive, rpc_handlers, client, network, transaction_pool); Ok((node, (inherent_data_providers, setup_handles.unwrap()))) }, |config| { - let (keep_alive, _, client, network, transaction_pool) = new_light_base(config)?; - Ok(sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool)) + let (keep_alive, rpc_handlers, client, network, transaction_pool) = new_light_base(config)?; + Ok(sc_service_test::TestNetComponents::new(keep_alive, rpc_handlers, client, network, transaction_pool)) }, |service, &mut (ref inherent_data_providers, (ref mut block_import, ref babe_link))| { let mut inherent_data = inherent_data_providers @@ -668,12 +669,12 @@ mod tests { sc_service_test::consensus( crate::chain_spec::tests::integration_test_config_with_two_authorities(), |config| { - let (keep_alive, _, client, network, transaction_pool) = new_full_base(config, |_, _| ())?; - Ok(sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool)) + let (keep_alive, _, rpc_handlers, client, network, transaction_pool) = new_full_base(config, |_, _| ())?; + Ok(sc_service_test::TestNetComponents::new(keep_alive, rpc_handlers, client, network, transaction_pool)) }, |config| { - let (keep_alive, _, client, network, transaction_pool) = new_light_base(config)?; - Ok(sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool)) + let (keep_alive, rpc_handlers, client, network, transaction_pool) = new_light_base(config)?; + Ok(sc_service_test::TestNetComponents::new(keep_alive, rpc_handlers, client, network, transaction_pool)) }, vec![ "//Alice".into(), diff --git a/client/network/src/service/out_events.rs b/client/network/src/service/out_events.rs index e77d2c43f0db1..4a631601a669e 100644 --- a/client/network/src/service/out_events.rs +++ b/client/network/src/service/out_events.rs @@ -35,7 +35,7 @@ use crate::Event; use super::maybe_utf8_bytes_to_string; -use futures::{prelude::*, channel::mpsc, ready, stream::FusedStream}; +use futures::{prelude::*, channel::mpsc, ready}; use parking_lot::Mutex; use prometheus_endpoint::{register, CounterVec, GaugeVec, Opts, PrometheusError, Registry, U64}; use std::{ @@ -120,16 +120,7 @@ impl fmt::Debug for Receiver { impl Drop for Receiver { fn drop(&mut self) { // Empty the list to properly decrease the metrics. - loop { - if self.inner.is_terminated() { - return; - } - - match self.next().now_or_never() { - Some(Some(_)) => {}, - _ => return, - } - } + while let Some(Some(_)) = self.next().now_or_never() {} } } diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index c16b01805afc4..2dc486308e0d4 100644 --- a/client/service/src/builder.rs +++ b/client/service/src/builder.rs @@ -17,7 +17,7 @@ // along with this program. If not, see . use crate::{ - NetworkStatus, NetworkState, error::Error, DEFAULT_PROTOCOL_ID, MallocSizeOfWasm, + NetworkStatus, NetworkState, error::Error, DEFAULT_PROTOCOL_ID, MallocSizeOfWasm, start_rpc_servers, build_network_future, TransactionPoolAdapter, TaskManager, SpawnTaskHandle, status_sinks, metrics::MetricsService, client::{light, Client, ClientConfig}, diff --git a/client/service/src/lib.rs b/client/service/src/lib.rs index 9894815a515c3..0c2327003e9da 100644 --- a/client/service/src/lib.rs +++ b/client/service/src/lib.rs @@ -44,7 +44,7 @@ use std::task::Poll; use parking_lot::Mutex; use client::Client; -use futures::{Future, FutureExt, Stream, StreamExt, stream::FusedStream, compat::*}; +use futures::{Future, FutureExt, Stream, StreamExt, compat::*}; use sc_network::{NetworkStatus, network_state::NetworkState, PeerId}; use log::{log, warn, debug, error, Level}; use codec::{Encode, Decode}; @@ -168,7 +168,7 @@ pub struct ChainComponents< /// The base path. pub base_path: Option>, /// RPC handlers that can perform RPC queries. - pub rpc_handlers: crate::RpcHandlers, + pub rpc_handlers: RpcHandlers, /// A shared instance of the chain selection algorithm. pub select_chain: Option, /// Sinks to propagate network status updates. @@ -234,81 +234,71 @@ fn build_network_future< } // Poll the RPC requests and answer them. - loop { - if rpc_rx.is_terminated() { - return Poll::Ready(()); - } - - match Pin::new(&mut rpc_rx).poll_next(cx) { - Poll::Ready(Some(request)) => { - match request { - sc_rpc::system::Request::Health(sender) => { - let _ = sender.send(sc_rpc::system::Health { - peers: network.peers_debug_info().len(), - is_syncing: network.service().is_major_syncing(), - should_have_peers, - }); - }, - sc_rpc::system::Request::LocalPeerId(sender) => { - let _ = sender.send(network.local_peer_id().to_base58()); - }, - sc_rpc::system::Request::LocalListenAddresses(sender) => { - let peer_id = network.local_peer_id().clone().into(); - let p2p_proto_suffix = sc_network::multiaddr::Protocol::P2p(peer_id); - let addresses = network.listen_addresses() - .map(|addr| addr.clone().with(p2p_proto_suffix.clone()).to_string()) - .collect(); - let _ = sender.send(addresses); - }, - sc_rpc::system::Request::Peers(sender) => { - let _ = sender.send(network.peers_debug_info().into_iter().map(|(peer_id, p)| - sc_rpc::system::PeerInfo { - peer_id: peer_id.to_base58(), - roles: format!("{:?}", p.roles), - protocol_version: p.protocol_version, - best_hash: p.best_hash, - best_number: p.best_number, - } - ).collect()); - } - sc_rpc::system::Request::NetworkState(sender) => { - if let Some(network_state) = serde_json::to_value(&network.network_state()).ok() { - let _ = sender.send(network_state); - } - } - sc_rpc::system::Request::NetworkAddReservedPeer(peer_addr, sender) => { - let x = network.add_reserved_peer(peer_addr) - .map_err(sc_rpc::system::error::Error::MalformattedPeerArg); - let _ = sender.send(x); - } - sc_rpc::system::Request::NetworkRemoveReservedPeer(peer_id, sender) => { - let _ = match peer_id.parse::() { - Ok(peer_id) => { - network.remove_reserved_peer(peer_id); - sender.send(Ok(())) - } - Err(e) => sender.send(Err(sc_rpc::system::error::Error::MalformattedPeerArg( - e.to_string(), - ))), - }; + while let Poll::Ready(Some(request)) = Pin::new(&mut rpc_rx).poll_next(cx) { + match request { + sc_rpc::system::Request::Health(sender) => { + let _ = sender.send(sc_rpc::system::Health { + peers: network.peers_debug_info().len(), + is_syncing: network.service().is_major_syncing(), + should_have_peers, + }); + }, + sc_rpc::system::Request::LocalPeerId(sender) => { + let _ = sender.send(network.local_peer_id().to_base58()); + }, + sc_rpc::system::Request::LocalListenAddresses(sender) => { + let peer_id = network.local_peer_id().clone().into(); + let p2p_proto_suffix = sc_network::multiaddr::Protocol::P2p(peer_id); + let addresses = network.listen_addresses() + .map(|addr| addr.clone().with(p2p_proto_suffix.clone()).to_string()) + .collect(); + let _ = sender.send(addresses); + }, + sc_rpc::system::Request::Peers(sender) => { + let _ = sender.send(network.peers_debug_info().into_iter().map(|(peer_id, p)| + sc_rpc::system::PeerInfo { + peer_id: peer_id.to_base58(), + roles: format!("{:?}", p.roles), + protocol_version: p.protocol_version, + best_hash: p.best_hash, + best_number: p.best_number, } - sc_rpc::system::Request::NodeRoles(sender) => { - use sc_rpc::system::NodeRole; - - let node_role = match role { - Role::Authority { .. } => NodeRole::Authority, - Role::Light => NodeRole::LightClient, - Role::Full => NodeRole::Full, - Role::Sentry { .. } => NodeRole::Sentry, - }; - - let _ = sender.send(vec![node_role]); + ).collect()); + } + sc_rpc::system::Request::NetworkState(sender) => { + if let Some(network_state) = serde_json::to_value(&network.network_state()).ok() { + let _ = sender.send(network_state); + } + } + sc_rpc::system::Request::NetworkAddReservedPeer(peer_addr, sender) => { + let x = network.add_reserved_peer(peer_addr) + .map_err(sc_rpc::system::error::Error::MalformattedPeerArg); + let _ = sender.send(x); + } + sc_rpc::system::Request::NetworkRemoveReservedPeer(peer_id, sender) => { + let _ = match peer_id.parse::() { + Ok(peer_id) => { + network.remove_reserved_peer(peer_id); + sender.send(Ok(())) } + Err(e) => sender.send(Err(sc_rpc::system::error::Error::MalformattedPeerArg( + e.to_string(), + ))), }; - }, - Poll::Ready(None) => return Poll::Ready(()), - Poll::Pending => break, - } + } + sc_rpc::system::Request::NodeRoles(sender) => { + use sc_rpc::system::NodeRole; + + let node_role = match role { + Role::Authority { .. } => NodeRole::Authority, + Role::Light => NodeRole::LightClient, + Role::Full => NodeRole::Full, + Role::Sentry { .. } => NodeRole::Sentry, + }; + + let _ = sender.send(vec![node_role]); + } + }; } // Interval report for the external API. diff --git a/client/service/test/src/lib.rs b/client/service/test/src/lib.rs index ca553c32fed78..6ab6f64acfb26 100644 --- a/client/service/test/src/lib.rs +++ b/client/service/test/src/lib.rs @@ -19,11 +19,11 @@ //! Service integration test utils. use std::iter; -use std::sync::{Arc}; +use std::sync::Arc; use std::net::Ipv4Addr; use std::pin::Pin; use std::time::Duration; -use log::info; +use log::{info, debug}; use futures01::{Future, Stream, Poll}; use futures::{FutureExt as _, TryFutureExt as _}; use tempfile::TempDir; @@ -40,6 +40,7 @@ use sc_service::{ Error, TaskType, client::Client, + RpcHandlers, }; use sp_blockchain::HeaderBackend; use sc_network::{multiaddr, Multiaddr}; @@ -49,7 +50,6 @@ use sp_transaction_pool::TransactionPool; use sc_client_api::{Backend, CallExecutor}; use parking_lot::Mutex; - #[cfg(test)] mod client; @@ -80,6 +80,7 @@ pub trait TestNetNode: Clone + Future + Se pub struct TestNetComponents { keep_alive: Arc>, + rpc_handlers: Arc, client: Arc>, transaction_pool: Arc, network: Arc::Hash>>, @@ -89,12 +90,14 @@ impl TestNetComponents { pub fn new( keep_alive: KeepAliveChainComponents, + rpc_handlers: RpcHandlers, client: Arc>, network: Arc::Hash>>, transaction_pool: Arc, ) -> Self { Self { client, transaction_pool, network, + rpc_handlers: Arc::new(rpc_handlers), keep_alive: Arc::new(Mutex::new(keep_alive)), } } @@ -109,6 +112,7 @@ TestNetComponents { client: self.client.clone(), transaction_pool: self.transaction_pool.clone(), network: self.network.clone(), + rpc_handlers: self.rpc_handlers.clone() } } } @@ -414,10 +418,16 @@ pub fn connectivity( } network.run_until_all_full( - move |_index, service| service.network().num_connected() - == expected_full_connections, - move |_index, service| service.network().num_connected() - == expected_light_connections, + move |_index, service| { + let connected = service.network().num_connected(); + debug!("Got {}/{} full connections...", connected, expected_full_connections); + connected == expected_full_connections + }, + move |_index, service| { + let connected = service.network().num_connected(); + debug!("Got {}/{} light connections...", connected, expected_light_connections); + connected == expected_light_connections + }, ); network.runtime @@ -460,10 +470,16 @@ pub fn connectivity( } network.run_until_all_full( - move |_index, service| service.network().num_connected() - == expected_full_connections, - move |_index, service| service.network().num_connected() - == expected_light_connections, + move |_index, service| { + let connected = service.network().num_connected(); + debug!("Got {}/{} full connections...", connected, expected_full_connections); + connected == expected_full_connections + }, + move |_index, service| { + let connected = service.network().num_connected(); + debug!("Got {}/{} light connections...", connected, expected_light_connections); + connected == expected_light_connections + }, ); } temp.close().expect("Error removing temp dir"); diff --git a/utils/browser/src/lib.rs b/utils/browser/src/lib.rs index d66447a59a868..a0901cd2b1b6b 100644 --- a/utils/browser/src/lib.rs +++ b/utils/browser/src/lib.rs @@ -20,9 +20,9 @@ use log::{debug, info}; use std::sync::Arc; use sc_network::config::TransportConfig; use sc_service::{ - RpcSession, Role, Configuration, + RpcSession, Role, Configuration, TaskManager, RpcHandlers, config::{DatabaseConfig, KeystoreConfig, NetworkConfiguration}, - GenericChainSpec, RuntimeGenesis, TaskManager, RpcHandlers, + GenericChainSpec, RuntimeGenesis }; use wasm_bindgen::prelude::*; use futures::{prelude::*, channel::{oneshot, mpsc}, future::{poll_fn, ok}, compat::*}; From 01e3ea013c7c846892c88cdf93994f2ceb94e07b Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Thu, 18 Jun 2020 16:08:56 +0200 Subject: [PATCH 08/25] Fix node cli warning --- bin/node/cli/src/browser.rs | 4 +++- bin/node/cli/src/service.rs | 5 ----- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/bin/node/cli/src/browser.rs b/bin/node/cli/src/browser.rs index 7115e665c11a3..048a3d0104de1 100644 --- a/bin/node/cli/src/browser.rs +++ b/bin/node/cli/src/browser.rs @@ -54,7 +54,9 @@ async fn start_inner(chain_spec: Option, log_level: String) -> Result Result Result<(KeepAliveChainComponents, RpcHandlers), ServiceError> { - new_light_base(config).map(|(components, rpc_handlers, _, _, _)| (components, rpc_handlers)) -} - #[cfg(test)] mod tests { use std::{sync::Arc, borrow::Cow, any::Any}; From 12904520e2871c03a177758c2cff2711bdefcca8 Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Thu, 18 Jun 2020 18:21:51 +0200 Subject: [PATCH 09/25] ChainComponents -> ServiceComponents, fix tests --- bin/node-template/node/src/service.rs | 16 +++++++-------- bin/node/cli/src/browser.rs | 4 ++-- bin/node/cli/src/service.rs | 28 ++++++++++++++++----------- client/cli/src/runner.rs | 20 +++++++++---------- client/service/src/builder.rs | 10 +++++----- client/service/src/lib.rs | 4 ++-- client/service/test/src/lib.rs | 6 +++--- 7 files changed, 47 insertions(+), 41 deletions(-) diff --git a/bin/node-template/node/src/service.rs b/bin/node-template/node/src/service.rs index 35371edcb88f3..0d4a45d5c102c 100644 --- a/bin/node-template/node/src/service.rs +++ b/bin/node-template/node/src/service.rs @@ -6,8 +6,8 @@ use sc_client_api::ExecutorProvider; use sc_consensus::LongestChain; use node_template_runtime::{self, opaque::Block, RuntimeApi}; use sc_service::{ - error::{Error as ServiceError}, Configuration, ServiceBuilder, ChainComponents, - KeepAliveChainComponents, + error::{Error as ServiceError}, Configuration, ServiceBuilder, ServiceComponents, + KeepAliveServiceComponents, }; use sp_inherents::InherentDataProviders; use sc_executor::native_executor_instance; @@ -97,7 +97,7 @@ macro_rules! new_full_start { /// Builds a new service for a full client. pub fn new_full(config: Configuration) --> Result +-> Result { let role = config.role.clone(); let force_authoring = config.force_authoring; @@ -110,7 +110,7 @@ pub fn new_full(config: Configuration) import_setup.take() .expect("Link Half and Block Import are present for Full Services or setup failed before. qed"); - let ChainComponents { + let ServiceComponents { client, transaction_pool, task_manager, keystore, network, telemetry, base_path, select_chain, prometheus_registry, telemetry_on_connect_sinks, rpc, .. } = builder @@ -203,13 +203,13 @@ pub fn new_full(config: Configuration) )?; } - Ok(KeepAliveChainComponents { + Ok(KeepAliveServiceComponents { task_manager, other: Box::new((telemetry, base_path, rpc)), }) } /// Builds a new service for a light client. -pub fn new_light(config: Configuration) -> Result { +pub fn new_light(config: Configuration) -> Result { let inherent_data_providers = InherentDataProviders::new(); ServiceBuilder::new_light::(config)? @@ -274,8 +274,8 @@ pub fn new_light(config: Configuration) -> Result, log_level: String) -> Result) ) -> Result<( - KeepAliveChainComponents, + KeepAliveServiceComponents, InherentDataProviders, RpcHandlers, Arc, Arc::Hash>>, Arc, Block>> @@ -185,7 +185,7 @@ pub fn new_full_base( let (builder, mut import_setup, inherent_data_providers, mut rpc_setup) = new_full_start!(config); - let ChainComponents { + let ServiceComponents { client, transaction_pool, task_manager, keystore, network, telemetry, base_path, select_chain, prometheus_registry, rpc, telemetry_on_connect_sinks, rpc_handlers, .. } = builder @@ -318,7 +318,7 @@ pub fn new_full_base( } Ok(( - KeepAliveChainComponents { + KeepAliveServiceComponents { task_manager, other: Box::new(( telemetry, base_path, rpc, )) @@ -330,21 +330,24 @@ pub fn new_full_base( /// Builds a new service for a full client. pub fn new_full(config: Configuration) --> Result { - new_full_base(config, |_, _| ()).map(|(keep_alive, _, _, _, _, _)| keep_alive) +-> Result { + new_full_base(config, |_, _| ()).map(|(mut keep_alive, rpc_handlers, _, _, _, _)| { + keep_alive.other = Box::new((keep_alive.other, rpc_handlers)); + keep_alive + }) } type LightClient = sc_service::TLightClient; type LightFetcher = sc_network::config::OnDemand; pub fn new_light_base(config: Configuration) -> Result<( - KeepAliveChainComponents, RpcHandlers, Arc, + KeepAliveServiceComponents, RpcHandlers, Arc, Arc::Hash>>, Arc, Block>> ), ServiceError> { let inherent_data_providers = InherentDataProviders::new(); - let ChainComponents { + let ServiceComponents { task_manager, telemetry, base_path, rpc_handlers, rpc, client, network, transaction_pool, .. } = ServiceBuilder::new_light::(config)? .with_select_chain(|_config, backend| { @@ -431,7 +434,7 @@ pub fn new_light_base(config: Configuration) -> Result<( .build_light()?; Ok(( - KeepAliveChainComponents { + KeepAliveServiceComponents { task_manager, other: Box::new((telemetry, base_path, rpc)), }, rpc_handlers, client, network, transaction_pool @@ -439,8 +442,11 @@ pub fn new_light_base(config: Configuration) -> Result<( } /// Builds a new service for a light client. -pub fn new_light(config: Configuration) -> Result { - new_light_base(config).map(|(components, _, _, _, _)| components) +pub fn new_light(config: Configuration) -> Result { + new_light_base(config).map(|(mut keep_alive, rpc_handlers, _, _, _)| { + keep_alive.other = Box::new((keep_alive.other, rpc_handlers)); + keep_alive + }) } #[cfg(test)] diff --git a/client/cli/src/runner.rs b/client/cli/src/runner.rs index 463d36d37aaa7..96bf77867986d 100644 --- a/client/cli/src/runner.rs +++ b/client/cli/src/runner.rs @@ -25,7 +25,7 @@ use futures::pin_mut; use futures::select; use futures::{future, future::FutureExt, Future}; use log::info; -use sc_service::{Configuration, Role, ServiceBuilderCommand, TaskType, KeepAliveChainComponents}; +use sc_service::{Configuration, Role, ServiceBuilderCommand, TaskType, KeepAliveServiceComponents}; use sp_runtime::traits::{Block as BlockT, Header as HeaderT}; use sp_utils::metrics::{TOKIO_THREADS_ALIVE, TOKIO_THREADS_TOTAL}; use sp_version::RuntimeVersion; @@ -174,13 +174,13 @@ impl Runner { info!("⛓ Native runtime: {}", runtime_version); } - /// A helper function that runs an `AbstractService` with tokio and stops if the process + /// A helper function that runs a node with tokio and stops if the process /// receives the signal `SIGTERM` or `SIGINT`. It can run a full or a light node depending on /// the node's configuration. pub fn run_node( self, - new_light: impl FnOnce(Configuration) -> sc_service::error::Result, - new_full: impl FnOnce(Configuration) -> sc_service::error::Result, + new_light: impl FnOnce(Configuration) -> sc_service::error::Result, + new_full: impl FnOnce(Configuration) -> sc_service::error::Result, runtime_version: RuntimeVersion, ) -> Result<()> { match self.config.role { @@ -189,12 +189,12 @@ impl Runner { } } - /// A helper function that runs an `AbstractService` with tokio and stops if the process + /// A helper function that runs a node with tokio and stops if the process /// receives the signal `SIGTERM` or `SIGINT`. It can only run a "full" node and will fail if /// the node's configuration uses a "light" role. pub fn run_full_node( self, - new_full: impl FnOnce(Configuration) -> sc_service::error::Result, + new_full: impl FnOnce(Configuration) -> sc_service::error::Result, runtime_version: RuntimeVersion, ) -> Result<()> { if matches!(self.config.role, Role::Light) { @@ -205,12 +205,12 @@ impl Runner { self.run_service_until_exit(new_full) } - /// A helper function that runs an `AbstractService` with tokio and stops if the process + /// A helper function that runs a node with tokio and stops if the process /// receives the signal `SIGTERM` or `SIGINT`. It can only run a "light" node and will fail if /// the node's configuration uses a "full" role. pub fn run_light_node( self, - new_light: impl FnOnce(Configuration) -> sc_service::error::Result, + new_light: impl FnOnce(Configuration) -> sc_service::error::Result, runtime_version: RuntimeVersion, ) -> Result<()> { if !matches!(self.config.role, Role::Light) { @@ -251,9 +251,9 @@ impl Runner { fn run_service_until_exit( mut self, - initialise: impl FnOnce(Configuration) -> sc_service::error::Result, + initialise: impl FnOnce(Configuration) -> sc_service::error::Result, ) -> Result<()> { - let KeepAliveChainComponents { + let KeepAliveServiceComponents { task_manager, // We need to keep a hold of these until the tasks are finished. other: _other, diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index 2dc486308e0d4..c073bae240b7c 100644 --- a/client/service/src/builder.rs +++ b/client/service/src/builder.rs @@ -64,7 +64,7 @@ use sp_core::traits::CodeExecutor; use sp_runtime::BuildStorage; use sc_client_api::execution_extensions::ExecutionExtensions; use sp_core::storage::Storage; -use crate::{ChainComponents, TelemetryOnConnectSinks, RpcHandlers, NetworkStatusSinks}; +use crate::{ServiceComponents, TelemetryOnConnectSinks, RpcHandlers, NetworkStatusSinks}; pub type BackgroundTask = Pin + Send>>; @@ -960,7 +960,7 @@ ServiceBuilder< Ok(self) } - fn build_common(self) -> Result, Error> + fn build_common(self) -> Result, Error> where TExec: CallExecutor, { let ServiceBuilder { @@ -1130,7 +1130,7 @@ ServiceBuilder< sc_informant::OutputFormat { enable_color: true, prefix: informant_prefix }, )); - Ok(ChainComponents { + Ok(ServiceComponents { client, task_manager, network, @@ -1149,7 +1149,7 @@ ServiceBuilder< } /// Builds the light service. - pub fn build_light(self) -> Result, Error> + pub fn build_light(self) -> Result, Error> where TExec: CallExecutor, { self.build_common() @@ -1192,7 +1192,7 @@ ServiceBuilder< { /// Builds the full service. - pub fn build_full(self) -> Result, Error> + pub fn build_full(self) -> Result, Error> where TExec: CallExecutor, { // make transaction pool available for off-chain runtime calls. diff --git a/client/service/src/lib.rs b/client/service/src/lib.rs index 0c2327003e9da..8eb115849a3fe 100644 --- a/client/service/src/lib.rs +++ b/client/service/src/lib.rs @@ -150,7 +150,7 @@ impl TelemetryOnConnectSinks { /// The individual components of the chain, built by the service builder. You are encouraged to /// deconstruct this into its fields. -pub struct ChainComponents< +pub struct ServiceComponents< TBl: BlockT, TBackend: sc_client_api::backend::Backend, TExec, TRtApi, TSc, TExPool, > { /// A blockchain client. @@ -188,7 +188,7 @@ pub struct ChainComponents< } /// The components of the chain that we need to keep alive until the node quits. -pub struct KeepAliveChainComponents { +pub struct KeepAliveServiceComponents { /// The chain task manager. pub task_manager: TaskManager, /// Everything else. diff --git a/client/service/test/src/lib.rs b/client/service/test/src/lib.rs index 6ab6f64acfb26..6b8370aefcafd 100644 --- a/client/service/test/src/lib.rs +++ b/client/service/test/src/lib.rs @@ -30,7 +30,7 @@ use tempfile::TempDir; use tokio::{runtime::Runtime, prelude::FutureExt}; use tokio::timer::Interval; use sc_service::{ - KeepAliveChainComponents, + KeepAliveServiceComponents, GenericChainSpec, ChainSpecExtension, Configuration, @@ -79,7 +79,7 @@ pub trait TestNetNode: Clone + Future + Se } pub struct TestNetComponents { - keep_alive: Arc>, + keep_alive: Arc>, rpc_handlers: Arc, client: Arc>, transaction_pool: Arc, @@ -89,7 +89,7 @@ pub struct TestNetComponents { impl TestNetComponents { pub fn new( - keep_alive: KeepAliveChainComponents, + keep_alive: KeepAliveServiceComponents, rpc_handlers: RpcHandlers, client: Arc>, network: Arc::Hash>>, From 11e32039a7e77ec9644fb204feba2bbc2c013059 Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Fri, 19 Jun 2020 12:40:23 +0200 Subject: [PATCH 10/25] make spawn_handle public --- client/service/src/task_manager.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/client/service/src/task_manager.rs b/client/service/src/task_manager.rs index e1430ffc325fe..f48594924be57 100644 --- a/client/service/src/task_manager.rs +++ b/client/service/src/task_manager.rs @@ -229,7 +229,8 @@ impl TaskManager { self.spawn_handle().spawn_blocking(name, task) } - pub(super) fn spawn_handle(&self) -> SpawnTaskHandle { + /// Get a handle for spawning tasks. + pub fn spawn_handle(&self) -> SpawnTaskHandle { SpawnTaskHandle { on_exit: self.on_exit.clone(), executor: self.executor.clone(), From 55df8af29ef7650aed4a777372208eb9bb35b43a Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Mon, 22 Jun 2020 11:44:16 +0200 Subject: [PATCH 11/25] Remove spawnnamed impl for taskmanager --- client/service/src/task_manager.rs | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/client/service/src/task_manager.rs b/client/service/src/task_manager.rs index 66041a46cd257..268e7164a3e37 100644 --- a/client/service/src/task_manager.rs +++ b/client/service/src/task_manager.rs @@ -265,17 +265,6 @@ impl TaskManager { } } -impl sp_core::traits::SpawnNamed for TaskManager { - fn spawn_blocking(&self, name: &'static str, future: BoxFuture<'static, ()>) { - self.spawn_blocking(name, future); - } - - fn spawn(&self, name: &'static str, future: BoxFuture<'static, ()>) { - self.spawn(name, future); - } -} - - impl Future for TaskManager { type Output = Result<(), crate::Error>; From 4e11e59a993f9e8b7bd4cc99b276da379f0ede77 Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Tue, 23 Jun 2020 15:16:40 +0200 Subject: [PATCH 12/25] Move the keep alive stuff to the task manager --- bin/node-template/node/src/service.rs | 23 ++++++------- bin/node/cli/src/browser.rs | 3 +- bin/node/cli/src/service.rs | 48 +++++++++++---------------- client/cli/src/runner.rs | 35 ++++++------------- client/service/src/lib.rs | 8 ----- client/service/src/task_manager.rs | 43 +++++++++++++----------- client/service/test/src/lib.rs | 12 +++---- utils/browser/src/lib.rs | 2 +- 8 files changed, 72 insertions(+), 102 deletions(-) diff --git a/bin/node-template/node/src/service.rs b/bin/node-template/node/src/service.rs index 66c0dcd0d5fa6..b5b6e5835fcd0 100644 --- a/bin/node-template/node/src/service.rs +++ b/bin/node-template/node/src/service.rs @@ -7,7 +7,7 @@ use sc_consensus::LongestChain; use node_template_runtime::{self, opaque::Block, RuntimeApi}; use sc_service::{ error::{Error as ServiceError}, Configuration, ServiceBuilder, ServiceComponents, - KeepAliveServiceComponents, + TaskManager, }; use sp_inherents::InherentDataProviders; use sc_executor::native_executor_instance; @@ -96,9 +96,7 @@ macro_rules! new_full_start { } /// Builds a new service for a full client. -pub fn new_full(config: Configuration) --> Result -{ +pub fn new_full(config: Configuration) -> Result { let role = config.role.clone(); let force_authoring = config.force_authoring; let name = config.network.node_name.clone(); @@ -111,7 +109,7 @@ pub fn new_full(config: Configuration) .expect("Link Half and Block Import are present for Full Services or setup failed before. qed"); let ServiceComponents { - client, transaction_pool, task_manager, keystore, network, telemetry, base_path, + client, transaction_pool, mut task_manager, keystore, network, telemetry, base_path, select_chain, prometheus_registry, telemetry_on_connect_sinks, rpc, .. } = builder .with_finality_proof_provider(|client, backend| { @@ -203,13 +201,13 @@ pub fn new_full(config: Configuration) )?; } - Ok(KeepAliveServiceComponents { - task_manager, other: Box::new((telemetry, base_path, rpc)), - }) + task_manager.keep_alive((telemetry, base_path, rpc)); + + Ok(task_manager) } /// Builds a new service for a light client. -pub fn new_light(config: Configuration) -> Result { +pub fn new_light(config: Configuration) -> Result { let inherent_data_providers = InherentDataProviders::new(); ServiceBuilder::new_light::(config)? @@ -274,9 +272,8 @@ pub fn new_light(config: Configuration) -> Result, log_level: String) -> Result) ) -> Result<( - KeepAliveServiceComponents, + TaskManager, InherentDataProviders, RpcHandlers, Arc, Arc::Hash>>, Arc, Block>> @@ -186,7 +186,7 @@ pub fn new_full_base( new_full_start!(config); let ServiceComponents { - client, transaction_pool, task_manager, keystore, network, telemetry, base_path, + client, transaction_pool, mut task_manager, keystore, network, telemetry, base_path, select_chain, prometheus_registry, rpc, telemetry_on_connect_sinks, rpc_handlers, .. } = builder .with_finality_proof_provider(|client, backend| { @@ -317,23 +317,17 @@ pub fn new_full_base( )?; } - Ok(( - KeepAliveServiceComponents { - task_manager, other: Box::new(( - telemetry, base_path, rpc, - )) - }, - inherent_data_providers, - rpc_handlers, client, network, transaction_pool - )) + task_manager.keep_alive((telemetry, base_path, rpc)); + + Ok((task_manager, inherent_data_providers, rpc_handlers, client, network, transaction_pool)) } /// Builds a new service for a full client. pub fn new_full(config: Configuration) --> Result { - new_full_base(config, |_, _| ()).map(|(mut keep_alive, rpc_handlers, _, _, _, _)| { - keep_alive.other = Box::new((keep_alive.other, rpc_handlers)); - keep_alive +-> Result { + new_full_base(config, |_, _| ()).map(|(mut task_manager, rpc_handlers, _, _, _, _)| { + task_manager.keep_alive(rpc_handlers); + task_manager }) } @@ -341,14 +335,15 @@ type LightClient = sc_service::TLightClient; pub fn new_light_base(config: Configuration) -> Result<( - KeepAliveServiceComponents, RpcHandlers, Arc, + TaskManager, RpcHandlers, Arc, Arc::Hash>>, Arc, Block>> ), ServiceError> { let inherent_data_providers = InherentDataProviders::new(); let ServiceComponents { - task_manager, telemetry, base_path, rpc_handlers, rpc, client, network, transaction_pool, .. + mut task_manager, telemetry, base_path, rpc_handlers, rpc, client, network, + transaction_pool, .. } = ServiceBuilder::new_light::(config)? .with_select_chain(|_config, backend| { Ok(LongestChain::new(backend.clone())) @@ -433,19 +428,16 @@ pub fn new_light_base(config: Configuration) -> Result<( })? .build_light()?; - Ok(( - KeepAliveServiceComponents { - task_manager, other: Box::new((telemetry, base_path, rpc)), - }, - rpc_handlers, client, network, transaction_pool - )) + task_manager.keep_alive((telemetry, base_path, rpc)); + + Ok((task_manager, rpc_handlers, client, network, transaction_pool)) } /// Builds a new service for a light client. -pub fn new_light(config: Configuration) -> Result { - new_light_base(config).map(|(mut keep_alive, rpc_handlers, _, _, _)| { - keep_alive.other = Box::new((keep_alive.other, rpc_handlers)); - keep_alive +pub fn new_light(config: Configuration) -> Result { + new_light_base(config).map(|(mut task_manager, rpc_handlers, _, _, _)| { + task_manager.keep_alive(rpc_handlers); + task_manager }) } diff --git a/client/cli/src/runner.rs b/client/cli/src/runner.rs index 96bf77867986d..edbc2e5dc4705 100644 --- a/client/cli/src/runner.rs +++ b/client/cli/src/runner.rs @@ -25,7 +25,7 @@ use futures::pin_mut; use futures::select; use futures::{future, future::FutureExt, Future}; use log::info; -use sc_service::{Configuration, Role, ServiceBuilderCommand, TaskType, KeepAliveServiceComponents}; +use sc_service::{Configuration, Role, ServiceBuilderCommand, TaskType, TaskManager}; use sp_runtime::traits::{Block as BlockT, Header as HeaderT}; use sp_utils::metrics::{TOKIO_THREADS_ALIVE, TOKIO_THREADS_TOTAL}; use sp_version::RuntimeVersion; @@ -179,8 +179,8 @@ impl Runner { /// the node's configuration. pub fn run_node( self, - new_light: impl FnOnce(Configuration) -> sc_service::error::Result, - new_full: impl FnOnce(Configuration) -> sc_service::error::Result, + new_light: impl FnOnce(Configuration) -> sc_service::error::Result, + new_full: impl FnOnce(Configuration) -> sc_service::error::Result, runtime_version: RuntimeVersion, ) -> Result<()> { match self.config.role { @@ -194,7 +194,7 @@ impl Runner { /// the node's configuration uses a "light" role. pub fn run_full_node( self, - new_full: impl FnOnce(Configuration) -> sc_service::error::Result, + new_full: impl FnOnce(Configuration) -> sc_service::error::Result, runtime_version: RuntimeVersion, ) -> Result<()> { if matches!(self.config.role, Role::Light) { @@ -210,7 +210,7 @@ impl Runner { /// the node's configuration uses a "full" role. pub fn run_light_node( self, - new_light: impl FnOnce(Configuration) -> sc_service::error::Result, + new_light: impl FnOnce(Configuration) -> sc_service::error::Result, runtime_version: RuntimeVersion, ) -> Result<()> { if !matches!(self.config.role, Role::Light) { @@ -251,26 +251,13 @@ impl Runner { fn run_service_until_exit( mut self, - initialise: impl FnOnce(Configuration) -> sc_service::error::Result, + initialise: impl FnOnce(Configuration) -> sc_service::error::Result, ) -> Result<()> { - let KeepAliveServiceComponents { - task_manager, - // We need to keep a hold of these until the tasks are finished. - other: _other, - } = initialise(self.config)?; - - { - let f = task_manager.fuse(); - self.tokio_runtime - .block_on(main(f)) - .map_err(|e| e.to_string())?; - } - - // The `service` **must** have been destroyed here for the shutdown signal to propagate - // to all the tasks. Dropping `tokio_runtime` will block the thread until all tasks have - // shut down. - drop(self.tokio_runtime); - + let mut task_manager = initialise(self.config)?; + self.tokio_runtime.block_on(main(task_manager.future().fuse())) + .map_err(|e| e.to_string())?; + task_manager.terminate(); + drop(task_manager); Ok(()) } diff --git a/client/service/src/lib.rs b/client/service/src/lib.rs index 77b09fe7bfc2e..57e59a0d04b08 100644 --- a/client/service/src/lib.rs +++ b/client/service/src/lib.rs @@ -187,14 +187,6 @@ pub struct ServiceComponents< >>>, } -/// The components of the chain that we need to keep alive until the node quits. -pub struct KeepAliveServiceComponents { - /// The chain task manager. - pub task_manager: TaskManager, - /// Everything else. - pub other: Box, -} - /// Builds a never-ending future that continuously polls the network. /// /// The `status_sink` contain a list of senders to send a periodic network status to. diff --git a/client/service/src/task_manager.rs b/client/service/src/task_manager.rs index 268e7164a3e37..40249cf6cd697 100644 --- a/client/service/src/task_manager.rs +++ b/client/service/src/task_manager.rs @@ -13,11 +13,11 @@ //! Substrate service tasks management module. -use std::{panic, pin::Pin, result::Result, sync::Arc, task::{Poll, Context}}; +use std::{panic, pin::Pin, result::Result, sync::Arc}; use exit_future::Signal; use log::debug; use futures::{ - Future, FutureExt, Stream, + Future, FutureExt, StreamExt, future::{select, Either, BoxFuture}, compat::*, task::{Spawn, FutureObj, SpawnError}, @@ -29,7 +29,7 @@ use prometheus_endpoint::{ CounterVec, HistogramOpts, HistogramVec, Opts, Registry, U64 }; use sc_client_api::CloneableSpawn; -use crate::config::TaskType; +use crate::{config::TaskType, Error}; use sp_utils::mpsc::{TracingUnboundedSender, TracingUnboundedReceiver, tracing_unbounded}; mod prometheus_future; @@ -224,6 +224,8 @@ pub struct TaskManager { essential_failed_tx: TracingUnboundedSender<()>, /// A receiver for spawned essential-tasks concluding. essential_failed_rx: TracingUnboundedReceiver<()>, + /// Things to keep alive until the task manager is dropped. + keep_alive: Box, } impl TaskManager { @@ -246,6 +248,7 @@ impl TaskManager { metrics, essential_failed_tx, essential_failed_rx, + keep_alive: Box::new(()), }) } @@ -263,34 +266,34 @@ impl TaskManager { pub fn spawn_essential_handle(&self) -> SpawnEssentialTaskHandle { SpawnEssentialTaskHandle::new(self.essential_failed_tx.clone(), self.spawn_handle()) } -} -impl Future for TaskManager { - type Output = Result<(), crate::Error>; + /// Return a future that will end if an essential task fails. + pub fn future<'a>(&'a mut self) -> Pin> + Send + 'a>> { + Box::pin(async move { + self.essential_failed_rx.next().await; - fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { - let this = Pin::into_inner(self); + Err(Error::Other("Essential task failed.".into())) + }) + } - match Pin::new(&mut this.essential_failed_rx).poll_next(cx) { - Poll::Pending => {}, - Poll::Ready(_) => { - // Ready(None) should not be possible since we hold a live - // sender. - return Poll::Ready(Err(crate::Error::Other("Essential task failed.".into()))); - } + /// Signal to terminate all the running tasks. + pub fn terminate(&mut self) { + if let Some(signal) = self.signal.take() { + let _ = signal.fire(); } + } - // The task manager future never ends. - Poll::Pending + /// Move a struct into the task manager to be kept alive. + pub fn keep_alive(&mut self, to_keep_alive: T) { + let keeping_alive = std::mem::replace(&mut self.keep_alive, Box::new(())); + self.keep_alive = Box::new((keeping_alive, to_keep_alive)); } } impl Drop for TaskManager { fn drop(&mut self) { debug!(target: "service", "Tasks manager shutdown"); - if let Some(signal) = self.signal.take() { - let _ = signal.fire(); - } + self.terminate(); } } diff --git a/client/service/test/src/lib.rs b/client/service/test/src/lib.rs index 6b8370aefcafd..1dfc0bc28d9a2 100644 --- a/client/service/test/src/lib.rs +++ b/client/service/test/src/lib.rs @@ -30,7 +30,7 @@ use tempfile::TempDir; use tokio::{runtime::Runtime, prelude::FutureExt}; use tokio::timer::Interval; use sc_service::{ - KeepAliveServiceComponents, + TaskManager, GenericChainSpec, ChainSpecExtension, Configuration, @@ -79,7 +79,7 @@ pub trait TestNetNode: Clone + Future + Se } pub struct TestNetComponents { - keep_alive: Arc>, + task_manager: Arc>, rpc_handlers: Arc, client: Arc>, transaction_pool: Arc, @@ -89,7 +89,7 @@ pub struct TestNetComponents { impl TestNetComponents { pub fn new( - keep_alive: KeepAliveServiceComponents, + task_manager: TaskManager, rpc_handlers: RpcHandlers, client: Arc>, network: Arc::Hash>>, @@ -98,7 +98,7 @@ TestNetComponents { Self { client, transaction_pool, network, rpc_handlers: Arc::new(rpc_handlers), - keep_alive: Arc::new(Mutex::new(keep_alive)), + task_manager: Arc::new(Mutex::new(task_manager)), } } } @@ -108,7 +108,7 @@ impl Clone for TestNetComponents { fn clone(&self) -> Self { Self { - keep_alive: self.keep_alive.clone(), + task_manager: self.task_manager.clone(), client: self.client.clone(), transaction_pool: self.transaction_pool.clone(), network: self.network.clone(), @@ -122,7 +122,7 @@ impl Future for TestNetComponents type Error = sc_service::Error; fn poll(&mut self) -> Poll { - futures::compat::Compat::new(&mut self.keep_alive.lock().task_manager).poll() + futures::compat::Compat::new(&mut self.task_manager.lock().future()).poll() } } diff --git a/utils/browser/src/lib.rs b/utils/browser/src/lib.rs index a0901cd2b1b6b..ec9003c072d1e 100644 --- a/utils/browser/src/lib.rs +++ b/utils/browser/src/lib.rs @@ -138,7 +138,7 @@ pub fn start_client(mut task_manager: TaskManager, rpc_handlers: RpcHandlers) -> } } - Pin::new(&mut task_manager) + Pin::new(&mut task_manager.future()) .poll(cx) .map(drop) })); From 22048cfb04aed8a402523ddda61c3a34281ec10b Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Tue, 23 Jun 2020 15:39:05 +0200 Subject: [PATCH 13/25] Move the telemetry, base path, rpc keep_alive to the service builder --- bin/node-template/node/src/service.rs | 11 +++-------- bin/node/cli/src/service.rs | 11 +++-------- client/service/src/builder.rs | 7 +++---- client/service/src/lib.rs | 6 ------ client/service/src/task_manager.rs | 9 ++++----- 5 files changed, 13 insertions(+), 31 deletions(-) diff --git a/bin/node-template/node/src/service.rs b/bin/node-template/node/src/service.rs index b5b6e5835fcd0..89bf159927fc6 100644 --- a/bin/node-template/node/src/service.rs +++ b/bin/node-template/node/src/service.rs @@ -109,8 +109,8 @@ pub fn new_full(config: Configuration) -> Result { .expect("Link Half and Block Import are present for Full Services or setup failed before. qed"); let ServiceComponents { - client, transaction_pool, mut task_manager, keystore, network, telemetry, base_path, - select_chain, prometheus_registry, telemetry_on_connect_sinks, rpc, .. + client, transaction_pool, task_manager, keystore, network, select_chain, + prometheus_registry, telemetry_on_connect_sinks, .. } = builder .with_finality_proof_provider(|client, backend| { // GenesisAuthoritySetProvider is implemented for StorageAndProofProvider @@ -201,8 +201,6 @@ pub fn new_full(config: Configuration) -> Result { )?; } - task_manager.keep_alive((telemetry, base_path, rpc)); - Ok(task_manager) } @@ -272,8 +270,5 @@ pub fn new_light(config: Configuration) -> Result { Ok(Arc::new(GrandpaFinalityProofProvider::new(backend, provider)) as _) })? .build_light() - .map(|ServiceComponents { mut task_manager, telemetry, base_path, rpc, .. }| { - task_manager.keep_alive((telemetry, base_path, rpc)); - task_manager - }) + .map(|ServiceComponents { task_manager, .. }| task_manager) } diff --git a/bin/node/cli/src/service.rs b/bin/node/cli/src/service.rs index 0ca1753b24961..ed246a80483ab 100644 --- a/bin/node/cli/src/service.rs +++ b/bin/node/cli/src/service.rs @@ -186,8 +186,8 @@ pub fn new_full_base( new_full_start!(config); let ServiceComponents { - client, transaction_pool, mut task_manager, keystore, network, telemetry, base_path, - select_chain, prometheus_registry, rpc, telemetry_on_connect_sinks, rpc_handlers, .. + client, transaction_pool, task_manager, keystore, network, select_chain, + prometheus_registry, telemetry_on_connect_sinks, rpc_handlers, .. } = builder .with_finality_proof_provider(|client, backend| { // GenesisAuthoritySetProvider is implemented for StorageAndProofProvider @@ -317,8 +317,6 @@ pub fn new_full_base( )?; } - task_manager.keep_alive((telemetry, base_path, rpc)); - Ok((task_manager, inherent_data_providers, rpc_handlers, client, network, transaction_pool)) } @@ -342,8 +340,7 @@ pub fn new_light_base(config: Configuration) -> Result<( let inherent_data_providers = InherentDataProviders::new(); let ServiceComponents { - mut task_manager, telemetry, base_path, rpc_handlers, rpc, client, network, - transaction_pool, .. + task_manager, rpc_handlers, client, network, transaction_pool, .. } = ServiceBuilder::new_light::(config)? .with_select_chain(|_config, backend| { Ok(LongestChain::new(backend.clone())) @@ -427,8 +424,6 @@ pub fn new_light_base(config: Configuration) -> Result<( Ok(node_rpc::create_light(light_deps)) })? .build_light()?; - - task_manager.keep_alive((telemetry, base_path, rpc)); Ok((task_manager, rpc_handlers, client, network, transaction_pool)) } diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index b991614020ead..1fb68bd3c50e3 100644 --- a/client/service/src/builder.rs +++ b/client/service/src/builder.rs @@ -967,7 +967,7 @@ ServiceBuilder< marker: _, mut config, client, - task_manager, + mut task_manager, fetcher: on_demand, backend, keystore, @@ -1132,6 +1132,8 @@ ServiceBuilder< sc_informant::OutputFormat { enable_color: true, prefix: informant_prefix }, )); + task_manager.keep_alive((telemetry, config.base_path, rpc)); + Ok(ServiceComponents { client, task_manager, @@ -1139,12 +1141,9 @@ ServiceBuilder< select_chain, transaction_pool, rpc_handlers, - rpc, - telemetry, keystore, offchain_workers, telemetry_on_connect_sinks: TelemetryOnConnectSinks(telemetry_connection_sinks), - base_path: config.base_path.map(Arc::new), network_status_sinks: NetworkStatusSinks::new(network_status_sinks), prometheus_registry: config.prometheus_config.map(|config| config.registry), }) diff --git a/client/service/src/lib.rs b/client/service/src/lib.rs index 57e59a0d04b08..98b1cd04f9075 100644 --- a/client/service/src/lib.rs +++ b/client/service/src/lib.rs @@ -163,10 +163,6 @@ pub struct ServiceComponents< pub keystore: sc_keystore::KeyStorePtr, /// A shared network instance. pub network: Arc::Hash>>, - /// A shared instance of Telemetry (if enabled). - pub telemetry: Option, - /// The base path. - pub base_path: Option>, /// RPC handlers that can perform RPC queries. pub rpc_handlers: RpcHandlers, /// A shared instance of the chain selection algorithm. @@ -175,8 +171,6 @@ pub struct ServiceComponents< pub network_status_sinks: NetworkStatusSinks, /// A prometheus metrics registry, (if enabled). pub prometheus_registry: Option, - /// A RPC instance. - pub rpc: Box, /// Shared Telemetry connection sinks, pub telemetry_on_connect_sinks: TelemetryOnConnectSinks, /// A shared offchain workers instance. diff --git a/client/service/src/task_manager.rs b/client/service/src/task_manager.rs index 40249cf6cd697..1675702bc2538 100644 --- a/client/service/src/task_manager.rs +++ b/client/service/src/task_manager.rs @@ -225,7 +225,7 @@ pub struct TaskManager { /// A receiver for spawned essential-tasks concluding. essential_failed_rx: TracingUnboundedReceiver<()>, /// Things to keep alive until the task manager is dropped. - keep_alive: Box, + keep_alive: Arc, } impl TaskManager { @@ -248,7 +248,7 @@ impl TaskManager { metrics, essential_failed_tx, essential_failed_rx, - keep_alive: Box::new(()), + keep_alive: Arc::new(()), }) } @@ -284,9 +284,8 @@ impl TaskManager { } /// Move a struct into the task manager to be kept alive. - pub fn keep_alive(&mut self, to_keep_alive: T) { - let keeping_alive = std::mem::replace(&mut self.keep_alive, Box::new(())); - self.keep_alive = Box::new((keeping_alive, to_keep_alive)); + pub fn keep_alive(&mut self, to_keep_alive: T) { + self.keep_alive = Arc::new((self.keep_alive.clone(), to_keep_alive)); } } From 8ae7a042b25661c31265558f37ee11f5a06f0001 Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Wed, 24 Jun 2020 12:35:03 +0200 Subject: [PATCH 14/25] Make the task manager keep alive an internal detail --- bin/node/cli/src/chain_spec.rs | 8 ++++---- bin/node/cli/src/service.rs | 30 ++++++++++++++---------------- client/service/src/builder.rs | 4 ++-- client/service/src/lib.rs | 2 +- client/service/src/task_manager.rs | 12 ++++++------ client/service/test/src/lib.rs | 5 ----- utils/browser/src/lib.rs | 5 ++--- 7 files changed, 29 insertions(+), 37 deletions(-) diff --git a/bin/node/cli/src/chain_spec.rs b/bin/node/cli/src/chain_spec.rs index 384b866c2956a..8617024255f28 100644 --- a/bin/node/cli/src/chain_spec.rs +++ b/bin/node/cli/src/chain_spec.rs @@ -431,12 +431,12 @@ pub(crate) mod tests { sc_service_test::connectivity( integration_test_config_with_two_authorities(), |config| { - let (keep_alive, _, rpc_handlers, client, network, transaction_pool) = new_full_base(config,|_, _| ())?; - Ok(sc_service_test::TestNetComponents::new(keep_alive, rpc_handlers, client, network, transaction_pool)) + let (keep_alive, _, client, network, transaction_pool) = new_full_base(config,|_, _| ())?; + Ok(sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool)) }, |config| { - let (keep_alive, rpc_handlers, client, network, transaction_pool) = new_light_base(config)?; - Ok(sc_service_test::TestNetComponents::new(keep_alive, rpc_handlers, client, network, transaction_pool)) + let (keep_alive, _, client, network, transaction_pool) = new_light_base(config)?; + Ok(sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool)) } ); } diff --git a/bin/node/cli/src/service.rs b/bin/node/cli/src/service.rs index ed246a80483ab..bcb1378cac312 100644 --- a/bin/node/cli/src/service.rs +++ b/bin/node/cli/src/service.rs @@ -167,7 +167,7 @@ pub fn new_full_base( ) -> Result<( TaskManager, InherentDataProviders, - RpcHandlers, Arc, Arc::Hash>>, + Arc, Arc::Hash>>, Arc, Block>> ), ServiceError> { let ( @@ -187,7 +187,7 @@ pub fn new_full_base( let ServiceComponents { client, transaction_pool, task_manager, keystore, network, select_chain, - prometheus_registry, telemetry_on_connect_sinks, rpc_handlers, .. + prometheus_registry, telemetry_on_connect_sinks, .. } = builder .with_finality_proof_provider(|client, backend| { // GenesisAuthoritySetProvider is implemented for StorageAndProofProvider @@ -317,14 +317,13 @@ pub fn new_full_base( )?; } - Ok((task_manager, inherent_data_providers, rpc_handlers, client, network, transaction_pool)) + Ok((task_manager, inherent_data_providers, client, network, transaction_pool)) } /// Builds a new service for a full client. pub fn new_full(config: Configuration) -> Result { - new_full_base(config, |_, _| ()).map(|(mut task_manager, rpc_handlers, _, _, _, _)| { - task_manager.keep_alive(rpc_handlers); + new_full_base(config, |_, _| ()).map(|(task_manager, _, _, _, _)| { task_manager }) } @@ -333,7 +332,7 @@ type LightClient = sc_service::TLightClient; pub fn new_light_base(config: Configuration) -> Result<( - TaskManager, RpcHandlers, Arc, + TaskManager, Arc, Arc, Arc::Hash>>, Arc, Block>> ), ServiceError> { @@ -430,8 +429,7 @@ pub fn new_light_base(config: Configuration) -> Result<( /// Builds a new service for a light client. pub fn new_light(config: Configuration) -> Result { - new_light_base(config).map(|(mut task_manager, rpc_handlers, _, _, _)| { - task_manager.keep_alive(rpc_handlers); + new_light_base(config).map(|(task_manager, _, _, _, _)| { task_manager }) } @@ -493,19 +491,19 @@ mod tests { chain_spec, |config| { let mut setup_handles = None; - let (keep_alive, inherent_data_providers, rpc_handlers, client, network, transaction_pool) = new_full_base(config, | + let (keep_alive, inherent_data_providers, client, network, transaction_pool) = new_full_base(config, | block_import: &sc_consensus_babe::BabeBlockImport, babe_link: &sc_consensus_babe::BabeLink, | { setup_handles = Some((block_import.clone(), babe_link.clone())); })?; - let node = sc_service_test::TestNetComponents::new(keep_alive, rpc_handlers, client, network, transaction_pool); + let node = sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool); Ok((node, (inherent_data_providers, setup_handles.unwrap()))) }, |config| { - let (keep_alive, rpc_handlers, client, network, transaction_pool) = new_light_base(config)?; - Ok(sc_service_test::TestNetComponents::new(keep_alive, rpc_handlers, client, network, transaction_pool)) + let (keep_alive, _, client, network, transaction_pool) = new_light_base(config)?; + Ok(sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool)) }, |service, &mut (ref inherent_data_providers, (ref mut block_import, ref babe_link))| { let mut inherent_data = inherent_data_providers @@ -657,12 +655,12 @@ mod tests { sc_service_test::consensus( crate::chain_spec::tests::integration_test_config_with_two_authorities(), |config| { - let (keep_alive, _, rpc_handlers, client, network, transaction_pool) = new_full_base(config, |_, _| ())?; - Ok(sc_service_test::TestNetComponents::new(keep_alive, rpc_handlers, client, network, transaction_pool)) + let (keep_alive, _, client, network, transaction_pool) = new_full_base(config, |_, _| ())?; + Ok(sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool)) }, |config| { - let (keep_alive, rpc_handlers, client, network, transaction_pool) = new_light_base(config)?; - Ok(sc_service_test::TestNetComponents::new(keep_alive, rpc_handlers, client, network, transaction_pool)) + let (keep_alive, _, client, network, transaction_pool) = new_light_base(config)?; + Ok(sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool)) }, vec![ "//Alice".into(), diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index 1fb68bd3c50e3..315b23e80543d 100644 --- a/client/service/src/builder.rs +++ b/client/service/src/builder.rs @@ -1095,7 +1095,7 @@ ServiceBuilder< ); let rpc = start_rpc_servers(&config, gen_handler)?; // This is used internally, so don't restrict access to unsafe RPC - let rpc_handlers = RpcHandlers(gen_handler(sc_rpc::DenyUnsafe::No)); + let rpc_handlers = Arc::new(RpcHandlers(gen_handler(sc_rpc::DenyUnsafe::No))); let telemetry_connection_sinks: Arc>>> = Default::default(); @@ -1132,7 +1132,7 @@ ServiceBuilder< sc_informant::OutputFormat { enable_color: true, prefix: informant_prefix }, )); - task_manager.keep_alive((telemetry, config.base_path, rpc)); + task_manager.keep_alive((telemetry, config.base_path, rpc, rpc_handlers.clone())); Ok(ServiceComponents { client, diff --git a/client/service/src/lib.rs b/client/service/src/lib.rs index feab89c2d107c..000b30d161265 100644 --- a/client/service/src/lib.rs +++ b/client/service/src/lib.rs @@ -166,7 +166,7 @@ pub struct ServiceComponents< /// A shared network instance. pub network: Arc::Hash>>, /// RPC handlers that can perform RPC queries. - pub rpc_handlers: RpcHandlers, + pub rpc_handlers: Arc, /// A shared instance of the chain selection algorithm. pub select_chain: Option, /// Sinks to propagate network status updates. diff --git a/client/service/src/task_manager.rs b/client/service/src/task_manager.rs index 9f7c30846fc9d..b6cc26005570a 100644 --- a/client/service/src/task_manager.rs +++ b/client/service/src/task_manager.rs @@ -13,7 +13,7 @@ //! Substrate service tasks management module. -use std::{panic, result::Result, sync::Arc, pin::Pin}; +use std::{panic, result::Result, pin::Pin}; use exit_future::Signal; use log::debug; use futures::{ @@ -222,7 +222,7 @@ pub struct TaskManager { /// A receiver for spawned essential-tasks concluding. essential_failed_rx: TracingUnboundedReceiver<()>, /// Things to keep alive until the task manager is dropped. - keep_alive: Arc, + keep_alive: Box, } impl TaskManager { @@ -245,7 +245,7 @@ impl TaskManager { metrics, essential_failed_tx, essential_failed_rx, - keep_alive: Arc::new(()), + keep_alive: Box::new(()), }) } @@ -280,9 +280,9 @@ impl TaskManager { } } - /// Move a struct into the task manager to be kept alive. - pub fn keep_alive(&mut self, to_keep_alive: T) { - self.keep_alive = Arc::new((self.keep_alive.clone(), to_keep_alive)); + /// Set what the task manager should keep alivei + pub(super) fn keep_alive(&mut self, to_keep_alive: T) { + self.keep_alive = Box::new(to_keep_alive); } } diff --git a/client/service/test/src/lib.rs b/client/service/test/src/lib.rs index dd6cbb7712fae..91632daea278e 100644 --- a/client/service/test/src/lib.rs +++ b/client/service/test/src/lib.rs @@ -40,7 +40,6 @@ use sc_service::{ Error, TaskExecutor, client::Client, - RpcHandlers, }; use sp_blockchain::HeaderBackend; use sc_network::{multiaddr, Multiaddr}; @@ -80,7 +79,6 @@ pub trait TestNetNode: Clone + Future + Se pub struct TestNetComponents { task_manager: Arc>, - rpc_handlers: Arc, client: Arc>, transaction_pool: Arc, network: Arc::Hash>>, @@ -90,14 +88,12 @@ impl TestNetComponents { pub fn new( task_manager: TaskManager, - rpc_handlers: RpcHandlers, client: Arc>, network: Arc::Hash>>, transaction_pool: Arc, ) -> Self { Self { client, transaction_pool, network, - rpc_handlers: Arc::new(rpc_handlers), task_manager: Arc::new(Mutex::new(task_manager)), } } @@ -112,7 +108,6 @@ TestNetComponents { client: self.client.clone(), transaction_pool: self.transaction_pool.clone(), network: self.network.clone(), - rpc_handlers: self.rpc_handlers.clone() } } } diff --git a/utils/browser/src/lib.rs b/utils/browser/src/lib.rs index a17a15f5cc770..27e7f12903613 100644 --- a/utils/browser/src/lib.rs +++ b/utils/browser/src/lib.rs @@ -25,8 +25,7 @@ use sc_service::{ }; use wasm_bindgen::prelude::*; use futures::{prelude::*, channel::{oneshot, mpsc}, future::{poll_fn, ok}, compat::*}; -use std::task::Poll; -use std::pin::Pin; +use std::{sync::Arc, task::Poll, pin::Pin}; use sc_chain_spec::Extension; use libp2p_wasm_ext::{ExtTransport, ffi}; @@ -116,7 +115,7 @@ struct RpcMessage { } /// Create a Client object that connects to a service. -pub fn start_client(mut task_manager: TaskManager, rpc_handlers: RpcHandlers) -> Client { +pub fn start_client(mut task_manager: TaskManager, rpc_handlers: Arc) -> Client { // We dispatch a background task responsible for processing the service. // // The main action performed by the code below consists in polling the service with From f0a8f358ad58ee89962d8cb6ee2a6831107ca453 Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Wed, 24 Jun 2020 12:58:42 +0200 Subject: [PATCH 15/25] Rewrite the browser start_client future --- bin/node/cli/browser-demo/build.sh | 2 +- utils/browser/src/lib.rs | 38 +++++++++++++----------------- 2 files changed, 18 insertions(+), 22 deletions(-) diff --git a/bin/node/cli/browser-demo/build.sh b/bin/node/cli/browser-demo/build.sh index be52b7a523f01..8339a8568268c 100755 --- a/bin/node/cli/browser-demo/build.sh +++ b/bin/node/cli/browser-demo/build.sh @@ -1,4 +1,4 @@ #!/usr/bin/env sh cargo +nightly build --release -p node-cli --target wasm32-unknown-unknown --no-default-features --features browser -Z features=itarget -wasm-bindgen ../../../../target/wasm32-unknown-unknown/release/node_cli.wasm --out-dir pkg --target web +wasm-bindgen-0-2-62 ../../../../target/wasm32-unknown-unknown/release/node_cli.wasm --out-dir pkg --target web python -m http.server 8000 diff --git a/utils/browser/src/lib.rs b/utils/browser/src/lib.rs index ffa2a5d558a81..c8034d9466fe7 100644 --- a/utils/browser/src/lib.rs +++ b/utils/browser/src/lib.rs @@ -24,8 +24,10 @@ use sc_service::{ GenericChainSpec, RuntimeGenesis }; use wasm_bindgen::prelude::*; -use futures::{prelude::*, channel::{oneshot, mpsc}, future::{poll_fn, ok}, compat::*}; -use std::{sync::Arc, task::Poll, pin::Pin}; +use futures::{ + prelude::*, channel::{oneshot, mpsc}, compat::*, future::{ready, ok, select} +}; +use std::{sync::Arc, pin::Pin}; use sc_chain_spec::Extension; use libp2p_wasm_ext::{ExtTransport, ffi}; @@ -125,25 +127,19 @@ pub fn start_client(mut task_manager: TaskManager, rpc_handlers: Arc(); - wasm_bindgen_futures::spawn_local(poll_fn(move |cx| { - loop { - match Pin::new(&mut rpc_send_rx).poll_next(cx) { - Poll::Ready(Some(message)) => { - let fut = rpc_handlers - .rpc_query(&message.session, &message.rpc_json) - .boxed(); - let _ = message.send_back.send(fut); - }, - Poll::Pending => break, - Poll::Ready(None) => return Poll::Ready(()), - } - } - - Pin::new(&mut task_manager.future()) - .poll(cx) - .map(drop) - })); + let (rpc_send_tx, rpc_send_rx) = mpsc::unbounded::(); + wasm_bindgen_futures::spawn_local( + select( + rpc_send_rx.for_each(move |message| { + let fut = rpc_handlers.rpc_query(&message.session, &message.rpc_json); + let _ = message.send_back.send(fut); + ready(()) + }), + Box::pin(async move { + let _ = task_manager.future().await; + }), + ).map(drop) + ); Client { rpc_send_tx, From 45464353064b86c7844d9959bcd6f7e9c3448eda Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Wed, 24 Jun 2020 15:59:24 +0200 Subject: [PATCH 16/25] Remove run_node etc --- bin/node-template/node/src/command.rs | 12 +++--- bin/node/cli/src/command.rs | 12 +++--- client/cli/src/runner.rs | 53 ++------------------------- 3 files changed, 18 insertions(+), 59 deletions(-) diff --git a/bin/node-template/node/src/command.rs b/bin/node-template/node/src/command.rs index 18e1b22a53f8e..a926639c25f56 100644 --- a/bin/node-template/node/src/command.rs +++ b/bin/node-template/node/src/command.rs @@ -19,6 +19,7 @@ use crate::chain_spec; use crate::cli::Cli; use crate::service; use sc_cli::SubstrateCli; +use sc_service::Role; impl SubstrateCli for Cli { fn impl_name() -> &'static str { @@ -71,11 +72,12 @@ pub fn run() -> sc_cli::Result<()> { } None => { let runner = cli.create_runner(&cli.run)?; - runner.run_node( - service::new_light, - service::new_full, - node_template_runtime::VERSION - ) + runner.print_node_infos(node_template_runtime::VERSION); + let service_fn = match runner.config().role { + Role::Light => service::new_light, + _ => service::new_full, + }; + runner.run_node_until_exit(service_fn) } } } diff --git a/bin/node/cli/src/command.rs b/bin/node/cli/src/command.rs index bd5483f2cd31e..2a04e8031f145 100644 --- a/bin/node/cli/src/command.rs +++ b/bin/node/cli/src/command.rs @@ -20,6 +20,7 @@ use crate::{chain_spec, service, Cli, Subcommand}; use node_executor::Executor; use node_runtime::{Block, RuntimeApi}; use sc_cli::{Result, SubstrateCli}; +use sc_service::Role; impl SubstrateCli for Cli { fn impl_name() -> &'static str { @@ -70,11 +71,12 @@ pub fn run() -> Result<()> { match &cli.subcommand { None => { let runner = cli.create_runner(&cli.run)?; - runner.run_node( - service::new_light, - service::new_full, - node_runtime::VERSION - ) + runner.print_node_infos(node_runtime::VERSION); + let service_fn = match runner.config().role { + Role::Light => service::new_light, + _ => service::new_full, + }; + runner.run_node_until_exit(service_fn) } Some(Subcommand::Inspect(cmd)) => { let runner = cli.create_runner(cmd)?; diff --git a/client/cli/src/runner.rs b/client/cli/src/runner.rs index 64e88a087507b..d6d103dbe2cf1 100644 --- a/client/cli/src/runner.rs +++ b/client/cli/src/runner.rs @@ -25,7 +25,7 @@ use futures::pin_mut; use futures::select; use futures::{future, future::FutureExt, Future}; use log::info; -use sc_service::{Configuration, Role, ServiceBuilderCommand, TaskType, TaskManager}; +use sc_service::{Configuration, ServiceBuilderCommand, TaskType, TaskManager}; use sp_runtime::traits::{Block as BlockT, Header as HeaderT}; use sp_utils::metrics::{TOKIO_THREADS_ALIVE, TOKIO_THREADS_TOTAL}; use sp_version::RuntimeVersion; @@ -172,53 +172,6 @@ impl Runner { info!("⛓ Native runtime: {}", runtime_version); } - /// A helper function that runs a node with tokio and stops if the process - /// receives the signal `SIGTERM` or `SIGINT`. It can run a full or a light node depending on - /// the node's configuration. - pub fn run_node( - self, - new_light: impl FnOnce(Configuration) -> sc_service::error::Result, - new_full: impl FnOnce(Configuration) -> sc_service::error::Result, - runtime_version: RuntimeVersion, - ) -> Result<()> { - match self.config.role { - Role::Light => self.run_light_node(new_light, runtime_version), - _ => self.run_full_node(new_full, runtime_version), - } - } - - /// A helper function that runs a node with tokio and stops if the process - /// receives the signal `SIGTERM` or `SIGINT`. It can only run a "full" node and will fail if - /// the node's configuration uses a "light" role. - pub fn run_full_node( - self, - new_full: impl FnOnce(Configuration) -> sc_service::error::Result, - runtime_version: RuntimeVersion, - ) -> Result<()> { - if matches!(self.config.role, Role::Light) { - return Err("Light node has been requested but this is not implemented".into()); - } - - self.print_node_infos(runtime_version); - self.run_service_until_exit(new_full) - } - - /// A helper function that runs a node with tokio and stops if the process - /// receives the signal `SIGTERM` or `SIGINT`. It can only run a "light" node and will fail if - /// the node's configuration uses a "full" role. - pub fn run_light_node( - self, - new_light: impl FnOnce(Configuration) -> sc_service::error::Result, - runtime_version: RuntimeVersion, - ) -> Result<()> { - if !matches!(self.config.role, Role::Light) { - return Err("Full node has been requested but this is not implemented".into()); - } - - self.print_node_infos(runtime_version); - self.run_service_until_exit(new_light) - } - /// A helper function that runs a future with tokio and stops if the process receives the signal /// `SIGTERM` or `SIGINT`. pub fn run_subcommand(self, subcommand: &Subcommand, builder: B) -> Result<()> @@ -247,7 +200,9 @@ impl Runner { } } - fn run_service_until_exit( + /// A helper function that runs a node with tokio and stops if the process receives the signal + /// `SIGTERM` or `SIGINT`. + pub fn run_node_until_exit( mut self, initialise: impl FnOnce(Configuration) -> sc_service::error::Result, ) -> Result<()> { From f42df6cf3e83fef047e40f659675cc7ffb9e63d8 Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Wed, 24 Jun 2020 16:02:39 +0200 Subject: [PATCH 17/25] Revert my personal changes to browser-demo/build.sh --- bin/node/cli/browser-demo/build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/node/cli/browser-demo/build.sh b/bin/node/cli/browser-demo/build.sh index 8339a8568268c..be52b7a523f01 100755 --- a/bin/node/cli/browser-demo/build.sh +++ b/bin/node/cli/browser-demo/build.sh @@ -1,4 +1,4 @@ #!/usr/bin/env sh cargo +nightly build --release -p node-cli --target wasm32-unknown-unknown --no-default-features --features browser -Z features=itarget -wasm-bindgen-0-2-62 ../../../../target/wasm32-unknown-unknown/release/node_cli.wasm --out-dir pkg --target web +wasm-bindgen ../../../../target/wasm32-unknown-unknown/release/node_cli.wasm --out-dir pkg --target web python -m http.server 8000 From 0ad4e40ca7499e7ea05d0194fd14da60845ecdad Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Wed, 24 Jun 2020 16:25:17 +0200 Subject: [PATCH 18/25] use |config| --- bin/node-template/node/src/command.rs | 13 +++++++------ bin/node/cli/src/command.rs | 13 +++++++------ client/cli/src/runner.rs | 4 +++- 3 files changed, 17 insertions(+), 13 deletions(-) diff --git a/bin/node-template/node/src/command.rs b/bin/node-template/node/src/command.rs index a926639c25f56..634a12d2dd17a 100644 --- a/bin/node-template/node/src/command.rs +++ b/bin/node-template/node/src/command.rs @@ -72,12 +72,13 @@ pub fn run() -> sc_cli::Result<()> { } None => { let runner = cli.create_runner(&cli.run)?; - runner.print_node_infos(node_template_runtime::VERSION); - let service_fn = match runner.config().role { - Role::Light => service::new_light, - _ => service::new_full, - }; - runner.run_node_until_exit(service_fn) + runner.run_node_until_exit( + |config| match config.role { + Role::Light => service::new_light(config), + _ => service::new_full(config), + }, + node_template_runtime::VERSION, + ) } } } diff --git a/bin/node/cli/src/command.rs b/bin/node/cli/src/command.rs index 2a04e8031f145..4ae4229ba54df 100644 --- a/bin/node/cli/src/command.rs +++ b/bin/node/cli/src/command.rs @@ -71,12 +71,13 @@ pub fn run() -> Result<()> { match &cli.subcommand { None => { let runner = cli.create_runner(&cli.run)?; - runner.print_node_infos(node_runtime::VERSION); - let service_fn = match runner.config().role { - Role::Light => service::new_light, - _ => service::new_full, - }; - runner.run_node_until_exit(service_fn) + runner.run_node_until_exit( + |config| match config.role { + Role::Light => service::new_light(config), + _ => service::new_full(config), + }, + node_runtime::VERSION, + ) } Some(Subcommand::Inspect(cmd)) => { let runner = cli.create_runner(cmd)?; diff --git a/client/cli/src/runner.rs b/client/cli/src/runner.rs index d6d103dbe2cf1..aedd41073bc09 100644 --- a/client/cli/src/runner.rs +++ b/client/cli/src/runner.rs @@ -153,7 +153,7 @@ impl Runner { /// 2020-06-03 16:14:21 💾 Database: RocksDb at /tmp/c/chains/flamingfir7/db /// 2020-06-03 16:14:21 ⛓ Native runtime: node-251 (substrate-node-1.tx1.au10) /// ``` - pub fn print_node_infos(&self, runtime_version: RuntimeVersion) { + fn print_node_infos(&self, runtime_version: RuntimeVersion) { info!("{}", C::impl_name()); info!("✌️ version {}", C::impl_version()); info!( @@ -205,7 +205,9 @@ impl Runner { pub fn run_node_until_exit( mut self, initialise: impl FnOnce(Configuration) -> sc_service::error::Result, + runtime_version: RuntimeVersion, ) -> Result<()> { + self.print_node_infos(runtime_version); let mut task_manager = initialise(self.config)?; self.tokio_runtime.block_on(main(task_manager.future().fuse())) .map_err(|e| e.to_string())?; From d066c026bb58ab6f6b66b25aeea06f7df271fef5 Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Wed, 24 Jun 2020 17:17:10 +0200 Subject: [PATCH 19/25] Add a runtime_version function to SubstrateCli --- Cargo.lock | 2 ++ bin/node-template/node/Cargo.toml | 1 + bin/node-template/node/src/command.rs | 16 +++++++++------- bin/node/cli/Cargo.toml | 1 + bin/node/cli/src/command.rs | 16 +++++++++------- client/cli/src/lib.rs | 4 ++++ client/cli/src/runner.rs | 8 +++----- 7 files changed, 29 insertions(+), 19 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 08e5102d3402a..16fd7d44858a2 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3396,6 +3396,7 @@ dependencies = [ "sp-runtime", "sp-timestamp", "sp-transaction-pool", + "sp-version", "structopt", "substrate-browser-utils", "substrate-build-script-utils", @@ -3601,6 +3602,7 @@ dependencies = [ "sp-inherents", "sp-runtime", "sp-transaction-pool", + "sp-version", "structopt", "substrate-build-script-utils", ] diff --git a/bin/node-template/node/Cargo.toml b/bin/node-template/node/Cargo.toml index 52fc1b4f8dacc..f74b2a3fa14a5 100644 --- a/bin/node-template/node/Cargo.toml +++ b/bin/node-template/node/Cargo.toml @@ -38,6 +38,7 @@ sp-finality-grandpa = { version = "2.0.0-rc3", path = "../../../primitives/final sc-client-api = { version = "2.0.0-rc3", path = "../../../client/api" } sp-runtime = { version = "2.0.0-rc3", path = "../../../primitives/runtime" } sc-basic-authorship = { path = "../../../client/basic-authorship", version = "0.8.0-rc3"} +sp-version = { version = "2.0.0-rc3", path = "../../../primitives/version" } node-template-runtime = { version = "2.0.0-rc3", path = "../runtime" } diff --git a/bin/node-template/node/src/command.rs b/bin/node-template/node/src/command.rs index 634a12d2dd17a..4f0d486f01f3e 100644 --- a/bin/node-template/node/src/command.rs +++ b/bin/node-template/node/src/command.rs @@ -20,6 +20,7 @@ use crate::cli::Cli; use crate::service; use sc_cli::SubstrateCli; use sc_service::Role; +use sp_version::RuntimeVersion; impl SubstrateCli for Cli { fn impl_name() -> &'static str { @@ -59,6 +60,10 @@ impl SubstrateCli for Cli { )?), }) } + + fn runtime_version() -> &'static RuntimeVersion { + &node_template_runtime::VERSION + } } /// Parse and run command line arguments @@ -72,13 +77,10 @@ pub fn run() -> sc_cli::Result<()> { } None => { let runner = cli.create_runner(&cli.run)?; - runner.run_node_until_exit( - |config| match config.role { - Role::Light => service::new_light(config), - _ => service::new_full(config), - }, - node_template_runtime::VERSION, - ) + runner.run_node_until_exit(|config| match config.role { + Role::Light => service::new_light(config), + _ => service::new_full(config), + }) } } } diff --git a/bin/node/cli/Cargo.toml b/bin/node/cli/Cargo.toml index 4e2c0151b7ea5..ab201690fbcc0 100644 --- a/bin/node/cli/Cargo.toml +++ b/bin/node/cli/Cargo.toml @@ -58,6 +58,7 @@ sp-keyring = { version = "2.0.0-rc3", path = "../../../primitives/keyring" } sp-io = { version = "2.0.0-rc3", path = "../../../primitives/io" } sp-consensus = { version = "0.8.0-rc3", path = "../../../primitives/consensus/common" } sp-transaction-pool = { version = "2.0.0-rc3", path = "../../../primitives/transaction-pool" } +sp-version = { version = "2.0.0-rc3", path = "../../../primitives/version" } # client dependencies sc-client-api = { version = "2.0.0-rc3", path = "../../../client/api" } diff --git a/bin/node/cli/src/command.rs b/bin/node/cli/src/command.rs index 4ae4229ba54df..9de38d1116444 100644 --- a/bin/node/cli/src/command.rs +++ b/bin/node/cli/src/command.rs @@ -21,6 +21,7 @@ use node_executor::Executor; use node_runtime::{Block, RuntimeApi}; use sc_cli::{Result, SubstrateCli}; use sc_service::Role; +use sp_version::RuntimeVersion; impl SubstrateCli for Cli { fn impl_name() -> &'static str { @@ -62,6 +63,10 @@ impl SubstrateCli for Cli { )?), }) } + + fn runtime_version() -> &'static RuntimeVersion { + &node_runtime::VERSION + } } /// Parse command line arguments into service configuration. @@ -71,13 +76,10 @@ pub fn run() -> Result<()> { match &cli.subcommand { None => { let runner = cli.create_runner(&cli.run)?; - runner.run_node_until_exit( - |config| match config.role { - Role::Light => service::new_light(config), - _ => service::new_full(config), - }, - node_runtime::VERSION, - ) + runner.run_node_until_exit(|config| match config.role { + Role::Light => service::new_light(config), + _ => service::new_full(config), + }) } Some(Subcommand::Inspect(cmd)) => { let runner = cli.create_runner(cmd)?; diff --git a/client/cli/src/lib.rs b/client/cli/src/lib.rs index 9623b08bfbb7f..da85e4ee16999 100644 --- a/client/cli/src/lib.rs +++ b/client/cli/src/lib.rs @@ -38,6 +38,7 @@ pub use params::*; use regex::Regex; pub use runner::*; use sc_service::{ChainSpec, Configuration, TaskExecutor}; +use sp_version::RuntimeVersion; use std::io::Write; pub use structopt; use structopt::{ @@ -207,6 +208,9 @@ pub trait SubstrateCli: Sized { command.init::()?; Runner::new(self, command) } + + /// Runtime version. + fn runtime_version() -> &'static RuntimeVersion; } /// Initialize the logger diff --git a/client/cli/src/runner.rs b/client/cli/src/runner.rs index aedd41073bc09..a1484e4f7c8a8 100644 --- a/client/cli/src/runner.rs +++ b/client/cli/src/runner.rs @@ -28,7 +28,6 @@ use log::info; use sc_service::{Configuration, ServiceBuilderCommand, TaskType, TaskManager}; use sp_runtime::traits::{Block as BlockT, Header as HeaderT}; use sp_utils::metrics::{TOKIO_THREADS_ALIVE, TOKIO_THREADS_TOTAL}; -use sp_version::RuntimeVersion; use std::{fmt::Debug, marker::PhantomData, str::FromStr}; #[cfg(target_family = "unix")] @@ -153,7 +152,7 @@ impl Runner { /// 2020-06-03 16:14:21 💾 Database: RocksDb at /tmp/c/chains/flamingfir7/db /// 2020-06-03 16:14:21 ⛓ Native runtime: node-251 (substrate-node-1.tx1.au10) /// ``` - fn print_node_infos(&self, runtime_version: RuntimeVersion) { + fn print_node_infos(&self) { info!("{}", C::impl_name()); info!("✌️ version {}", C::impl_version()); info!( @@ -169,7 +168,7 @@ impl Runner { self.config.database, self.config.database.path().map_or_else(|| "".to_owned(), |p| p.display().to_string()) ); - info!("⛓ Native runtime: {}", runtime_version); + info!("⛓ Native runtime: {}", C::runtime_version()); } /// A helper function that runs a future with tokio and stops if the process receives the signal @@ -205,9 +204,8 @@ impl Runner { pub fn run_node_until_exit( mut self, initialise: impl FnOnce(Configuration) -> sc_service::error::Result, - runtime_version: RuntimeVersion, ) -> Result<()> { - self.print_node_infos(runtime_version); + self.print_node_infos(); let mut task_manager = initialise(self.config)?; self.tokio_runtime.block_on(main(task_manager.future().fuse())) .map_err(|e| e.to_string())?; From 1305100ce5f6b4191223d5ed44f5bcc6fda5f2d7 Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Wed, 24 Jun 2020 17:52:17 +0200 Subject: [PATCH 20/25] Reexport role and runtime version from sc cli --- bin/node-template/node/Cargo.toml | 1 - bin/node-template/node/src/command.rs | 4 +--- bin/node/cli/Cargo.toml | 1 - bin/node/cli/src/command.rs | 4 +--- client/cli/src/lib.rs | 3 ++- 5 files changed, 4 insertions(+), 9 deletions(-) diff --git a/bin/node-template/node/Cargo.toml b/bin/node-template/node/Cargo.toml index f74b2a3fa14a5..52fc1b4f8dacc 100644 --- a/bin/node-template/node/Cargo.toml +++ b/bin/node-template/node/Cargo.toml @@ -38,7 +38,6 @@ sp-finality-grandpa = { version = "2.0.0-rc3", path = "../../../primitives/final sc-client-api = { version = "2.0.0-rc3", path = "../../../client/api" } sp-runtime = { version = "2.0.0-rc3", path = "../../../primitives/runtime" } sc-basic-authorship = { path = "../../../client/basic-authorship", version = "0.8.0-rc3"} -sp-version = { version = "2.0.0-rc3", path = "../../../primitives/version" } node-template-runtime = { version = "2.0.0-rc3", path = "../runtime" } diff --git a/bin/node-template/node/src/command.rs b/bin/node-template/node/src/command.rs index 4f0d486f01f3e..ef2635018759a 100644 --- a/bin/node-template/node/src/command.rs +++ b/bin/node-template/node/src/command.rs @@ -18,9 +18,7 @@ use crate::chain_spec; use crate::cli::Cli; use crate::service; -use sc_cli::SubstrateCli; -use sc_service::Role; -use sp_version::RuntimeVersion; +use sc_cli::{SubstrateCli, RuntimeVersion, Role}; impl SubstrateCli for Cli { fn impl_name() -> &'static str { diff --git a/bin/node/cli/Cargo.toml b/bin/node/cli/Cargo.toml index ab201690fbcc0..4e2c0151b7ea5 100644 --- a/bin/node/cli/Cargo.toml +++ b/bin/node/cli/Cargo.toml @@ -58,7 +58,6 @@ sp-keyring = { version = "2.0.0-rc3", path = "../../../primitives/keyring" } sp-io = { version = "2.0.0-rc3", path = "../../../primitives/io" } sp-consensus = { version = "0.8.0-rc3", path = "../../../primitives/consensus/common" } sp-transaction-pool = { version = "2.0.0-rc3", path = "../../../primitives/transaction-pool" } -sp-version = { version = "2.0.0-rc3", path = "../../../primitives/version" } # client dependencies sc-client-api = { version = "2.0.0-rc3", path = "../../../client/api" } diff --git a/bin/node/cli/src/command.rs b/bin/node/cli/src/command.rs index 9de38d1116444..6d23883c29bb1 100644 --- a/bin/node/cli/src/command.rs +++ b/bin/node/cli/src/command.rs @@ -19,9 +19,7 @@ use crate::{chain_spec, service, Cli, Subcommand}; use node_executor::Executor; use node_runtime::{Block, RuntimeApi}; -use sc_cli::{Result, SubstrateCli}; -use sc_service::Role; -use sp_version::RuntimeVersion; +use sc_cli::{Result, SubstrateCli, RuntimeVersion, Role}; impl SubstrateCli for Cli { fn impl_name() -> &'static str { diff --git a/client/cli/src/lib.rs b/client/cli/src/lib.rs index da85e4ee16999..c710b228b9c18 100644 --- a/client/cli/src/lib.rs +++ b/client/cli/src/lib.rs @@ -38,7 +38,8 @@ pub use params::*; use regex::Regex; pub use runner::*; use sc_service::{ChainSpec, Configuration, TaskExecutor}; -use sp_version::RuntimeVersion; +pub use sc_service::Role; +pub use sp_version::RuntimeVersion; use std::io::Write; pub use structopt; use structopt::{ From 1326a02ac26f6f3e7bab2594e1393450c6b94557 Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Wed, 24 Jun 2020 17:52:37 +0200 Subject: [PATCH 21/25] Update Cargo.lock --- Cargo.lock | 2 -- 1 file changed, 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 16fd7d44858a2..08e5102d3402a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3396,7 +3396,6 @@ dependencies = [ "sp-runtime", "sp-timestamp", "sp-transaction-pool", - "sp-version", "structopt", "substrate-browser-utils", "substrate-build-script-utils", @@ -3602,7 +3601,6 @@ dependencies = [ "sp-inherents", "sp-runtime", "sp-transaction-pool", - "sp-version", "structopt", "substrate-build-script-utils", ] From 772b7572bb20f9bb7e5b262edbbf47033c40a07e Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Thu, 25 Jun 2020 10:58:00 +0200 Subject: [PATCH 22/25] runtime_version -> native_runtime_version --- bin/node-template/node/src/command.rs | 2 +- bin/node/cli/src/command.rs | 2 +- client/cli/src/lib.rs | 2 +- client/cli/src/runner.rs | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/bin/node-template/node/src/command.rs b/bin/node-template/node/src/command.rs index ef2635018759a..716ff51b82ede 100644 --- a/bin/node-template/node/src/command.rs +++ b/bin/node-template/node/src/command.rs @@ -59,7 +59,7 @@ impl SubstrateCli for Cli { }) } - fn runtime_version() -> &'static RuntimeVersion { + fn native_runtime_version() -> &'static RuntimeVersion { &node_template_runtime::VERSION } } diff --git a/bin/node/cli/src/command.rs b/bin/node/cli/src/command.rs index 6d23883c29bb1..d4d165cbd99b9 100644 --- a/bin/node/cli/src/command.rs +++ b/bin/node/cli/src/command.rs @@ -62,7 +62,7 @@ impl SubstrateCli for Cli { }) } - fn runtime_version() -> &'static RuntimeVersion { + fn native_runtime_version() -> &'static RuntimeVersion { &node_runtime::VERSION } } diff --git a/client/cli/src/lib.rs b/client/cli/src/lib.rs index c710b228b9c18..b875856c4daf0 100644 --- a/client/cli/src/lib.rs +++ b/client/cli/src/lib.rs @@ -211,7 +211,7 @@ pub trait SubstrateCli: Sized { } /// Runtime version. - fn runtime_version() -> &'static RuntimeVersion; + fn native_runtime_version() -> &'static RuntimeVersion; } /// Initialize the logger diff --git a/client/cli/src/runner.rs b/client/cli/src/runner.rs index a1484e4f7c8a8..c556f86ddd0d7 100644 --- a/client/cli/src/runner.rs +++ b/client/cli/src/runner.rs @@ -168,7 +168,7 @@ impl Runner { self.config.database, self.config.database.path().map_or_else(|| "".to_owned(), |p| p.display().to_string()) ); - info!("⛓ Native runtime: {}", C::runtime_version()); + info!("⛓ Native runtime: {}", C::native_runtime_version()); } /// A helper function that runs a future with tokio and stops if the process receives the signal From fe9d37a286fbb99fae1aed9ae61c0f9852b62c7c Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Thu, 25 Jun 2020 11:17:55 +0200 Subject: [PATCH 23/25] Pass chain spec to native_runtime_version for polkadot --- bin/node-template/node/src/command.rs | 4 ++-- bin/node/cli/src/command.rs | 4 ++-- client/cli/src/lib.rs | 8 ++++---- client/cli/src/runner.rs | 2 +- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/bin/node-template/node/src/command.rs b/bin/node-template/node/src/command.rs index 716ff51b82ede..4f2fd3aad6fd3 100644 --- a/bin/node-template/node/src/command.rs +++ b/bin/node-template/node/src/command.rs @@ -18,7 +18,7 @@ use crate::chain_spec; use crate::cli::Cli; use crate::service; -use sc_cli::{SubstrateCli, RuntimeVersion, Role}; +use sc_cli::{SubstrateCli, RuntimeVersion, Role, ChainSpec}; impl SubstrateCli for Cli { fn impl_name() -> &'static str { @@ -59,7 +59,7 @@ impl SubstrateCli for Cli { }) } - fn native_runtime_version() -> &'static RuntimeVersion { + fn native_runtime_version(_: &Box) -> &'static RuntimeVersion { &node_template_runtime::VERSION } } diff --git a/bin/node/cli/src/command.rs b/bin/node/cli/src/command.rs index d4d165cbd99b9..b07e0cdc907e0 100644 --- a/bin/node/cli/src/command.rs +++ b/bin/node/cli/src/command.rs @@ -19,7 +19,7 @@ use crate::{chain_spec, service, Cli, Subcommand}; use node_executor::Executor; use node_runtime::{Block, RuntimeApi}; -use sc_cli::{Result, SubstrateCli, RuntimeVersion, Role}; +use sc_cli::{Result, SubstrateCli, RuntimeVersion, Role, ChainSpec}; impl SubstrateCli for Cli { fn impl_name() -> &'static str { @@ -62,7 +62,7 @@ impl SubstrateCli for Cli { }) } - fn native_runtime_version() -> &'static RuntimeVersion { + fn native_runtime_version(_: &Box) -> &'static RuntimeVersion { &node_runtime::VERSION } } diff --git a/client/cli/src/lib.rs b/client/cli/src/lib.rs index b875856c4daf0..a702edba65784 100644 --- a/client/cli/src/lib.rs +++ b/client/cli/src/lib.rs @@ -37,8 +37,8 @@ use log::info; pub use params::*; use regex::Regex; pub use runner::*; -use sc_service::{ChainSpec, Configuration, TaskExecutor}; -pub use sc_service::Role; +use sc_service::{Configuration, TaskExecutor}; +pub use sc_service::{ChainSpec, Role}; pub use sp_version::RuntimeVersion; use std::io::Write; pub use structopt; @@ -210,8 +210,8 @@ pub trait SubstrateCli: Sized { Runner::new(self, command) } - /// Runtime version. - fn native_runtime_version() -> &'static RuntimeVersion; + /// Native runtime version. + fn native_runtime_version(chain_spec: &Box) -> &'static RuntimeVersion; } /// Initialize the logger diff --git a/client/cli/src/runner.rs b/client/cli/src/runner.rs index c556f86ddd0d7..fcc869dc87069 100644 --- a/client/cli/src/runner.rs +++ b/client/cli/src/runner.rs @@ -168,7 +168,7 @@ impl Runner { self.config.database, self.config.database.path().map_or_else(|| "".to_owned(), |p| p.display().to_string()) ); - info!("⛓ Native runtime: {}", C::native_runtime_version()); + info!("⛓ Native runtime: {}", C::native_runtime_version(&self.config.chain_spec)); } /// A helper function that runs a future with tokio and stops if the process receives the signal From 9c2e7823c6a4745aa6438212713d80466959bb60 Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Mon, 29 Jun 2020 14:18:24 +0200 Subject: [PATCH 24/25] Fix line widths --- bin/node/cli/src/service.rs | 40 +++++++++++++++++++--------------- client/service/src/builder.rs | 12 +++++++--- client/service/test/src/lib.rs | 25 ++++++++++++++------- 3 files changed, 49 insertions(+), 28 deletions(-) diff --git a/bin/node/cli/src/service.rs b/bin/node/cli/src/service.rs index 44016bde4c18f..608d175a76316 100644 --- a/bin/node/cli/src/service.rs +++ b/bin/node/cli/src/service.rs @@ -29,7 +29,7 @@ use node_executor; use node_primitives::Block; use node_runtime::RuntimeApi; use sc_service::{ - ServiceBuilder, config::Configuration, error::{Error as ServiceError}, + ServiceBuilder, config::{Role, Configuration}, error::{Error as ServiceError}, RpcHandlers, ServiceComponents, TaskManager, }; use sp_inherents::InherentDataProviders; @@ -235,7 +235,7 @@ pub fn new_full_base( } // Spawn authority discovery module. - if matches!(role, sc_service::config::Role::Authority{..} | sc_service::config::Role::Sentry {..}) { + if matches!(role, Role::Authority{..} | Role::Sentry {..}) { let (sentries, authority_discovery_role) = match role { sc_service::config::Role::Authority { ref sentry_nodes } => ( sentry_nodes.clone(), @@ -250,10 +250,11 @@ pub fn new_full_base( _ => unreachable!("Due to outer matches! constraint; qed.") }; - let dht_event_stream = network.event_stream("authority-discovery").filter_map(|e| async move { match e { - Event::Dht(e) => Some(e), - _ => None, - }}).boxed(); + let dht_event_stream = network.event_stream("authority-discovery") + .filter_map(|e| async move { match e { + Event::Dht(e) => Some(e), + _ => None, + }}).boxed(); let authority_discovery = sc_authority_discovery::AuthorityDiscovery::new( client.clone(), network.clone(), @@ -334,7 +335,9 @@ type LightFetcher = sc_network::config::OnDemand; pub fn new_light_base(config: Configuration) -> Result<( TaskManager, Arc, Arc, Arc::Hash>>, - Arc, Block>> + Arc, Block + >> ), ServiceError> { let inherent_data_providers = InherentDataProviders::new(); @@ -437,9 +440,7 @@ pub fn new_light(config: Configuration) -> Result { #[cfg(test)] mod tests { use std::{sync::Arc, borrow::Cow, any::Any}; - use sc_consensus_babe::{ - CompatibleDigestItem, BabeIntermediate, INTERMEDIATE_KEY - }; + use sc_consensus_babe::{CompatibleDigestItem, BabeIntermediate, INTERMEDIATE_KEY}; use sc_consensus_epochs::descendent_query; use sp_consensus::{ Environment, Proposer, BlockImportParams, BlockOrigin, ForkChoiceStrategy, BlockImport, @@ -490,14 +491,19 @@ mod tests { chain_spec, |config| { let mut setup_handles = None; - let (keep_alive, inherent_data_providers, client, network, transaction_pool) = new_full_base(config, | - block_import: &sc_consensus_babe::BabeBlockImport, - babe_link: &sc_consensus_babe::BabeLink, - | { - setup_handles = Some((block_import.clone(), babe_link.clone())); - })?; + let (keep_alive, inherent_data_providers, client, network, transaction_pool) = + new_full_base(config, + | + block_import: &sc_consensus_babe::BabeBlockImport, + babe_link: &sc_consensus_babe::BabeLink, + | { + setup_handles = Some((block_import.clone(), babe_link.clone())); + } + )?; - let node = sc_service_test::TestNetComponents::new(keep_alive, client, network, transaction_pool); + let node = sc_service_test::TestNetComponents::new( + keep_alive, client, network, transaction_pool + ); Ok((node, (inherent_data_providers, setup_handles.unwrap()))) }, |config| { diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index 1cad3a0463ce8..ebf373642219d 100644 --- a/client/service/src/builder.rs +++ b/client/service/src/builder.rs @@ -917,7 +917,9 @@ ServiceBuilder< Ok(self) } - fn build_common(self) -> Result, Error> + fn build_common(self) -> Result< + ServiceComponents, Error + > where TExec: CallExecutor, { let ServiceBuilder { @@ -1115,7 +1117,9 @@ ServiceBuilder< } /// Builds the light service. - pub fn build_light(self) -> Result, Error> + pub fn build_light(self) -> Result< + ServiceComponents, Error + > where TExec: CallExecutor, { self.build_common() @@ -1158,7 +1162,9 @@ ServiceBuilder< { /// Builds the full service. - pub fn build_full(self) -> Result, Error> + pub fn build_full(self) -> Result< + ServiceComponents, Error + > where TExec: CallExecutor, { // make transaction pool available for off-chain runtime calls. diff --git a/client/service/test/src/lib.rs b/client/service/test/src/lib.rs index 77f35f41e42e9..5a676e5263c8a 100644 --- a/client/service/test/src/lib.rs +++ b/client/service/test/src/lib.rs @@ -112,7 +112,9 @@ TestNetComponents { } } -impl Future for TestNetComponents { +impl Future for + TestNetComponents +{ type Item = (); type Error = sc_service::Error; @@ -540,10 +542,12 @@ pub fn sync( info!("Running sync"); for (_, service, _, _) in network.full_nodes.iter().skip(1) { - service.network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); + service.network().add_reserved_peer(first_address.to_string()) + .expect("Error adding reserved peer"); } for (_, service, _) in network.light_nodes.iter() { - service.network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); + service.network().add_reserved_peer(first_address.to_string()) + .expect("Error adding reserved peer"); } network.run_until_all_full( |_index, service| @@ -598,13 +602,16 @@ pub fn consensus( info!("Checking consensus"); let first_address = network.authority_nodes[0].3.clone(); for (_, service, _, _) in network.full_nodes.iter() { - service.network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); + service.network().add_reserved_peer(first_address.to_string()) + .expect("Error adding reserved peer"); } for (_, service, _) in network.light_nodes.iter() { - service.network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); + service.network().add_reserved_peer(first_address.to_string()) + .expect("Error adding reserved peer"); } for (_, service, _, _) in network.authority_nodes.iter().skip(1) { - service.network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); + service.network().add_reserved_peer(first_address.to_string()) + .expect("Error adding reserved peer"); } network.run_until_all_full( |_index, service| @@ -623,10 +630,12 @@ pub fn consensus( (0..0).map(|_| (String::new(), { |cfg| full_builder(cfg).map(|s| (s, ())) })), ); for (_, service, _, _) in network.full_nodes.iter() { - service.network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); + service.network().add_reserved_peer(first_address.to_string()) + .expect("Error adding reserved peer"); } for (_, service, _) in network.light_nodes.iter() { - service.network().add_reserved_peer(first_address.to_string()).expect("Error adding reserved peer"); + service.network().add_reserved_peer(first_address.to_string()) + .expect("Error adding reserved peer"); } network.run_until_all_full( |_index, service| From 1f7107633477a9e3e4383eb49dd0ea4d247aca0d Mon Sep 17 00:00:00 2001 From: Ashley Ruglys Date: Tue, 30 Jun 2020 10:55:10 +0200 Subject: [PATCH 25/25] Traitify ServiceComponents Client --- bin/node/cli/src/service.rs | 1 + bin/node/testing/src/bench.rs | 2 +- client/api/src/client.rs | 3 + client/finality-grandpa/src/light_import.rs | 2 +- client/service/src/builder.rs | 104 ++++++++++---------- client/service/src/client/client.rs | 11 +-- client/service/src/lib.rs | 13 +-- client/service/test/src/client/light.rs | 8 +- test-utils/client/src/client_ext.rs | 1 + 9 files changed, 72 insertions(+), 73 deletions(-) diff --git a/bin/node/cli/src/service.rs b/bin/node/cli/src/service.rs index 608d175a76316..9707e3d8caf08 100644 --- a/bin/node/cli/src/service.rs +++ b/bin/node/cli/src/service.rs @@ -463,6 +463,7 @@ mod tests { use crate::service::{new_full_base, new_light_base}; use sp_runtime::traits::IdentifyAccount; use sp_transaction_pool::{MaintainedTransactionPool, ChainEvent}; + use sc_client_api::BlockBackend; type AccountPublic = ::Signer; diff --git a/bin/node/testing/src/bench.rs b/bin/node/testing/src/bench.rs index fc5daa80ad63e..5df2709f87053 100644 --- a/bin/node/testing/src/bench.rs +++ b/bin/node/testing/src/bench.rs @@ -55,7 +55,7 @@ use sp_api::ProvideRuntimeApi; use sp_block_builder::BlockBuilder; use sp_inherents::InherentData; use sc_client_api::{ - ExecutionStrategy, + ExecutionStrategy, BlockBackend, execution_extensions::{ExecutionExtensions, ExecutionStrategies}, }; use sp_core::{Pair, Public, sr25519, ed25519}; diff --git a/client/api/src/client.rs b/client/api/src/client.rs index 42dd5d53b19da..35d40965e6425 100644 --- a/client/api/src/client.rs +++ b/client/api/src/client.rs @@ -90,6 +90,9 @@ pub trait BlockBackend { /// Get block justification set by id. fn justification(&self, id: &BlockId) -> sp_blockchain::Result>; + + /// Get block hash by number. + fn block_hash(&self, number: NumberFor) -> sp_blockchain::Result>; } /// Provide a list of potential uncle headers for a given block. diff --git a/client/finality-grandpa/src/light_import.rs b/client/finality-grandpa/src/light_import.rs index b63c6f0bd7c1d..a7c9a655467c7 100644 --- a/client/finality-grandpa/src/light_import.rs +++ b/client/finality-grandpa/src/light_import.rs @@ -573,7 +573,7 @@ pub mod tests { use sp_consensus::{import_queue::CacheKeyId, ForkChoiceStrategy, BlockImport}; use sp_finality_grandpa::AuthorityId; use sp_core::{H256, crypto::Public}; - use sc_client_api::{in_mem::Blockchain as InMemoryAuxStore, StorageProof}; + use sc_client_api::{in_mem::Blockchain as InMemoryAuxStore, StorageProof, BlockBackend}; use substrate_test_runtime_client::runtime::{Block, Header}; use crate::tests::TestApi; use crate::finality_proof::{ diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index ebf373642219d..8c96f514ddaee 100644 --- a/client/service/src/builder.rs +++ b/client/service/src/builder.rs @@ -24,14 +24,13 @@ use crate::{ config::{Configuration, KeystoreConfig, PrometheusConfig, OffchainWorkerConfig}, }; use sc_client_api::{ - self, light::RemoteBlockchain, execution_extensions::ExtensionsFactory, - ExecutorProvider, CallExecutor, ForkBlocks, BadBlocks, CloneableSpawn, UsageProvider, - backend::RemoteBackend, + self, light::RemoteBlockchain, execution_extensions::ExtensionsFactory, ExecutorProvider, + ForkBlocks, BadBlocks, CloneableSpawn, UsageProvider, backend::RemoteBackend, }; use sp_utils::mpsc::{tracing_unbounded, TracingUnboundedSender, TracingUnboundedReceiver}; use sc_chain_spec::get_extension; use sp_consensus::{ - block_validation::{BlockAnnounceValidator, DefaultBlockAnnounceValidator}, + block_validation::{BlockAnnounceValidator, DefaultBlockAnnounceValidator, Chain}, import_queue::ImportQueue, }; use futures::{ @@ -46,9 +45,9 @@ use sc_network::NetworkService; use parking_lot::{Mutex, RwLock}; use sp_runtime::generic::BlockId; use sp_runtime::traits::{ - Block as BlockT, NumberFor, SaturatedConversion, HashFor, Zero, + Block as BlockT, NumberFor, SaturatedConversion, HashFor, Zero, BlockIdTo, }; -use sp_api::ProvideRuntimeApi; +use sp_api::{ProvideRuntimeApi, CallApiAt}; use sc_executor::{NativeExecutor, NativeExecutionDispatch, RuntimeInfo}; use std::{ collections::HashMap, @@ -62,8 +61,14 @@ use prometheus_endpoint::Registry; use sc_client_db::{Backend, DatabaseSettings}; use sp_core::traits::CodeExecutor; use sp_runtime::BuildStorage; -use sc_client_api::execution_extensions::ExecutionExtensions; +use sc_client_api::{ + BlockBackend, BlockchainEvents, + backend::StorageProvider, + proof_provider::ProofProvider, + execution_extensions::ExecutionExtensions +}; use sp_core::storage::Storage; +use sp_blockchain::{HeaderMetadata, HeaderBackend}; use crate::{ServiceComponents, TelemetryOnConnectSinks, RpcHandlers, NetworkStatusSinks}; pub type BackgroundTask = Pin + Send>>; @@ -879,11 +884,11 @@ pub trait ServiceBuilderCommand { ) -> Result; } -impl +impl ServiceBuilder< TBl, TRtApi, - Client, + TCl, Arc>, TSc, TImpQu, @@ -893,8 +898,12 @@ ServiceBuilder< TRpc, TBackend, > where - Client: ProvideRuntimeApi, - as ProvideRuntimeApi>::Api: + TCl: ProvideRuntimeApi + HeaderMetadata + Chain + + BlockBackend + BlockIdTo + ProofProvider + + HeaderBackend + BlockchainEvents + ExecutorProvider + UsageProvider + + StorageProvider + CallApiAt + + Send + 'static, + >::Api: sp_api::Metadata + sc_offchain::OffchainWorkerApi + sp_transaction_pool::runtime_api::TaggedTransactionQueue + @@ -904,7 +913,6 @@ ServiceBuilder< TBl: BlockT, TRtApi: 'static + Send + Sync, TBackend: 'static + sc_client_api::backend::Backend + Send, - TExec: 'static + CallExecutor + Send + Sync + Clone, TSc: Clone, TImpQu: 'static + ImportQueue, TExPool: MaintainedTransactionPool::Hash> + MallocSizeOfWasm + 'static, @@ -917,11 +925,7 @@ ServiceBuilder< Ok(self) } - fn build_common(self) -> Result< - ServiceComponents, Error - > - where TExec: CallExecutor, - { + fn build_common(self) -> Result, Error> { let ServiceBuilder { marker: _, mut config, @@ -940,14 +944,14 @@ ServiceBuilder< block_announce_validator_builder, } = self; + let chain_info = client.usage_info().chain; + sp_session::generate_initial_session_keys( client.clone(), - &BlockId::Hash(client.chain_info().best_hash), + &BlockId::Hash(chain_info.best_hash), config.dev_key_seed.clone().map(|s| vec![s]).unwrap_or_default(), )?; - let chain_info = client.chain_info(); - info!("📦 Highest known block at #{}", chain_info.best_number); telemetry!( SUBSTRATE_INFO; @@ -1117,20 +1121,16 @@ ServiceBuilder< } /// Builds the light service. - pub fn build_light(self) -> Result< - ServiceComponents, Error - > - where TExec: CallExecutor, - { + pub fn build_light(self) -> Result, Error> { self.build_common() } } -impl +impl ServiceBuilder< TBl, TRtApi, - Client, + TCl, Arc>, TSc, TImpQu, @@ -1140,8 +1140,12 @@ ServiceBuilder< TRpc, TBackend, > where - Client: ProvideRuntimeApi, - as ProvideRuntimeApi>::Api: + TCl: ProvideRuntimeApi + HeaderMetadata + Chain + + BlockBackend + BlockIdTo + ProofProvider + + HeaderBackend + BlockchainEvents + ExecutorProvider + UsageProvider + + StorageProvider + CallApiAt + + Send + 'static, + >::Api: sp_api::Metadata + sc_offchain::OffchainWorkerApi + sp_transaction_pool::runtime_api::TaggedTransactionQueue + @@ -1151,7 +1155,6 @@ ServiceBuilder< TBl: BlockT, TRtApi: 'static + Send + Sync, TBackend: 'static + sc_client_api::backend::Backend + Send, - TExec: 'static + CallExecutor + Send + Sync + Clone, TSc: Clone, TImpQu: 'static + ImportQueue, TExPool: MaintainedTransactionPool::Hash> + @@ -1162,11 +1165,7 @@ ServiceBuilder< { /// Builds the full service. - pub fn build_full(self) -> Result< - ServiceComponents, Error - > - where TExec: CallExecutor, - { + pub fn build_full(self) -> Result, Error> { // make transaction pool available for off-chain runtime calls. self.client.execution_extensions() .register_transaction_pool(Arc::downgrade(&self.transaction_pool) as _); @@ -1198,18 +1197,16 @@ async fn transaction_notifications( } // Periodically notify the telemetry. -async fn telemetry_periodic_send( - client: Arc>, +async fn telemetry_periodic_send( + client: Arc, transaction_pool: Arc, mut metrics_service: MetricsService, network_status_sinks: Arc, NetworkState)>>> ) where TBl: BlockT, - TExec: CallExecutor, - Client: ProvideRuntimeApi, + TCl: ProvideRuntimeApi + UsageProvider, TExPool: MaintainedTransactionPool::Hash>, - TBackend: sc_client_api::backend::Backend, { let (state_tx, state_rx) = tracing_unbounded::<(NetworkStatus<_>, NetworkState)>("mpsc_netstat1"); network_status_sinks.lock().push(std::time::Duration::from_millis(5000), state_tx); @@ -1287,11 +1284,11 @@ fn build_telemetry( (telemetry, future) } -fn gen_handler( +fn gen_handler( deny_unsafe: sc_rpc::DenyUnsafe, config: &Configuration, task_manager: &TaskManager, - client: Arc>, + client: Arc, transaction_pool: Arc, keystore: Arc>, on_demand: Option>>, @@ -1302,13 +1299,14 @@ fn gen_handler( ) -> jsonrpc_pubsub::PubSubHandler where TBl: BlockT, - TExec: CallExecutor + Send + Sync + 'static, - TRtApi: Send + Sync + 'static, - Client: ProvideRuntimeApi, + TCl: ProvideRuntimeApi + BlockchainEvents + HeaderBackend + + HeaderMetadata + ExecutorProvider + + CallApiAt + ProofProvider + + StorageProvider + BlockBackend + Send + Sync + 'static, TExPool: MaintainedTransactionPool::Hash> + 'static, TBackend: sc_client_api::backend::Backend + 'static, TRpc: sc_rpc::RpcExtension, - as ProvideRuntimeApi>::Api: + >::Api: sp_session::SessionKeys + sp_api::Metadata, { @@ -1377,15 +1375,14 @@ fn gen_handler( )) } -fn build_network( +fn build_network( config: &Configuration, - client: Arc>, + client: Arc, transaction_pool: Arc, spawn_handle: SpawnTaskHandle, on_demand: Option>>, block_announce_validator_builder: Option>) -> - Box + Send> + Send + dyn FnOnce(Arc) -> Box + Send> + Send >>, finality_proof_request_builder: Option>, finality_proof_provider: Option>>, @@ -1401,11 +1398,10 @@ fn build_network( > where TBl: BlockT, - TExec: CallExecutor + Send + Sync + 'static, - TRtApi: Send + Sync + 'static, - Client: ProvideRuntimeApi, + TCl: ProvideRuntimeApi + HeaderMetadata + Chain + + BlockBackend + BlockIdTo + ProofProvider + + HeaderBackend + BlockchainEvents + 'static, TExPool: MaintainedTransactionPool::Hash> + 'static, - TBackend: sc_client_api::backend::Backend + 'static, TImpQu: ImportQueue + 'static, { let transaction_pool_adapter = Arc::new(TransactionPoolAdapter { diff --git a/client/service/src/client/client.rs b/client/service/src/client/client.rs index 922f34b656813..2f101465d516f 100644 --- a/client/service/src/client/client.rs +++ b/client/service/src/client/client.rs @@ -353,13 +353,6 @@ impl Client where self.executor.runtime_version(id) } - /// Get block hash by number. - pub fn block_hash(&self, - block_number: <::Header as HeaderT>::Number - ) -> sp_blockchain::Result> { - self.backend.blockchain().hash(block_number) - } - /// Reads given header and generates CHT-based header proof for CHT of given size. pub fn header_proof_with_cht_size( &self, @@ -1925,6 +1918,10 @@ impl BlockBackend for Client fn justification(&self, id: &BlockId) -> sp_blockchain::Result> { self.backend.blockchain().justification(*id) } + + fn block_hash(&self, number: NumberFor) -> sp_blockchain::Result> { + self.backend.blockchain().hash(number) + } } impl backend::AuxStore for Client diff --git a/client/service/src/lib.rs b/client/service/src/lib.rs index 000b30d161265..c3c8f60e689ad 100644 --- a/client/service/src/lib.rs +++ b/client/service/src/lib.rs @@ -43,7 +43,6 @@ use wasm_timer::Instant; use std::task::Poll; use parking_lot::Mutex; -use client::Client; use futures::{Future, FutureExt, Stream, StreamExt, compat::*}; use sc_network::{NetworkStatus, network_state::NetworkState, PeerId}; use log::{log, warn, debug, error, Level}; @@ -80,7 +79,7 @@ pub use sc_network::config::{ pub use sc_tracing::TracingReceiver; pub use task_manager::SpawnTaskHandle; pub use task_manager::TaskManager; -use sc_client_api::BlockchainEvents; +use sc_client_api::{Backend, BlockchainEvents}; const DEFAULT_PROTOCOL_ID: &str = "sup"; @@ -152,11 +151,9 @@ impl TelemetryOnConnectSinks { /// The individual components of the chain, built by the service builder. You are encouraged to /// deconstruct this into its fields. -pub struct ServiceComponents< - TBl: BlockT, TBackend: sc_client_api::backend::Backend, TExec, TRtApi, TSc, TExPool, -> { +pub struct ServiceComponents, TSc, TExPool, TCl> { /// A blockchain client. - pub client: Arc>, + pub client: Arc, /// A shared transaction pool instance. pub transaction_pool: Arc, /// The chain task manager. @@ -177,9 +174,7 @@ pub struct ServiceComponents< pub telemetry_on_connect_sinks: TelemetryOnConnectSinks, /// A shared offchain workers instance. pub offchain_workers: Option, - TBackend::OffchainStorage, - TBl + TCl, TBackend::OffchainStorage, TBl >>>, } diff --git a/client/service/test/src/client/light.rs b/client/service/test/src/client/light.rs index 994d846c6a088..e72c290d43bbe 100644 --- a/client/service/test/src/client/light.rs +++ b/client/service/test/src/client/light.rs @@ -40,7 +40,13 @@ use sp_api::{InitializeBlock, StorageTransactionCache, ProofRecorder, OffchainOv use sp_consensus::{BlockOrigin}; use sc_executor::{NativeExecutor, WasmExecutionMethod, RuntimeVersion, NativeVersion}; use sp_core::{H256, tasks::executor as tasks_executor, NativeOrEncoded}; -use sc_client_api::{blockchain::Info, backend::NewBlockState, Backend as ClientBackend, ProofProvider, in_mem::{Backend as InMemBackend, Blockchain as InMemoryBlockchain}, AuxStore, Storage, CallExecutor, cht, ExecutionStrategy, StorageProof, BlockImportOperation, RemoteCallRequest, StorageProvider, ChangesProof, RemoteBodyRequest, RemoteReadRequest, RemoteChangesRequest, FetchChecker, RemoteReadChildRequest, RemoteHeaderRequest}; +use sc_client_api::{ + blockchain::Info, backend::NewBlockState, Backend as ClientBackend, ProofProvider, + in_mem::{Backend as InMemBackend, Blockchain as InMemoryBlockchain}, + AuxStore, Storage, CallExecutor, cht, ExecutionStrategy, StorageProof, BlockImportOperation, + RemoteCallRequest, StorageProvider, ChangesProof, RemoteBodyRequest, RemoteReadRequest, + RemoteChangesRequest, FetchChecker, RemoteReadChildRequest, RemoteHeaderRequest, BlockBackend, +}; use sp_externalities::Extensions; use sc_block_builder::BlockBuilderProvider; use sp_blockchain::{ diff --git a/test-utils/client/src/client_ext.rs b/test-utils/client/src/client_ext.rs index 706a7b6e95a85..a74bd3258ef0f 100644 --- a/test-utils/client/src/client_ext.rs +++ b/test-utils/client/src/client_ext.rs @@ -19,6 +19,7 @@ use sc_service::client::Client; use sc_client_api::backend::Finalizer; +use sc_client_api::client::BlockBackend; use sp_consensus::{ BlockImportParams, BlockImport, BlockOrigin, Error as ConsensusError, ForkChoiceStrategy,