From 7bf7c9450ec54b8d2d04a6d391443e2b3cfa2f80 Mon Sep 17 00:00:00 2001 From: Aaro Altonen Date: Thu, 9 Mar 2023 15:55:05 +0200 Subject: [PATCH 01/10] Move service tests to `client/network/tests` These tests depend on `sc-network` and `sc-network-sync` so they should live outside the crate. --- client/network/src/service.rs | 12 +- client/network/src/service/tests/mod.rs | 240 ------------------ client/network/test/src/lib.rs | 2 + .../service/tests => test/src}/service.rs | 228 ++++++++++++++++- 4 files changed, 224 insertions(+), 258 deletions(-) delete mode 100644 client/network/src/service/tests/mod.rs rename client/network/{src/service/tests => test/src}/service.rs (73%) diff --git a/client/network/src/service.rs b/client/network/src/service.rs index 4a05393618cfb..e0f369c8b62a5 100644 --- a/client/network/src/service.rs +++ b/client/network/src/service.rs @@ -93,8 +93,6 @@ pub use behaviour::{InboundFailure, OutboundFailure, ResponseFailure}; mod metrics; mod out_events; -#[cfg(test)] -mod tests; pub use libp2p::identity::{error::DecodingError, Keypair, PublicKey}; use sc_network_common::service::NetworkRequest; @@ -1432,10 +1430,11 @@ where }, } }, - SwarmEvent::Behaviour(BehaviourOut::ReputationChanges { peer, changes }) => + SwarmEvent::Behaviour(BehaviourOut::ReputationChanges { peer, changes }) => { for change in changes { self.network_service.behaviour().user_protocol().report_peer(peer, change); - }, + } + }, SwarmEvent::Behaviour(BehaviourOut::PeerIdentify { peer_id, info: @@ -1467,10 +1466,11 @@ where .user_protocol_mut() .add_default_set_discovered_nodes(iter::once(peer_id)); }, - SwarmEvent::Behaviour(BehaviourOut::RandomKademliaStarted) => + SwarmEvent::Behaviour(BehaviourOut::RandomKademliaStarted) => { if let Some(metrics) = self.metrics.as_ref() { metrics.kademlia_random_queries_total.inc(); - }, + } + }, SwarmEvent::Behaviour(BehaviourOut::NotificationStreamOpened { remote, protocol, diff --git a/client/network/src/service/tests/mod.rs b/client/network/src/service/tests/mod.rs deleted file mode 100644 index f29e43e6ce366..0000000000000 --- a/client/network/src/service/tests/mod.rs +++ /dev/null @@ -1,240 +0,0 @@ -// This file is part of Substrate. - -// Copyright (C) Parity Technologies (UK) Ltd. -// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0 - -// This program is free software: you can redistribute it and/or modify -// it under the terms of the GNU General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. - -// This program is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU General Public License for more details. - -// You should have received a copy of the GNU General Public License -// along with this program. If not, see . - -use crate::{config, NetworkService, NetworkWorker}; - -use futures::prelude::*; -use libp2p::Multiaddr; -use sc_consensus::{ImportQueue, Link}; -use sc_network_common::{ - config::{NonDefaultSetConfig, ProtocolId, SetConfig, TransportConfig}, - protocol::{event::Event, role::Roles}, - service::NetworkEventStream, -}; -use sc_network_light::light_client_requests::handler::LightClientRequestHandler; -use sc_network_sync::{ - block_request_handler::BlockRequestHandler, - engine::SyncingEngine, - service::network::{NetworkServiceHandle, NetworkServiceProvider}, - state_request_handler::StateRequestHandler, -}; -use sp_runtime::traits::Block as BlockT; -use std::sync::Arc; -use substrate_test_runtime_client::{ - runtime::{Block as TestBlock, Hash as TestHash}, - TestClientBuilder, TestClientBuilderExt as _, -}; - -#[cfg(test)] -mod service; - -type TestNetworkWorker = NetworkWorker; -type TestNetworkService = NetworkService; - -const PROTOCOL_NAME: &str = "/foo"; - -struct TestNetwork { - network: TestNetworkWorker, -} - -impl TestNetwork { - pub fn new(network: TestNetworkWorker) -> Self { - Self { network } - } - - pub fn start_network( - self, - ) -> (Arc, (impl Stream + std::marker::Unpin)) { - let worker = self.network; - let service = worker.service().clone(); - let event_stream = service.event_stream("test"); - - tokio::spawn(worker.run()); - - (service, event_stream) - } -} - -struct TestNetworkBuilder { - import_queue: Option>>, - link: Option>>, - client: Option>, - listen_addresses: Vec, - set_config: Option, - chain_sync_network: Option<(NetworkServiceProvider, NetworkServiceHandle)>, - config: Option, -} - -impl TestNetworkBuilder { - pub fn new() -> Self { - Self { - import_queue: None, - link: None, - client: None, - listen_addresses: Vec::new(), - set_config: None, - chain_sync_network: None, - config: None, - } - } - - pub fn with_config(mut self, config: config::NetworkConfiguration) -> Self { - self.config = Some(config); - self - } - - pub fn with_listen_addresses(mut self, addresses: Vec) -> Self { - self.listen_addresses = addresses; - self - } - - pub fn with_set_config(mut self, set_config: SetConfig) -> Self { - self.set_config = Some(set_config); - self - } - - pub fn build(mut self) -> TestNetwork { - let client = self.client.as_mut().map_or( - Arc::new(TestClientBuilder::with_default_backend().build_with_longest_chain().0), - |v| v.clone(), - ); - - let network_config = self.config.unwrap_or(config::NetworkConfiguration { - extra_sets: vec![NonDefaultSetConfig { - notifications_protocol: PROTOCOL_NAME.into(), - fallback_names: Vec::new(), - max_notification_size: 1024 * 1024, - handshake: None, - set_config: self.set_config.unwrap_or_default(), - }], - listen_addresses: self.listen_addresses, - transport: TransportConfig::MemoryOnly, - ..config::NetworkConfiguration::new_local() - }); - - #[derive(Clone)] - struct PassThroughVerifier(bool); - - #[async_trait::async_trait] - impl sc_consensus::Verifier for PassThroughVerifier { - async fn verify( - &mut self, - mut block: sc_consensus::BlockImportParams, - ) -> Result, String> { - block.finalized = self.0; - block.fork_choice = Some(sc_consensus::ForkChoiceStrategy::LongestChain); - Ok(block) - } - } - - let mut import_queue = - self.import_queue.unwrap_or(Box::new(sc_consensus::BasicQueue::new( - PassThroughVerifier(false), - Box::new(client.clone()), - None, - &sp_core::testing::TaskExecutor::new(), - None, - ))); - - let protocol_id = ProtocolId::from("test-protocol-name"); - let fork_id = Some(String::from("test-fork-id")); - - let block_request_protocol_config = { - let (handler, protocol_config) = - BlockRequestHandler::new(&protocol_id, None, client.clone(), 50); - tokio::spawn(handler.run().boxed()); - protocol_config - }; - - let state_request_protocol_config = { - let (handler, protocol_config) = - StateRequestHandler::new(&protocol_id, None, client.clone(), 50); - tokio::spawn(handler.run().boxed()); - protocol_config - }; - - let light_client_request_protocol_config = { - let (handler, protocol_config) = - LightClientRequestHandler::new(&protocol_id, None, client.clone()); - tokio::spawn(handler.run().boxed()); - protocol_config - }; - - let (chain_sync_network_provider, chain_sync_network_handle) = - self.chain_sync_network.unwrap_or(NetworkServiceProvider::new()); - - let (engine, chain_sync_service, block_announce_config) = SyncingEngine::new( - Roles::from(&config::Role::Full), - client.clone(), - None, - &network_config, - protocol_id.clone(), - &None, - Box::new(sp_consensus::block_validation::DefaultBlockAnnounceValidator), - None, - chain_sync_network_handle, - import_queue.service(), - block_request_protocol_config.name.clone(), - state_request_protocol_config.name.clone(), - None, - ) - .unwrap(); - let mut link = self.link.unwrap_or(Box::new(chain_sync_service.clone())); - let worker = NetworkWorker::< - substrate_test_runtime_client::runtime::Block, - substrate_test_runtime_client::runtime::Hash, - >::new(config::Params { - block_announce_config, - role: config::Role::Full, - executor: Box::new(|f| { - tokio::spawn(f); - }), - network_config, - chain: client.clone(), - protocol_id, - fork_id, - metrics_registry: None, - request_response_protocol_configs: [ - block_request_protocol_config, - state_request_protocol_config, - light_client_request_protocol_config, - ] - .to_vec(), - }) - .unwrap(); - - let service = worker.service().clone(); - tokio::spawn(async move { - let _ = chain_sync_network_provider.run(service).await; - }); - tokio::spawn(async move { - loop { - futures::future::poll_fn(|cx| { - import_queue.poll_actions(cx, &mut *link); - std::task::Poll::Ready(()) - }) - .await; - tokio::time::sleep(std::time::Duration::from_millis(250)).await; - } - }); - let stream = worker.service().event_stream("syncing"); - tokio::spawn(engine.run(stream)); - - TestNetwork::new(worker) - } -} diff --git a/client/network/test/src/lib.rs b/client/network/test/src/lib.rs index a79e17e2fb6d1..6a895687f149e 100644 --- a/client/network/test/src/lib.rs +++ b/client/network/test/src/lib.rs @@ -20,6 +20,8 @@ #[cfg(test)] mod block_import; #[cfg(test)] +mod service; +#[cfg(test)] mod sync; use std::{ diff --git a/client/network/src/service/tests/service.rs b/client/network/test/src/service.rs similarity index 73% rename from client/network/src/service/tests/service.rs rename to client/network/test/src/service.rs index 9c4c0ad6e0cd9..4b065a16d8699 100644 --- a/client/network/src/service/tests/service.rs +++ b/client/network/test/src/service.rs @@ -16,24 +16,227 @@ // You should have received a copy of the GNU General Public License // along with this program. If not, see . -use crate::{config, service::tests::TestNetworkBuilder, NetworkService}; - use futures::prelude::*; -use libp2p::PeerId; +use libp2p::{Multiaddr, PeerId}; + +use sc_consensus::{ImportQueue, Link}; +use sc_network::{config, NetworkService, NetworkWorker}; use sc_network_common::{ - config::{MultiaddrWithPeerId, NonDefaultSetConfig, SetConfig, TransportConfig}, - protocol::event::Event, - service::{NetworkNotification, NetworkPeers, NetworkStateInfo}, + config::{MultiaddrWithPeerId, NonDefaultSetConfig, ProtocolId, SetConfig, TransportConfig}, + protocol::{event::Event, role::Roles}, + service::{NetworkEventStream, NetworkNotification, NetworkPeers, NetworkStateInfo}, +}; +use sc_network_light::light_client_requests::handler::LightClientRequestHandler; +use sc_network_sync::{ + block_request_handler::BlockRequestHandler, + engine::SyncingEngine, + service::network::{NetworkServiceHandle, NetworkServiceProvider}, + state_request_handler::StateRequestHandler, +}; +use sp_runtime::traits::Block as BlockT; +use substrate_test_runtime_client::{ + runtime::{Block as TestBlock, Hash as TestHash}, + TestClientBuilder, TestClientBuilderExt as _, }; + use std::{sync::Arc, time::Duration}; -type TestNetworkService = NetworkService< - substrate_test_runtime_client::runtime::Block, - substrate_test_runtime_client::runtime::Hash, ->; +type TestNetworkWorker = NetworkWorker; +type TestNetworkService = NetworkService; const PROTOCOL_NAME: &str = "/foo"; +struct TestNetwork { + network: TestNetworkWorker, +} + +impl TestNetwork { + pub fn new(network: TestNetworkWorker) -> Self { + Self { network } + } + + pub fn start_network( + self, + ) -> (Arc, (impl Stream + std::marker::Unpin)) { + let worker = self.network; + let service = worker.service().clone(); + let event_stream = service.event_stream("test"); + + tokio::spawn(worker.run()); + + (service, event_stream) + } +} + +struct TestNetworkBuilder { + import_queue: Option>>, + link: Option>>, + client: Option>, + listen_addresses: Vec, + set_config: Option, + chain_sync_network: Option<(NetworkServiceProvider, NetworkServiceHandle)>, + config: Option, +} + +impl TestNetworkBuilder { + pub fn new() -> Self { + Self { + import_queue: None, + link: None, + client: None, + listen_addresses: Vec::new(), + set_config: None, + chain_sync_network: None, + config: None, + } + } + + pub fn with_config(mut self, config: config::NetworkConfiguration) -> Self { + self.config = Some(config); + self + } + + pub fn with_listen_addresses(mut self, addresses: Vec) -> Self { + self.listen_addresses = addresses; + self + } + + pub fn with_set_config(mut self, set_config: SetConfig) -> Self { + self.set_config = Some(set_config); + self + } + + pub fn build(mut self) -> TestNetwork { + let client = self.client.as_mut().map_or( + Arc::new(TestClientBuilder::with_default_backend().build_with_longest_chain().0), + |v| v.clone(), + ); + + let network_config = self.config.unwrap_or(config::NetworkConfiguration { + extra_sets: vec![NonDefaultSetConfig { + notifications_protocol: PROTOCOL_NAME.into(), + fallback_names: Vec::new(), + max_notification_size: 1024 * 1024, + handshake: None, + set_config: self.set_config.unwrap_or_default(), + }], + listen_addresses: self.listen_addresses, + transport: TransportConfig::MemoryOnly, + ..config::NetworkConfiguration::new_local() + }); + + #[derive(Clone)] + struct PassThroughVerifier(bool); + + #[async_trait::async_trait] + impl sc_consensus::Verifier for PassThroughVerifier { + async fn verify( + &mut self, + mut block: sc_consensus::BlockImportParams, + ) -> Result, String> { + block.finalized = self.0; + block.fork_choice = Some(sc_consensus::ForkChoiceStrategy::LongestChain); + Ok(block) + } + } + + let mut import_queue = + self.import_queue.unwrap_or(Box::new(sc_consensus::BasicQueue::new( + PassThroughVerifier(false), + Box::new(client.clone()), + None, + &sp_core::testing::TaskExecutor::new(), + None, + ))); + + let protocol_id = ProtocolId::from("test-protocol-name"); + let fork_id = Some(String::from("test-fork-id")); + + let block_request_protocol_config = { + let (handler, protocol_config) = + BlockRequestHandler::new(&protocol_id, None, client.clone(), 50); + tokio::spawn(handler.run().boxed()); + protocol_config + }; + + let state_request_protocol_config = { + let (handler, protocol_config) = + StateRequestHandler::new(&protocol_id, None, client.clone(), 50); + tokio::spawn(handler.run().boxed()); + protocol_config + }; + + let light_client_request_protocol_config = { + let (handler, protocol_config) = + LightClientRequestHandler::new(&protocol_id, None, client.clone()); + tokio::spawn(handler.run().boxed()); + protocol_config + }; + + let (chain_sync_network_provider, chain_sync_network_handle) = + self.chain_sync_network.unwrap_or(NetworkServiceProvider::new()); + + let (engine, chain_sync_service, block_announce_config) = SyncingEngine::new( + Roles::from(&config::Role::Full), + client.clone(), + None, + &network_config, + protocol_id.clone(), + &None, + Box::new(sp_consensus::block_validation::DefaultBlockAnnounceValidator), + None, + chain_sync_network_handle, + import_queue.service(), + block_request_protocol_config.name.clone(), + state_request_protocol_config.name.clone(), + None, + ) + .unwrap(); + let mut link = self.link.unwrap_or(Box::new(chain_sync_service.clone())); + let worker = NetworkWorker::< + substrate_test_runtime_client::runtime::Block, + substrate_test_runtime_client::runtime::Hash, + >::new(config::Params { + block_announce_config, + role: config::Role::Full, + executor: Box::new(|f| { + tokio::spawn(f); + }), + network_config, + chain: client.clone(), + protocol_id, + fork_id, + metrics_registry: None, + request_response_protocol_configs: [ + block_request_protocol_config, + state_request_protocol_config, + light_client_request_protocol_config, + ] + .to_vec(), + }) + .unwrap(); + + let service = worker.service().clone(); + tokio::spawn(async move { + let _ = chain_sync_network_provider.run(service).await; + }); + tokio::spawn(async move { + loop { + futures::future::poll_fn(|cx| { + import_queue.poll_actions(cx, &mut *link); + std::task::Poll::Ready(()) + }) + .await; + tokio::time::sleep(std::time::Duration::from_millis(250)).await; + } + }); + let stream = worker.service().event_stream("syncing"); + tokio::spawn(engine.run(stream)); + + TestNetwork::new(worker) + } +} + /// Builds two nodes and their associated events stream. /// The nodes are connected together and have the `PROTOCOL_NAME` protocol registered. fn build_nodes_one_proto() -> ( @@ -289,10 +492,11 @@ async fn notifications_back_pressure() { while received_notifications < TOTAL_NOTIFS { match events_stream2.next().await.unwrap() { - Event::NotificationStreamOpened { protocol, .. } => + Event::NotificationStreamOpened { protocol, .. } => { if let None = sync_protocol_name { sync_protocol_name = Some(protocol); - }, + } + }, Event::NotificationStreamClosed { protocol, .. } => { if Some(&protocol) != sync_protocol_name.as_ref() { panic!() From 4fb0634f1a85be1fa548d02397a88d291a94245f Mon Sep 17 00:00:00 2001 From: Aaro Altonen Date: Thu, 9 Mar 2023 17:38:45 +0200 Subject: [PATCH 02/10] Move some configs from `sc-network-common` to `sc-network` --- Cargo.lock | 2 + .../consensus/beefy/src/communication/mod.rs | 5 +- client/consensus/grandpa/src/lib.rs | 4 +- client/network/common/src/config.rs | 204 --------------- client/network/common/src/lib.rs | 1 - client/network/src/config.rs | 236 +++++++++++++++++- client/network/src/error.rs | 79 ++++++ client/network/src/lib.rs | 1 + client/network/src/protocol.rs | 20 +- client/network/src/service.rs | 16 +- client/network/sync/Cargo.toml | 1 + client/network/sync/src/engine.rs | 13 +- client/network/sync/src/lib.rs | 44 ++-- client/network/test/src/lib.rs | 9 +- client/network/test/src/service.rs | 6 +- client/network/transactions/Cargo.toml | 1 + client/network/transactions/src/lib.rs | 11 +- client/service/src/config.rs | 7 +- client/service/src/error.rs | 2 +- 19 files changed, 395 insertions(+), 267 deletions(-) create mode 100644 client/network/src/error.rs diff --git a/Cargo.lock b/Cargo.lock index 9a63e25b4878c..f6c124c2818cd 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8939,6 +8939,7 @@ dependencies = [ "sc-block-builder", "sc-client-api", "sc-consensus", + "sc-network", "sc-network-common", "sc-peerset", "sc-utils", @@ -8997,6 +8998,7 @@ dependencies = [ "log", "parity-scale-codec", "pin-project", + "sc-network", "sc-network-common", "sc-peerset", "sc-utils", diff --git a/client/consensus/beefy/src/communication/mod.rs b/client/consensus/beefy/src/communication/mod.rs index f91b08bbc97ec..295d549bb1ba8 100644 --- a/client/consensus/beefy/src/communication/mod.rs +++ b/client/consensus/beefy/src/communication/mod.rs @@ -67,9 +67,8 @@ pub(crate) mod beefy_protocol_name { /// For standard protocol name see [`beefy_protocol_name::gossip_protocol_name`]. pub fn beefy_peers_set_config( gossip_protocol_name: sc_network::ProtocolName, -) -> sc_network_common::config::NonDefaultSetConfig { - let mut cfg = - sc_network_common::config::NonDefaultSetConfig::new(gossip_protocol_name, 1024 * 1024); +) -> sc_network::config::NonDefaultSetConfig { + let mut cfg = sc_network::config::NonDefaultSetConfig::new(gossip_protocol_name, 1024 * 1024); cfg.allow_non_reserved(25, 25); cfg } diff --git a/client/consensus/grandpa/src/lib.rs b/client/consensus/grandpa/src/lib.rs index 7bf48a498fabb..7c50b2f255577 100644 --- a/client/consensus/grandpa/src/lib.rs +++ b/client/consensus/grandpa/src/lib.rs @@ -695,9 +695,9 @@ pub struct GrandpaParams { /// For standard protocol name see [`crate::protocol_standard_name`]. pub fn grandpa_peers_set_config( protocol_name: ProtocolName, -) -> sc_network_common::config::NonDefaultSetConfig { +) -> sc_network::config::NonDefaultSetConfig { use communication::grandpa_protocol_name; - sc_network_common::config::NonDefaultSetConfig { + sc_network::config::NonDefaultSetConfig { notifications_protocol: protocol_name, fallback_names: grandpa_protocol_name::LEGACY_NAMES.iter().map(|&n| n.into()).collect(), // Notifications reach ~256kiB in size at the time of writing on Kusama and Polkadot. diff --git a/client/network/common/src/config.rs b/client/network/common/src/config.rs index 6a02129fcfe06..1281f1b60e4a6 100644 --- a/client/network/common/src/config.rs +++ b/client/network/common/src/config.rs @@ -39,8 +39,6 @@ use std::{ error::Error, fmt, fs, io::{self, Write}, - iter, - net::Ipv4Addr, path::{Path, PathBuf}, str, str::FromStr, @@ -246,74 +244,6 @@ impl std::ops::Deref for NotificationHandshake { } } -/// Extension to [`SetConfig`] for sets that aren't the default set. -/// -/// > **Note**: As new fields might be added in the future, please consider using the `new` method -/// > and modifiers instead of creating this struct manually. -#[derive(Clone, Debug)] -pub struct NonDefaultSetConfig { - /// Name of the notifications protocols of this set. A substream on this set will be - /// considered established once this protocol is open. - /// - /// > **Note**: This field isn't present for the default set, as this is handled internally - /// > by the networking code. - pub notifications_protocol: protocol::ProtocolName, - /// If the remote reports that it doesn't support the protocol indicated in the - /// `notifications_protocol` field, then each of these fallback names will be tried one by - /// one. - /// - /// If a fallback is used, it will be reported in - /// `sc_network::protocol::event::Event::NotificationStreamOpened::negotiated_fallback` - pub fallback_names: Vec, - /// Handshake of the protocol - /// - /// NOTE: Currently custom handshakes are not fully supported. See issue #5685 for more - /// details. This field is temporarily used to allow moving the hardcoded block announcement - /// protocol out of `protocol.rs`. - pub handshake: Option, - /// Maximum allowed size of single notifications. - pub max_notification_size: u64, - /// Base configuration. - pub set_config: SetConfig, -} - -impl NonDefaultSetConfig { - /// Creates a new [`NonDefaultSetConfig`]. Zero slots and accepts only reserved nodes. - pub fn new(notifications_protocol: protocol::ProtocolName, max_notification_size: u64) -> Self { - Self { - notifications_protocol, - max_notification_size, - fallback_names: Vec::new(), - handshake: None, - set_config: SetConfig { - in_peers: 0, - out_peers: 0, - reserved_nodes: Vec::new(), - non_reserved_mode: NonReservedPeerMode::Deny, - }, - } - } - - /// Modifies the configuration to allow non-reserved nodes. - pub fn allow_non_reserved(&mut self, in_peers: u32, out_peers: u32) { - self.set_config.in_peers = in_peers; - self.set_config.out_peers = out_peers; - self.set_config.non_reserved_mode = NonReservedPeerMode::Accept; - } - - /// Add a node to the list of reserved nodes. - pub fn add_reserved(&mut self, peer: MultiaddrWithPeerId) { - self.set_config.reserved_nodes.push(peer); - } - - /// Add a list of protocol names used for backward compatibility. - /// - /// See the explanations in [`NonDefaultSetConfig::fallback_names`]. - pub fn add_fallback_names(&mut self, fallback_names: Vec) { - self.fallback_names.extend(fallback_names); - } -} - /// Configuration for the transport layer. #[derive(Clone, Debug)] pub enum TransportConfig { @@ -388,140 +318,6 @@ impl Default for SyncMode { } } -/// Network service configuration. -#[derive(Clone, Debug)] -pub struct NetworkConfiguration { - /// Directory path to store network-specific configuration. None means nothing will be saved. - pub net_config_path: Option, - /// Multiaddresses to listen for incoming connections. - pub listen_addresses: Vec, - /// Multiaddresses to advertise. Detected automatically if empty. - pub public_addresses: Vec, - /// List of initial node addresses - pub boot_nodes: Vec, - /// The node key configuration, which determines the node's network identity keypair. - pub node_key: NodeKeyConfig, - /// List of request-response protocols that the node supports. - pub request_response_protocols: Vec, - /// Configuration for the default set of nodes used for block syncing and transactions. - pub default_peers_set: SetConfig, - /// Number of substreams to reserve for full nodes for block syncing and transactions. - /// Any other slot will be dedicated to light nodes. - /// - /// This value is implicitly capped to `default_set.out_peers + default_set.in_peers`. - pub default_peers_set_num_full: u32, - /// Configuration for extra sets of nodes. - pub extra_sets: Vec, - /// Client identifier. Sent over the wire for debugging purposes. - pub client_version: String, - /// Name of the node. Sent over the wire for debugging purposes. - pub node_name: String, - /// Configuration for the transport layer. - pub transport: TransportConfig, - /// Maximum number of peers to ask the same blocks in parallel. - pub max_parallel_downloads: u32, - /// Initial syncing mode. - pub sync_mode: SyncMode, - - /// True if Kademlia random discovery should be enabled. - /// - /// If true, the node will automatically randomly walk the DHT in order to find new peers. - pub enable_dht_random_walk: bool, - - /// Should we insert non-global addresses into the DHT? - pub allow_non_globals_in_dht: bool, - - /// Require iterative Kademlia DHT queries to use disjoint paths for increased resiliency in - /// the presence of potentially adversarial nodes. - pub kademlia_disjoint_query_paths: bool, - /// Enable serving block data over IPFS bitswap. - pub ipfs_server: bool, - - /// Size of Yamux receive window of all substreams. `None` for the default (256kiB). - /// Any value less than 256kiB is invalid. - /// - /// # Context - /// - /// By design, notifications substreams on top of Yamux connections only allow up to `N` bytes - /// to be transferred at a time, where `N` is the Yamux receive window size configurable here. - /// This means, in practice, that every `N` bytes must be acknowledged by the receiver before - /// the sender can send more data. The maximum bandwidth of each notifications substream is - /// therefore `N / round_trip_time`. - /// - /// It is recommended to leave this to `None`, and use a request-response protocol instead if - /// a large amount of data must be transferred. The reason why the value is configurable is - /// that some Substrate users mis-use notification protocols to send large amounts of data. - /// As such, this option isn't designed to stay and will likely get removed in the future. - /// - /// Note that configuring a value here isn't a modification of the Yamux protocol, but rather - /// a modification of the way the implementation works. Different nodes with different - /// configured values remain compatible with each other. - pub yamux_window_size: Option, -} - -impl NetworkConfiguration { - /// Create new default configuration - pub fn new, SV: Into>( - node_name: SN, - client_version: SV, - node_key: NodeKeyConfig, - net_config_path: Option, - ) -> Self { - let default_peers_set = SetConfig::default(); - Self { - net_config_path, - listen_addresses: Vec::new(), - public_addresses: Vec::new(), - boot_nodes: Vec::new(), - node_key, - request_response_protocols: Vec::new(), - default_peers_set_num_full: default_peers_set.in_peers + default_peers_set.out_peers, - default_peers_set, - extra_sets: Vec::new(), - client_version: client_version.into(), - node_name: node_name.into(), - transport: TransportConfig::Normal { enable_mdns: false, allow_private_ip: true }, - max_parallel_downloads: 5, - sync_mode: SyncMode::Full, - enable_dht_random_walk: true, - allow_non_globals_in_dht: false, - kademlia_disjoint_query_paths: false, - yamux_window_size: None, - ipfs_server: false, - } - } - - /// Create new default configuration for localhost-only connection with random port (useful for - /// testing) - pub fn new_local() -> NetworkConfiguration { - let mut config = - NetworkConfiguration::new("test-node", "test-client", Default::default(), None); - - config.listen_addresses = - vec![iter::once(multiaddr::Protocol::Ip4(Ipv4Addr::new(127, 0, 0, 1))) - .chain(iter::once(multiaddr::Protocol::Tcp(0))) - .collect()]; - - config.allow_non_globals_in_dht = true; - config - } - - /// Create new default configuration for localhost-only connection with random port (useful for - /// testing) - pub fn new_memory() -> NetworkConfiguration { - let mut config = - NetworkConfiguration::new("test-node", "test-client", Default::default(), None); - - config.listen_addresses = - vec![iter::once(multiaddr::Protocol::Ip4(Ipv4Addr::new(127, 0, 0, 1))) - .chain(iter::once(multiaddr::Protocol::Tcp(0))) - .collect()]; - - config.allow_non_globals_in_dht = true; - config - } -} - /// The configuration of a node's secret key, describing the type of key /// and how it is obtained. A node's identity keypair is the result of /// the evaluation of the node key configuration. diff --git a/client/network/common/src/lib.rs b/client/network/common/src/lib.rs index 10d2449388bb9..dfae3d0a0c2be 100644 --- a/client/network/common/src/lib.rs +++ b/client/network/common/src/lib.rs @@ -19,7 +19,6 @@ //! Common data structures of the networking layer. pub mod config; -pub mod error; pub mod message; pub mod protocol; pub mod request_responses; diff --git a/client/network/src/config.rs b/client/network/src/config.rs index 90d02af848bf4..ef2936be9e176 100644 --- a/client/network/src/config.rs +++ b/client/network/src/config.rs @@ -21,9 +21,17 @@ //! The [`Params`] struct is the struct that must be passed in order to initialize the networking. //! See the documentation of [`Params`]. +// External dependencies +pub use libp2p::{build_multiaddr, core::PublicKey, identity, multiaddr, Multiaddr}; + +// Substrate dependencies +use prometheus_endpoint::Registry; pub use sc_network_common::{ - config::{NetworkConfiguration, ProtocolId}, - protocol::role::Role, + config::{ + MultiaddrWithPeerId, NodeKeyConfig, NonReservedPeerMode, NotificationHandshake, ProtocolId, + SetConfig, SyncMode, TransportConfig, + }, + protocol::{role::Role, ProtocolName}, request_responses::{ IncomingRequest, OutgoingResponse, ProtocolConfig as RequestResponseConfig, }, @@ -31,11 +39,227 @@ pub use sc_network_common::{ ExHashT, }; -pub use libp2p::{build_multiaddr, core::PublicKey, identity}; +// `std` dependencies +use std::{future::Future, iter, net::Ipv4Addr, path::PathBuf, pin::Pin, sync::Arc}; -use prometheus_endpoint::Registry; -use sc_network_common::config::NonDefaultSetConfig; -use std::{future::Future, pin::Pin, sync::Arc}; +/// Extension to [`SetConfig`] for sets that aren't the default set. +/// +/// > **Note**: As new fields might be added in the future, please consider using the `new` method +/// > and modifiers instead of creating this struct manually. +#[derive(Clone, Debug)] +pub struct NonDefaultSetConfig { + /// Name of the notifications protocols of this set. A substream on this set will be + /// considered established once this protocol is open. + /// + /// > **Note**: This field isn't present for the default set, as this is handled internally + /// > by the networking code. + pub notifications_protocol: ProtocolName, + + /// If the remote reports that it doesn't support the protocol indicated in the + /// `notifications_protocol` field, then each of these fallback names will be tried one by + /// one. + /// + /// If a fallback is used, it will be reported in + /// `sc_network::protocol::event::Event::NotificationStreamOpened::negotiated_fallback` + pub fallback_names: Vec, + + /// Handshake of the protocol + /// + /// NOTE: Currently custom handshakes are not fully supported. See issue #5685 for more + /// details. This field is temporarily used to allow moving the hardcoded block announcement + /// protocol out of `protocol.rs`. + pub handshake: Option, + + /// Maximum allowed size of single notifications. + pub max_notification_size: u64, + + /// Base configuration. + pub set_config: SetConfig, +} + +impl NonDefaultSetConfig { + /// Creates a new [`NonDefaultSetConfig`]. Zero slots and accepts only reserved nodes. + pub fn new(notifications_protocol: ProtocolName, max_notification_size: u64) -> Self { + Self { + notifications_protocol, + max_notification_size, + fallback_names: Vec::new(), + handshake: None, + set_config: SetConfig { + in_peers: 0, + out_peers: 0, + reserved_nodes: Vec::new(), + non_reserved_mode: NonReservedPeerMode::Deny, + }, + } + } + + /// Modifies the configuration to allow non-reserved nodes. + pub fn allow_non_reserved(&mut self, in_peers: u32, out_peers: u32) { + self.set_config.in_peers = in_peers; + self.set_config.out_peers = out_peers; + self.set_config.non_reserved_mode = NonReservedPeerMode::Accept; + } + + /// Add a node to the list of reserved nodes. + pub fn add_reserved(&mut self, peer: MultiaddrWithPeerId) { + self.set_config.reserved_nodes.push(peer); + } + + /// Add a list of protocol names used for backward compatibility. + /// + /// See the explanations in [`NonDefaultSetConfig::fallback_names`]. + pub fn add_fallback_names(&mut self, fallback_names: Vec) { + self.fallback_names.extend(fallback_names); + } +} + +/// Network service configuration. +#[derive(Clone, Debug)] +pub struct NetworkConfiguration { + /// Directory path to store network-specific configuration. None means nothing will be saved. + pub net_config_path: Option, + + /// Multiaddresses to listen for incoming connections. + pub listen_addresses: Vec, + + /// Multiaddresses to advertise. Detected automatically if empty. + pub public_addresses: Vec, + + /// List of initial node addresses + pub boot_nodes: Vec, + + /// The node key configuration, which determines the node's network identity keypair. + pub node_key: NodeKeyConfig, + + /// List of request-response protocols that the node supports. + pub request_response_protocols: Vec, + /// Configuration for the default set of nodes used for block syncing and transactions. + pub default_peers_set: SetConfig, + + /// Number of substreams to reserve for full nodes for block syncing and transactions. + /// Any other slot will be dedicated to light nodes. + /// + /// This value is implicitly capped to `default_set.out_peers + default_set.in_peers`. + pub default_peers_set_num_full: u32, + + /// Configuration for extra sets of nodes. + pub extra_sets: Vec, + + /// Client identifier. Sent over the wire for debugging purposes. + pub client_version: String, + + /// Name of the node. Sent over the wire for debugging purposes. + pub node_name: String, + + /// Configuration for the transport layer. + pub transport: TransportConfig, + + /// Maximum number of peers to ask the same blocks in parallel. + pub max_parallel_downloads: u32, + + /// Initial syncing mode. + pub sync_mode: SyncMode, + + /// True if Kademlia random discovery should be enabled. + /// + /// If true, the node will automatically randomly walk the DHT in order to find new peers. + pub enable_dht_random_walk: bool, + + /// Should we insert non-global addresses into the DHT? + pub allow_non_globals_in_dht: bool, + + /// Require iterative Kademlia DHT queries to use disjoint paths for increased resiliency in + /// the presence of potentially adversarial nodes. + pub kademlia_disjoint_query_paths: bool, + + /// Enable serving block data over IPFS bitswap. + pub ipfs_server: bool, + + /// Size of Yamux receive window of all substreams. `None` for the default (256kiB). + /// Any value less than 256kiB is invalid. + /// + /// # Context + /// + /// By design, notifications substreams on top of Yamux connections only allow up to `N` bytes + /// to be transferred at a time, where `N` is the Yamux receive window size configurable here. + /// This means, in practice, that every `N` bytes must be acknowledged by the receiver before + /// the sender can send more data. The maximum bandwidth of each notifications substream is + /// therefore `N / round_trip_time`. + /// + /// It is recommended to leave this to `None`, and use a request-response protocol instead if + /// a large amount of data must be transferred. The reason why the value is configurable is + /// that some Substrate users mis-use notification protocols to send large amounts of data. + /// As such, this option isn't designed to stay and will likely get removed in the future. + /// + /// Note that configuring a value here isn't a modification of the Yamux protocol, but rather + /// a modification of the way the implementation works. Different nodes with different + /// configured values remain compatible with each other. + pub yamux_window_size: Option, +} + +impl NetworkConfiguration { + /// Create new default configuration + pub fn new, SV: Into>( + node_name: SN, + client_version: SV, + node_key: NodeKeyConfig, + net_config_path: Option, + ) -> Self { + let default_peers_set = SetConfig::default(); + Self { + net_config_path, + listen_addresses: Vec::new(), + public_addresses: Vec::new(), + boot_nodes: Vec::new(), + node_key, + request_response_protocols: Vec::new(), + default_peers_set_num_full: default_peers_set.in_peers + default_peers_set.out_peers, + default_peers_set, + extra_sets: Vec::new(), + client_version: client_version.into(), + node_name: node_name.into(), + transport: TransportConfig::Normal { enable_mdns: false, allow_private_ip: true }, + max_parallel_downloads: 5, + sync_mode: SyncMode::Full, + enable_dht_random_walk: true, + allow_non_globals_in_dht: false, + kademlia_disjoint_query_paths: false, + yamux_window_size: None, + ipfs_server: false, + } + } + + /// Create new default configuration for localhost-only connection with random port (useful for + /// testing) + pub fn new_local() -> NetworkConfiguration { + let mut config = + NetworkConfiguration::new("test-node", "test-client", Default::default(), None); + + config.listen_addresses = + vec![iter::once(multiaddr::Protocol::Ip4(Ipv4Addr::new(127, 0, 0, 1))) + .chain(iter::once(multiaddr::Protocol::Tcp(0))) + .collect()]; + + config.allow_non_globals_in_dht = true; + config + } + + /// Create new default configuration for localhost-only connection with random port (useful for + /// testing) + pub fn new_memory() -> NetworkConfiguration { + let mut config = + NetworkConfiguration::new("test-node", "test-client", Default::default(), None); + + config.listen_addresses = + vec![iter::once(multiaddr::Protocol::Ip4(Ipv4Addr::new(127, 0, 0, 1))) + .chain(iter::once(multiaddr::Protocol::Tcp(0))) + .collect()]; + + config.allow_non_globals_in_dht = true; + config + } +} /// Network initialization parameters. pub struct Params { diff --git a/client/network/src/error.rs b/client/network/src/error.rs new file mode 100644 index 0000000000000..51c0f78ff335a --- /dev/null +++ b/client/network/src/error.rs @@ -0,0 +1,79 @@ +// This file is part of Substrate. + +// Copyright (C) Parity Technologies (UK) Ltd. +// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0 + +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with this program. If not, see . + +//! Substrate network possible errors. + +use libp2p::{Multiaddr, PeerId}; + +// TODO: remove +use sc_network_common::{config::TransportConfig, protocol::ProtocolName}; + +use std::fmt; + +/// Result type alias for the network. +pub type Result = std::result::Result; + +/// Error type for the network. +#[derive(thiserror::Error)] +pub enum Error { + /// Io error + #[error(transparent)] + Io(#[from] std::io::Error), + + /// Client error + #[error(transparent)] + Client(#[from] Box), + /// The same bootnode (based on address) is registered with two different peer ids. + #[error( + "The same bootnode (`{address}`) is registered with two different peer ids: `{first_id}` and `{second_id}`" + )] + DuplicateBootnode { + /// The address of the bootnode. + address: Multiaddr, + /// The first peer id that was found for the bootnode. + first_id: PeerId, + /// The second peer id that was found for the bootnode. + second_id: PeerId, + }, + /// Prometheus metrics error. + #[error(transparent)] + Prometheus(#[from] prometheus_endpoint::PrometheusError), + /// The network addresses are invalid because they don't match the transport. + #[error( + "The following addresses are invalid because they don't match the transport: {addresses:?}" + )] + AddressesForAnotherTransport { + /// Transport used. + transport: TransportConfig, + /// The invalid addresses. + addresses: Vec, + }, + /// The same request-response protocol has been registered multiple times. + #[error("Request-response protocol registered multiple times: {protocol}")] + DuplicateRequestResponseProtocol { + /// Name of the protocol registered multiple times. + protocol: ProtocolName, + }, +} + +// Make `Debug` use the `Display` implementation. +impl fmt::Debug for Error { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + fmt::Display::fmt(self, f) + } +} diff --git a/client/network/src/lib.rs b/client/network/src/lib.rs index f94a71681cd3d..dc912f15584cb 100644 --- a/client/network/src/lib.rs +++ b/client/network/src/lib.rs @@ -253,6 +253,7 @@ mod service; mod transport; pub mod config; +pub mod error; pub mod network_state; #[doc(inline)] diff --git a/client/network/src/protocol.rs b/client/network/src/protocol.rs index 8d604124233a0..063240876357a 100644 --- a/client/network/src/protocol.rs +++ b/client/network/src/protocol.rs @@ -16,8 +16,10 @@ // You should have received a copy of the GNU General Public License // along with this program. If not, see . -use crate::config; +// Local dependencies +use crate::{config, error}; +// External dependencies use bytes::Bytes; use codec::{DecodeAll, Encode}; use libp2p::{ @@ -29,27 +31,31 @@ use libp2p::{ Multiaddr, PeerId, }; use log::{debug, error, warn}; -use message::{generic::Message as GenericMessage, Message}; -use notifications::{Notifications, NotificationsOut}; + +// Substrate dependencies use sc_network_common::{ config::NonReservedPeerMode, - error, protocol::{role::Roles, ProtocolName}, sync::message::BlockAnnouncesHandshake, }; use sp_runtime::traits::Block as BlockT; + +// `std` dependencies use std::{ collections::{HashMap, HashSet, VecDeque}, iter, task::Poll, }; +use message::{generic::Message as GenericMessage, Message}; +use notifications::{Notifications, NotificationsOut}; + +pub use notifications::{NotificationsSink, NotifsHandlerError, Ready}; + mod notifications; pub mod message; -pub use notifications::{NotificationsSink, NotifsHandlerError, Ready}; - /// Maximum size used for notifications in the block announce and transaction protocols. // Must be equal to `max(MAX_BLOCK_ANNOUNCE_SIZE, MAX_TRANSACTIONS_SIZE)`. pub(crate) const BLOCK_ANNOUNCES_TRANSACTIONS_SUBSTREAM_SIZE: u64 = 16 * 1024 * 1024; @@ -93,7 +99,7 @@ impl Protocol { pub fn new( roles: Roles, network_config: &config::NetworkConfiguration, - block_announces_protocol: sc_network_common::config::NonDefaultSetConfig, + block_announces_protocol: config::NonDefaultSetConfig, ) -> error::Result<(Self, sc_peerset::PeersetHandle, Vec<(PeerId, Multiaddr)>)> { let mut known_addresses = Vec::new(); diff --git a/client/network/src/service.rs b/client/network/src/service.rs index e0f369c8b62a5..c0eca554f52c1 100644 --- a/client/network/src/service.rs +++ b/client/network/src/service.rs @@ -27,10 +27,12 @@ //! The methods of the [`NetworkService`] are implemented by sending a message over a channel, //! which is then processed by [`NetworkWorker::next_action`]. +// Local dependencies use crate::{ behaviour::{self, Behaviour, BehaviourOut}, config::Params, discovery::DiscoveryConfig, + error::Error, network_state::{ NetworkState, NotConnectedPeer as NetworkStateNotConnectedPeer, Peer as NetworkStatePeer, }, @@ -38,6 +40,7 @@ use crate::{ transport, ReputationChange, }; +// External dependencies use futures::{channel::oneshot, prelude::*}; use libp2p::{ core::{either::EitherError, upgrade, ConnectedPoint}, @@ -55,17 +58,18 @@ use libp2p::{ use log::{debug, error, info, trace, warn}; use metrics::{Histogram, HistogramVec, MetricSources, Metrics}; use parking_lot::Mutex; + +// Substrate dependencies use sc_network_common::{ config::{MultiaddrWithPeerId, TransportConfig}, - error::Error, protocol::{ event::{DhtEvent, Event}, ProtocolName, }, request_responses::{IfDisconnected, RequestFailure}, service::{ - NetworkDHTProvider, NetworkEventStream, NetworkNotification, NetworkPeers, NetworkSigner, - NetworkStateInfo, NetworkStatus, NetworkStatusProvider, + NetworkDHTProvider, NetworkEventStream, NetworkNotification, NetworkPeers, NetworkRequest, + NetworkSigner, NetworkStateInfo, NetworkStatus, NetworkStatusProvider, NotificationSender as NotificationSenderT, NotificationSenderError, NotificationSenderReady as NotificationSenderReadyT, Signature, SigningError, }, @@ -75,6 +79,8 @@ use sc_peerset::PeersetHandle; use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; use sp_blockchain::HeaderBackend; use sp_runtime::traits::{Block as BlockT, Zero}; + +// `std` dependencies use std::{ cmp, collections::{HashMap, HashSet}, @@ -90,13 +96,11 @@ use std::{ }; pub use behaviour::{InboundFailure, OutboundFailure, ResponseFailure}; +pub use libp2p::identity::{error::DecodingError, Keypair, PublicKey}; mod metrics; mod out_events; -pub use libp2p::identity::{error::DecodingError, Keypair, PublicKey}; -use sc_network_common::service::NetworkRequest; - /// Custom error that can be produced by the [`ConnectionHandler`] of the [`NetworkBehaviour`]. /// Used as a template parameter of [`SwarmEvent`] below. type ConnectionHandlerErr = diff --git a/client/network/sync/Cargo.toml b/client/network/sync/Cargo.toml index 52ab0d15e1670..298cbf1801f2e 100644 --- a/client/network/sync/Cargo.toml +++ b/client/network/sync/Cargo.toml @@ -32,6 +32,7 @@ fork-tree = { version = "3.0.0", path = "../../../utils/fork-tree" } prometheus-endpoint = { package = "substrate-prometheus-endpoint", version = "0.10.0-dev", path = "../../../utils/prometheus" } sc-client-api = { version = "4.0.0-dev", path = "../../api" } sc-consensus = { version = "0.10.0-dev", path = "../../consensus/common" } +sc-network = { version = "0.10.0-dev", path = "../" } sc-network-common = { version = "0.10.0-dev", path = "../common" } sc-peerset = { version = "4.0.0-dev", path = "../../peerset" } sc-utils = { version = "4.0.0-dev", path = "../../utils" } diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index 25cd3968c2924..c17dde1170c21 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -19,26 +19,28 @@ //! `SyncingEngine` is the actor responsible for syncing Substrate chain //! to tip and keep the blockchain up to date with network updates. +// Local dependencies use crate::{ service::{self, chain_sync::ToServiceCommand}, ChainSync, ClientError, SyncingService, }; +// External dependencies +use codec::{Decode, DecodeAll, Encode}; use futures::{FutureExt, Stream, StreamExt}; +use futures_timer::Delay; use libp2p::PeerId; use lru::LruCache; use prometheus_endpoint::{ register, Gauge, GaugeVec, MetricSource, Opts, PrometheusError, Registry, SourcedGauge, U64, }; -use codec::{Decode, DecodeAll, Encode}; -use futures_timer::Delay; +// Substrate dependencies use sc_client_api::{BlockBackend, HeaderBackend, ProofProvider}; use sc_consensus::import_queue::ImportQueueService; +use sc_network::config::{NetworkConfiguration, NonDefaultSetConfig}; use sc_network_common::{ - config::{ - NetworkConfiguration, NonDefaultSetConfig, ProtocolId, SyncMode as SyncOperationMode, - }, + config::{ProtocolId, SyncMode as SyncOperationMode}, protocol::{event::Event, role::Roles, ProtocolName}, sync::{ message::{ @@ -59,6 +61,7 @@ use sp_runtime::{ SaturatedConversion, }; +// `std` dependencies use std::{ collections::{HashMap, HashSet}, num::NonZeroUsize, diff --git a/client/network/sync/src/lib.rs b/client/network/sync/src/lib.rs index 23269b02bd128..969bf396bb2b6 100644 --- a/client/network/sync/src/lib.rs +++ b/client/network/sync/src/lib.rs @@ -28,23 +28,15 @@ //! the network, or whenever a block has been successfully verified, call the appropriate method in //! order to update it. -pub mod block_request_handler; -pub mod blocks; -pub mod engine; -pub mod mock; -mod schema; -pub mod service; -pub mod state; -pub mod state_request_handler; -pub mod warp; -pub mod warp_request_handler; - +// Local dependencies use crate::{ blocks::BlockCollection, schema::v1::{StateRequest, StateResponse}, state::StateSync, warp::{WarpProofImportResult, WarpSync}, }; + +// External dependencies use codec::{Decode, DecodeAll, Encode}; use extra_requests::ExtraRequests; use futures::{ @@ -52,16 +44,17 @@ use futures::{ }; use libp2p::{request_response::OutboundFailure, PeerId}; use log::{debug, error, info, trace, warn}; -use prometheus_endpoint::{register, Counter, PrometheusError, Registry, U64}; use prost::Message; + +// Substrate dependencies +use prometheus_endpoint::{register, Counter, PrometheusError, Registry, U64}; use sc_client_api::{BlockBackend, ProofProvider}; use sc_consensus::{ import_queue::ImportQueueService, BlockImportError, BlockImportStatus, IncomingBlock, }; +use sc_network::config::NonDefaultSetConfig; use sc_network_common::{ - config::{ - NonDefaultSetConfig, NonReservedPeerMode, NotificationHandshake, ProtocolId, SetConfig, - }, + config::{NonReservedPeerMode, NotificationHandshake, ProtocolId, SetConfig}, protocol::{role::Roles, ProtocolName}, request_responses::{IfDisconnected, RequestFailure}, sync::{ @@ -76,7 +69,6 @@ use sc_network_common::{ SyncState, SyncStatus, }, }; -pub use service::chain_sync::SyncingService; use sp_arithmetic::traits::Saturating; use sp_blockchain::{Error as ClientError, HeaderBackend, HeaderMetadata}; use sp_consensus::{ @@ -90,6 +82,8 @@ use sp_runtime::{ }, EncodedJustification, Justifications, }; + +// `std` dependencies use std::{ collections::{hash_map::Entry, HashMap, HashSet}, iter, @@ -97,9 +91,21 @@ use std::{ pin::Pin, sync::Arc, }; -use warp::TargetBlockImportResult; + +pub use service::chain_sync::SyncingService; mod extra_requests; +mod schema; + +pub mod block_request_handler; +pub mod blocks; +pub mod engine; +pub mod mock; +pub mod service; +pub mod state; +pub mod state_request_handler; +pub mod warp; +pub mod warp_request_handler; /// Maximum blocks to request in a single packet. const MAX_BLOCKS_TO_REQUEST: usize = 64; @@ -927,9 +933,9 @@ where match warp_sync.import_target_block( blocks.pop().expect("`blocks` len checked above."), ) { - TargetBlockImportResult::Success => + warp::TargetBlockImportResult::Success => return Ok(OnBlockData::Continue), - TargetBlockImportResult::BadResponse => + warp::TargetBlockImportResult::BadResponse => return Err(BadPeer(*who, rep::VERIFICATION_FAIL)), } } else if blocks.is_empty() { diff --git a/client/network/test/src/lib.rs b/client/network/test/src/lib.rs index 6a895687f149e..366c4738d5dae 100644 --- a/client/network/test/src/lib.rs +++ b/client/network/test/src/lib.rs @@ -48,11 +48,14 @@ use sc_consensus::{ ForkChoiceStrategy, ImportQueue, ImportResult, JustificationImport, JustificationSyncLink, LongestChain, Verifier, }; -use sc_network::{Multiaddr, NetworkService, NetworkWorker}; +use sc_network::{ + config::{NetworkConfiguration, NonDefaultSetConfig}, + Multiaddr, NetworkService, NetworkWorker, +}; use sc_network_common::{ config::{ - MultiaddrWithPeerId, NetworkConfiguration, NonDefaultSetConfig, NonReservedPeerMode, - ProtocolId, RequestResponseConfig, Role, SyncMode, TransportConfig, + MultiaddrWithPeerId, NonReservedPeerMode, ProtocolId, RequestResponseConfig, Role, + SyncMode, TransportConfig, }, protocol::{role::Roles, ProtocolName}, service::{NetworkBlock, NetworkEventStream, NetworkStateInfo, NetworkSyncForkRequest}, diff --git a/client/network/test/src/service.rs b/client/network/test/src/service.rs index 4b065a16d8699..ea835a10771a4 100644 --- a/client/network/test/src/service.rs +++ b/client/network/test/src/service.rs @@ -22,7 +22,7 @@ use libp2p::{Multiaddr, PeerId}; use sc_consensus::{ImportQueue, Link}; use sc_network::{config, NetworkService, NetworkWorker}; use sc_network_common::{ - config::{MultiaddrWithPeerId, NonDefaultSetConfig, ProtocolId, SetConfig, TransportConfig}, + config::{MultiaddrWithPeerId, ProtocolId, SetConfig, TransportConfig}, protocol::{event::Event, role::Roles}, service::{NetworkEventStream, NetworkNotification, NetworkPeers, NetworkStateInfo}, }; @@ -113,7 +113,7 @@ impl TestNetworkBuilder { ); let network_config = self.config.unwrap_or(config::NetworkConfiguration { - extra_sets: vec![NonDefaultSetConfig { + extra_sets: vec![config::NonDefaultSetConfig { notifications_protocol: PROTOCOL_NAME.into(), fallback_names: Vec::new(), max_notification_size: 1024 * 1024, @@ -548,7 +548,7 @@ async fn fallback_name_working() { let listen_addr = config::build_multiaddr![Memory(rand::random::())]; let (node1, mut events_stream1) = TestNetworkBuilder::new() .with_config(config::NetworkConfiguration { - extra_sets: vec![NonDefaultSetConfig { + extra_sets: vec![config::NonDefaultSetConfig { notifications_protocol: NEW_PROTOCOL_NAME.into(), fallback_names: vec![PROTOCOL_NAME.into()], max_notification_size: 1024 * 1024, diff --git a/client/network/transactions/Cargo.toml b/client/network/transactions/Cargo.toml index cab702fbd7a6c..3616473d3baed 100644 --- a/client/network/transactions/Cargo.toml +++ b/client/network/transactions/Cargo.toml @@ -20,6 +20,7 @@ libp2p = "0.50.0" log = "0.4.17" pin-project = "1.0.12" prometheus-endpoint = { package = "substrate-prometheus-endpoint", version = "0.10.0-dev", path = "../../../utils/prometheus" } +sc-network = { version = "0.10.0-dev", path = "../" } sc-network-common = { version = "0.10.0-dev", path = "../common" } sc-peerset = { version = "4.0.0-dev", path = "../../peerset" } sc-utils = { version = "4.0.0-dev", path = "../../utils" } diff --git a/client/network/transactions/src/lib.rs b/client/network/transactions/src/lib.rs index 48bb394946671..c22c8c6fca549 100644 --- a/client/network/transactions/src/lib.rs +++ b/client/network/transactions/src/lib.rs @@ -26,15 +26,20 @@ //! - Use [`TransactionsHandlerPrototype::build`] then [`TransactionsHandler::run`] to obtain a //! `Future` that processes transactions. +// Local dependencies use crate::config::*; + +// External dependencies use codec::{Decode, Encode}; use futures::{prelude::*, stream::FuturesUnordered}; use libp2p::{multiaddr, PeerId}; use log::{debug, trace, warn}; + +// Substrate dependencies use prometheus_endpoint::{register, Counter, PrometheusError, Registry, U64}; +use sc_network::{config::NonDefaultSetConfig, error}; use sc_network_common::{ - config::{NonDefaultSetConfig, NonReservedPeerMode, ProtocolId, SetConfig}, - error, + config::{NonReservedPeerMode, ProtocolId, SetConfig}, protocol::{event::Event, role::ObservedRole, ProtocolName}, service::{NetworkEventStream, NetworkNotification, NetworkPeers}, sync::{SyncEvent, SyncEventStream}, @@ -43,6 +48,8 @@ use sc_network_common::{ }; use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; use sp_runtime::traits::Block as BlockT; + +// `std` dependencies use std::{ collections::{hash_map::Entry, HashMap}, iter, diff --git a/client/service/src/config.rs b/client/service/src/config.rs index 8e843b58f285e..14aae046776e9 100644 --- a/client/service/src/config.rs +++ b/client/service/src/config.rs @@ -22,14 +22,11 @@ pub use sc_client_api::execution_extensions::{ExecutionStrategies, ExecutionStra pub use sc_client_db::{BlocksPruning, Database, DatabaseSource, PruningMode}; pub use sc_executor::{WasmExecutionMethod, WasmtimeInstantiationStrategy}; pub use sc_network::{ - config::{NetworkConfiguration, Role}, + config::{NetworkConfiguration, NonDefaultSetConfig, Role}, Multiaddr, }; pub use sc_network_common::{ - config::{ - MultiaddrWithPeerId, NodeKeyConfig, NonDefaultSetConfig, ProtocolId, SetConfig, - TransportConfig, - }, + config::{MultiaddrWithPeerId, NodeKeyConfig, ProtocolId, SetConfig, TransportConfig}, request_responses::{ IncomingRequest, OutgoingResponse, ProtocolConfig as RequestResponseConfig, }, diff --git a/client/service/src/error.rs b/client/service/src/error.rs index 6b71e46b4e0ef..c871342c771eb 100644 --- a/client/service/src/error.rs +++ b/client/service/src/error.rs @@ -40,7 +40,7 @@ pub enum Error { Consensus(#[from] sp_consensus::Error), #[error(transparent)] - Network(#[from] sc_network_common::error::Error), + Network(#[from] sc_network::error::Error), #[error(transparent)] Keystore(#[from] sc_keystore::Error), From 8bd8c5c142883f1948c23e50c721d937a0266e63 Mon Sep 17 00:00:00 2001 From: Aaro Altonen Date: Fri, 10 Mar 2023 12:05:03 +0200 Subject: [PATCH 03/10] Move `NetworkService` traits to `sc-network` --- Cargo.lock | 4 + bin/node/cli/src/service.rs | 6 +- client/authority-discovery/Cargo.toml | 1 + client/authority-discovery/src/worker.rs | 6 +- .../authority-discovery/src/worker/tests.rs | 9 +- client/cli/src/params/network_params.rs | 7 +- .../outgoing_requests_engine.rs | 7 +- client/consensus/beefy/src/lib.rs | 3 +- .../grandpa/src/communication/mod.rs | 7 +- .../grandpa/src/communication/tests.rs | 10 +- client/consensus/grandpa/src/lib.rs | 2 +- client/informant/Cargo.toml | 1 + client/informant/src/display.rs | 10 +- client/informant/src/lib.rs | 3 +- client/network-gossip/Cargo.toml | 1 + client/network-gossip/src/bridge.rs | 12 +- client/network-gossip/src/lib.rs | 7 +- client/network-gossip/src/state_machine.rs | 11 +- client/network/common/src/config.rs | 25 - client/network/common/src/error.rs | 77 --- client/network/common/src/service.rs | 632 +---------------- .../network/common/src/service/signature.rs | 4 +- client/network/src/config.rs | 30 +- client/network/src/lib.rs | 11 +- client/network/src/service.rs | 15 +- client/network/src/service/traits.rs | 635 ++++++++++++++++++ client/network/src/types.rs | 0 client/network/sync/src/lib.rs | 4 +- client/network/sync/src/service/chain_sync.rs | 6 +- client/network/sync/src/service/mock.rs | 10 +- client/network/sync/src/service/network.rs | 2 +- client/network/test/src/lib.rs | 4 +- client/network/test/src/service.rs | 24 +- client/network/transactions/src/lib.rs | 8 +- client/offchain/Cargo.toml | 1 + client/offchain/src/api.rs | 7 +- client/offchain/src/lib.rs | 2 +- client/service/src/builder.rs | 9 +- client/service/src/config.rs | 4 +- client/service/src/lib.rs | 7 +- client/service/src/metrics.rs | 7 +- client/service/test/src/lib.rs | 7 +- 42 files changed, 780 insertions(+), 858 deletions(-) delete mode 100644 client/network/common/src/error.rs create mode 100644 client/network/src/service/traits.rs create mode 100644 client/network/src/types.rs diff --git a/Cargo.lock b/Cargo.lock index f6c124c2818cd..c16860f0c921b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8101,6 +8101,7 @@ dependencies = [ "quickcheck", "rand 0.8.5", "sc-client-api", + "sc-network", "sc-network-common", "sp-api", "sp-authority-discovery", @@ -8751,6 +8752,7 @@ dependencies = [ "futures-timer", "log", "sc-client-api", + "sc-network", "sc-network-common", "sp-blockchain", "sp-runtime", @@ -8890,6 +8892,7 @@ dependencies = [ "log", "lru", "quickcheck", + "sc-network", "sc-network-common", "sc-peerset", "sp-runtime", @@ -9028,6 +9031,7 @@ dependencies = [ "sc-block-builder", "sc-client-api", "sc-client-db", + "sc-network", "sc-network-common", "sc-peerset", "sc-transaction-pool", diff --git a/bin/node/cli/src/service.rs b/bin/node/cli/src/service.rs index e9a34b2a5c728..376bae4962bde 100644 --- a/bin/node/cli/src/service.rs +++ b/bin/node/cli/src/service.rs @@ -31,10 +31,8 @@ use node_primitives::Block; use sc_client_api::BlockBackend; use sc_consensus_babe::{self, SlotProportion}; use sc_executor::NativeElseWasmExecutor; -use sc_network::NetworkService; -use sc_network_common::{ - protocol::event::Event, service::NetworkEventStream, sync::warp::WarpSyncParams, -}; +use sc_network::{NetworkEventStream, NetworkService}; +use sc_network_common::{protocol::event::Event, sync::warp::WarpSyncParams}; use sc_network_sync::SyncingService; use sc_service::{config::Configuration, error::Error as ServiceError, RpcHandlers, TaskManager}; use sc_telemetry::{Telemetry, TelemetryWorker}; diff --git a/client/authority-discovery/Cargo.toml b/client/authority-discovery/Cargo.toml index b37507662bfeb..900d9c59dfdae 100644 --- a/client/authority-discovery/Cargo.toml +++ b/client/authority-discovery/Cargo.toml @@ -28,6 +28,7 @@ rand = "0.8.5" thiserror = "1.0" prometheus-endpoint = { package = "substrate-prometheus-endpoint", version = "0.10.0-dev", path = "../../utils/prometheus" } sc-client-api = { version = "4.0.0-dev", path = "../api" } +sc-network = { version = "0.10.0-dev", path = "../network/" } sc-network-common = { version = "0.10.0-dev", path = "../network/common" } sp-api = { version = "4.0.0-dev", path = "../../primitives/api" } sp-authority-discovery = { version = "4.0.0-dev", path = "../../primitives/authority-discovery" } diff --git a/client/authority-discovery/src/worker.rs b/client/authority-discovery/src/worker.rs index bc5fe2848d5a1..7e5807f88d775 100644 --- a/client/authority-discovery/src/worker.rs +++ b/client/authority-discovery/src/worker.rs @@ -43,10 +43,8 @@ use log::{debug, error, log_enabled}; use prometheus_endpoint::{register, Counter, CounterVec, Gauge, Opts, U64}; use prost::Message; use rand::{seq::SliceRandom, thread_rng}; -use sc_network_common::{ - protocol::event::DhtEvent, - service::{KademliaKey, NetworkDHTProvider, NetworkSigner, NetworkStateInfo, Signature}, -}; +use sc_network::{KademliaKey, NetworkDHTProvider, NetworkSigner, NetworkStateInfo}; +use sc_network_common::{protocol::event::DhtEvent, service::signature::Signature}; use sp_api::{ApiError, ProvideRuntimeApi}; use sp_authority_discovery::{ AuthorityDiscoveryApi, AuthorityId, AuthorityPair, AuthoritySignature, diff --git a/client/authority-discovery/src/worker/tests.rs b/client/authority-discovery/src/worker/tests.rs index 22f984d9718ee..4b4b814d91e97 100644 --- a/client/authority-discovery/src/worker/tests.rs +++ b/client/authority-discovery/src/worker/tests.rs @@ -29,11 +29,16 @@ use futures::{ sink::SinkExt, task::LocalSpawn, }; -use libp2p::{core::multiaddr, identity::Keypair, PeerId}; +use libp2p::{ + core::multiaddr, + identity::{error::SigningError, Keypair}, + kad::record::Key as KademliaKey, + PeerId, +}; use prometheus_endpoint::prometheus::default_registry; use sc_client_api::HeaderBackend; -use sc_network_common::service::{KademliaKey, Signature, SigningError}; +use sc_network_common::service::signature::Signature; use sp_api::{ApiRef, ProvideRuntimeApi}; use sp_keystore::{testing::KeyStore, CryptoStore}; use sp_runtime::traits::{Block as BlockT, NumberFor, Zero}; diff --git a/client/cli/src/params/network_params.rs b/client/cli/src/params/network_params.rs index 31b761938df84..78bdfc58cf64e 100644 --- a/client/cli/src/params/network_params.rs +++ b/client/cli/src/params/network_params.rs @@ -18,8 +18,11 @@ use crate::{arg_enums::SyncMode, params::node_key_params::NodeKeyParams}; use clap::Args; -use sc_network::{config::NetworkConfiguration, multiaddr::Protocol}; -use sc_network_common::config::{NodeKeyConfig, NonReservedPeerMode, SetConfig, TransportConfig}; +use sc_network::{ + config::{NetworkConfiguration, SetConfig}, + multiaddr::Protocol, +}; +use sc_network_common::config::{NodeKeyConfig, NonReservedPeerMode, TransportConfig}; use sc_service::{ config::{Multiaddr, MultiaddrWithPeerId}, ChainSpec, ChainType, diff --git a/client/consensus/beefy/src/communication/request_response/outgoing_requests_engine.rs b/client/consensus/beefy/src/communication/request_response/outgoing_requests_engine.rs index 7450d4b3239f1..7c6112ab101c9 100644 --- a/client/consensus/beefy/src/communication/request_response/outgoing_requests_engine.rs +++ b/client/consensus/beefy/src/communication/request_response/outgoing_requests_engine.rs @@ -22,11 +22,8 @@ use codec::Encode; use futures::channel::{oneshot, oneshot::Canceled}; use log::{debug, warn}; use parking_lot::Mutex; -use sc_network::{PeerId, ProtocolName}; -use sc_network_common::{ - request_responses::{IfDisconnected, RequestFailure}, - service::NetworkRequest, -}; +use sc_network::{NetworkRequest, PeerId, ProtocolName}; +use sc_network_common::request_responses::{IfDisconnected, RequestFailure}; use sp_consensus_beefy::{crypto::AuthorityId, ValidatorSet}; use sp_runtime::traits::{Block, NumberFor}; use std::{collections::VecDeque, result::Result, sync::Arc}; diff --git a/client/consensus/beefy/src/lib.rs b/client/consensus/beefy/src/lib.rs index 67758a4979b19..eb56a97de1dd9 100644 --- a/client/consensus/beefy/src/lib.rs +++ b/client/consensus/beefy/src/lib.rs @@ -38,8 +38,7 @@ use parking_lot::Mutex; use prometheus::Registry; use sc_client_api::{Backend, BlockBackend, BlockchainEvents, FinalityNotifications, Finalizer}; use sc_consensus::BlockImport; -use sc_network::ProtocolName; -use sc_network_common::service::NetworkRequest; +use sc_network::{NetworkRequest, ProtocolName}; use sc_network_gossip::{GossipEngine, Network as GossipNetwork, Syncing as GossipSyncing}; use sp_api::{HeaderT, NumberFor, ProvideRuntimeApi}; use sp_blockchain::{ diff --git a/client/consensus/grandpa/src/communication/mod.rs b/client/consensus/grandpa/src/communication/mod.rs index 6c7e3ea46511e..522c092493837 100644 --- a/client/consensus/grandpa/src/communication/mod.rs +++ b/client/consensus/grandpa/src/communication/mod.rs @@ -46,7 +46,7 @@ use finality_grandpa::{ Message::{Precommit, Prevote, PrimaryPropose}, }; use parity_scale_codec::{Decode, Encode}; -use sc_network::ReputationChange; +use sc_network::{NetworkBlock, NetworkSyncForkRequest, ReputationChange}; use sc_network_gossip::{GossipEngine, Network as GossipNetwork}; use sc_telemetry::{telemetry, TelemetryHandle, CONSENSUS_DEBUG, CONSENSUS_INFO}; use sp_keystore::SyncCryptoStorePtr; @@ -59,10 +59,7 @@ use crate::{ use gossip::{ FullCatchUpMessage, FullCommitMessage, GossipMessage, GossipValidator, PeerReport, VoteMessage, }; -use sc_network_common::{ - service::{NetworkBlock, NetworkSyncForkRequest}, - sync::SyncEventStream, -}; +use sc_network_common::sync::SyncEventStream; use sc_utils::mpsc::TracingUnboundedReceiver; use sp_consensus_grandpa::{AuthorityId, AuthoritySignature, RoundNumber, SetId as SetIdNumber}; diff --git a/client/consensus/grandpa/src/communication/tests.rs b/client/consensus/grandpa/src/communication/tests.rs index 843e5467910e8..711c14f350506 100644 --- a/client/consensus/grandpa/src/communication/tests.rs +++ b/client/consensus/grandpa/src/communication/tests.rs @@ -25,14 +25,14 @@ use super::{ use crate::{communication::grandpa_protocol_name, environment::SharedVoterSetState}; use futures::prelude::*; use parity_scale_codec::Encode; -use sc_network::{config::Role, Multiaddr, PeerId, ReputationChange}; +use sc_network::{ + config::Role, Multiaddr, NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers, + NetworkSyncForkRequest, NotificationSenderError, NotificationSenderT as NotificationSender, + PeerId, ReputationChange, +}; use sc_network_common::{ config::MultiaddrWithPeerId, protocol::{event::Event as NetworkEvent, role::ObservedRole, ProtocolName}, - service::{ - NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers, - NetworkSyncForkRequest, NotificationSender, NotificationSenderError, - }, sync::{SyncEvent as SyncStreamEvent, SyncEventStream}, }; use sc_network_gossip::Validator; diff --git a/client/consensus/grandpa/src/lib.rs b/client/consensus/grandpa/src/lib.rs index 7c50b2f255577..690835e3a6639 100644 --- a/client/consensus/grandpa/src/lib.rs +++ b/client/consensus/grandpa/src/lib.rs @@ -703,7 +703,7 @@ pub fn grandpa_peers_set_config( // Notifications reach ~256kiB in size at the time of writing on Kusama and Polkadot. max_notification_size: 1024 * 1024, handshake: None, - set_config: sc_network_common::config::SetConfig { + set_config: sc_network::config::SetConfig { in_peers: 0, out_peers: 0, reserved_nodes: Vec::new(), diff --git a/client/informant/Cargo.toml b/client/informant/Cargo.toml index ca0ae03af3265..cd84dcb5a0dfe 100644 --- a/client/informant/Cargo.toml +++ b/client/informant/Cargo.toml @@ -19,5 +19,6 @@ futures-timer = "3.0.1" log = "0.4.17" sc-client-api = { version = "4.0.0-dev", path = "../api" } sc-network-common = { version = "0.10.0-dev", path = "../network/common" } +sc-network = { version = "0.10.0-dev", path = "../network" } sp-blockchain = { version = "4.0.0-dev", path = "../../primitives/blockchain" } sp-runtime = { version = "7.0.0", path = "../../primitives/runtime" } diff --git a/client/informant/src/display.rs b/client/informant/src/display.rs index fc68e5603c134..46e7229273d79 100644 --- a/client/informant/src/display.rs +++ b/client/informant/src/display.rs @@ -20,12 +20,10 @@ use crate::OutputFormat; use ansi_term::Colour; use log::info; use sc_client_api::ClientInfo; -use sc_network_common::{ - service::NetworkStatus, - sync::{ - warp::{WarpSyncPhase, WarpSyncProgress}, - SyncState, SyncStatus, - }, +use sc_network::NetworkStatus; +use sc_network_common::sync::{ + warp::{WarpSyncPhase, WarpSyncProgress}, + SyncState, SyncStatus, }; use sp_runtime::traits::{Block as BlockT, CheckedDiv, NumberFor, Saturating, Zero}; use std::{fmt, time::Instant}; diff --git a/client/informant/src/lib.rs b/client/informant/src/lib.rs index dc6aebc2ed658..03f9075055e2f 100644 --- a/client/informant/src/lib.rs +++ b/client/informant/src/lib.rs @@ -23,7 +23,8 @@ use futures::prelude::*; use futures_timer::Delay; use log::{debug, info, trace}; use sc_client_api::{BlockchainEvents, UsageProvider}; -use sc_network_common::{service::NetworkStatusProvider, sync::SyncStatusProvider}; +use sc_network::NetworkStatusProvider; +use sc_network_common::sync::SyncStatusProvider; use sp_blockchain::HeaderMetadata; use sp_runtime::traits::{Block as BlockT, Header}; use std::{collections::VecDeque, fmt::Display, sync::Arc, time::Duration}; diff --git a/client/network-gossip/Cargo.toml b/client/network-gossip/Cargo.toml index 7811e86c095ba..5c1bc91f105c8 100644 --- a/client/network-gossip/Cargo.toml +++ b/client/network-gossip/Cargo.toml @@ -22,6 +22,7 @@ log = "0.4.17" lru = "0.8.1" tracing = "0.1.29" prometheus-endpoint = { package = "substrate-prometheus-endpoint", version = "0.10.0-dev", path = "../../utils/prometheus" } +sc-network = { version = "0.10.0-dev", path = "../network/" } sc-network-common = { version = "0.10.0-dev", path = "../network/common" } sc-peerset = { version = "4.0.0-dev", path = "../peerset" } sp-runtime = { version = "7.0.0", path = "../../primitives/runtime" } diff --git a/client/network-gossip/src/bridge.rs b/client/network-gossip/src/bridge.rs index 7368cc770306e..b2ccf22ed57f4 100644 --- a/client/network-gossip/src/bridge.rs +++ b/client/network-gossip/src/bridge.rs @@ -340,14 +340,12 @@ mod tests { future::poll_fn, }; use quickcheck::{Arbitrary, Gen, QuickCheck}; + use sc_network::{ + NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers, + NotificationSenderError, NotificationSenderT as NotificationSender, + }; use sc_network_common::{ - config::MultiaddrWithPeerId, - protocol::role::ObservedRole, - service::{ - NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers, - NotificationSender, NotificationSenderError, - }, - sync::SyncEventStream, + config::MultiaddrWithPeerId, protocol::role::ObservedRole, sync::SyncEventStream, }; use sp_runtime::{ testing::H256, diff --git a/client/network-gossip/src/lib.rs b/client/network-gossip/src/lib.rs index e3448ba01c373..c6a20ce8e950c 100644 --- a/client/network-gossip/src/lib.rs +++ b/client/network-gossip/src/lib.rs @@ -68,11 +68,8 @@ pub use self::{ }; use libp2p::{multiaddr, PeerId}; -use sc_network_common::{ - protocol::ProtocolName, - service::{NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers}, - sync::SyncEventStream, -}; +use sc_network::{NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers}; +use sc_network_common::{protocol::ProtocolName, sync::SyncEventStream}; use sp_runtime::traits::{Block as BlockT, NumberFor}; use std::iter; diff --git a/client/network-gossip/src/state_machine.rs b/client/network-gossip/src/state_machine.rs index 3a0d5fc0de173..2d2cfab021028 100644 --- a/client/network-gossip/src/state_machine.rs +++ b/client/network-gossip/src/state_machine.rs @@ -525,14 +525,11 @@ mod tests { use super::*; use crate::multiaddr::Multiaddr; use futures::prelude::*; - use sc_network_common::{ - config::MultiaddrWithPeerId, - protocol::event::Event, - service::{ - NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers, - NotificationSender, NotificationSenderError, - }, + use sc_network::{ + NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers, + NotificationSenderError, NotificationSenderT as NotificationSender, }; + use sc_network_common::{config::MultiaddrWithPeerId, protocol::event::Event}; use sc_peerset::ReputationChange; use sp_runtime::{ testing::{Block as RawBlock, ExtrinsicWrapper, H256}, diff --git a/client/network/common/src/config.rs b/client/network/common/src/config.rs index 1281f1b60e4a6..4faf0bb4d443d 100644 --- a/client/network/common/src/config.rs +++ b/client/network/common/src/config.rs @@ -195,31 +195,6 @@ impl From for ParseErr { } } -/// Configuration for a set of nodes. -#[derive(Clone, Debug)] -pub struct SetConfig { - /// Maximum allowed number of incoming substreams related to this set. - pub in_peers: u32, - /// Number of outgoing substreams related to this set that we're trying to maintain. - pub out_peers: u32, - /// List of reserved node addresses. - pub reserved_nodes: Vec, - /// Whether nodes that aren't in [`SetConfig::reserved_nodes`] are accepted or automatically - /// refused. - pub non_reserved_mode: NonReservedPeerMode, -} - -impl Default for SetConfig { - fn default() -> Self { - Self { - in_peers: 25, - out_peers: 75, - reserved_nodes: Vec::new(), - non_reserved_mode: NonReservedPeerMode::Accept, - } - } -} - /// Custom handshake for the notification protocol #[derive(Debug, Clone)] pub struct NotificationHandshake(Vec); diff --git a/client/network/common/src/error.rs b/client/network/common/src/error.rs deleted file mode 100644 index 712b8c5de3f66..0000000000000 --- a/client/network/common/src/error.rs +++ /dev/null @@ -1,77 +0,0 @@ -// This file is part of Substrate. - -// Copyright (C) Parity Technologies (UK) Ltd. -// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0 - -// This program is free software: you can redistribute it and/or modify -// it under the terms of the GNU General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. - -// This program is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU General Public License for more details. - -// You should have received a copy of the GNU General Public License -// along with this program. If not, see . - -//! Substrate network possible errors. - -use crate::{config::TransportConfig, protocol::ProtocolName}; -use libp2p::{Multiaddr, PeerId}; - -use std::fmt; - -/// Result type alias for the network. -pub type Result = std::result::Result; - -/// Error type for the network. -#[derive(thiserror::Error)] -pub enum Error { - /// Io error - #[error(transparent)] - Io(#[from] std::io::Error), - - /// Client error - #[error(transparent)] - Client(#[from] Box), - /// The same bootnode (based on address) is registered with two different peer ids. - #[error( - "The same bootnode (`{address}`) is registered with two different peer ids: `{first_id}` and `{second_id}`" - )] - DuplicateBootnode { - /// The address of the bootnode. - address: Multiaddr, - /// The first peer id that was found for the bootnode. - first_id: PeerId, - /// The second peer id that was found for the bootnode. - second_id: PeerId, - }, - /// Prometheus metrics error. - #[error(transparent)] - Prometheus(#[from] prometheus_endpoint::PrometheusError), - /// The network addresses are invalid because they don't match the transport. - #[error( - "The following addresses are invalid because they don't match the transport: {addresses:?}" - )] - AddressesForAnotherTransport { - /// Transport used. - transport: TransportConfig, - /// The invalid addresses. - addresses: Vec, - }, - /// The same request-response protocol has been registered multiple times. - #[error("Request-response protocol registered multiple times: {protocol}")] - DuplicateRequestResponseProtocol { - /// Name of the protocol registered multiple times. - protocol: ProtocolName, - }, -} - -// Make `Debug` use the `Display` implementation. -impl fmt::Debug for Error { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - fmt::Display::fmt(self, f) - } -} diff --git a/client/network/common/src/service.rs b/client/network/common/src/service.rs index d3c5c2f439450..4917bfba75fab 100644 --- a/client/network/common/src/service.rs +++ b/client/network/common/src/service.rs @@ -1,631 +1 @@ -// This file is part of Substrate. -// -// Copyright (C) Parity Technologies (UK) Ltd. -// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0 -// -// This program is free software: you can redistribute it and/or modify -// it under the terms of the GNU General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// This program is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU General Public License for more details. -// -// You should have received a copy of the GNU General Public License -// along with this program. If not, see . -// -// If you read this, you are very thorough, congratulations. - -use crate::{ - config::MultiaddrWithPeerId, - protocol::{event::Event, ProtocolName}, - request_responses::{IfDisconnected, RequestFailure}, -}; -use futures::{channel::oneshot, Stream}; -pub use libp2p::{identity::error::SigningError, kad::record::Key as KademliaKey}; -use libp2p::{Multiaddr, PeerId}; -use sc_peerset::ReputationChange; -pub use signature::Signature; -use std::{collections::HashSet, future::Future, pin::Pin, sync::Arc}; - -mod signature; - -/// Signer with network identity -pub trait NetworkSigner { - /// Signs the message with the `KeyPair` that defines the local [`PeerId`]. - fn sign_with_local_identity(&self, msg: impl AsRef<[u8]>) -> Result; -} - -impl NetworkSigner for Arc -where - T: ?Sized, - T: NetworkSigner, -{ - fn sign_with_local_identity(&self, msg: impl AsRef<[u8]>) -> Result { - T::sign_with_local_identity(self, msg) - } -} - -/// Provides access to the networking DHT. -pub trait NetworkDHTProvider { - /// Start getting a value from the DHT. - fn get_value(&self, key: &KademliaKey); - - /// Start putting a value in the DHT. - fn put_value(&self, key: KademliaKey, value: Vec); -} - -impl NetworkDHTProvider for Arc -where - T: ?Sized, - T: NetworkDHTProvider, -{ - fn get_value(&self, key: &KademliaKey) { - T::get_value(self, key) - } - - fn put_value(&self, key: KademliaKey, value: Vec) { - T::put_value(self, key, value) - } -} - -/// Provides an ability to set a fork sync request for a particular block. -pub trait NetworkSyncForkRequest { - /// Notifies the sync service to try and sync the given block from the given - /// peers. - /// - /// If the given vector of peers is empty then the underlying implementation - /// should make a best effort to fetch the block from any peers it is - /// connected to (NOTE: this assumption will change in the future #3629). - fn set_sync_fork_request(&self, peers: Vec, hash: BlockHash, number: BlockNumber); -} - -impl NetworkSyncForkRequest for Arc -where - T: ?Sized, - T: NetworkSyncForkRequest, -{ - fn set_sync_fork_request(&self, peers: Vec, hash: BlockHash, number: BlockNumber) { - T::set_sync_fork_request(self, peers, hash, number) - } -} - -/// Overview status of the network. -#[derive(Clone)] -pub struct NetworkStatus { - /// Total number of connected peers. - pub num_connected_peers: usize, - /// The total number of bytes received. - pub total_bytes_inbound: u64, - /// The total number of bytes sent. - pub total_bytes_outbound: u64, -} - -/// Provides high-level status information about network. -#[async_trait::async_trait] -pub trait NetworkStatusProvider { - /// High-level network status information. - /// - /// Returns an error if the `NetworkWorker` is no longer running. - async fn status(&self) -> Result; -} - -// Manual implementation to avoid extra boxing here -impl NetworkStatusProvider for Arc -where - T: ?Sized, - T: NetworkStatusProvider, -{ - fn status<'life0, 'async_trait>( - &'life0 self, - ) -> Pin> + Send + 'async_trait>> - where - 'life0: 'async_trait, - Self: 'async_trait, - { - T::status(self) - } -} - -/// Provides low-level API for manipulating network peers. -pub trait NetworkPeers { - /// Set authorized peers. - /// - /// Need a better solution to manage authorized peers, but now just use reserved peers for - /// prototyping. - fn set_authorized_peers(&self, peers: HashSet); - - /// Set authorized_only flag. - /// - /// Need a better solution to decide authorized_only, but now just use reserved_only flag for - /// prototyping. - fn set_authorized_only(&self, reserved_only: bool); - - /// Adds an address known to a node. - fn add_known_address(&self, peer_id: PeerId, addr: Multiaddr); - - /// Report a given peer as either beneficial (+) or costly (-) according to the - /// given scalar. - fn report_peer(&self, who: PeerId, cost_benefit: ReputationChange); - - /// Disconnect from a node as soon as possible. - /// - /// This triggers the same effects as if the connection had closed itself spontaneously. - /// - /// See also [`NetworkPeers::remove_from_peers_set`], which has the same effect but also - /// prevents the local node from re-establishing an outgoing substream to this peer until it - /// is added again. - fn disconnect_peer(&self, who: PeerId, protocol: ProtocolName); - - /// Connect to unreserved peers and allow unreserved peers to connect for syncing purposes. - fn accept_unreserved_peers(&self); - - /// Disconnect from unreserved peers and deny new unreserved peers to connect for syncing - /// purposes. - fn deny_unreserved_peers(&self); - - /// Adds a `PeerId` and its `Multiaddr` as reserved for a sync protocol (default peer set). - /// - /// Returns an `Err` if the given string is not a valid multiaddress - /// or contains an invalid peer ID (which includes the local peer ID). - fn add_reserved_peer(&self, peer: MultiaddrWithPeerId) -> Result<(), String>; - - /// Removes a `PeerId` from the list of reserved peers for a sync protocol (default peer set). - fn remove_reserved_peer(&self, peer_id: PeerId); - - /// Sets the reserved set of a protocol to the given set of peers. - /// - /// Each `Multiaddr` must end with a `/p2p/` component containing the `PeerId`. It can also - /// consist of only `/p2p/`. - /// - /// The node will start establishing/accepting connections and substreams to/from peers in this - /// set, if it doesn't have any substream open with them yet. - /// - /// Note however, if a call to this function results in less peers on the reserved set, they - /// will not necessarily get disconnected (depending on available free slots in the peer set). - /// If you want to also disconnect those removed peers, you will have to call - /// `remove_from_peers_set` on those in addition to updating the reserved set. You can omit - /// this step if the peer set is in reserved only mode. - /// - /// Returns an `Err` if one of the given addresses is invalid or contains an - /// invalid peer ID (which includes the local peer ID). - fn set_reserved_peers( - &self, - protocol: ProtocolName, - peers: HashSet, - ) -> Result<(), String>; - - /// Add peers to a peer set. - /// - /// Each `Multiaddr` must end with a `/p2p/` component containing the `PeerId`. It can also - /// consist of only `/p2p/`. - /// - /// Returns an `Err` if one of the given addresses is invalid or contains an - /// invalid peer ID (which includes the local peer ID). - fn add_peers_to_reserved_set( - &self, - protocol: ProtocolName, - peers: HashSet, - ) -> Result<(), String>; - - /// Remove peers from a peer set. - fn remove_peers_from_reserved_set(&self, protocol: ProtocolName, peers: Vec); - - /// Add a peer to a set of peers. - /// - /// If the set has slots available, it will try to open a substream with this peer. - /// - /// Each `Multiaddr` must end with a `/p2p/` component containing the `PeerId`. It can also - /// consist of only `/p2p/`. - /// - /// Returns an `Err` if one of the given addresses is invalid or contains an - /// invalid peer ID (which includes the local peer ID). - fn add_to_peers_set( - &self, - protocol: ProtocolName, - peers: HashSet, - ) -> Result<(), String>; - - /// Remove peers from a peer set. - /// - /// If we currently have an open substream with this peer, it will soon be closed. - fn remove_from_peers_set(&self, protocol: ProtocolName, peers: Vec); - - /// Returns the number of peers in the sync peer set we're connected to. - fn sync_num_connected(&self) -> usize; -} - -// Manual implementation to avoid extra boxing here -impl NetworkPeers for Arc -where - T: ?Sized, - T: NetworkPeers, -{ - fn set_authorized_peers(&self, peers: HashSet) { - T::set_authorized_peers(self, peers) - } - - fn set_authorized_only(&self, reserved_only: bool) { - T::set_authorized_only(self, reserved_only) - } - - fn add_known_address(&self, peer_id: PeerId, addr: Multiaddr) { - T::add_known_address(self, peer_id, addr) - } - - fn report_peer(&self, who: PeerId, cost_benefit: ReputationChange) { - T::report_peer(self, who, cost_benefit) - } - - fn disconnect_peer(&self, who: PeerId, protocol: ProtocolName) { - T::disconnect_peer(self, who, protocol) - } - - fn accept_unreserved_peers(&self) { - T::accept_unreserved_peers(self) - } - - fn deny_unreserved_peers(&self) { - T::deny_unreserved_peers(self) - } - - fn add_reserved_peer(&self, peer: MultiaddrWithPeerId) -> Result<(), String> { - T::add_reserved_peer(self, peer) - } - - fn remove_reserved_peer(&self, peer_id: PeerId) { - T::remove_reserved_peer(self, peer_id) - } - - fn set_reserved_peers( - &self, - protocol: ProtocolName, - peers: HashSet, - ) -> Result<(), String> { - T::set_reserved_peers(self, protocol, peers) - } - - fn add_peers_to_reserved_set( - &self, - protocol: ProtocolName, - peers: HashSet, - ) -> Result<(), String> { - T::add_peers_to_reserved_set(self, protocol, peers) - } - - fn remove_peers_from_reserved_set(&self, protocol: ProtocolName, peers: Vec) { - T::remove_peers_from_reserved_set(self, protocol, peers) - } - - fn add_to_peers_set( - &self, - protocol: ProtocolName, - peers: HashSet, - ) -> Result<(), String> { - T::add_to_peers_set(self, protocol, peers) - } - - fn remove_from_peers_set(&self, protocol: ProtocolName, peers: Vec) { - T::remove_from_peers_set(self, protocol, peers) - } - - fn sync_num_connected(&self) -> usize { - T::sync_num_connected(self) - } -} - -/// Provides access to network-level event stream. -pub trait NetworkEventStream { - /// Returns a stream containing the events that happen on the network. - /// - /// If this method is called multiple times, the events are duplicated. - /// - /// The stream never ends (unless the `NetworkWorker` gets shut down). - /// - /// The name passed is used to identify the channel in the Prometheus metrics. Note that the - /// parameter is a `&'static str`, and not a `String`, in order to avoid accidentally having - /// an unbounded set of Prometheus metrics, which would be quite bad in terms of memory - fn event_stream(&self, name: &'static str) -> Pin + Send>>; -} - -impl NetworkEventStream for Arc -where - T: ?Sized, - T: NetworkEventStream, -{ - fn event_stream(&self, name: &'static str) -> Pin + Send>> { - T::event_stream(self, name) - } -} - -/// Trait for providing information about the local network state -pub trait NetworkStateInfo { - /// Returns the local external addresses. - fn external_addresses(&self) -> Vec; - - /// Returns the listening addresses (without trailing `/p2p/` with our `PeerId`). - fn listen_addresses(&self) -> Vec; - - /// Returns the local Peer ID. - fn local_peer_id(&self) -> PeerId; -} - -impl NetworkStateInfo for Arc -where - T: ?Sized, - T: NetworkStateInfo, -{ - fn external_addresses(&self) -> Vec { - T::external_addresses(self) - } - - fn listen_addresses(&self) -> Vec { - T::listen_addresses(self) - } - - fn local_peer_id(&self) -> PeerId { - T::local_peer_id(self) - } -} - -/// Reserved slot in the notifications buffer, ready to accept data. -pub trait NotificationSenderReady { - /// Consumes this slots reservation and actually queues the notification. - /// - /// NOTE: Traits can't consume itself, but calling this method second time will return an error. - fn send(&mut self, notification: Vec) -> Result<(), NotificationSenderError>; -} - -/// A `NotificationSender` allows for sending notifications to a peer with a chosen protocol. -#[async_trait::async_trait] -pub trait NotificationSender: Send + Sync + 'static { - /// Returns a future that resolves when the `NotificationSender` is ready to send a - /// notification. - async fn ready(&self) - -> Result, NotificationSenderError>; -} - -/// Error returned by [`NetworkNotification::notification_sender`]. -#[derive(Debug, thiserror::Error)] -pub enum NotificationSenderError { - /// The notification receiver has been closed, usually because the underlying connection - /// closed. - /// - /// Some of the notifications most recently sent may not have been received. However, - /// the peer may still be connected and a new `NotificationSender` for the same - /// protocol obtained from [`NetworkNotification::notification_sender`]. - #[error("The notification receiver has been closed")] - Closed, - /// Protocol name hasn't been registered. - #[error("Protocol name hasn't been registered")] - BadProtocol, -} - -/// Provides ability to send network notifications. -pub trait NetworkNotification { - /// Appends a notification to the buffer of pending outgoing notifications with the given peer. - /// Has no effect if the notifications channel with this protocol name is not open. - /// - /// If the buffer of pending outgoing notifications with that peer is full, the notification - /// is silently dropped and the connection to the remote will start being shut down. This - /// happens if you call this method at a higher rate than the rate at which the peer processes - /// these notifications, or if the available network bandwidth is too low. - /// - /// For this reason, this method is considered soft-deprecated. You are encouraged to use - /// [`NetworkNotification::notification_sender`] instead. - /// - /// > **Note**: The reason why this is a no-op in the situation where we have no channel is - /// > that we don't guarantee message delivery anyway. Networking issues can cause - /// > connections to drop at any time, and higher-level logic shouldn't differentiate - /// > between the remote voluntarily closing a substream or a network error - /// > preventing the message from being delivered. - /// - /// The protocol must have been registered with - /// `crate::config::NetworkConfiguration::notifications_protocols`. - fn write_notification(&self, target: PeerId, protocol: ProtocolName, message: Vec); - - /// Obtains a [`NotificationSender`] for a connected peer, if it exists. - /// - /// A `NotificationSender` is scoped to a particular connection to the peer that holds - /// a receiver. With a `NotificationSender` at hand, sending a notification is done in two - /// steps: - /// - /// 1. [`NotificationSender::ready`] is used to wait for the sender to become ready - /// for another notification, yielding a [`NotificationSenderReady`] token. - /// 2. [`NotificationSenderReady::send`] enqueues the notification for sending. This operation - /// can only fail if the underlying notification substream or connection has suddenly closed. - /// - /// An error is returned by [`NotificationSenderReady::send`] if there exists no open - /// notifications substream with that combination of peer and protocol, or if the remote - /// has asked to close the notifications substream. If that happens, it is guaranteed that an - /// [`Event::NotificationStreamClosed`] has been generated on the stream returned by - /// [`NetworkEventStream::event_stream`]. - /// - /// If the remote requests to close the notifications substream, all notifications successfully - /// enqueued using [`NotificationSenderReady::send`] will finish being sent out before the - /// substream actually gets closed, but attempting to enqueue more notifications will now - /// return an error. It is however possible for the entire connection to be abruptly closed, - /// in which case enqueued notifications will be lost. - /// - /// The protocol must have been registered with - /// `crate::config::NetworkConfiguration::notifications_protocols`. - /// - /// # Usage - /// - /// This method returns a struct that allows waiting until there is space available in the - /// buffer of messages towards the given peer. If the peer processes notifications at a slower - /// rate than we send them, this buffer will quickly fill up. - /// - /// As such, you should never do something like this: - /// - /// ```ignore - /// // Do NOT do this - /// for peer in peers { - /// if let Ok(n) = network.notification_sender(peer, ...) { - /// if let Ok(s) = n.ready().await { - /// let _ = s.send(...); - /// } - /// } - /// } - /// ``` - /// - /// Doing so would slow down all peers to the rate of the slowest one. A malicious or - /// malfunctioning peer could intentionally process notifications at a very slow rate. - /// - /// Instead, you are encouraged to maintain your own buffer of notifications on top of the one - /// maintained by `sc-network`, and use `notification_sender` to progressively send out - /// elements from your buffer. If this additional buffer is full (which will happen at some - /// point if the peer is too slow to process notifications), appropriate measures can be taken, - /// such as removing non-critical notifications from the buffer or disconnecting the peer - /// using [`NetworkPeers::disconnect_peer`]. - /// - /// - /// Notifications Per-peer buffer - /// broadcast +-------> of notifications +--> `notification_sender` +--> Internet - /// ^ (not covered by - /// | sc-network) - /// + - /// Notifications should be dropped - /// if buffer is full - /// - /// - /// See also the `sc-network-gossip` crate for a higher-level way to send notifications. - fn notification_sender( - &self, - target: PeerId, - protocol: ProtocolName, - ) -> Result, NotificationSenderError>; - - /// Set handshake for the notification protocol. - fn set_notification_handshake(&self, protocol: ProtocolName, handshake: Vec); -} - -impl NetworkNotification for Arc -where - T: ?Sized, - T: NetworkNotification, -{ - fn write_notification(&self, target: PeerId, protocol: ProtocolName, message: Vec) { - T::write_notification(self, target, protocol, message) - } - - fn notification_sender( - &self, - target: PeerId, - protocol: ProtocolName, - ) -> Result, NotificationSenderError> { - T::notification_sender(self, target, protocol) - } - - fn set_notification_handshake(&self, protocol: ProtocolName, handshake: Vec) { - T::set_notification_handshake(self, protocol, handshake) - } -} - -/// Provides ability to send network requests. -#[async_trait::async_trait] -pub trait NetworkRequest { - /// Sends a single targeted request to a specific peer. On success, returns the response of - /// the peer. - /// - /// Request-response protocols are a way to complement notifications protocols, but - /// notifications should remain the default ways of communicating information. For example, a - /// peer can announce something through a notification, after which the recipient can obtain - /// more information by performing a request. - /// As such, call this function with `IfDisconnected::ImmediateError` for `connect`. This way - /// you will get an error immediately for disconnected peers, instead of waiting for a - /// potentially very long connection attempt, which would suggest that something is wrong - /// anyway, as you are supposed to be connected because of the notification protocol. - /// - /// No limit or throttling of concurrent outbound requests per peer and protocol are enforced. - /// Such restrictions, if desired, need to be enforced at the call site(s). - /// - /// The protocol must have been registered through - /// `NetworkConfiguration::request_response_protocols`. - async fn request( - &self, - target: PeerId, - protocol: ProtocolName, - request: Vec, - connect: IfDisconnected, - ) -> Result, RequestFailure>; - - /// Variation of `request` which starts a request whose response is delivered on a provided - /// channel. - /// - /// Instead of blocking and waiting for a reply, this function returns immediately, sending - /// responses via the passed in sender. This alternative API exists to make it easier to - /// integrate with message passing APIs. - /// - /// Keep in mind that the connected receiver might receive a `Canceled` event in case of a - /// closing connection. This is expected behaviour. With `request` you would get a - /// `RequestFailure::Network(OutboundFailure::ConnectionClosed)` in that case. - fn start_request( - &self, - target: PeerId, - protocol: ProtocolName, - request: Vec, - tx: oneshot::Sender, RequestFailure>>, - connect: IfDisconnected, - ); -} - -// Manual implementation to avoid extra boxing here -impl NetworkRequest for Arc -where - T: ?Sized, - T: NetworkRequest, -{ - fn request<'life0, 'async_trait>( - &'life0 self, - target: PeerId, - protocol: ProtocolName, - request: Vec, - connect: IfDisconnected, - ) -> Pin, RequestFailure>> + Send + 'async_trait>> - where - 'life0: 'async_trait, - Self: 'async_trait, - { - T::request(self, target, protocol, request, connect) - } - - fn start_request( - &self, - target: PeerId, - protocol: ProtocolName, - request: Vec, - tx: oneshot::Sender, RequestFailure>>, - connect: IfDisconnected, - ) { - T::start_request(self, target, protocol, request, tx, connect) - } -} - -/// Provides ability to announce blocks to the network. -pub trait NetworkBlock { - /// Make sure an important block is propagated to peers. - /// - /// In chain-based consensus, we often need to make sure non-best forks are - /// at least temporarily synced. This function forces such an announcement. - fn announce_block(&self, hash: BlockHash, data: Option>); - - /// Inform the network service about new best imported block. - fn new_best_block_imported(&self, hash: BlockHash, number: BlockNumber); -} - -impl NetworkBlock for Arc -where - T: ?Sized, - T: NetworkBlock, -{ - fn announce_block(&self, hash: BlockHash, data: Option>) { - T::announce_block(self, hash, data) - } - - fn new_best_block_imported(&self, hash: BlockHash, number: BlockNumber) { - T::new_best_block_imported(self, hash, number) - } -} +pub mod signature; diff --git a/client/network/common/src/service/signature.rs b/client/network/common/src/service/signature.rs index 14addafc6bba2..e52dd6b1d2a29 100644 --- a/client/network/common/src/service/signature.rs +++ b/client/network/common/src/service/signature.rs @@ -19,10 +19,12 @@ // If you read this, you are very thorough, congratulations. use libp2p::{ - identity::{error::SigningError, Keypair, PublicKey}, + identity::{Keypair, PublicKey}, PeerId, }; +pub use libp2p::identity::error::SigningError; + /// A result of signing a message with a network identity. Since `PeerId` is potentially a hash of a /// `PublicKey`, you need to reveal the `PublicKey` next to the signature, so the verifier can check /// if the signature was made by the entity that controls a given `PeerId`. diff --git a/client/network/src/config.rs b/client/network/src/config.rs index ef2936be9e176..9d4a3087ac0c2 100644 --- a/client/network/src/config.rs +++ b/client/network/src/config.rs @@ -29,7 +29,7 @@ use prometheus_endpoint::Registry; pub use sc_network_common::{ config::{ MultiaddrWithPeerId, NodeKeyConfig, NonReservedPeerMode, NotificationHandshake, ProtocolId, - SetConfig, SyncMode, TransportConfig, + SyncMode, TransportConfig, }, protocol::{role::Role, ProtocolName}, request_responses::{ @@ -42,6 +42,34 @@ pub use sc_network_common::{ // `std` dependencies use std::{future::Future, iter, net::Ipv4Addr, path::PathBuf, pin::Pin, sync::Arc}; +/// Configuration for a set of nodes. +#[derive(Clone, Debug)] +pub struct SetConfig { + /// Maximum allowed number of incoming substreams related to this set. + pub in_peers: u32, + + /// Number of outgoing substreams related to this set that we're trying to maintain. + pub out_peers: u32, + + /// List of reserved node addresses. + pub reserved_nodes: Vec, + + /// Whether nodes that aren't in [`SetConfig::reserved_nodes`] are accepted or automatically + /// refused. + pub non_reserved_mode: NonReservedPeerMode, +} + +impl Default for SetConfig { + fn default() -> Self { + Self { + in_peers: 25, + out_peers: 75, + reserved_nodes: Vec::new(), + non_reserved_mode: NonReservedPeerMode::Accept, + } + } +} + /// Extension to [`SetConfig`] for sets that aren't the default set. /// /// > **Note**: As new fields might be added in the future, please consider using the `new` method diff --git a/client/network/src/lib.rs b/client/network/src/lib.rs index dc912f15584cb..88674eb438a5d 100644 --- a/client/network/src/lib.rs +++ b/client/network/src/lib.rs @@ -265,17 +265,18 @@ pub use sc_network_common::{ ProtocolName, }, request_responses::{IfDisconnected, RequestFailure}, - service::{ - KademliaKey, NetworkBlock, NetworkDHTProvider, NetworkRequest, NetworkSigner, - NetworkStateInfo, NetworkStatus, NetworkStatusProvider, NetworkSyncForkRequest, Signature, - SigningError, - }, sync::{ warp::{WarpSyncPhase, WarpSyncProgress}, ExtendedPeerInfo, StateDownloadProgress, SyncEventStream, SyncState, SyncStatusProvider, }, }; pub use service::{ + traits::{ + KademliaKey, NetworkBlock, NetworkDHTProvider, NetworkEventStream, NetworkNotification, + NetworkPeers, NetworkRequest, NetworkSigner, NetworkStateInfo, NetworkStatus, + NetworkStatusProvider, NetworkSyncForkRequest, NotificationSender as NotificationSenderT, + NotificationSenderError, + }, DecodingError, Keypair, NetworkService, NetworkWorker, NotificationSender, NotificationSenderReady, OutboundFailure, PublicKey, }; diff --git a/client/network/src/service.rs b/client/network/src/service.rs index c0eca554f52c1..f80684b11b41c 100644 --- a/client/network/src/service.rs +++ b/client/network/src/service.rs @@ -37,6 +37,12 @@ use crate::{ NetworkState, NotConnectedPeer as NetworkStateNotConnectedPeer, Peer as NetworkStatePeer, }, protocol::{self, NotificationsSink, NotifsHandlerError, Protocol, Ready}, + service::traits::{ + NetworkDHTProvider, NetworkEventStream, NetworkNotification, NetworkPeers, NetworkRequest, + NetworkSigner, NetworkStateInfo, NetworkStatus, NetworkStatusProvider, + NotificationSender as NotificationSenderT, NotificationSenderError, + NotificationSenderReady as NotificationSenderReadyT, + }, transport, ReputationChange, }; @@ -67,12 +73,7 @@ use sc_network_common::{ ProtocolName, }, request_responses::{IfDisconnected, RequestFailure}, - service::{ - NetworkDHTProvider, NetworkEventStream, NetworkNotification, NetworkPeers, NetworkRequest, - NetworkSigner, NetworkStateInfo, NetworkStatus, NetworkStatusProvider, - NotificationSender as NotificationSenderT, NotificationSenderError, - NotificationSenderReady as NotificationSenderReadyT, Signature, SigningError, - }, + service::signature::{Signature, SigningError}, ExHashT, }; use sc_peerset::PeersetHandle; @@ -101,6 +102,8 @@ pub use libp2p::identity::{error::DecodingError, Keypair, PublicKey}; mod metrics; mod out_events; +pub mod traits; + /// Custom error that can be produced by the [`ConnectionHandler`] of the [`NetworkBehaviour`]. /// Used as a template parameter of [`SwarmEvent`] below. type ConnectionHandlerErr = diff --git a/client/network/src/service/traits.rs b/client/network/src/service/traits.rs new file mode 100644 index 0000000000000..bdbd435d87a7b --- /dev/null +++ b/client/network/src/service/traits.rs @@ -0,0 +1,635 @@ +// This file is part of Substrate. +// +// Copyright (C) Parity Technologies (UK) Ltd. +// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0 +// +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. +// +// You should have received a copy of the GNU General Public License +// along with this program. If not, see . +// +// If you read this, you are very thorough, congratulations. + +// External dependencies +use futures::{channel::oneshot, Stream}; +use libp2p::{Multiaddr, PeerId}; + +// Substrate dependencies +use sc_network_common::{ + config::MultiaddrWithPeerId, + protocol::{event::Event, ProtocolName}, + request_responses::{IfDisconnected, RequestFailure}, + service::signature::Signature, +}; +use sc_peerset::ReputationChange; + +// `std` dependencies +use std::{collections::HashSet, future::Future, pin::Pin, sync::Arc}; + +pub use libp2p::{identity::error::SigningError, kad::record::Key as KademliaKey}; + +/// Signer with network identity +pub trait NetworkSigner { + /// Signs the message with the `KeyPair` that defines the local [`PeerId`]. + fn sign_with_local_identity(&self, msg: impl AsRef<[u8]>) -> Result; +} + +impl NetworkSigner for Arc +where + T: ?Sized, + T: NetworkSigner, +{ + fn sign_with_local_identity(&self, msg: impl AsRef<[u8]>) -> Result { + T::sign_with_local_identity(self, msg) + } +} + +/// Provides access to the networking DHT. +pub trait NetworkDHTProvider { + /// Start getting a value from the DHT. + fn get_value(&self, key: &KademliaKey); + + /// Start putting a value in the DHT. + fn put_value(&self, key: KademliaKey, value: Vec); +} + +impl NetworkDHTProvider for Arc +where + T: ?Sized, + T: NetworkDHTProvider, +{ + fn get_value(&self, key: &KademliaKey) { + T::get_value(self, key) + } + + fn put_value(&self, key: KademliaKey, value: Vec) { + T::put_value(self, key, value) + } +} + +/// Provides an ability to set a fork sync request for a particular block. +pub trait NetworkSyncForkRequest { + /// Notifies the sync service to try and sync the given block from the given + /// peers. + /// + /// If the given vector of peers is empty then the underlying implementation + /// should make a best effort to fetch the block from any peers it is + /// connected to (NOTE: this assumption will change in the future #3629). + fn set_sync_fork_request(&self, peers: Vec, hash: BlockHash, number: BlockNumber); +} + +impl NetworkSyncForkRequest for Arc +where + T: ?Sized, + T: NetworkSyncForkRequest, +{ + fn set_sync_fork_request(&self, peers: Vec, hash: BlockHash, number: BlockNumber) { + T::set_sync_fork_request(self, peers, hash, number) + } +} + +/// Overview status of the network. +#[derive(Clone)] +pub struct NetworkStatus { + /// Total number of connected peers. + pub num_connected_peers: usize, + /// The total number of bytes received. + pub total_bytes_inbound: u64, + /// The total number of bytes sent. + pub total_bytes_outbound: u64, +} + +/// Provides high-level status information about network. +#[async_trait::async_trait] +pub trait NetworkStatusProvider { + /// High-level network status information. + /// + /// Returns an error if the `NetworkWorker` is no longer running. + async fn status(&self) -> Result; +} + +// Manual implementation to avoid extra boxing here +impl NetworkStatusProvider for Arc +where + T: ?Sized, + T: NetworkStatusProvider, +{ + fn status<'life0, 'async_trait>( + &'life0 self, + ) -> Pin> + Send + 'async_trait>> + where + 'life0: 'async_trait, + Self: 'async_trait, + { + T::status(self) + } +} + +/// Provides low-level API for manipulating network peers. +pub trait NetworkPeers { + /// Set authorized peers. + /// + /// Need a better solution to manage authorized peers, but now just use reserved peers for + /// prototyping. + fn set_authorized_peers(&self, peers: HashSet); + + /// Set authorized_only flag. + /// + /// Need a better solution to decide authorized_only, but now just use reserved_only flag for + /// prototyping. + fn set_authorized_only(&self, reserved_only: bool); + + /// Adds an address known to a node. + fn add_known_address(&self, peer_id: PeerId, addr: Multiaddr); + + /// Report a given peer as either beneficial (+) or costly (-) according to the + /// given scalar. + fn report_peer(&self, who: PeerId, cost_benefit: ReputationChange); + + /// Disconnect from a node as soon as possible. + /// + /// This triggers the same effects as if the connection had closed itself spontaneously. + /// + /// See also [`NetworkPeers::remove_from_peers_set`], which has the same effect but also + /// prevents the local node from re-establishing an outgoing substream to this peer until it + /// is added again. + fn disconnect_peer(&self, who: PeerId, protocol: ProtocolName); + + /// Connect to unreserved peers and allow unreserved peers to connect for syncing purposes. + fn accept_unreserved_peers(&self); + + /// Disconnect from unreserved peers and deny new unreserved peers to connect for syncing + /// purposes. + fn deny_unreserved_peers(&self); + + /// Adds a `PeerId` and its `Multiaddr` as reserved for a sync protocol (default peer set). + /// + /// Returns an `Err` if the given string is not a valid multiaddress + /// or contains an invalid peer ID (which includes the local peer ID). + fn add_reserved_peer(&self, peer: MultiaddrWithPeerId) -> Result<(), String>; + + /// Removes a `PeerId` from the list of reserved peers for a sync protocol (default peer set). + fn remove_reserved_peer(&self, peer_id: PeerId); + + /// Sets the reserved set of a protocol to the given set of peers. + /// + /// Each `Multiaddr` must end with a `/p2p/` component containing the `PeerId`. It can also + /// consist of only `/p2p/`. + /// + /// The node will start establishing/accepting connections and substreams to/from peers in this + /// set, if it doesn't have any substream open with them yet. + /// + /// Note however, if a call to this function results in less peers on the reserved set, they + /// will not necessarily get disconnected (depending on available free slots in the peer set). + /// If you want to also disconnect those removed peers, you will have to call + /// `remove_from_peers_set` on those in addition to updating the reserved set. You can omit + /// this step if the peer set is in reserved only mode. + /// + /// Returns an `Err` if one of the given addresses is invalid or contains an + /// invalid peer ID (which includes the local peer ID). + fn set_reserved_peers( + &self, + protocol: ProtocolName, + peers: HashSet, + ) -> Result<(), String>; + + /// Add peers to a peer set. + /// + /// Each `Multiaddr` must end with a `/p2p/` component containing the `PeerId`. It can also + /// consist of only `/p2p/`. + /// + /// Returns an `Err` if one of the given addresses is invalid or contains an + /// invalid peer ID (which includes the local peer ID). + fn add_peers_to_reserved_set( + &self, + protocol: ProtocolName, + peers: HashSet, + ) -> Result<(), String>; + + /// Remove peers from a peer set. + fn remove_peers_from_reserved_set(&self, protocol: ProtocolName, peers: Vec); + + /// Add a peer to a set of peers. + /// + /// If the set has slots available, it will try to open a substream with this peer. + /// + /// Each `Multiaddr` must end with a `/p2p/` component containing the `PeerId`. It can also + /// consist of only `/p2p/`. + /// + /// Returns an `Err` if one of the given addresses is invalid or contains an + /// invalid peer ID (which includes the local peer ID). + fn add_to_peers_set( + &self, + protocol: ProtocolName, + peers: HashSet, + ) -> Result<(), String>; + + /// Remove peers from a peer set. + /// + /// If we currently have an open substream with this peer, it will soon be closed. + fn remove_from_peers_set(&self, protocol: ProtocolName, peers: Vec); + + /// Returns the number of peers in the sync peer set we're connected to. + fn sync_num_connected(&self) -> usize; +} + +// Manual implementation to avoid extra boxing here +impl NetworkPeers for Arc +where + T: ?Sized, + T: NetworkPeers, +{ + fn set_authorized_peers(&self, peers: HashSet) { + T::set_authorized_peers(self, peers) + } + + fn set_authorized_only(&self, reserved_only: bool) { + T::set_authorized_only(self, reserved_only) + } + + fn add_known_address(&self, peer_id: PeerId, addr: Multiaddr) { + T::add_known_address(self, peer_id, addr) + } + + fn report_peer(&self, who: PeerId, cost_benefit: ReputationChange) { + T::report_peer(self, who, cost_benefit) + } + + fn disconnect_peer(&self, who: PeerId, protocol: ProtocolName) { + T::disconnect_peer(self, who, protocol) + } + + fn accept_unreserved_peers(&self) { + T::accept_unreserved_peers(self) + } + + fn deny_unreserved_peers(&self) { + T::deny_unreserved_peers(self) + } + + fn add_reserved_peer(&self, peer: MultiaddrWithPeerId) -> Result<(), String> { + T::add_reserved_peer(self, peer) + } + + fn remove_reserved_peer(&self, peer_id: PeerId) { + T::remove_reserved_peer(self, peer_id) + } + + fn set_reserved_peers( + &self, + protocol: ProtocolName, + peers: HashSet, + ) -> Result<(), String> { + T::set_reserved_peers(self, protocol, peers) + } + + fn add_peers_to_reserved_set( + &self, + protocol: ProtocolName, + peers: HashSet, + ) -> Result<(), String> { + T::add_peers_to_reserved_set(self, protocol, peers) + } + + fn remove_peers_from_reserved_set(&self, protocol: ProtocolName, peers: Vec) { + T::remove_peers_from_reserved_set(self, protocol, peers) + } + + fn add_to_peers_set( + &self, + protocol: ProtocolName, + peers: HashSet, + ) -> Result<(), String> { + T::add_to_peers_set(self, protocol, peers) + } + + fn remove_from_peers_set(&self, protocol: ProtocolName, peers: Vec) { + T::remove_from_peers_set(self, protocol, peers) + } + + fn sync_num_connected(&self) -> usize { + T::sync_num_connected(self) + } +} + +/// Provides access to network-level event stream. +pub trait NetworkEventStream { + /// Returns a stream containing the events that happen on the network. + /// + /// If this method is called multiple times, the events are duplicated. + /// + /// The stream never ends (unless the `NetworkWorker` gets shut down). + /// + /// The name passed is used to identify the channel in the Prometheus metrics. Note that the + /// parameter is a `&'static str`, and not a `String`, in order to avoid accidentally having + /// an unbounded set of Prometheus metrics, which would be quite bad in terms of memory + fn event_stream(&self, name: &'static str) -> Pin + Send>>; +} + +impl NetworkEventStream for Arc +where + T: ?Sized, + T: NetworkEventStream, +{ + fn event_stream(&self, name: &'static str) -> Pin + Send>> { + T::event_stream(self, name) + } +} + +/// Trait for providing information about the local network state +pub trait NetworkStateInfo { + /// Returns the local external addresses. + fn external_addresses(&self) -> Vec; + + /// Returns the listening addresses (without trailing `/p2p/` with our `PeerId`). + fn listen_addresses(&self) -> Vec; + + /// Returns the local Peer ID. + fn local_peer_id(&self) -> PeerId; +} + +impl NetworkStateInfo for Arc +where + T: ?Sized, + T: NetworkStateInfo, +{ + fn external_addresses(&self) -> Vec { + T::external_addresses(self) + } + + fn listen_addresses(&self) -> Vec { + T::listen_addresses(self) + } + + fn local_peer_id(&self) -> PeerId { + T::local_peer_id(self) + } +} + +/// Reserved slot in the notifications buffer, ready to accept data. +pub trait NotificationSenderReady { + /// Consumes this slots reservation and actually queues the notification. + /// + /// NOTE: Traits can't consume itself, but calling this method second time will return an error. + fn send(&mut self, notification: Vec) -> Result<(), NotificationSenderError>; +} + +/// A `NotificationSender` allows for sending notifications to a peer with a chosen protocol. +#[async_trait::async_trait] +pub trait NotificationSender: Send + Sync + 'static { + /// Returns a future that resolves when the `NotificationSender` is ready to send a + /// notification. + async fn ready(&self) + -> Result, NotificationSenderError>; +} + +/// Error returned by [`NetworkNotification::notification_sender`]. +#[derive(Debug, thiserror::Error)] +pub enum NotificationSenderError { + /// The notification receiver has been closed, usually because the underlying connection + /// closed. + /// + /// Some of the notifications most recently sent may not have been received. However, + /// the peer may still be connected and a new `NotificationSender` for the same + /// protocol obtained from [`NetworkNotification::notification_sender`]. + #[error("The notification receiver has been closed")] + Closed, + /// Protocol name hasn't been registered. + #[error("Protocol name hasn't been registered")] + BadProtocol, +} + +/// Provides ability to send network notifications. +pub trait NetworkNotification { + /// Appends a notification to the buffer of pending outgoing notifications with the given peer. + /// Has no effect if the notifications channel with this protocol name is not open. + /// + /// If the buffer of pending outgoing notifications with that peer is full, the notification + /// is silently dropped and the connection to the remote will start being shut down. This + /// happens if you call this method at a higher rate than the rate at which the peer processes + /// these notifications, or if the available network bandwidth is too low. + /// + /// For this reason, this method is considered soft-deprecated. You are encouraged to use + /// [`NetworkNotification::notification_sender`] instead. + /// + /// > **Note**: The reason why this is a no-op in the situation where we have no channel is + /// > that we don't guarantee message delivery anyway. Networking issues can cause + /// > connections to drop at any time, and higher-level logic shouldn't differentiate + /// > between the remote voluntarily closing a substream or a network error + /// > preventing the message from being delivered. + /// + /// The protocol must have been registered with + /// `crate::config::NetworkConfiguration::notifications_protocols`. + fn write_notification(&self, target: PeerId, protocol: ProtocolName, message: Vec); + + /// Obtains a [`NotificationSender`] for a connected peer, if it exists. + /// + /// A `NotificationSender` is scoped to a particular connection to the peer that holds + /// a receiver. With a `NotificationSender` at hand, sending a notification is done in two + /// steps: + /// + /// 1. [`NotificationSender::ready`] is used to wait for the sender to become ready + /// for another notification, yielding a [`NotificationSenderReady`] token. + /// 2. [`NotificationSenderReady::send`] enqueues the notification for sending. This operation + /// can only fail if the underlying notification substream or connection has suddenly closed. + /// + /// An error is returned by [`NotificationSenderReady::send`] if there exists no open + /// notifications substream with that combination of peer and protocol, or if the remote + /// has asked to close the notifications substream. If that happens, it is guaranteed that an + /// [`Event::NotificationStreamClosed`] has been generated on the stream returned by + /// [`NetworkEventStream::event_stream`]. + /// + /// If the remote requests to close the notifications substream, all notifications successfully + /// enqueued using [`NotificationSenderReady::send`] will finish being sent out before the + /// substream actually gets closed, but attempting to enqueue more notifications will now + /// return an error. It is however possible for the entire connection to be abruptly closed, + /// in which case enqueued notifications will be lost. + /// + /// The protocol must have been registered with + /// `crate::config::NetworkConfiguration::notifications_protocols`. + /// + /// # Usage + /// + /// This method returns a struct that allows waiting until there is space available in the + /// buffer of messages towards the given peer. If the peer processes notifications at a slower + /// rate than we send them, this buffer will quickly fill up. + /// + /// As such, you should never do something like this: + /// + /// ```ignore + /// // Do NOT do this + /// for peer in peers { + /// if let Ok(n) = network.notification_sender(peer, ...) { + /// if let Ok(s) = n.ready().await { + /// let _ = s.send(...); + /// } + /// } + /// } + /// ``` + /// + /// Doing so would slow down all peers to the rate of the slowest one. A malicious or + /// malfunctioning peer could intentionally process notifications at a very slow rate. + /// + /// Instead, you are encouraged to maintain your own buffer of notifications on top of the one + /// maintained by `sc-network`, and use `notification_sender` to progressively send out + /// elements from your buffer. If this additional buffer is full (which will happen at some + /// point if the peer is too slow to process notifications), appropriate measures can be taken, + /// such as removing non-critical notifications from the buffer or disconnecting the peer + /// using [`NetworkPeers::disconnect_peer`]. + /// + /// + /// Notifications Per-peer buffer + /// broadcast +-------> of notifications +--> `notification_sender` +--> Internet + /// ^ (not covered by + /// | sc-network) + /// + + /// Notifications should be dropped + /// if buffer is full + /// + /// + /// See also the `sc-network-gossip` crate for a higher-level way to send notifications. + fn notification_sender( + &self, + target: PeerId, + protocol: ProtocolName, + ) -> Result, NotificationSenderError>; + + /// Set handshake for the notification protocol. + fn set_notification_handshake(&self, protocol: ProtocolName, handshake: Vec); +} + +impl NetworkNotification for Arc +where + T: ?Sized, + T: NetworkNotification, +{ + fn write_notification(&self, target: PeerId, protocol: ProtocolName, message: Vec) { + T::write_notification(self, target, protocol, message) + } + + fn notification_sender( + &self, + target: PeerId, + protocol: ProtocolName, + ) -> Result, NotificationSenderError> { + T::notification_sender(self, target, protocol) + } + + fn set_notification_handshake(&self, protocol: ProtocolName, handshake: Vec) { + T::set_notification_handshake(self, protocol, handshake) + } +} + +/// Provides ability to send network requests. +#[async_trait::async_trait] +pub trait NetworkRequest { + /// Sends a single targeted request to a specific peer. On success, returns the response of + /// the peer. + /// + /// Request-response protocols are a way to complement notifications protocols, but + /// notifications should remain the default ways of communicating information. For example, a + /// peer can announce something through a notification, after which the recipient can obtain + /// more information by performing a request. + /// As such, call this function with `IfDisconnected::ImmediateError` for `connect`. This way + /// you will get an error immediately for disconnected peers, instead of waiting for a + /// potentially very long connection attempt, which would suggest that something is wrong + /// anyway, as you are supposed to be connected because of the notification protocol. + /// + /// No limit or throttling of concurrent outbound requests per peer and protocol are enforced. + /// Such restrictions, if desired, need to be enforced at the call site(s). + /// + /// The protocol must have been registered through + /// `NetworkConfiguration::request_response_protocols`. + async fn request( + &self, + target: PeerId, + protocol: ProtocolName, + request: Vec, + connect: IfDisconnected, + ) -> Result, RequestFailure>; + + /// Variation of `request` which starts a request whose response is delivered on a provided + /// channel. + /// + /// Instead of blocking and waiting for a reply, this function returns immediately, sending + /// responses via the passed in sender. This alternative API exists to make it easier to + /// integrate with message passing APIs. + /// + /// Keep in mind that the connected receiver might receive a `Canceled` event in case of a + /// closing connection. This is expected behaviour. With `request` you would get a + /// `RequestFailure::Network(OutboundFailure::ConnectionClosed)` in that case. + fn start_request( + &self, + target: PeerId, + protocol: ProtocolName, + request: Vec, + tx: oneshot::Sender, RequestFailure>>, + connect: IfDisconnected, + ); +} + +// Manual implementation to avoid extra boxing here +impl NetworkRequest for Arc +where + T: ?Sized, + T: NetworkRequest, +{ + fn request<'life0, 'async_trait>( + &'life0 self, + target: PeerId, + protocol: ProtocolName, + request: Vec, + connect: IfDisconnected, + ) -> Pin, RequestFailure>> + Send + 'async_trait>> + where + 'life0: 'async_trait, + Self: 'async_trait, + { + T::request(self, target, protocol, request, connect) + } + + fn start_request( + &self, + target: PeerId, + protocol: ProtocolName, + request: Vec, + tx: oneshot::Sender, RequestFailure>>, + connect: IfDisconnected, + ) { + T::start_request(self, target, protocol, request, tx, connect) + } +} + +/// Provides ability to announce blocks to the network. +pub trait NetworkBlock { + /// Make sure an important block is propagated to peers. + /// + /// In chain-based consensus, we often need to make sure non-best forks are + /// at least temporarily synced. This function forces such an announcement. + fn announce_block(&self, hash: BlockHash, data: Option>); + + /// Inform the network service about new best imported block. + fn new_best_block_imported(&self, hash: BlockHash, number: BlockNumber); +} + +impl NetworkBlock for Arc +where + T: ?Sized, + T: NetworkBlock, +{ + fn announce_block(&self, hash: BlockHash, data: Option>) { + T::announce_block(self, hash, data) + } + + fn new_best_block_imported(&self, hash: BlockHash, number: BlockNumber) { + T::new_best_block_imported(self, hash, number) + } +} diff --git a/client/network/src/types.rs b/client/network/src/types.rs new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/client/network/sync/src/lib.rs b/client/network/sync/src/lib.rs index 969bf396bb2b6..46c87c6866c33 100644 --- a/client/network/sync/src/lib.rs +++ b/client/network/sync/src/lib.rs @@ -52,9 +52,9 @@ use sc_client_api::{BlockBackend, ProofProvider}; use sc_consensus::{ import_queue::ImportQueueService, BlockImportError, BlockImportStatus, IncomingBlock, }; -use sc_network::config::NonDefaultSetConfig; +use sc_network::config::{NonDefaultSetConfig, SetConfig}; use sc_network_common::{ - config::{NonReservedPeerMode, NotificationHandshake, ProtocolId, SetConfig}, + config::{NonReservedPeerMode, NotificationHandshake, ProtocolId}, protocol::{role::Roles, ProtocolName}, request_responses::{IfDisconnected, RequestFailure}, sync::{ diff --git a/client/network/sync/src/service/chain_sync.rs b/client/network/sync/src/service/chain_sync.rs index 99b4197740eda..f9e0e401fdf8f 100644 --- a/client/network/sync/src/service/chain_sync.rs +++ b/client/network/sync/src/service/chain_sync.rs @@ -20,9 +20,9 @@ use futures::{channel::oneshot, Stream}; use libp2p::PeerId; use sc_consensus::{BlockImportError, BlockImportStatus, JustificationSyncLink, Link}; -use sc_network_common::{ - service::{NetworkBlock, NetworkSyncForkRequest}, - sync::{ExtendedPeerInfo, SyncEvent, SyncEventStream, SyncStatus, SyncStatusProvider}, +use sc_network::{NetworkBlock, NetworkSyncForkRequest}; +use sc_network_common::sync::{ + ExtendedPeerInfo, SyncEvent, SyncEventStream, SyncStatus, SyncStatusProvider, }; use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedSender}; use sp_runtime::traits::{Block as BlockT, NumberFor}; diff --git a/client/network/sync/src/service/mock.rs b/client/network/sync/src/service/mock.rs index 2853616adfb4d..6b3f1095c5e8d 100644 --- a/client/network/sync/src/service/mock.rs +++ b/client/network/sync/src/service/mock.rs @@ -19,14 +19,14 @@ use futures::channel::oneshot; use libp2p::{Multiaddr, PeerId}; use sc_consensus::{BlockImportError, BlockImportStatus}; +use sc_network::{ + NetworkNotification, NetworkPeers, NetworkRequest, NetworkSyncForkRequest, + NotificationSenderError, NotificationSenderT, +}; use sc_network_common::{ config::MultiaddrWithPeerId, protocol::ProtocolName, request_responses::{IfDisconnected, RequestFailure}, - service::{ - NetworkNotification, NetworkPeers, NetworkRequest, NetworkSyncForkRequest, - NotificationSender, NotificationSenderError, - }, }; use sc_peerset::ReputationChange; use sp_runtime::traits::{Block as BlockT, NumberFor}; @@ -135,7 +135,7 @@ mockall::mock! { &self, target: PeerId, protocol: ProtocolName, - ) -> Result, NotificationSenderError>; + ) -> Result, NotificationSenderError>; fn set_notification_handshake(&self, protocol: ProtocolName, handshake: Vec); } } diff --git a/client/network/sync/src/service/network.rs b/client/network/sync/src/service/network.rs index a1f4f27bbbd99..0521aeefc253f 100644 --- a/client/network/sync/src/service/network.rs +++ b/client/network/sync/src/service/network.rs @@ -18,10 +18,10 @@ use futures::{channel::oneshot, StreamExt}; use libp2p::PeerId; +use sc_network::{NetworkNotification, NetworkPeers, NetworkRequest}; use sc_network_common::{ protocol::ProtocolName, request_responses::{IfDisconnected, RequestFailure}, - service::{NetworkNotification, NetworkPeers, NetworkRequest}, }; use sc_peerset::ReputationChange; use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; diff --git a/client/network/test/src/lib.rs b/client/network/test/src/lib.rs index 366c4738d5dae..374890e9ae55c 100644 --- a/client/network/test/src/lib.rs +++ b/client/network/test/src/lib.rs @@ -50,7 +50,8 @@ use sc_consensus::{ }; use sc_network::{ config::{NetworkConfiguration, NonDefaultSetConfig}, - Multiaddr, NetworkService, NetworkWorker, + Multiaddr, NetworkBlock, NetworkEventStream, NetworkService, NetworkStateInfo, + NetworkSyncForkRequest, NetworkWorker, }; use sc_network_common::{ config::{ @@ -58,7 +59,6 @@ use sc_network_common::{ SyncMode, TransportConfig, }, protocol::{role::Roles, ProtocolName}, - service::{NetworkBlock, NetworkEventStream, NetworkStateInfo, NetworkSyncForkRequest}, sync::warp::{ AuthorityList, EncodedProof, SetId, VerificationResult, WarpSyncParams, WarpSyncProvider, }, diff --git a/client/network/test/src/service.rs b/client/network/test/src/service.rs index ea835a10771a4..f7cc32b84ce9f 100644 --- a/client/network/test/src/service.rs +++ b/client/network/test/src/service.rs @@ -20,11 +20,13 @@ use futures::prelude::*; use libp2p::{Multiaddr, PeerId}; use sc_consensus::{ImportQueue, Link}; -use sc_network::{config, NetworkService, NetworkWorker}; +use sc_network::{ + config, NetworkEventStream, NetworkNotification, NetworkPeers, NetworkService, + NetworkStateInfo, NetworkWorker, +}; use sc_network_common::{ - config::{MultiaddrWithPeerId, ProtocolId, SetConfig, TransportConfig}, + config::{MultiaddrWithPeerId, ProtocolId, TransportConfig}, protocol::{event::Event, role::Roles}, - service::{NetworkEventStream, NetworkNotification, NetworkPeers, NetworkStateInfo}, }; use sc_network_light::light_client_requests::handler::LightClientRequestHandler; use sc_network_sync::{ @@ -73,7 +75,7 @@ struct TestNetworkBuilder { link: Option>>, client: Option>, listen_addresses: Vec, - set_config: Option, + set_config: Option, chain_sync_network: Option<(NetworkServiceProvider, NetworkServiceHandle)>, config: Option, } @@ -101,7 +103,7 @@ impl TestNetworkBuilder { self } - pub fn with_set_config(mut self, set_config: SetConfig) -> Self { + pub fn with_set_config(mut self, set_config: config::SetConfig) -> Self { self.set_config = Some(set_config); self } @@ -253,7 +255,7 @@ fn build_nodes_one_proto() -> ( .start_network(); let (node2, events_stream2) = TestNetworkBuilder::new() - .with_set_config(SetConfig { + .with_set_config(config::SetConfig { reserved_nodes: vec![MultiaddrWithPeerId { multiaddr: listen_addr, peer_id: node1.local_peer_id(), @@ -411,7 +413,7 @@ async fn lots_of_incoming_peers_works() { let (main_node, _) = TestNetworkBuilder::new() .with_listen_addresses(vec![listen_addr.clone()]) - .with_set_config(SetConfig { in_peers: u32::MAX, ..Default::default() }) + .with_set_config(config::SetConfig { in_peers: u32::MAX, ..Default::default() }) .build() .start_network(); @@ -423,7 +425,7 @@ async fn lots_of_incoming_peers_works() { for _ in 0..32 { let (_dialing_node, event_stream) = TestNetworkBuilder::new() - .with_set_config(SetConfig { + .with_set_config(config::SetConfig { reserved_nodes: vec![MultiaddrWithPeerId { multiaddr: listen_addr.clone(), peer_id: main_node_peer_id, @@ -563,7 +565,7 @@ async fn fallback_name_working() { .start_network(); let (_, mut events_stream2) = TestNetworkBuilder::new() - .with_set_config(SetConfig { + .with_set_config(config::SetConfig { reserved_nodes: vec![MultiaddrWithPeerId { multiaddr: listen_addr, peer_id: node1.local_peer_id(), @@ -704,7 +706,7 @@ async fn ensure_reserved_node_addresses_consistent_with_transport_memory() { .with_config(config::NetworkConfiguration { listen_addresses: vec![listen_addr.clone()], transport: TransportConfig::MemoryOnly, - default_peers_set: SetConfig { + default_peers_set: config::SetConfig { reserved_nodes: vec![reserved_node], ..Default::default() }, @@ -731,7 +733,7 @@ async fn ensure_reserved_node_addresses_consistent_with_transport_not_memory() { let _ = TestNetworkBuilder::new() .with_config(config::NetworkConfiguration { listen_addresses: vec![listen_addr.clone()], - default_peers_set: SetConfig { + default_peers_set: config::SetConfig { reserved_nodes: vec![reserved_node], ..Default::default() }, diff --git a/client/network/transactions/src/lib.rs b/client/network/transactions/src/lib.rs index c22c8c6fca549..e39657d004199 100644 --- a/client/network/transactions/src/lib.rs +++ b/client/network/transactions/src/lib.rs @@ -37,11 +37,13 @@ use log::{debug, trace, warn}; // Substrate dependencies use prometheus_endpoint::{register, Counter, PrometheusError, Registry, U64}; -use sc_network::{config::NonDefaultSetConfig, error}; +use sc_network::{ + config::{NonDefaultSetConfig, SetConfig}, + error, NetworkEventStream, NetworkNotification, NetworkPeers, +}; use sc_network_common::{ - config::{NonReservedPeerMode, ProtocolId, SetConfig}, + config::{NonReservedPeerMode, ProtocolId}, protocol::{event::Event, role::ObservedRole, ProtocolName}, - service::{NetworkEventStream, NetworkNotification, NetworkPeers}, sync::{SyncEvent, SyncEventStream}, utils::{interval, LruHashSet}, ExHashT, diff --git a/client/offchain/Cargo.toml b/client/offchain/Cargo.toml index dd6e2e44caf55..0307e3125f3ee 100644 --- a/client/offchain/Cargo.toml +++ b/client/offchain/Cargo.toml @@ -29,6 +29,7 @@ rand = "0.8.5" threadpool = "1.7" tracing = "0.1.29" sc-client-api = { version = "4.0.0-dev", path = "../api" } +sc-network = { version = "0.10.0-dev", path = "../network" } sc-network-common = { version = "0.10.0-dev", path = "../network/common" } sc-peerset = { version = "4.0.0-dev", path = "../peerset" } sc-utils = { version = "4.0.0-dev", path = "../utils" } diff --git a/client/offchain/src/api.rs b/client/offchain/src/api.rs index 6fdd91cda2390..880d2019dd725 100644 --- a/client/offchain/src/api.rs +++ b/client/offchain/src/api.rs @@ -326,11 +326,8 @@ mod tests { use super::*; use libp2p::PeerId; use sc_client_db::offchain::LocalStorage; - use sc_network_common::{ - config::MultiaddrWithPeerId, - protocol::ProtocolName, - service::{NetworkPeers, NetworkStateInfo}, - }; + use sc_network::{NetworkPeers, NetworkStateInfo}; + use sc_network_common::{config::MultiaddrWithPeerId, protocol::ProtocolName}; use sc_peerset::ReputationChange; use sp_core::offchain::{DbExternalities, Externalities}; use std::time::SystemTime; diff --git a/client/offchain/src/lib.rs b/client/offchain/src/lib.rs index c69f01de19bf6..4f9a7bedf4e80 100644 --- a/client/offchain/src/lib.rs +++ b/client/offchain/src/lib.rs @@ -42,7 +42,7 @@ use futures::{ prelude::*, }; use parking_lot::Mutex; -use sc_network_common::service::{NetworkPeers, NetworkStateInfo}; +use sc_network::{NetworkPeers, NetworkStateInfo}; use sp_api::{ApiExt, ProvideRuntimeApi}; use sp_core::{offchain, traits::SpawnNamed, ExecutionContext}; use sp_runtime::traits::{self, Header}; diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index ea4b630003123..dea75017f6c45 100644 --- a/client/service/src/builder.rs +++ b/client/service/src/builder.rs @@ -38,14 +38,9 @@ use sc_client_db::{Backend, DatabaseSettings}; use sc_consensus::import_queue::ImportQueue; use sc_executor::RuntimeVersionOf; use sc_keystore::LocalKeystore; -use sc_network::NetworkService; +use sc_network::{NetworkEventStream, NetworkService, NetworkStateInfo, NetworkStatusProvider}; use sc_network_bitswap::BitswapRequestHandler; -use sc_network_common::{ - config::SyncMode, - protocol::role::Roles, - service::{NetworkEventStream, NetworkStateInfo, NetworkStatusProvider}, - sync::warp::WarpSyncParams, -}; +use sc_network_common::{config::SyncMode, protocol::role::Roles, sync::warp::WarpSyncParams}; use sc_network_light::light_client_requests::handler::LightClientRequestHandler; use sc_network_sync::{ block_request_handler::BlockRequestHandler, engine::SyncingEngine, diff --git a/client/service/src/config.rs b/client/service/src/config.rs index 14aae046776e9..312a022d1c998 100644 --- a/client/service/src/config.rs +++ b/client/service/src/config.rs @@ -22,11 +22,11 @@ pub use sc_client_api::execution_extensions::{ExecutionStrategies, ExecutionStra pub use sc_client_db::{BlocksPruning, Database, DatabaseSource, PruningMode}; pub use sc_executor::{WasmExecutionMethod, WasmtimeInstantiationStrategy}; pub use sc_network::{ - config::{NetworkConfiguration, NonDefaultSetConfig, Role}, + config::{NetworkConfiguration, NonDefaultSetConfig, Role, SetConfig}, Multiaddr, }; pub use sc_network_common::{ - config::{MultiaddrWithPeerId, NodeKeyConfig, ProtocolId, SetConfig, TransportConfig}, + config::{MultiaddrWithPeerId, NodeKeyConfig, ProtocolId, TransportConfig}, request_responses::{ IncomingRequest, OutgoingResponse, ProtocolConfig as RequestResponseConfig, }, diff --git a/client/service/src/lib.rs b/client/service/src/lib.rs index 9dab81a5b4ed9..44b88f6525c9b 100644 --- a/client/service/src/lib.rs +++ b/client/service/src/lib.rs @@ -41,11 +41,8 @@ use futures::{channel::mpsc, pin_mut, FutureExt, StreamExt}; use jsonrpsee::{core::Error as JsonRpseeError, RpcModule}; use log::{debug, error, warn}; use sc_client_api::{blockchain::HeaderBackend, BlockBackend, BlockchainEvents, ProofProvider}; -use sc_network::{NetworkStateInfo, PeerId}; -use sc_network_common::{ - config::MultiaddrWithPeerId, - service::{NetworkBlock, NetworkPeers}, -}; +use sc_network::{NetworkBlock, NetworkPeers, NetworkStateInfo, PeerId}; +use sc_network_common::config::MultiaddrWithPeerId; use sc_network_sync::SyncingService; use sc_utils::mpsc::TracingUnboundedReceiver; use sp_blockchain::HeaderMetadata; diff --git a/client/service/src/metrics.rs b/client/service/src/metrics.rs index 967e3133dbea9..ece5758be7718 100644 --- a/client/service/src/metrics.rs +++ b/client/service/src/metrics.rs @@ -22,11 +22,8 @@ use crate::config::Configuration; use futures_timer::Delay; use prometheus_endpoint::{register, Gauge, GaugeVec, Opts, PrometheusError, Registry, U64}; use sc_client_api::{ClientInfo, UsageProvider}; -use sc_network::config::Role; -use sc_network_common::{ - service::{NetworkStatus, NetworkStatusProvider}, - sync::{SyncStatus, SyncStatusProvider}, -}; +use sc_network::{config::Role, NetworkStatus, NetworkStatusProvider}; +use sc_network_common::sync::{SyncStatus, SyncStatusProvider}; use sc_telemetry::{telemetry, TelemetryHandle, SUBSTRATE_INFO}; use sc_transaction_pool_api::{MaintainedTransactionPool, PoolStatus}; use sc_utils::metrics::register_globals; diff --git a/client/service/test/src/lib.rs b/client/service/test/src/lib.rs index b1a09a0620fcd..191052fc0b57e 100644 --- a/client/service/test/src/lib.rs +++ b/client/service/test/src/lib.rs @@ -22,11 +22,10 @@ use futures::{task::Poll, Future, TryFutureExt as _}; use log::{debug, info}; use parking_lot::Mutex; use sc_client_api::{Backend, CallExecutor}; -use sc_network::{config::NetworkConfiguration, multiaddr}; -use sc_network_common::{ - config::{MultiaddrWithPeerId, TransportConfig}, - service::{NetworkBlock, NetworkPeers, NetworkStateInfo}, +use sc_network::{ + config::NetworkConfiguration, multiaddr, NetworkBlock, NetworkPeers, NetworkStateInfo, }; +use sc_network_common::config::{MultiaddrWithPeerId, TransportConfig}; use sc_network_sync::SyncingService; use sc_service::{ client::Client, From 5d25a52d264553dc47c998f1f41f71caa80a7401 Mon Sep 17 00:00:00 2001 From: Aaro Altonen Date: Mon, 13 Mar 2023 11:42:19 +0200 Subject: [PATCH 04/10] Move request-responses to `sc-network` --- Cargo.lock | 2 + .../outgoing_requests_engine.rs | 6 +- client/network/bitswap/Cargo.toml | 1 + client/network/bitswap/src/lib.rs | 11 +- client/network/common/src/config.rs | 3 - client/network/common/src/lib.rs | 1 - .../network/common/src/request_responses.rs | 155 ------------------ client/network/light/Cargo.toml | 1 + .../light/src/light_client_requests.rs | 9 +- .../src/light_client_requests/handler.rs | 6 +- client/network/src/behaviour.rs | 13 +- client/network/src/config.rs | 10 +- client/network/src/lib.rs | 4 +- client/network/src/protocol.rs | 4 - client/network/src/request_responses.rs | 153 +++++++++++++++-- client/network/src/service.rs | 6 +- client/network/src/service/traits.rs | 6 +- .../network/sync/src/block_request_handler.rs | 10 +- client/network/sync/src/engine.rs | 4 - client/network/sync/src/lib.rs | 10 +- client/network/sync/src/service/mock.rs | 9 +- client/network/sync/src/service/network.rs | 8 +- .../network/sync/src/state_request_handler.rs | 9 +- .../network/sync/src/warp_request_handler.rs | 8 +- client/network/test/src/lib.rs | 4 +- client/network/transactions/src/lib.rs | 4 - client/service/src/config.rs | 8 +- 27 files changed, 210 insertions(+), 255 deletions(-) delete mode 100644 client/network/common/src/request_responses.rs diff --git a/Cargo.lock b/Cargo.lock index c16860f0c921b..da12cf7918073 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8840,6 +8840,7 @@ dependencies = [ "sc-block-builder", "sc-client-api", "sc-consensus", + "sc-network", "sc-network-common", "sp-blockchain", "sp-consensus", @@ -8914,6 +8915,7 @@ dependencies = [ "prost", "prost-build", "sc-client-api", + "sc-network", "sc-network-common", "sc-peerset", "sp-blockchain", diff --git a/client/consensus/beefy/src/communication/request_response/outgoing_requests_engine.rs b/client/consensus/beefy/src/communication/request_response/outgoing_requests_engine.rs index 7c6112ab101c9..c642e3dfe581c 100644 --- a/client/consensus/beefy/src/communication/request_response/outgoing_requests_engine.rs +++ b/client/consensus/beefy/src/communication/request_response/outgoing_requests_engine.rs @@ -22,8 +22,10 @@ use codec::Encode; use futures::channel::{oneshot, oneshot::Canceled}; use log::{debug, warn}; use parking_lot::Mutex; -use sc_network::{NetworkRequest, PeerId, ProtocolName}; -use sc_network_common::request_responses::{IfDisconnected, RequestFailure}; +use sc_network::{ + request_responses::{IfDisconnected, RequestFailure}, + NetworkRequest, PeerId, ProtocolName, +}; use sp_consensus_beefy::{crypto::AuthorityId, ValidatorSet}; use sp_runtime::traits::{Block, NumberFor}; use std::{collections::VecDeque, result::Result, sync::Arc}; diff --git a/client/network/bitswap/Cargo.toml b/client/network/bitswap/Cargo.toml index 0bbb5ff09e4b9..ee2e0cfc79ff7 100644 --- a/client/network/bitswap/Cargo.toml +++ b/client/network/bitswap/Cargo.toml @@ -24,6 +24,7 @@ prost = "0.11" thiserror = "1.0" unsigned-varint = { version = "0.7.1", features = ["futures", "asynchronous_codec"] } sc-client-api = { version = "4.0.0-dev", path = "../../api" } +sc-network = { version = "0.10.0-dev", path = "../" } sc-network-common = { version = "0.10.0-dev", path = "../common" } sp-blockchain = { version = "4.0.0-dev", path = "../../../primitives/blockchain" } sp-runtime = { version = "7.0.0", path = "../../../primitives/runtime" } diff --git a/client/network/bitswap/src/lib.rs b/client/network/bitswap/src/lib.rs index dd5fc46f6ca71..dd3b9a659a458 100644 --- a/client/network/bitswap/src/lib.rs +++ b/client/network/bitswap/src/lib.rs @@ -26,10 +26,8 @@ use libp2p::core::PeerId; use log::{debug, error, trace}; use prost::Message; use sc_client_api::BlockBackend; -use sc_network_common::{ - protocol::ProtocolName, - request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}, -}; +use sc_network::request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}; +use sc_network_common::protocol::ProtocolName; use schema::bitswap::{ message::{wantlist::WantType, Block as MessageBlock, BlockPresence, BlockPresenceType}, Message as BitswapMessage, @@ -127,8 +125,9 @@ impl BitswapRequestHandler { }; match pending_response.send(response) { - Ok(()) => - trace!(target: LOG_TARGET, "Handled bitswap request from {peer}.",), + Ok(()) => { + trace!(target: LOG_TARGET, "Handled bitswap request from {peer}.",) + }, Err(_) => debug!( target: LOG_TARGET, "Failed to handle light client request from {peer}: {}", diff --git a/client/network/common/src/config.rs b/client/network/common/src/config.rs index 4faf0bb4d443d..723aeff950c24 100644 --- a/client/network/common/src/config.rs +++ b/client/network/common/src/config.rs @@ -20,9 +20,6 @@ pub use crate::{ protocol::{self, role::Role}, - request_responses::{ - IncomingRequest, OutgoingResponse, ProtocolConfig as RequestResponseConfig, - }, sync::warp::WarpSyncProvider, ExHashT, }; diff --git a/client/network/common/src/lib.rs b/client/network/common/src/lib.rs index dfae3d0a0c2be..025a7415c5591 100644 --- a/client/network/common/src/lib.rs +++ b/client/network/common/src/lib.rs @@ -21,7 +21,6 @@ pub mod config; pub mod message; pub mod protocol; -pub mod request_responses; pub mod service; pub mod sync; pub mod utils; diff --git a/client/network/common/src/request_responses.rs b/client/network/common/src/request_responses.rs deleted file mode 100644 index 4fa53a736ac33..0000000000000 --- a/client/network/common/src/request_responses.rs +++ /dev/null @@ -1,155 +0,0 @@ -// This file is part of Substrate. - -// Copyright (C) Parity Technologies (UK) Ltd. -// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0 - -// This program is free software: you can redistribute it and/or modify -// it under the terms of the GNU General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. - -// This program is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU General Public License for more details. - -// You should have received a copy of the GNU General Public License -// along with this program. If not, see . - -//! Collection of generic data structures for request-response protocols. - -use crate::protocol::ProtocolName; -use futures::channel::{mpsc, oneshot}; -use libp2p::{request_response::OutboundFailure, PeerId}; -use sc_peerset::ReputationChange; -use std::time::Duration; - -/// Configuration for a single request-response protocol. -#[derive(Debug, Clone)] -pub struct ProtocolConfig { - /// Name of the protocol on the wire. Should be something like `/foo/bar`. - pub name: ProtocolName, - - /// Fallback on the wire protocol names to support. - pub fallback_names: Vec, - - /// Maximum allowed size, in bytes, of a request. - /// - /// Any request larger than this value will be declined as a way to avoid allocating too - /// much memory for it. - pub max_request_size: u64, - - /// Maximum allowed size, in bytes, of a response. - /// - /// Any response larger than this value will be declined as a way to avoid allocating too - /// much memory for it. - pub max_response_size: u64, - - /// Duration after which emitted requests are considered timed out. - /// - /// If you expect the response to come back quickly, you should set this to a smaller duration. - pub request_timeout: Duration, - - /// Channel on which the networking service will send incoming requests. - /// - /// Every time a peer sends a request to the local node using this protocol, the networking - /// service will push an element on this channel. The receiving side of this channel then has - /// to pull this element, process the request, and send back the response to send back to the - /// peer. - /// - /// The size of the channel has to be carefully chosen. If the channel is full, the networking - /// service will discard the incoming request send back an error to the peer. Consequently, - /// the channel being full is an indicator that the node is overloaded. - /// - /// You can typically set the size of the channel to `T / d`, where `T` is the - /// `request_timeout` and `d` is the expected average duration of CPU and I/O it takes to - /// build a response. - /// - /// Can be `None` if the local node does not support answering incoming requests. - /// If this is `None`, then the local node will not advertise support for this protocol towards - /// other peers. If this is `Some` but the channel is closed, then the local node will - /// advertise support for this protocol, but any incoming request will lead to an error being - /// sent back. - pub inbound_queue: Option>, -} - -/// A single request received by a peer on a request-response protocol. -#[derive(Debug)] -pub struct IncomingRequest { - /// Who sent the request. - pub peer: PeerId, - - /// Request sent by the remote. Will always be smaller than - /// [`ProtocolConfig::max_request_size`]. - pub payload: Vec, - - /// Channel to send back the response. - /// - /// There are two ways to indicate that handling the request failed: - /// - /// 1. Drop `pending_response` and thus not changing the reputation of the peer. - /// - /// 2. Sending an `Err(())` via `pending_response`, optionally including reputation changes for - /// the given peer. - pub pending_response: oneshot::Sender, -} - -/// Response for an incoming request to be send by a request protocol handler. -#[derive(Debug)] -pub struct OutgoingResponse { - /// The payload of the response. - /// - /// `Err(())` if none is available e.g. due an error while handling the request. - pub result: Result, ()>, - - /// Reputation changes accrued while handling the request. To be applied to the reputation of - /// the peer sending the request. - pub reputation_changes: Vec, - - /// If provided, the `oneshot::Sender` will be notified when the request has been sent to the - /// peer. - /// - /// > **Note**: Operating systems typically maintain a buffer of a few dozen kilobytes of - /// > outgoing data for each TCP socket, and it is not possible for a user - /// > application to inspect this buffer. This channel here is not actually notified - /// > when the response has been fully sent out, but rather when it has fully been - /// > written to the buffer managed by the operating system. - pub sent_feedback: Option>, -} - -/// When sending a request, what to do on a disconnected recipient. -#[derive(Debug, Copy, Clone, PartialEq, Eq, Hash)] -pub enum IfDisconnected { - /// Try to connect to the peer. - TryConnect, - /// Just fail if the destination is not yet connected. - ImmediateError, -} - -/// Convenience functions for `IfDisconnected`. -impl IfDisconnected { - /// Shall we connect to a disconnected peer? - pub fn should_connect(self) -> bool { - match self { - Self::TryConnect => true, - Self::ImmediateError => false, - } - } -} - -/// Error in a request. -#[derive(Debug, thiserror::Error)] -#[allow(missing_docs)] -pub enum RequestFailure { - #[error("We are not currently connected to the requested peer.")] - NotConnected, - #[error("Given protocol hasn't been registered.")] - UnknownProtocol, - #[error("Remote has closed the substream before answering, thereby signaling that it considers the request as valid, but refused to answer it.")] - Refused, - #[error("The remote replied, but the local node is no longer interested in the response.")] - Obsolete, - /// Problem on the network. - #[error("Problem on the network: {0}")] - Network(OutboundFailure), -} diff --git a/client/network/light/Cargo.toml b/client/network/light/Cargo.toml index f3e94790df61e..ed2a5d6cb4ec6 100644 --- a/client/network/light/Cargo.toml +++ b/client/network/light/Cargo.toml @@ -26,6 +26,7 @@ log = "0.4.16" prost = "0.11" sp-blockchain = { version = "4.0.0-dev", path = "../../../primitives/blockchain" } sc-client-api = { version = "4.0.0-dev", path = "../../api" } +sc-network = { version = "0.10.0-dev", path = "../" } sc-network-common = { version = "0.10.0-dev", path = "../common" } sc-peerset = { version = "4.0.0-dev", path = "../../peerset" } sp-core = { version = "7.0.0", path = "../../../primitives/core" } diff --git a/client/network/light/src/light_client_requests.rs b/client/network/light/src/light_client_requests.rs index 5b6daf174cce6..2f6f766fd6439 100644 --- a/client/network/light/src/light_client_requests.rs +++ b/client/network/light/src/light_client_requests.rs @@ -18,13 +18,14 @@ //! Helpers for outgoing and incoming light client requests. -/// For incoming light client requests. -pub mod handler; - -use sc_network_common::{config::ProtocolId, request_responses::ProtocolConfig}; +use sc_network::request_responses::ProtocolConfig; +use sc_network_common::config::ProtocolId; use std::time::Duration; +/// For incoming light client requests. +pub mod handler; + /// Generate the light client protocol name from the genesis hash and fork id. fn generate_protocol_name>(genesis_hash: Hash, fork_id: Option<&str>) -> String { let genesis_hash = genesis_hash.as_ref(); diff --git a/client/network/light/src/light_client_requests/handler.rs b/client/network/light/src/light_client_requests/handler.rs index 0a1215a1dc6a9..7c38f299c444e 100644 --- a/client/network/light/src/light_client_requests/handler.rs +++ b/client/network/light/src/light_client_requests/handler.rs @@ -29,10 +29,8 @@ use libp2p::PeerId; use log::{debug, trace}; use prost::Message; use sc_client_api::{BlockBackend, ProofProvider}; -use sc_network_common::{ - config::ProtocolId, - request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}, -}; +use sc_network::request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}; +use sc_network_common::config::ProtocolId; use sc_peerset::ReputationChange; use sp_core::{ hexdisplay::HexDisplay, diff --git a/client/network/src/behaviour.rs b/client/network/src/behaviour.rs index 5591290200361..cd963a044fb02 100644 --- a/client/network/src/behaviour.rs +++ b/client/network/src/behaviour.rs @@ -20,7 +20,7 @@ use crate::{ discovery::{DiscoveryBehaviour, DiscoveryConfig, DiscoveryOut}, peer_info, protocol::{CustomMessageOutcome, NotificationsSink, Protocol}, - request_responses, + request_responses::{self, IfDisconnected, ProtocolConfig, RequestFailure}, }; use bytes::Bytes; @@ -32,13 +32,10 @@ use libp2p::{ swarm::NetworkBehaviour, }; -use sc_network_common::{ - protocol::{ - event::DhtEvent, - role::{ObservedRole, Roles}, - ProtocolName, - }, - request_responses::{IfDisconnected, ProtocolConfig, RequestFailure}, +use sc_network_common::protocol::{ + event::DhtEvent, + role::{ObservedRole, Roles}, + ProtocolName, }; use sc_peerset::{PeersetHandle, ReputationChange}; use sp_runtime::traits::Block as BlockT; diff --git a/client/network/src/config.rs b/client/network/src/config.rs index 9d4a3087ac0c2..60ec35f88df45 100644 --- a/client/network/src/config.rs +++ b/client/network/src/config.rs @@ -21,10 +21,12 @@ //! The [`Params`] struct is the struct that must be passed in order to initialize the networking. //! See the documentation of [`Params`]. -// External dependencies +pub use crate::request_responses::{ + IncomingRequest, OutgoingResponse, ProtocolConfig as RequestResponseConfig, +}; + pub use libp2p::{build_multiaddr, core::PublicKey, identity, multiaddr, Multiaddr}; -// Substrate dependencies use prometheus_endpoint::Registry; pub use sc_network_common::{ config::{ @@ -32,14 +34,10 @@ pub use sc_network_common::{ SyncMode, TransportConfig, }, protocol::{role::Role, ProtocolName}, - request_responses::{ - IncomingRequest, OutgoingResponse, ProtocolConfig as RequestResponseConfig, - }, sync::warp::WarpSyncProvider, ExHashT, }; -// `std` dependencies use std::{future::Future, iter, net::Ipv4Addr, path::PathBuf, pin::Pin, sync::Arc}; /// Configuration for a set of nodes. diff --git a/client/network/src/lib.rs b/client/network/src/lib.rs index 88674eb438a5d..a74bb926c7a1f 100644 --- a/client/network/src/lib.rs +++ b/client/network/src/lib.rs @@ -248,23 +248,23 @@ mod behaviour; mod discovery; mod peer_info; mod protocol; -mod request_responses; mod service; mod transport; pub mod config; pub mod error; pub mod network_state; +pub mod request_responses; #[doc(inline)] pub use libp2p::{multiaddr, Multiaddr, PeerId}; +pub use request_responses::{IfDisconnected, RequestFailure, RequestResponseConfig}; pub use sc_network_common::{ protocol::{ event::{DhtEvent, Event}, role::ObservedRole, ProtocolName, }, - request_responses::{IfDisconnected, RequestFailure}, sync::{ warp::{WarpSyncPhase, WarpSyncProgress}, ExtendedPeerInfo, StateDownloadProgress, SyncEventStream, SyncState, SyncStatusProvider, diff --git a/client/network/src/protocol.rs b/client/network/src/protocol.rs index 063240876357a..522f8ae7fcdaf 100644 --- a/client/network/src/protocol.rs +++ b/client/network/src/protocol.rs @@ -16,10 +16,8 @@ // You should have received a copy of the GNU General Public License // along with this program. If not, see . -// Local dependencies use crate::{config, error}; -// External dependencies use bytes::Bytes; use codec::{DecodeAll, Encode}; use libp2p::{ @@ -32,7 +30,6 @@ use libp2p::{ }; use log::{debug, error, warn}; -// Substrate dependencies use sc_network_common::{ config::NonReservedPeerMode, protocol::{role::Roles, ProtocolName}, @@ -40,7 +37,6 @@ use sc_network_common::{ }; use sp_runtime::traits::Block as BlockT; -// `std` dependencies use std::{ collections::{HashMap, HashSet, VecDeque}, iter, diff --git a/client/network/src/request_responses.rs b/client/network/src/request_responses.rs index 37c97e8282b4c..c32de5feb8eaf 100644 --- a/client/network/src/request_responses.rs +++ b/client/network/src/request_responses.rs @@ -35,6 +35,7 @@ //! is used to handle incoming requests. use crate::ReputationChange; + use futures::{ channel::{mpsc, oneshot}, prelude::*, @@ -43,7 +44,7 @@ use libp2p::{ core::{connection::ConnectionId, Multiaddr, PeerId}, request_response::{ handler::RequestResponseHandler, ProtocolSupport, RequestResponse, RequestResponseCodec, - RequestResponseConfig, RequestResponseEvent, RequestResponseMessage, ResponseChannel, + RequestResponseEvent, RequestResponseMessage, ResponseChannel, }, swarm::{ behaviour::{ConnectionClosed, DialFailure, FromSwarm, ListenFailure}, @@ -52,12 +53,10 @@ use libp2p::{ PollParameters, }, }; -use sc_network_common::{ - protocol::ProtocolName, - request_responses::{ - IfDisconnected, IncomingRequest, OutgoingResponse, ProtocolConfig, RequestFailure, - }, -}; + +use sc_network_common::protocol::ProtocolName; +use sc_peerset::{PeersetHandle, BANNED_THRESHOLD}; + use std::{ collections::{hash_map::Entry, HashMap}, io, iter, @@ -66,8 +65,139 @@ use std::{ time::{Duration, Instant}, }; -pub use libp2p::request_response::{InboundFailure, OutboundFailure, RequestId}; -use sc_peerset::{PeersetHandle, BANNED_THRESHOLD}; +pub use libp2p::request_response::{ + InboundFailure, OutboundFailure, RequestId, RequestResponseConfig, +}; + +/// Error in a request. +#[derive(Debug, thiserror::Error)] +#[allow(missing_docs)] +pub enum RequestFailure { + #[error("We are not currently connected to the requested peer.")] + NotConnected, + #[error("Given protocol hasn't been registered.")] + UnknownProtocol, + #[error("Remote has closed the substream before answering, thereby signaling that it considers the request as valid, but refused to answer it.")] + Refused, + #[error("The remote replied, but the local node is no longer interested in the response.")] + Obsolete, + /// Problem on the network. + #[error("Problem on the network: {0}")] + Network(OutboundFailure), +} + +/// Configuration for a single request-response protocol. +#[derive(Debug, Clone)] +pub struct ProtocolConfig { + /// Name of the protocol on the wire. Should be something like `/foo/bar`. + pub name: ProtocolName, + + /// Fallback on the wire protocol names to support. + pub fallback_names: Vec, + + /// Maximum allowed size, in bytes, of a request. + /// + /// Any request larger than this value will be declined as a way to avoid allocating too + /// much memory for it. + pub max_request_size: u64, + + /// Maximum allowed size, in bytes, of a response. + /// + /// Any response larger than this value will be declined as a way to avoid allocating too + /// much memory for it. + pub max_response_size: u64, + + /// Duration after which emitted requests are considered timed out. + /// + /// If you expect the response to come back quickly, you should set this to a smaller duration. + pub request_timeout: Duration, + + /// Channel on which the networking service will send incoming requests. + /// + /// Every time a peer sends a request to the local node using this protocol, the networking + /// service will push an element on this channel. The receiving side of this channel then has + /// to pull this element, process the request, and send back the response to send back to the + /// peer. + /// + /// The size of the channel has to be carefully chosen. If the channel is full, the networking + /// service will discard the incoming request send back an error to the peer. Consequently, + /// the channel being full is an indicator that the node is overloaded. + /// + /// You can typically set the size of the channel to `T / d`, where `T` is the + /// `request_timeout` and `d` is the expected average duration of CPU and I/O it takes to + /// build a response. + /// + /// Can be `None` if the local node does not support answering incoming requests. + /// If this is `None`, then the local node will not advertise support for this protocol towards + /// other peers. If this is `Some` but the channel is closed, then the local node will + /// advertise support for this protocol, but any incoming request will lead to an error being + /// sent back. + pub inbound_queue: Option>, +} + +/// A single request received by a peer on a request-response protocol. +#[derive(Debug)] +pub struct IncomingRequest { + /// Who sent the request. + pub peer: PeerId, + + /// Request sent by the remote. Will always be smaller than + /// [`ProtocolConfig::max_request_size`]. + pub payload: Vec, + + /// Channel to send back the response. + /// + /// There are two ways to indicate that handling the request failed: + /// + /// 1. Drop `pending_response` and thus not changing the reputation of the peer. + /// + /// 2. Sending an `Err(())` via `pending_response`, optionally including reputation changes for + /// the given peer. + pub pending_response: oneshot::Sender, +} + +/// Response for an incoming request to be send by a request protocol handler. +#[derive(Debug)] +pub struct OutgoingResponse { + /// The payload of the response. + /// + /// `Err(())` if none is available e.g. due an error while handling the request. + pub result: Result, ()>, + + /// Reputation changes accrued while handling the request. To be applied to the reputation of + /// the peer sending the request. + pub reputation_changes: Vec, + + /// If provided, the `oneshot::Sender` will be notified when the request has been sent to the + /// peer. + /// + /// > **Note**: Operating systems typically maintain a buffer of a few dozen kilobytes of + /// > outgoing data for each TCP socket, and it is not possible for a user + /// > application to inspect this buffer. This channel here is not actually notified + /// > when the response has been fully sent out, but rather when it has fully been + /// > written to the buffer managed by the operating system. + pub sent_feedback: Option>, +} + +/// When sending a request, what to do on a disconnected recipient. +#[derive(Debug, Copy, Clone, PartialEq, Eq, Hash)] +pub enum IfDisconnected { + /// Try to connect to the peer. + TryConnect, + /// Just fail if the destination is not yet connected. + ImmediateError, +} + +/// Convenience functions for `IfDisconnected`. +impl IfDisconnected { + /// Shall we connect to a disconnected peer? + pub fn should_connect(self) -> bool { + match self { + Self::TryConnect => true, + Self::ImmediateError => false, + } + } +} /// Event generated by the [`RequestResponsesBehaviour`]. #[derive(Debug)] @@ -344,7 +474,7 @@ impl NetworkBehaviour for RequestResponsesBehaviour { ) } }, - FromSwarm::DialFailure(DialFailure { peer_id, error, handler }) => + FromSwarm::DialFailure(DialFailure { peer_id, error, handler }) => { for (p_name, p_handler) in handler.into_iter() { if let Some((proto, _)) = self.protocols.get_mut(p_name.as_str()) { proto.on_swarm_event(FromSwarm::DialFailure(DialFailure { @@ -359,7 +489,8 @@ impl NetworkBehaviour for RequestResponsesBehaviour { p_name, ) } - }, + } + }, FromSwarm::ListenerClosed(e) => for (p, _) in self.protocols.values_mut() { NetworkBehaviour::on_swarm_event(p, FromSwarm::ListenerClosed(e)); diff --git a/client/network/src/service.rs b/client/network/src/service.rs index f80684b11b41c..cf2ba82274ddb 100644 --- a/client/network/src/service.rs +++ b/client/network/src/service.rs @@ -27,7 +27,6 @@ //! The methods of the [`NetworkService`] are implemented by sending a message over a channel, //! which is then processed by [`NetworkWorker::next_action`]. -// Local dependencies use crate::{ behaviour::{self, Behaviour, BehaviourOut}, config::Params, @@ -37,6 +36,7 @@ use crate::{ NetworkState, NotConnectedPeer as NetworkStateNotConnectedPeer, Peer as NetworkStatePeer, }, protocol::{self, NotificationsSink, NotifsHandlerError, Protocol, Ready}, + request_responses::{IfDisconnected, RequestFailure}, service::traits::{ NetworkDHTProvider, NetworkEventStream, NetworkNotification, NetworkPeers, NetworkRequest, NetworkSigner, NetworkStateInfo, NetworkStatus, NetworkStatusProvider, @@ -46,7 +46,6 @@ use crate::{ transport, ReputationChange, }; -// External dependencies use futures::{channel::oneshot, prelude::*}; use libp2p::{ core::{either::EitherError, upgrade, ConnectedPoint}, @@ -65,14 +64,12 @@ use log::{debug, error, info, trace, warn}; use metrics::{Histogram, HistogramVec, MetricSources, Metrics}; use parking_lot::Mutex; -// Substrate dependencies use sc_network_common::{ config::{MultiaddrWithPeerId, TransportConfig}, protocol::{ event::{DhtEvent, Event}, ProtocolName, }, - request_responses::{IfDisconnected, RequestFailure}, service::signature::{Signature, SigningError}, ExHashT, }; @@ -81,7 +78,6 @@ use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnbound use sp_blockchain::HeaderBackend; use sp_runtime::traits::{Block as BlockT, Zero}; -// `std` dependencies use std::{ cmp, collections::{HashMap, HashSet}, diff --git a/client/network/src/service/traits.rs b/client/network/src/service/traits.rs index bdbd435d87a7b..a3bf536e4bbdc 100644 --- a/client/network/src/service/traits.rs +++ b/client/network/src/service/traits.rs @@ -18,20 +18,18 @@ // // If you read this, you are very thorough, congratulations. -// External dependencies +use crate::request_responses::{IfDisconnected, RequestFailure}; + use futures::{channel::oneshot, Stream}; use libp2p::{Multiaddr, PeerId}; -// Substrate dependencies use sc_network_common::{ config::MultiaddrWithPeerId, protocol::{event::Event, ProtocolName}, - request_responses::{IfDisconnected, RequestFailure}, service::signature::Signature, }; use sc_peerset::ReputationChange; -// `std` dependencies use std::{collections::HashSet, future::Future, pin::Pin, sync::Arc}; pub use libp2p::{identity::error::SigningError, kad::record::Key as KademliaKey}; diff --git a/client/network/sync/src/block_request_handler.rs b/client/network/sync/src/block_request_handler.rs index 38aca2cde3fd2..d126786138ef0 100644 --- a/client/network/sync/src/block_request_handler.rs +++ b/client/network/sync/src/block_request_handler.rs @@ -18,6 +18,7 @@ //! `crate::request_responses::RequestResponsesBehaviour`. use crate::schema::v1::{block_request::FromBlock, BlockResponse, Direction}; + use codec::{Decode, Encode}; use futures::{ channel::{mpsc, oneshot}, @@ -27,17 +28,16 @@ use libp2p::PeerId; use log::debug; use lru::LruCache; use prost::Message; + use sc_client_api::BlockBackend; -use sc_network_common::{ - config::ProtocolId, - request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}, - sync::message::BlockAttributes, -}; +use sc_network::request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}; +use sc_network_common::{config::ProtocolId, sync::message::BlockAttributes}; use sp_blockchain::HeaderBackend; use sp_runtime::{ generic::BlockId, traits::{Block as BlockT, Header, One, Zero}, }; + use std::{ cmp::min, hash::{Hash, Hasher}, diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index c17dde1170c21..a6619aca96a7e 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -19,13 +19,11 @@ //! `SyncingEngine` is the actor responsible for syncing Substrate chain //! to tip and keep the blockchain up to date with network updates. -// Local dependencies use crate::{ service::{self, chain_sync::ToServiceCommand}, ChainSync, ClientError, SyncingService, }; -// External dependencies use codec::{Decode, DecodeAll, Encode}; use futures::{FutureExt, Stream, StreamExt}; use futures_timer::Delay; @@ -35,7 +33,6 @@ use prometheus_endpoint::{ register, Gauge, GaugeVec, MetricSource, Opts, PrometheusError, Registry, SourcedGauge, U64, }; -// Substrate dependencies use sc_client_api::{BlockBackend, HeaderBackend, ProofProvider}; use sc_consensus::import_queue::ImportQueueService; use sc_network::config::{NetworkConfiguration, NonDefaultSetConfig}; @@ -61,7 +58,6 @@ use sp_runtime::{ SaturatedConversion, }; -// `std` dependencies use std::{ collections::{HashMap, HashSet}, num::NonZeroUsize, diff --git a/client/network/sync/src/lib.rs b/client/network/sync/src/lib.rs index 46c87c6866c33..b32783bf2f24e 100644 --- a/client/network/sync/src/lib.rs +++ b/client/network/sync/src/lib.rs @@ -28,7 +28,6 @@ //! the network, or whenever a block has been successfully verified, call the appropriate method in //! order to update it. -// Local dependencies use crate::{ blocks::BlockCollection, schema::v1::{StateRequest, StateResponse}, @@ -36,7 +35,6 @@ use crate::{ warp::{WarpProofImportResult, WarpSync}, }; -// External dependencies use codec::{Decode, DecodeAll, Encode}; use extra_requests::ExtraRequests; use futures::{ @@ -46,17 +44,18 @@ use libp2p::{request_response::OutboundFailure, PeerId}; use log::{debug, error, info, trace, warn}; use prost::Message; -// Substrate dependencies use prometheus_endpoint::{register, Counter, PrometheusError, Registry, U64}; use sc_client_api::{BlockBackend, ProofProvider}; use sc_consensus::{ import_queue::ImportQueueService, BlockImportError, BlockImportStatus, IncomingBlock, }; -use sc_network::config::{NonDefaultSetConfig, SetConfig}; +use sc_network::{ + config::{NonDefaultSetConfig, SetConfig}, + request_responses::{IfDisconnected, RequestFailure}, +}; use sc_network_common::{ config::{NonReservedPeerMode, NotificationHandshake, ProtocolId}, protocol::{role::Roles, ProtocolName}, - request_responses::{IfDisconnected, RequestFailure}, sync::{ message::{ BlockAnnounce, BlockAnnouncesHandshake, BlockAttributes, BlockData, BlockRequest, @@ -83,7 +82,6 @@ use sp_runtime::{ EncodedJustification, Justifications, }; -// `std` dependencies use std::{ collections::{hash_map::Entry, HashMap, HashSet}, iter, diff --git a/client/network/sync/src/service/mock.rs b/client/network/sync/src/service/mock.rs index 6b3f1095c5e8d..47dbc674b8ab3 100644 --- a/client/network/sync/src/service/mock.rs +++ b/client/network/sync/src/service/mock.rs @@ -18,18 +18,17 @@ use futures::channel::oneshot; use libp2p::{Multiaddr, PeerId}; + use sc_consensus::{BlockImportError, BlockImportStatus}; use sc_network::{ + request_responses::{IfDisconnected, RequestFailure}, NetworkNotification, NetworkPeers, NetworkRequest, NetworkSyncForkRequest, NotificationSenderError, NotificationSenderT, }; -use sc_network_common::{ - config::MultiaddrWithPeerId, - protocol::ProtocolName, - request_responses::{IfDisconnected, RequestFailure}, -}; +use sc_network_common::{config::MultiaddrWithPeerId, protocol::ProtocolName}; use sc_peerset::ReputationChange; use sp_runtime::traits::{Block as BlockT, NumberFor}; + use std::collections::HashSet; mockall::mock! { diff --git a/client/network/sync/src/service/network.rs b/client/network/sync/src/service/network.rs index 0521aeefc253f..e59ba1e90d151 100644 --- a/client/network/sync/src/service/network.rs +++ b/client/network/sync/src/service/network.rs @@ -18,13 +18,15 @@ use futures::{channel::oneshot, StreamExt}; use libp2p::PeerId; -use sc_network::{NetworkNotification, NetworkPeers, NetworkRequest}; -use sc_network_common::{ - protocol::ProtocolName, + +use sc_network::{ request_responses::{IfDisconnected, RequestFailure}, + NetworkNotification, NetworkPeers, NetworkRequest, }; +use sc_network_common::protocol::ProtocolName; use sc_peerset::ReputationChange; use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; + use std::sync::Arc; /// Network-related services required by `sc-network-sync` diff --git a/client/network/sync/src/state_request_handler.rs b/client/network/sync/src/state_request_handler.rs index 2f615f4ead753..b6729dc036a8d 100644 --- a/client/network/sync/src/state_request_handler.rs +++ b/client/network/sync/src/state_request_handler.rs @@ -18,6 +18,7 @@ //! `crate::request_responses::RequestResponsesBehaviour`. use crate::schema::v1::{KeyValueStateEntry, StateEntry, StateRequest, StateResponse}; + use codec::{Decode, Encode}; use futures::{ channel::{mpsc, oneshot}, @@ -27,12 +28,12 @@ use libp2p::PeerId; use log::{debug, trace}; use lru::LruCache; use prost::Message; + use sc_client_api::{BlockBackend, ProofProvider}; -use sc_network_common::{ - config::ProtocolId, - request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}, -}; +use sc_network::request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}; +use sc_network_common::config::ProtocolId; use sp_runtime::traits::Block as BlockT; + use std::{ hash::{Hash, Hasher}, num::NonZeroUsize, diff --git a/client/network/sync/src/warp_request_handler.rs b/client/network/sync/src/warp_request_handler.rs index fcb0d116bcdac..8936f71eb1892 100644 --- a/client/network/sync/src/warp_request_handler.rs +++ b/client/network/sync/src/warp_request_handler.rs @@ -22,14 +22,16 @@ use futures::{ stream::StreamExt, }; use log::debug; + +use sc_network::request_responses::{ + IncomingRequest, OutgoingResponse, ProtocolConfig as RequestResponseConfig, +}; use sc_network_common::{ config::ProtocolId, - request_responses::{ - IncomingRequest, OutgoingResponse, ProtocolConfig as RequestResponseConfig, - }, sync::warp::{EncodedProof, WarpProofRequest, WarpSyncProvider}, }; use sp_runtime::traits::Block as BlockT; + use std::{sync::Arc, time::Duration}; const MAX_RESPONSE_SIZE: u64 = 16 * 1024 * 1024; diff --git a/client/network/test/src/lib.rs b/client/network/test/src/lib.rs index 374890e9ae55c..42f03d02636c2 100644 --- a/client/network/test/src/lib.rs +++ b/client/network/test/src/lib.rs @@ -50,13 +50,13 @@ use sc_consensus::{ }; use sc_network::{ config::{NetworkConfiguration, NonDefaultSetConfig}, + request_responses::ProtocolConfig as RequestResponseConfig, Multiaddr, NetworkBlock, NetworkEventStream, NetworkService, NetworkStateInfo, NetworkSyncForkRequest, NetworkWorker, }; use sc_network_common::{ config::{ - MultiaddrWithPeerId, NonReservedPeerMode, ProtocolId, RequestResponseConfig, Role, - SyncMode, TransportConfig, + MultiaddrWithPeerId, NonReservedPeerMode, ProtocolId, Role, SyncMode, TransportConfig, }, protocol::{role::Roles, ProtocolName}, sync::warp::{ diff --git a/client/network/transactions/src/lib.rs b/client/network/transactions/src/lib.rs index e39657d004199..4ccee2cce239c 100644 --- a/client/network/transactions/src/lib.rs +++ b/client/network/transactions/src/lib.rs @@ -26,16 +26,13 @@ //! - Use [`TransactionsHandlerPrototype::build`] then [`TransactionsHandler::run`] to obtain a //! `Future` that processes transactions. -// Local dependencies use crate::config::*; -// External dependencies use codec::{Decode, Encode}; use futures::{prelude::*, stream::FuturesUnordered}; use libp2p::{multiaddr, PeerId}; use log::{debug, trace, warn}; -// Substrate dependencies use prometheus_endpoint::{register, Counter, PrometheusError, Registry, U64}; use sc_network::{ config::{NonDefaultSetConfig, SetConfig}, @@ -51,7 +48,6 @@ use sc_network_common::{ use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; use sp_runtime::traits::Block as BlockT; -// `std` dependencies use std::{ collections::{hash_map::Entry, HashMap}, iter, diff --git a/client/service/src/config.rs b/client/service/src/config.rs index 312a022d1c998..99de079feb780 100644 --- a/client/service/src/config.rs +++ b/client/service/src/config.rs @@ -23,13 +23,13 @@ pub use sc_client_db::{BlocksPruning, Database, DatabaseSource, PruningMode}; pub use sc_executor::{WasmExecutionMethod, WasmtimeInstantiationStrategy}; pub use sc_network::{ config::{NetworkConfiguration, NonDefaultSetConfig, Role, SetConfig}, - Multiaddr, -}; -pub use sc_network_common::{ - config::{MultiaddrWithPeerId, NodeKeyConfig, ProtocolId, TransportConfig}, request_responses::{ IncomingRequest, OutgoingResponse, ProtocolConfig as RequestResponseConfig, }, + Multiaddr, +}; +pub use sc_network_common::config::{ + MultiaddrWithPeerId, NodeKeyConfig, ProtocolId, TransportConfig, }; use prometheus_endpoint::Registry; From df378333dfe363b8c6a93ac08171d61f703b42c0 Mon Sep 17 00:00:00 2001 From: Aaro Altonen Date: Mon, 13 Mar 2023 12:04:07 +0200 Subject: [PATCH 05/10] Remove more stuff --- Cargo.lock | 2 +- client/authority-discovery/src/worker.rs | 4 ++-- client/authority-discovery/src/worker/tests.rs | 2 +- client/network/Cargo.toml | 1 + client/network/common/Cargo.toml | 1 - client/network/common/src/config.rs | 1 - client/network/common/src/lib.rs | 2 -- client/network/common/src/service.rs | 1 - client/network/src/discovery.rs | 4 +++- client/network/src/lib.rs | 2 ++ client/network/src/peer_info.rs | 4 +++- client/network/src/service.rs | 15 +++++++++------ .../network/{common => }/src/service/signature.rs | 0 client/network/src/service/traits.rs | 6 ++++-- client/network/{common => }/src/utils.rs | 1 + client/network/sync/src/engine.rs | 6 ++++-- client/network/transactions/src/lib.rs | 5 +++-- 17 files changed, 34 insertions(+), 23 deletions(-) delete mode 100644 client/network/common/src/service.rs rename client/network/{common => }/src/service/signature.rs (100%) rename client/network/{common => }/src/utils.rs (99%) diff --git a/Cargo.lock b/Cargo.lock index da12cf7918073..5d3f2852b2a9f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8789,6 +8789,7 @@ dependencies = [ "futures-timer", "ip_network", "libp2p", + "linked_hash_set", "log", "lru", "mockall", @@ -8864,7 +8865,6 @@ dependencies = [ "futures", "futures-timer", "libp2p", - "linked_hash_set", "parity-scale-codec", "prost-build", "sc-consensus", diff --git a/client/authority-discovery/src/worker.rs b/client/authority-discovery/src/worker.rs index 7e5807f88d775..1213348047553 100644 --- a/client/authority-discovery/src/worker.rs +++ b/client/authority-discovery/src/worker.rs @@ -43,8 +43,8 @@ use log::{debug, error, log_enabled}; use prometheus_endpoint::{register, Counter, CounterVec, Gauge, Opts, U64}; use prost::Message; use rand::{seq::SliceRandom, thread_rng}; -use sc_network::{KademliaKey, NetworkDHTProvider, NetworkSigner, NetworkStateInfo}; -use sc_network_common::{protocol::event::DhtEvent, service::signature::Signature}; +use sc_network::{KademliaKey, NetworkDHTProvider, NetworkSigner, NetworkStateInfo, Signature}; +use sc_network_common::protocol::event::DhtEvent; use sp_api::{ApiError, ProvideRuntimeApi}; use sp_authority_discovery::{ AuthorityDiscoveryApi, AuthorityId, AuthorityPair, AuthoritySignature, diff --git a/client/authority-discovery/src/worker/tests.rs b/client/authority-discovery/src/worker/tests.rs index 4b4b814d91e97..febe40657e06a 100644 --- a/client/authority-discovery/src/worker/tests.rs +++ b/client/authority-discovery/src/worker/tests.rs @@ -38,7 +38,7 @@ use libp2p::{ use prometheus_endpoint::prometheus::default_registry; use sc_client_api::HeaderBackend; -use sc_network_common::service::signature::Signature; +use sc_network::Signature; use sp_api::{ApiRef, ProvideRuntimeApi}; use sp_keystore::{testing::KeyStore, CryptoStore}; use sp_runtime::traits::{Block as BlockT, NumberFor, Zero}; diff --git a/client/network/Cargo.toml b/client/network/Cargo.toml index 5a918bebd626e..90b5ce871ef1a 100644 --- a/client/network/Cargo.toml +++ b/client/network/Cargo.toml @@ -26,6 +26,7 @@ futures = "0.3.21" futures-timer = "3.0.2" ip_network = "0.4.1" libp2p = { version = "0.50.0", features = ["dns", "identify", "kad", "macros", "mdns", "mplex", "noise", "ping", "tcp", "tokio", "yamux", "websocket"] } +linked_hash_set = "0.1.3" log = "0.4.17" lru = "0.8.1" mockall = "0.11.3" diff --git a/client/network/common/Cargo.toml b/client/network/common/Cargo.toml index c5f9d6d4cb027..983342b014b82 100644 --- a/client/network/common/Cargo.toml +++ b/client/network/common/Cargo.toml @@ -26,7 +26,6 @@ codec = { package = "parity-scale-codec", version = "3.2.2", features = [ futures = "0.3.21" futures-timer = "3.0.2" libp2p = { version = "0.50.0", features = ["request-response", "kad"] } -linked_hash_set = "0.1.3" prometheus-endpoint = { package = "substrate-prometheus-endpoint", version = "0.10.0-dev", path = "../../../utils/prometheus" } smallvec = "1.8.0" sc-consensus = { version = "0.10.0-dev", path = "../../consensus/common" } diff --git a/client/network/common/src/config.rs b/client/network/common/src/config.rs index 723aeff950c24..7e2259491538e 100644 --- a/client/network/common/src/config.rs +++ b/client/network/common/src/config.rs @@ -20,7 +20,6 @@ pub use crate::{ protocol::{self, role::Role}, - sync::warp::WarpSyncProvider, ExHashT, }; pub use libp2p::{build_multiaddr, core::PublicKey, identity}; diff --git a/client/network/common/src/lib.rs b/client/network/common/src/lib.rs index 025a7415c5591..acd7c1b6ca141 100644 --- a/client/network/common/src/lib.rs +++ b/client/network/common/src/lib.rs @@ -21,9 +21,7 @@ pub mod config; pub mod message; pub mod protocol; -pub mod service; pub mod sync; -pub mod utils; /// Minimum Requirements for a Hash within Networking pub trait ExHashT: std::hash::Hash + Eq + std::fmt::Debug + Clone + Send + Sync + 'static {} diff --git a/client/network/common/src/service.rs b/client/network/common/src/service.rs deleted file mode 100644 index 4917bfba75fab..0000000000000 --- a/client/network/common/src/service.rs +++ /dev/null @@ -1 +0,0 @@ -pub mod signature; diff --git a/client/network/src/discovery.rs b/client/network/src/discovery.rs index 6a8c777cac9c6..9ae4f913ac77d 100644 --- a/client/network/src/discovery.rs +++ b/client/network/src/discovery.rs @@ -46,6 +46,8 @@ //! active mechanism that asks nodes for the addresses they are listening on. Whenever we learn //! of a node's address, you must call `add_self_reported_address`. +use crate::utils::LruHashSet; + use array_bytes::bytes2hex; use futures::prelude::*; use futures_timer::Delay; @@ -73,7 +75,7 @@ use libp2p::{ }, }; use log::{debug, info, trace, warn}; -use sc_network_common::{config::ProtocolId, utils::LruHashSet}; +use sc_network_common::config::ProtocolId; use sp_core::hexdisplay::HexDisplay; use std::{ cmp, diff --git a/client/network/src/lib.rs b/client/network/src/lib.rs index a74bb926c7a1f..388e07a35909d 100644 --- a/client/network/src/lib.rs +++ b/client/network/src/lib.rs @@ -255,6 +255,7 @@ pub mod config; pub mod error; pub mod network_state; pub mod request_responses; +pub mod utils; #[doc(inline)] pub use libp2p::{multiaddr, Multiaddr, PeerId}; @@ -271,6 +272,7 @@ pub use sc_network_common::{ }, }; pub use service::{ + signature::Signature, traits::{ KademliaKey, NetworkBlock, NetworkDHTProvider, NetworkEventStream, NetworkNotification, NetworkPeers, NetworkRequest, NetworkSigner, NetworkStateInfo, NetworkStatus, diff --git a/client/network/src/peer_info.rs b/client/network/src/peer_info.rs index 1cf0c928a4067..3f769736ff10e 100644 --- a/client/network/src/peer_info.rs +++ b/client/network/src/peer_info.rs @@ -16,6 +16,8 @@ // You should have received a copy of the GNU General Public License // along with this program. If not, see . +use crate::utils::interval; + use fnv::FnvHashMap; use futures::prelude::*; use libp2p::{ @@ -36,8 +38,8 @@ use libp2p::{ Multiaddr, }; use log::{debug, error, trace}; -use sc_network_common::utils::interval; use smallvec::SmallVec; + use std::{ collections::hash_map::Entry, pin::Pin, diff --git a/client/network/src/service.rs b/client/network/src/service.rs index cf2ba82274ddb..94e374f2b8f57 100644 --- a/client/network/src/service.rs +++ b/client/network/src/service.rs @@ -37,11 +37,14 @@ use crate::{ }, protocol::{self, NotificationsSink, NotifsHandlerError, Protocol, Ready}, request_responses::{IfDisconnected, RequestFailure}, - service::traits::{ - NetworkDHTProvider, NetworkEventStream, NetworkNotification, NetworkPeers, NetworkRequest, - NetworkSigner, NetworkStateInfo, NetworkStatus, NetworkStatusProvider, - NotificationSender as NotificationSenderT, NotificationSenderError, - NotificationSenderReady as NotificationSenderReadyT, + service::{ + signature::{Signature, SigningError}, + traits::{ + NetworkDHTProvider, NetworkEventStream, NetworkNotification, NetworkPeers, + NetworkRequest, NetworkSigner, NetworkStateInfo, NetworkStatus, NetworkStatusProvider, + NotificationSender as NotificationSenderT, NotificationSenderError, + NotificationSenderReady as NotificationSenderReadyT, + }, }, transport, ReputationChange, }; @@ -70,7 +73,6 @@ use sc_network_common::{ event::{DhtEvent, Event}, ProtocolName, }, - service::signature::{Signature, SigningError}, ExHashT, }; use sc_peerset::PeersetHandle; @@ -98,6 +100,7 @@ pub use libp2p::identity::{error::DecodingError, Keypair, PublicKey}; mod metrics; mod out_events; +pub mod signature; pub mod traits; /// Custom error that can be produced by the [`ConnectionHandler`] of the [`NetworkBehaviour`]. diff --git a/client/network/common/src/service/signature.rs b/client/network/src/service/signature.rs similarity index 100% rename from client/network/common/src/service/signature.rs rename to client/network/src/service/signature.rs diff --git a/client/network/src/service/traits.rs b/client/network/src/service/traits.rs index a3bf536e4bbdc..b4eabec5d2cd2 100644 --- a/client/network/src/service/traits.rs +++ b/client/network/src/service/traits.rs @@ -18,7 +18,10 @@ // // If you read this, you are very thorough, congratulations. -use crate::request_responses::{IfDisconnected, RequestFailure}; +use crate::{ + request_responses::{IfDisconnected, RequestFailure}, + service::signature::Signature, +}; use futures::{channel::oneshot, Stream}; use libp2p::{Multiaddr, PeerId}; @@ -26,7 +29,6 @@ use libp2p::{Multiaddr, PeerId}; use sc_network_common::{ config::MultiaddrWithPeerId, protocol::{event::Event, ProtocolName}, - service::signature::Signature, }; use sc_peerset::ReputationChange; diff --git a/client/network/common/src/utils.rs b/client/network/src/utils.rs similarity index 99% rename from client/network/common/src/utils.rs rename to client/network/src/utils.rs index c32d264a943e7..a89ec8ed36943 100644 --- a/client/network/common/src/utils.rs +++ b/client/network/src/utils.rs @@ -19,6 +19,7 @@ use futures::{stream::unfold, FutureExt, Stream, StreamExt}; use futures_timer::Delay; use linked_hash_set::LinkedHashSet; + use std::{hash::Hash, num::NonZeroUsize, time::Duration}; /// Creates a stream that returns a new value every `duration`. diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index a6619aca96a7e..c25b6732b8c7b 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -35,7 +35,10 @@ use prometheus_endpoint::{ use sc_client_api::{BlockBackend, HeaderBackend, ProofProvider}; use sc_consensus::import_queue::ImportQueueService; -use sc_network::config::{NetworkConfiguration, NonDefaultSetConfig}; +use sc_network::{ + config::{NetworkConfiguration, NonDefaultSetConfig}, + utils::LruHashSet, +}; use sc_network_common::{ config::{ProtocolId, SyncMode as SyncOperationMode}, protocol::{event::Event, role::Roles, ProtocolName}, @@ -48,7 +51,6 @@ use sc_network_common::{ BadPeer, ChainSync as ChainSyncT, ExtendedPeerInfo, PollBlockAnnounceValidation, SyncEvent, SyncMode, }, - utils::LruHashSet, }; use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; use sp_blockchain::HeaderMetadata; diff --git a/client/network/transactions/src/lib.rs b/client/network/transactions/src/lib.rs index 4ccee2cce239c..97c4737a7e146 100644 --- a/client/network/transactions/src/lib.rs +++ b/client/network/transactions/src/lib.rs @@ -36,13 +36,14 @@ use log::{debug, trace, warn}; use prometheus_endpoint::{register, Counter, PrometheusError, Registry, U64}; use sc_network::{ config::{NonDefaultSetConfig, SetConfig}, - error, NetworkEventStream, NetworkNotification, NetworkPeers, + error, + utils::{interval, LruHashSet}, + NetworkEventStream, NetworkNotification, NetworkPeers, }; use sc_network_common::{ config::{NonReservedPeerMode, ProtocolId}, protocol::{event::Event, role::ObservedRole, ProtocolName}, sync::{SyncEvent, SyncEventStream}, - utils::{interval, LruHashSet}, ExHashT, }; use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; From f9f9fd0c9d7228f103af6a8ab43330edcc017dbd Mon Sep 17 00:00:00 2001 From: Aaro Altonen Date: Mon, 13 Mar 2023 13:46:21 +0200 Subject: [PATCH 06/10] Remove rest of configs from `sc-network-common` to `sc-network` --- Cargo.lock | 2 +- client/chain-spec/Cargo.toml | 2 +- client/chain-spec/src/chain_spec.rs | 2 +- client/chain-spec/src/lib.rs | 2 +- client/cli/src/arg_enums.rs | 20 +- client/cli/src/params/network_params.rs | 5 +- client/cli/src/params/node_key_params.rs | 14 +- .../grandpa/src/communication/tests.rs | 4 +- client/consensus/grandpa/src/lib.rs | 2 +- client/network-gossip/src/bridge.rs | 8 +- client/network-gossip/src/state_machine.rs | 6 +- client/network/common/src/config.rs | 469 ------------------ client/network/common/src/lib.rs | 1 - .../light/src/light_client_requests.rs | 3 +- .../src/light_client_requests/handler.rs | 6 +- client/network/src/config.rs | 455 ++++++++++++++++- client/network/src/discovery.rs | 5 +- client/network/src/error.rs | 4 +- client/network/src/protocol.rs | 6 +- client/network/src/service.rs | 3 +- client/network/src/service/traits.rs | 6 +- .../network/sync/src/block_request_handler.rs | 7 +- client/network/sync/src/engine.rs | 5 +- client/network/sync/src/lib.rs | 5 +- client/network/sync/src/service/mock.rs | 3 +- .../network/sync/src/state_request_handler.rs | 6 +- .../network/sync/src/warp_request_handler.rs | 10 +- client/network/test/src/lib.rs | 8 +- client/network/test/src/service.rs | 10 +- client/network/transactions/src/lib.rs | 3 +- client/offchain/src/api.rs | 4 +- client/offchain/src/lib.rs | 3 +- client/rpc/src/system/tests.rs | 2 +- client/service/src/builder.rs | 6 +- client/service/src/config.rs | 9 +- client/service/src/lib.rs | 5 +- client/service/test/src/lib.rs | 4 +- 37 files changed, 545 insertions(+), 570 deletions(-) delete mode 100644 client/network/common/src/config.rs diff --git a/Cargo.lock b/Cargo.lock index 5d3f2852b2a9f..4aac7f3c7ed99 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8164,7 +8164,7 @@ dependencies = [ "sc-chain-spec-derive", "sc-client-api", "sc-executor", - "sc-network-common", + "sc-network", "sc-telemetry", "serde", "serde_json", diff --git a/client/chain-spec/Cargo.toml b/client/chain-spec/Cargo.toml index 6168a897c962c..b1188b3bd4625 100644 --- a/client/chain-spec/Cargo.toml +++ b/client/chain-spec/Cargo.toml @@ -19,7 +19,7 @@ serde_json = "1.0.85" sc-client-api = { version = "4.0.0-dev", path = "../api" } sc-chain-spec-derive = { version = "4.0.0-dev", path = "./derive" } sc-executor = { version = "0.10.0-dev", path = "../executor" } -sc-network-common = { version = "0.10.0-dev", path = "../network/common" } +sc-network = { version = "0.10.0-dev", path = "../network" } sc-telemetry = { version = "4.0.0-dev", path = "../telemetry" } sp-blockchain = { version = "4.0.0-dev", path = "../../primitives/blockchain" } sp-core = { version = "7.0.0", path = "../../primitives/core" } diff --git a/client/chain-spec/src/chain_spec.rs b/client/chain-spec/src/chain_spec.rs index 9eed5bbbc4e69..96e36d8399ed5 100644 --- a/client/chain-spec/src/chain_spec.rs +++ b/client/chain-spec/src/chain_spec.rs @@ -20,7 +20,7 @@ #![warn(missing_docs)] use crate::{extension::GetExtension, ChainType, Properties, RuntimeGenesis}; -use sc_network_common::config::MultiaddrWithPeerId; +use sc_network::config::MultiaddrWithPeerId; use sc_telemetry::TelemetryEndpoints; use serde::{Deserialize, Serialize}; use serde_json as json; diff --git a/client/chain-spec/src/lib.rs b/client/chain-spec/src/lib.rs index e43a247961f02..6239eb7326b78 100644 --- a/client/chain-spec/src/lib.rs +++ b/client/chain-spec/src/lib.rs @@ -189,7 +189,7 @@ pub use self::{ }; pub use sc_chain_spec_derive::{ChainSpecExtension, ChainSpecGroup}; -use sc_network_common::config::MultiaddrWithPeerId; +use sc_network::config::MultiaddrWithPeerId; use sc_telemetry::TelemetryEndpoints; use serde::{de::DeserializeOwned, Serialize}; use sp_core::storage::Storage; diff --git a/client/cli/src/arg_enums.rs b/client/cli/src/arg_enums.rs index 472c1722f9ed8..c3399a89680d1 100644 --- a/client/cli/src/arg_enums.rs +++ b/client/cli/src/arg_enums.rs @@ -251,19 +251,15 @@ pub enum SyncMode { Warp, } -impl Into for SyncMode { - fn into(self) -> sc_network_common::config::SyncMode { +impl Into for SyncMode { + fn into(self) -> sc_network::config::SyncMode { match self { - SyncMode::Full => sc_network_common::config::SyncMode::Full, - SyncMode::Fast => sc_network_common::config::SyncMode::Fast { - skip_proofs: false, - storage_chain_mode: false, - }, - SyncMode::FastUnsafe => sc_network_common::config::SyncMode::Fast { - skip_proofs: true, - storage_chain_mode: false, - }, - SyncMode::Warp => sc_network_common::config::SyncMode::Warp, + SyncMode::Full => sc_network::config::SyncMode::Full, + SyncMode::Fast => + sc_network::config::SyncMode::Fast { skip_proofs: false, storage_chain_mode: false }, + SyncMode::FastUnsafe => + sc_network::config::SyncMode::Fast { skip_proofs: true, storage_chain_mode: false }, + SyncMode::Warp => sc_network::config::SyncMode::Warp, } } } diff --git a/client/cli/src/params/network_params.rs b/client/cli/src/params/network_params.rs index 78bdfc58cf64e..106fba75aa727 100644 --- a/client/cli/src/params/network_params.rs +++ b/client/cli/src/params/network_params.rs @@ -19,10 +19,11 @@ use crate::{arg_enums::SyncMode, params::node_key_params::NodeKeyParams}; use clap::Args; use sc_network::{ - config::{NetworkConfiguration, SetConfig}, + config::{ + NetworkConfiguration, NodeKeyConfig, NonReservedPeerMode, SetConfig, TransportConfig, + }, multiaddr::Protocol, }; -use sc_network_common::config::{NodeKeyConfig, NonReservedPeerMode, TransportConfig}; use sc_service::{ config::{Multiaddr, MultiaddrWithPeerId}, ChainSpec, ChainType, diff --git a/client/cli/src/params/node_key_params.rs b/client/cli/src/params/node_key_params.rs index d470ef1fad12a..074b95bea0f3a 100644 --- a/client/cli/src/params/node_key_params.rs +++ b/client/cli/src/params/node_key_params.rs @@ -17,7 +17,7 @@ // along with this program. If not, see . use clap::Args; -use sc_network_common::config::{identity::ed25519, NodeKeyConfig}; +use sc_network::config::{identity::ed25519, NodeKeyConfig}; use sp_core::H256; use std::{path::PathBuf, str::FromStr}; @@ -92,7 +92,7 @@ impl NodeKeyParams { let secret = if let Some(node_key) = self.node_key.as_ref() { parse_ed25519_secret(node_key)? } else { - sc_network_common::config::Secret::File( + sc_network::config::Secret::File( self.node_key_file .clone() .unwrap_or_else(|| net_config_dir.join(NODE_KEY_ED25519_FILE)), @@ -111,10 +111,10 @@ fn invalid_node_key(e: impl std::fmt::Display) -> error::Error { } /// Parse a Ed25519 secret key from a hex string into a `sc_network::Secret`. -fn parse_ed25519_secret(hex: &str) -> error::Result { +fn parse_ed25519_secret(hex: &str) -> error::Result { H256::from_str(hex).map_err(invalid_node_key).and_then(|bytes| { ed25519::SecretKey::from_bytes(bytes) - .map(sc_network_common::config::Secret::Input) + .map(sc_network::config::Secret::Input) .map_err(invalid_node_key) }) } @@ -123,7 +123,7 @@ fn parse_ed25519_secret(hex: &str) -> error::Result Ok(()), _ => Err(error::Error::Input("Unexpected node key config".into())), @@ -200,7 +200,7 @@ mod tests { let dir = PathBuf::from(net_config_dir.clone()); let typ = params.node_key_type; params.node_key(net_config_dir).and_then(move |c| match c { - NodeKeyConfig::Ed25519(sc_network_common::config::Secret::File(ref f)) + NodeKeyConfig::Ed25519(sc_network::config::Secret::File(ref f)) if typ == NodeKeyType::Ed25519 && f == &dir.join(NODE_KEY_ED25519_FILE) => Ok(()), _ => Err(error::Error::Input("Unexpected node key config".into())), diff --git a/client/consensus/grandpa/src/communication/tests.rs b/client/consensus/grandpa/src/communication/tests.rs index 711c14f350506..5c0d8036d5a81 100644 --- a/client/consensus/grandpa/src/communication/tests.rs +++ b/client/consensus/grandpa/src/communication/tests.rs @@ -26,12 +26,12 @@ use crate::{communication::grandpa_protocol_name, environment::SharedVoterSetSta use futures::prelude::*; use parity_scale_codec::Encode; use sc_network::{ - config::Role, Multiaddr, NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers, + config::{MultiaddrWithPeerId, Role}, + Multiaddr, NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers, NetworkSyncForkRequest, NotificationSenderError, NotificationSenderT as NotificationSender, PeerId, ReputationChange, }; use sc_network_common::{ - config::MultiaddrWithPeerId, protocol::{event::Event as NetworkEvent, role::ObservedRole, ProtocolName}, sync::{SyncEvent as SyncStreamEvent, SyncEventStream}, }; diff --git a/client/consensus/grandpa/src/lib.rs b/client/consensus/grandpa/src/lib.rs index 690835e3a6639..66bb209615635 100644 --- a/client/consensus/grandpa/src/lib.rs +++ b/client/consensus/grandpa/src/lib.rs @@ -707,7 +707,7 @@ pub fn grandpa_peers_set_config( in_peers: 0, out_peers: 0, reserved_nodes: Vec::new(), - non_reserved_mode: sc_network_common::config::NonReservedPeerMode::Deny, + non_reserved_mode: sc_network::config::NonReservedPeerMode::Deny, }, } } diff --git a/client/network-gossip/src/bridge.rs b/client/network-gossip/src/bridge.rs index b2ccf22ed57f4..995ebd4e5bfa3 100644 --- a/client/network-gossip/src/bridge.rs +++ b/client/network-gossip/src/bridge.rs @@ -341,12 +341,10 @@ mod tests { }; use quickcheck::{Arbitrary, Gen, QuickCheck}; use sc_network::{ - NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers, - NotificationSenderError, NotificationSenderT as NotificationSender, - }; - use sc_network_common::{ - config::MultiaddrWithPeerId, protocol::role::ObservedRole, sync::SyncEventStream, + config::MultiaddrWithPeerId, NetworkBlock, NetworkEventStream, NetworkNotification, + NetworkPeers, NotificationSenderError, NotificationSenderT as NotificationSender, }; + use sc_network_common::{protocol::role::ObservedRole, sync::SyncEventStream}; use sp_runtime::{ testing::H256, traits::{Block as BlockT, NumberFor}, diff --git a/client/network-gossip/src/state_machine.rs b/client/network-gossip/src/state_machine.rs index 2d2cfab021028..922702242d6b0 100644 --- a/client/network-gossip/src/state_machine.rs +++ b/client/network-gossip/src/state_machine.rs @@ -526,10 +526,10 @@ mod tests { use crate::multiaddr::Multiaddr; use futures::prelude::*; use sc_network::{ - NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers, - NotificationSenderError, NotificationSenderT as NotificationSender, + config::MultiaddrWithPeerId, NetworkBlock, NetworkEventStream, NetworkNotification, + NetworkPeers, NotificationSenderError, NotificationSenderT as NotificationSender, }; - use sc_network_common::{config::MultiaddrWithPeerId, protocol::event::Event}; + use sc_network_common::protocol::event::Event; use sc_peerset::ReputationChange; use sp_runtime::{ testing::{Block as RawBlock, ExtrinsicWrapper, H256}, diff --git a/client/network/common/src/config.rs b/client/network/common/src/config.rs deleted file mode 100644 index 7e2259491538e..0000000000000 --- a/client/network/common/src/config.rs +++ /dev/null @@ -1,469 +0,0 @@ -// This file is part of Substrate. - -// Copyright (C) Parity Technologies (UK) Ltd. -// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0 - -// This program is free software: you can redistribute it and/or modify -// it under the terms of the GNU General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. - -// This program is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU General Public License for more details. - -// You should have received a copy of the GNU General Public License -// along with this program. If not, see . - -//! Configuration of the networking layer. - -pub use crate::{ - protocol::{self, role::Role}, - ExHashT, -}; -pub use libp2p::{build_multiaddr, core::PublicKey, identity}; - -use codec::Encode; -use libp2p::{ - identity::{ed25519, Keypair}, - multiaddr, Multiaddr, PeerId, -}; -use zeroize::Zeroize; - -use std::{ - error::Error, - fmt, fs, - io::{self, Write}, - path::{Path, PathBuf}, - str, - str::FromStr, -}; - -/// Protocol name prefix, transmitted on the wire for legacy protocol names. -/// I.e., `dot` in `/dot/sync/2`. Should be unique for each chain. Always UTF-8. -/// Deprecated in favour of genesis hash & fork ID based protocol names. -#[derive(Clone, PartialEq, Eq, Hash)] -pub struct ProtocolId(smallvec::SmallVec<[u8; 6]>); - -impl<'a> From<&'a str> for ProtocolId { - fn from(bytes: &'a str) -> ProtocolId { - Self(bytes.as_bytes().into()) - } -} - -impl AsRef for ProtocolId { - fn as_ref(&self) -> &str { - str::from_utf8(&self.0[..]) - .expect("the only way to build a ProtocolId is through a UTF-8 String; qed") - } -} - -impl fmt::Debug for ProtocolId { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - fmt::Debug::fmt(self.as_ref(), f) - } -} - -/// Parses a string address and splits it into Multiaddress and PeerId, if -/// valid. -/// -/// # Example -/// -/// ``` -/// # use libp2p::{Multiaddr, PeerId}; -/// # use sc_network_common::config::parse_str_addr; -/// let (peer_id, addr) = parse_str_addr( -/// "/ip4/198.51.100.19/tcp/30333/p2p/QmSk5HQbn6LhUwDiNMseVUjuRYhEtYj4aUZ6WfWoGURpdV" -/// ).unwrap(); -/// assert_eq!(peer_id, "QmSk5HQbn6LhUwDiNMseVUjuRYhEtYj4aUZ6WfWoGURpdV".parse::().unwrap()); -/// assert_eq!(addr, "/ip4/198.51.100.19/tcp/30333".parse::().unwrap()); -/// ``` -pub fn parse_str_addr(addr_str: &str) -> Result<(PeerId, Multiaddr), ParseErr> { - let addr: Multiaddr = addr_str.parse()?; - parse_addr(addr) -} - -/// Splits a Multiaddress into a Multiaddress and PeerId. -pub fn parse_addr(mut addr: Multiaddr) -> Result<(PeerId, Multiaddr), ParseErr> { - let who = match addr.pop() { - Some(multiaddr::Protocol::P2p(key)) => - PeerId::from_multihash(key).map_err(|_| ParseErr::InvalidPeerId)?, - _ => return Err(ParseErr::PeerIdMissing), - }; - - Ok((who, addr)) -} - -/// Address of a node, including its identity. -/// -/// This struct represents a decoded version of a multiaddress that ends with `/p2p/`. -/// -/// # Example -/// -/// ``` -/// # use libp2p::{Multiaddr, PeerId}; -/// # use sc_network_common::config::MultiaddrWithPeerId; -/// let addr: MultiaddrWithPeerId = -/// "/ip4/198.51.100.19/tcp/30333/p2p/QmSk5HQbn6LhUwDiNMseVUjuRYhEtYj4aUZ6WfWoGURpdV".parse().unwrap(); -/// assert_eq!(addr.peer_id.to_base58(), "QmSk5HQbn6LhUwDiNMseVUjuRYhEtYj4aUZ6WfWoGURpdV"); -/// assert_eq!(addr.multiaddr.to_string(), "/ip4/198.51.100.19/tcp/30333"); -/// ``` -#[derive(Debug, Clone, serde::Serialize, serde::Deserialize, PartialEq)] -#[serde(try_from = "String", into = "String")] -pub struct MultiaddrWithPeerId { - /// Address of the node. - pub multiaddr: Multiaddr, - /// Its identity. - pub peer_id: PeerId, -} - -impl MultiaddrWithPeerId { - /// Concatenates the multiaddress and peer ID into one multiaddress containing both. - pub fn concat(&self) -> Multiaddr { - let proto = multiaddr::Protocol::P2p(From::from(self.peer_id)); - self.multiaddr.clone().with(proto) - } -} - -impl fmt::Display for MultiaddrWithPeerId { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - fmt::Display::fmt(&self.concat(), f) - } -} - -impl FromStr for MultiaddrWithPeerId { - type Err = ParseErr; - - fn from_str(s: &str) -> Result { - let (peer_id, multiaddr) = parse_str_addr(s)?; - Ok(Self { peer_id, multiaddr }) - } -} - -impl From for String { - fn from(ma: MultiaddrWithPeerId) -> String { - format!("{}", ma) - } -} - -impl TryFrom for MultiaddrWithPeerId { - type Error = ParseErr; - fn try_from(string: String) -> Result { - string.parse() - } -} - -/// Error that can be generated by `parse_str_addr`. -#[derive(Debug)] -pub enum ParseErr { - /// Error while parsing the multiaddress. - MultiaddrParse(multiaddr::Error), - /// Multihash of the peer ID is invalid. - InvalidPeerId, - /// The peer ID is missing from the address. - PeerIdMissing, -} - -impl fmt::Display for ParseErr { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match self { - Self::MultiaddrParse(err) => write!(f, "{}", err), - Self::InvalidPeerId => write!(f, "Peer id at the end of the address is invalid"), - Self::PeerIdMissing => write!(f, "Peer id is missing from the address"), - } - } -} - -impl std::error::Error for ParseErr { - fn source(&self) -> Option<&(dyn std::error::Error + 'static)> { - match self { - Self::MultiaddrParse(err) => Some(err), - Self::InvalidPeerId => None, - Self::PeerIdMissing => None, - } - } -} - -impl From for ParseErr { - fn from(err: multiaddr::Error) -> ParseErr { - Self::MultiaddrParse(err) - } -} - -/// Custom handshake for the notification protocol -#[derive(Debug, Clone)] -pub struct NotificationHandshake(Vec); - -impl NotificationHandshake { - /// Create new `NotificationHandshake` from an object that implements `Encode` - pub fn new(handshake: H) -> Self { - Self(handshake.encode()) - } - - /// Create new `NotificationHandshake` from raw bytes - pub fn from_bytes(bytes: Vec) -> Self { - Self(bytes) - } -} - -impl std::ops::Deref for NotificationHandshake { - type Target = Vec; - - fn deref(&self) -> &Self::Target { - &self.0 - } -} - -/// Configuration for the transport layer. -#[derive(Clone, Debug)] -pub enum TransportConfig { - /// Normal transport mode. - Normal { - /// If true, the network will use mDNS to discover other libp2p nodes on the local network - /// and connect to them if they support the same chain. - enable_mdns: bool, - - /// If true, allow connecting to private IPv4/IPv6 addresses (as defined in - /// [RFC1918](https://tools.ietf.org/html/rfc1918)). Irrelevant for addresses that have - /// been passed in `::sc_network::config::NetworkConfiguration::boot_nodes`. - allow_private_ip: bool, - }, - - /// Only allow connections within the same process. - /// Only addresses of the form `/memory/...` will be supported. - MemoryOnly, -} - -/// The policy for connections to non-reserved peers. -#[derive(Clone, Debug, PartialEq, Eq)] -pub enum NonReservedPeerMode { - /// Accept them. This is the default. - Accept, - /// Deny them. - Deny, -} - -impl NonReservedPeerMode { - /// Attempt to parse the peer mode from a string. - pub fn parse(s: &str) -> Option { - match s { - "accept" => Some(Self::Accept), - "deny" => Some(Self::Deny), - _ => None, - } - } -} - -/// Sync operation mode. -#[derive(Copy, Clone, Debug, Eq, PartialEq)] -pub enum SyncMode { - /// Full block download and verification. - Full, - /// Download blocks and the latest state. - Fast { - /// Skip state proof download and verification. - skip_proofs: bool, - /// Download indexed transactions for recent blocks. - storage_chain_mode: bool, - }, - /// Warp sync - verify authority set transitions and the latest state. - Warp, -} - -impl SyncMode { - /// Returns if `self` is [`Self::Warp`]. - pub fn is_warp(&self) -> bool { - matches!(self, Self::Warp) - } - - /// Returns if `self` is [`Self::Fast`]. - pub fn is_fast(&self) -> bool { - matches!(self, Self::Fast { .. }) - } -} - -impl Default for SyncMode { - fn default() -> Self { - Self::Full - } -} - -/// The configuration of a node's secret key, describing the type of key -/// and how it is obtained. A node's identity keypair is the result of -/// the evaluation of the node key configuration. -#[derive(Clone, Debug)] -pub enum NodeKeyConfig { - /// A Ed25519 secret key configuration. - Ed25519(Secret), -} - -impl Default for NodeKeyConfig { - fn default() -> NodeKeyConfig { - Self::Ed25519(Secret::New) - } -} - -/// The options for obtaining a Ed25519 secret key. -pub type Ed25519Secret = Secret; - -/// The configuration options for obtaining a secret key `K`. -#[derive(Clone)] -pub enum Secret { - /// Use the given secret key `K`. - Input(K), - /// Read the secret key from a file. If the file does not exist, - /// it is created with a newly generated secret key `K`. The format - /// of the file is determined by `K`: - /// - /// * `ed25519::SecretKey`: An unencoded 32 bytes Ed25519 secret key. - File(PathBuf), - /// Always generate a new secret key `K`. - New, -} - -impl fmt::Debug for Secret { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - match self { - Self::Input(_) => f.debug_tuple("Secret::Input").finish(), - Self::File(path) => f.debug_tuple("Secret::File").field(path).finish(), - Self::New => f.debug_tuple("Secret::New").finish(), - } - } -} - -impl NodeKeyConfig { - /// Evaluate a `NodeKeyConfig` to obtain an identity `Keypair`: - /// - /// * If the secret is configured as input, the corresponding keypair is returned. - /// - /// * If the secret is configured as a file, it is read from that file, if it exists. Otherwise - /// a new secret is generated and stored. In either case, the keypair obtained from the - /// secret is returned. - /// - /// * If the secret is configured to be new, it is generated and the corresponding keypair is - /// returned. - pub fn into_keypair(self) -> io::Result { - use NodeKeyConfig::*; - match self { - Ed25519(Secret::New) => Ok(Keypair::generate_ed25519()), - - Ed25519(Secret::Input(k)) => Ok(Keypair::Ed25519(k.into())), - - Ed25519(Secret::File(f)) => get_secret( - f, - |mut b| match String::from_utf8(b.to_vec()).ok().and_then(|s| { - if s.len() == 64 { - array_bytes::hex2bytes(&s).ok() - } else { - None - } - }) { - Some(s) => ed25519::SecretKey::from_bytes(s), - _ => ed25519::SecretKey::from_bytes(&mut b), - }, - ed25519::SecretKey::generate, - |b| b.as_ref().to_vec(), - ) - .map(ed25519::Keypair::from) - .map(Keypair::Ed25519), - } - } -} - -/// Load a secret key from a file, if it exists, or generate a -/// new secret key and write it to that file. In either case, -/// the secret key is returned. -fn get_secret(file: P, parse: F, generate: G, serialize: W) -> io::Result -where - P: AsRef, - F: for<'r> FnOnce(&'r mut [u8]) -> Result, - G: FnOnce() -> K, - E: Error + Send + Sync + 'static, - W: Fn(&K) -> Vec, -{ - std::fs::read(&file) - .and_then(|mut sk_bytes| { - parse(&mut sk_bytes).map_err(|e| io::Error::new(io::ErrorKind::InvalidData, e)) - }) - .or_else(|e| { - if e.kind() == io::ErrorKind::NotFound { - file.as_ref().parent().map_or(Ok(()), fs::create_dir_all)?; - let sk = generate(); - let mut sk_vec = serialize(&sk); - write_secret_file(file, &sk_vec)?; - sk_vec.zeroize(); - Ok(sk) - } else { - Err(e) - } - }) -} - -/// Write secret bytes to a file. -fn write_secret_file

(path: P, sk_bytes: &[u8]) -> io::Result<()> -where - P: AsRef, -{ - let mut file = open_secret_file(&path)?; - file.write_all(sk_bytes) -} - -/// Opens a file containing a secret key in write mode. -#[cfg(unix)] -fn open_secret_file

(path: P) -> io::Result -where - P: AsRef, -{ - use std::os::unix::fs::OpenOptionsExt; - fs::OpenOptions::new().write(true).create_new(true).mode(0o600).open(path) -} - -/// Opens a file containing a secret key in write mode. -#[cfg(not(unix))] -fn open_secret_file

(path: P) -> Result -where - P: AsRef, -{ - fs::OpenOptions::new().write(true).create_new(true).open(path) -} - -#[cfg(test)] -mod tests { - use super::*; - use tempfile::TempDir; - - fn tempdir_with_prefix(prefix: &str) -> TempDir { - tempfile::Builder::new().prefix(prefix).tempdir().unwrap() - } - - fn secret_bytes(kp: &Keypair) -> Vec { - let Keypair::Ed25519(p) = kp; - p.secret().as_ref().iter().cloned().collect() - } - - #[test] - fn test_secret_file() { - let tmp = tempdir_with_prefix("x"); - std::fs::remove_dir(tmp.path()).unwrap(); // should be recreated - let file = tmp.path().join("x").to_path_buf(); - let kp1 = NodeKeyConfig::Ed25519(Secret::File(file.clone())).into_keypair().unwrap(); - let kp2 = NodeKeyConfig::Ed25519(Secret::File(file.clone())).into_keypair().unwrap(); - assert!(file.is_file() && secret_bytes(&kp1) == secret_bytes(&kp2)) - } - - #[test] - fn test_secret_input() { - let sk = ed25519::SecretKey::generate(); - let kp1 = NodeKeyConfig::Ed25519(Secret::Input(sk.clone())).into_keypair().unwrap(); - let kp2 = NodeKeyConfig::Ed25519(Secret::Input(sk)).into_keypair().unwrap(); - assert!(secret_bytes(&kp1) == secret_bytes(&kp2)); - } - - #[test] - fn test_secret_new() { - let kp1 = NodeKeyConfig::Ed25519(Secret::New).into_keypair().unwrap(); - let kp2 = NodeKeyConfig::Ed25519(Secret::New).into_keypair().unwrap(); - assert!(secret_bytes(&kp1) != secret_bytes(&kp2)); - } -} diff --git a/client/network/common/src/lib.rs b/client/network/common/src/lib.rs index acd7c1b6ca141..998c56d756f5b 100644 --- a/client/network/common/src/lib.rs +++ b/client/network/common/src/lib.rs @@ -18,7 +18,6 @@ //! Common data structures of the networking layer. -pub mod config; pub mod message; pub mod protocol; pub mod sync; diff --git a/client/network/light/src/light_client_requests.rs b/client/network/light/src/light_client_requests.rs index 2f6f766fd6439..4d2a301c00e6b 100644 --- a/client/network/light/src/light_client_requests.rs +++ b/client/network/light/src/light_client_requests.rs @@ -18,8 +18,7 @@ //! Helpers for outgoing and incoming light client requests. -use sc_network::request_responses::ProtocolConfig; -use sc_network_common::config::ProtocolId; +use sc_network::{config::ProtocolId, request_responses::ProtocolConfig}; use std::time::Duration; diff --git a/client/network/light/src/light_client_requests/handler.rs b/client/network/light/src/light_client_requests/handler.rs index 7c38f299c444e..db2630b79f498 100644 --- a/client/network/light/src/light_client_requests/handler.rs +++ b/client/network/light/src/light_client_requests/handler.rs @@ -29,8 +29,10 @@ use libp2p::PeerId; use log::{debug, trace}; use prost::Message; use sc_client_api::{BlockBackend, ProofProvider}; -use sc_network::request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}; -use sc_network_common::config::ProtocolId; +use sc_network::{ + config::ProtocolId, + request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}, +}; use sc_peerset::ReputationChange; use sp_core::{ hexdisplay::HexDisplay, diff --git a/client/network/src/config.rs b/client/network/src/config.rs index 60ec35f88df45..b0367550748ec 100644 --- a/client/network/src/config.rs +++ b/client/network/src/config.rs @@ -25,20 +25,421 @@ pub use crate::request_responses::{ IncomingRequest, OutgoingResponse, ProtocolConfig as RequestResponseConfig, }; -pub use libp2p::{build_multiaddr, core::PublicKey, identity, multiaddr, Multiaddr}; - +use codec::Encode; +use libp2p::{core::PublicKey, identity::Keypair, multiaddr, Multiaddr, PeerId}; use prometheus_endpoint::Registry; pub use sc_network_common::{ - config::{ - MultiaddrWithPeerId, NodeKeyConfig, NonReservedPeerMode, NotificationHandshake, ProtocolId, - SyncMode, TransportConfig, - }, protocol::{role::Role, ProtocolName}, sync::warp::WarpSyncProvider, ExHashT, }; +use zeroize::Zeroize; + +use std::{ + error::Error, + fmt, fs, + future::Future, + io::{self, Write}, + iter, + net::Ipv4Addr, + path::{Path, PathBuf}, + pin::Pin, + str::{self, FromStr}, + sync::Arc, +}; + +pub use libp2p::{ + build_multiaddr, + identity::{self, ed25519}, +}; + +/// Protocol name prefix, transmitted on the wire for legacy protocol names. +/// I.e., `dot` in `/dot/sync/2`. Should be unique for each chain. Always UTF-8. +/// Deprecated in favour of genesis hash & fork ID based protocol names. +#[derive(Clone, PartialEq, Eq, Hash)] +pub struct ProtocolId(smallvec::SmallVec<[u8; 6]>); + +impl<'a> From<&'a str> for ProtocolId { + fn from(bytes: &'a str) -> ProtocolId { + Self(bytes.as_bytes().into()) + } +} + +impl AsRef for ProtocolId { + fn as_ref(&self) -> &str { + str::from_utf8(&self.0[..]) + .expect("the only way to build a ProtocolId is through a UTF-8 String; qed") + } +} + +impl fmt::Debug for ProtocolId { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fmt::Debug::fmt(self.as_ref(), f) + } +} + +/// Parses a string address and splits it into Multiaddress and PeerId, if +/// valid. +/// +/// # Example +/// +/// ``` +/// # use libp2p::{Multiaddr, PeerId}; +/// use sc_network::config::parse_str_addr; +/// let (peer_id, addr) = parse_str_addr( +/// "/ip4/198.51.100.19/tcp/30333/p2p/QmSk5HQbn6LhUwDiNMseVUjuRYhEtYj4aUZ6WfWoGURpdV" +/// ).unwrap(); +/// assert_eq!(peer_id, "QmSk5HQbn6LhUwDiNMseVUjuRYhEtYj4aUZ6WfWoGURpdV".parse::().unwrap()); +/// assert_eq!(addr, "/ip4/198.51.100.19/tcp/30333".parse::().unwrap()); +/// ``` +pub fn parse_str_addr(addr_str: &str) -> Result<(PeerId, Multiaddr), ParseErr> { + let addr: Multiaddr = addr_str.parse()?; + parse_addr(addr) +} + +/// Splits a Multiaddress into a Multiaddress and PeerId. +pub fn parse_addr(mut addr: Multiaddr) -> Result<(PeerId, Multiaddr), ParseErr> { + let who = match addr.pop() { + Some(multiaddr::Protocol::P2p(key)) => + PeerId::from_multihash(key).map_err(|_| ParseErr::InvalidPeerId)?, + _ => return Err(ParseErr::PeerIdMissing), + }; + + Ok((who, addr)) +} + +/// Address of a node, including its identity. +/// +/// This struct represents a decoded version of a multiaddress that ends with `/p2p/`. +/// +/// # Example +/// +/// ``` +/// # use libp2p::{Multiaddr, PeerId}; +/// use sc_network::config::MultiaddrWithPeerId; +/// let addr: MultiaddrWithPeerId = +/// "/ip4/198.51.100.19/tcp/30333/p2p/QmSk5HQbn6LhUwDiNMseVUjuRYhEtYj4aUZ6WfWoGURpdV".parse().unwrap(); +/// assert_eq!(addr.peer_id.to_base58(), "QmSk5HQbn6LhUwDiNMseVUjuRYhEtYj4aUZ6WfWoGURpdV"); +/// assert_eq!(addr.multiaddr.to_string(), "/ip4/198.51.100.19/tcp/30333"); +/// ``` +#[derive(Debug, Clone, serde::Serialize, serde::Deserialize, PartialEq)] +#[serde(try_from = "String", into = "String")] +pub struct MultiaddrWithPeerId { + /// Address of the node. + pub multiaddr: Multiaddr, + /// Its identity. + pub peer_id: PeerId, +} + +impl MultiaddrWithPeerId { + /// Concatenates the multiaddress and peer ID into one multiaddress containing both. + pub fn concat(&self) -> Multiaddr { + let proto = multiaddr::Protocol::P2p(From::from(self.peer_id)); + self.multiaddr.clone().with(proto) + } +} + +impl fmt::Display for MultiaddrWithPeerId { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fmt::Display::fmt(&self.concat(), f) + } +} + +impl FromStr for MultiaddrWithPeerId { + type Err = ParseErr; + + fn from_str(s: &str) -> Result { + let (peer_id, multiaddr) = parse_str_addr(s)?; + Ok(Self { peer_id, multiaddr }) + } +} + +impl From for String { + fn from(ma: MultiaddrWithPeerId) -> String { + format!("{}", ma) + } +} + +impl TryFrom for MultiaddrWithPeerId { + type Error = ParseErr; + fn try_from(string: String) -> Result { + string.parse() + } +} + +/// Error that can be generated by `parse_str_addr`. +#[derive(Debug)] +pub enum ParseErr { + /// Error while parsing the multiaddress. + MultiaddrParse(multiaddr::Error), + /// Multihash of the peer ID is invalid. + InvalidPeerId, + /// The peer ID is missing from the address. + PeerIdMissing, +} + +impl fmt::Display for ParseErr { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Self::MultiaddrParse(err) => write!(f, "{}", err), + Self::InvalidPeerId => write!(f, "Peer id at the end of the address is invalid"), + Self::PeerIdMissing => write!(f, "Peer id is missing from the address"), + } + } +} + +impl std::error::Error for ParseErr { + fn source(&self) -> Option<&(dyn std::error::Error + 'static)> { + match self { + Self::MultiaddrParse(err) => Some(err), + Self::InvalidPeerId => None, + Self::PeerIdMissing => None, + } + } +} + +impl From for ParseErr { + fn from(err: multiaddr::Error) -> ParseErr { + Self::MultiaddrParse(err) + } +} + +/// Custom handshake for the notification protocol +#[derive(Debug, Clone)] +pub struct NotificationHandshake(Vec); + +impl NotificationHandshake { + /// Create new `NotificationHandshake` from an object that implements `Encode` + pub fn new(handshake: H) -> Self { + Self(handshake.encode()) + } + + /// Create new `NotificationHandshake` from raw bytes + pub fn from_bytes(bytes: Vec) -> Self { + Self(bytes) + } +} + +impl std::ops::Deref for NotificationHandshake { + type Target = Vec; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +/// Configuration for the transport layer. +#[derive(Clone, Debug)] +pub enum TransportConfig { + /// Normal transport mode. + Normal { + /// If true, the network will use mDNS to discover other libp2p nodes on the local network + /// and connect to them if they support the same chain. + enable_mdns: bool, + + /// If true, allow connecting to private IPv4/IPv6 addresses (as defined in + /// [RFC1918](https://tools.ietf.org/html/rfc1918)). Irrelevant for addresses that have + /// been passed in `::sc_network::config::NetworkConfiguration::boot_nodes`. + allow_private_ip: bool, + }, + + /// Only allow connections within the same process. + /// Only addresses of the form `/memory/...` will be supported. + MemoryOnly, +} + +/// The policy for connections to non-reserved peers. +#[derive(Clone, Debug, PartialEq, Eq)] +pub enum NonReservedPeerMode { + /// Accept them. This is the default. + Accept, + /// Deny them. + Deny, +} + +impl NonReservedPeerMode { + /// Attempt to parse the peer mode from a string. + pub fn parse(s: &str) -> Option { + match s { + "accept" => Some(Self::Accept), + "deny" => Some(Self::Deny), + _ => None, + } + } +} + +/// Sync operation mode. +#[derive(Copy, Clone, Debug, Eq, PartialEq)] +pub enum SyncMode { + /// Full block download and verification. + Full, + /// Download blocks and the latest state. + Fast { + /// Skip state proof download and verification. + skip_proofs: bool, + /// Download indexed transactions for recent blocks. + storage_chain_mode: bool, + }, + /// Warp sync - verify authority set transitions and the latest state. + Warp, +} + +impl SyncMode { + /// Returns if `self` is [`Self::Warp`]. + pub fn is_warp(&self) -> bool { + matches!(self, Self::Warp) + } + + /// Returns if `self` is [`Self::Fast`]. + pub fn is_fast(&self) -> bool { + matches!(self, Self::Fast { .. }) + } +} + +impl Default for SyncMode { + fn default() -> Self { + Self::Full + } +} + +/// The configuration of a node's secret key, describing the type of key +/// and how it is obtained. A node's identity keypair is the result of +/// the evaluation of the node key configuration. +#[derive(Clone, Debug)] +pub enum NodeKeyConfig { + /// A Ed25519 secret key configuration. + Ed25519(Secret), +} + +impl Default for NodeKeyConfig { + fn default() -> NodeKeyConfig { + Self::Ed25519(Secret::New) + } +} + +/// The options for obtaining a Ed25519 secret key. +pub type Ed25519Secret = Secret; + +/// The configuration options for obtaining a secret key `K`. +#[derive(Clone)] +pub enum Secret { + /// Use the given secret key `K`. + Input(K), + /// Read the secret key from a file. If the file does not exist, + /// it is created with a newly generated secret key `K`. The format + /// of the file is determined by `K`: + /// + /// * `ed25519::SecretKey`: An unencoded 32 bytes Ed25519 secret key. + File(PathBuf), + /// Always generate a new secret key `K`. + New, +} + +impl fmt::Debug for Secret { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + match self { + Self::Input(_) => f.debug_tuple("Secret::Input").finish(), + Self::File(path) => f.debug_tuple("Secret::File").field(path).finish(), + Self::New => f.debug_tuple("Secret::New").finish(), + } + } +} + +impl NodeKeyConfig { + /// Evaluate a `NodeKeyConfig` to obtain an identity `Keypair`: + /// + /// * If the secret is configured as input, the corresponding keypair is returned. + /// + /// * If the secret is configured as a file, it is read from that file, if it exists. Otherwise + /// a new secret is generated and stored. In either case, the keypair obtained from the + /// secret is returned. + /// + /// * If the secret is configured to be new, it is generated and the corresponding keypair is + /// returned. + pub fn into_keypair(self) -> io::Result { + use NodeKeyConfig::*; + match self { + Ed25519(Secret::New) => Ok(Keypair::generate_ed25519()), + + Ed25519(Secret::Input(k)) => Ok(Keypair::Ed25519(k.into())), + + Ed25519(Secret::File(f)) => get_secret( + f, + |mut b| match String::from_utf8(b.to_vec()).ok().and_then(|s| { + if s.len() == 64 { + array_bytes::hex2bytes(&s).ok() + } else { + None + } + }) { + Some(s) => ed25519::SecretKey::from_bytes(s), + _ => ed25519::SecretKey::from_bytes(&mut b), + }, + ed25519::SecretKey::generate, + |b| b.as_ref().to_vec(), + ) + .map(ed25519::Keypair::from) + .map(Keypair::Ed25519), + } + } +} + +/// Load a secret key from a file, if it exists, or generate a +/// new secret key and write it to that file. In either case, +/// the secret key is returned. +fn get_secret(file: P, parse: F, generate: G, serialize: W) -> io::Result +where + P: AsRef, + F: for<'r> FnOnce(&'r mut [u8]) -> Result, + G: FnOnce() -> K, + E: Error + Send + Sync + 'static, + W: Fn(&K) -> Vec, +{ + std::fs::read(&file) + .and_then(|mut sk_bytes| { + parse(&mut sk_bytes).map_err(|e| io::Error::new(io::ErrorKind::InvalidData, e)) + }) + .or_else(|e| { + if e.kind() == io::ErrorKind::NotFound { + file.as_ref().parent().map_or(Ok(()), fs::create_dir_all)?; + let sk = generate(); + let mut sk_vec = serialize(&sk); + write_secret_file(file, &sk_vec)?; + sk_vec.zeroize(); + Ok(sk) + } else { + Err(e) + } + }) +} + +/// Write secret bytes to a file. +fn write_secret_file

(path: P, sk_bytes: &[u8]) -> io::Result<()> +where + P: AsRef, +{ + let mut file = open_secret_file(&path)?; + file.write_all(sk_bytes) +} + +/// Opens a file containing a secret key in write mode. +#[cfg(unix)] +fn open_secret_file

(path: P) -> io::Result +where + P: AsRef, +{ + use std::os::unix::fs::OpenOptionsExt; + fs::OpenOptions::new().write(true).create_new(true).mode(0o600).open(path) +} -use std::{future::Future, iter, net::Ipv4Addr, path::PathBuf, pin::Pin, sync::Arc}; +/// Opens a file containing a secret key in write mode. +#[cfg(not(unix))] +fn open_secret_file

(path: P) -> Result +where + P: AsRef, +{ + fs::OpenOptions::new().write(true).create_new(true).open(path) +} /// Configuration for a set of nodes. #[derive(Clone, Debug)] @@ -317,3 +718,43 @@ pub struct Params { /// Request response protocol configurations pub request_response_protocol_configs: Vec, } + +#[cfg(test)] +mod tests { + use super::*; + use tempfile::TempDir; + + fn tempdir_with_prefix(prefix: &str) -> TempDir { + tempfile::Builder::new().prefix(prefix).tempdir().unwrap() + } + + fn secret_bytes(kp: &Keypair) -> Vec { + let Keypair::Ed25519(p) = kp; + p.secret().as_ref().iter().cloned().collect() + } + + #[test] + fn test_secret_file() { + let tmp = tempdir_with_prefix("x"); + std::fs::remove_dir(tmp.path()).unwrap(); // should be recreated + let file = tmp.path().join("x").to_path_buf(); + let kp1 = NodeKeyConfig::Ed25519(Secret::File(file.clone())).into_keypair().unwrap(); + let kp2 = NodeKeyConfig::Ed25519(Secret::File(file.clone())).into_keypair().unwrap(); + assert!(file.is_file() && secret_bytes(&kp1) == secret_bytes(&kp2)) + } + + #[test] + fn test_secret_input() { + let sk = ed25519::SecretKey::generate(); + let kp1 = NodeKeyConfig::Ed25519(Secret::Input(sk.clone())).into_keypair().unwrap(); + let kp2 = NodeKeyConfig::Ed25519(Secret::Input(sk)).into_keypair().unwrap(); + assert!(secret_bytes(&kp1) == secret_bytes(&kp2)); + } + + #[test] + fn test_secret_new() { + let kp1 = NodeKeyConfig::Ed25519(Secret::New).into_keypair().unwrap(); + let kp2 = NodeKeyConfig::Ed25519(Secret::New).into_keypair().unwrap(); + assert!(secret_bytes(&kp1) != secret_bytes(&kp2)); + } +} diff --git a/client/network/src/discovery.rs b/client/network/src/discovery.rs index 9ae4f913ac77d..7100c0c70d525 100644 --- a/client/network/src/discovery.rs +++ b/client/network/src/discovery.rs @@ -46,7 +46,7 @@ //! active mechanism that asks nodes for the addresses they are listening on. Whenever we learn //! of a node's address, you must call `add_self_reported_address`. -use crate::utils::LruHashSet; +use crate::{config::ProtocolId, utils::LruHashSet}; use array_bytes::bytes2hex; use futures::prelude::*; @@ -75,7 +75,6 @@ use libp2p::{ }, }; use log::{debug, info, trace, warn}; -use sc_network_common::config::ProtocolId; use sp_core::hexdisplay::HexDisplay; use std::{ cmp, @@ -906,6 +905,7 @@ mod tests { use super::{ kademlia_protocol_name, legacy_kademlia_protocol_name, DiscoveryConfig, DiscoveryOut, }; + use crate::config::ProtocolId; use futures::prelude::*; use libp2p::{ core::{ @@ -917,7 +917,6 @@ mod tests { swarm::{Executor, Swarm, SwarmEvent}, yamux, Multiaddr, }; - use sc_network_common::config::ProtocolId; use sp_core::hash::H256; use std::{collections::HashSet, pin::Pin, task::Poll}; diff --git a/client/network/src/error.rs b/client/network/src/error.rs index 51c0f78ff335a..8ba309cdb16c7 100644 --- a/client/network/src/error.rs +++ b/client/network/src/error.rs @@ -18,10 +18,12 @@ //! Substrate network possible errors. +use crate::config::TransportConfig; + use libp2p::{Multiaddr, PeerId}; // TODO: remove -use sc_network_common::{config::TransportConfig, protocol::ProtocolName}; +use sc_network_common::protocol::ProtocolName; use std::fmt; diff --git a/client/network/src/protocol.rs b/client/network/src/protocol.rs index 522f8ae7fcdaf..a8ed20aacabfe 100644 --- a/client/network/src/protocol.rs +++ b/client/network/src/protocol.rs @@ -16,7 +16,10 @@ // You should have received a copy of the GNU General Public License // along with this program. If not, see . -use crate::{config, error}; +use crate::{ + config::{self, NonReservedPeerMode}, + error, +}; use bytes::Bytes; use codec::{DecodeAll, Encode}; @@ -31,7 +34,6 @@ use libp2p::{ use log::{debug, error, warn}; use sc_network_common::{ - config::NonReservedPeerMode, protocol::{role::Roles, ProtocolName}, sync::message::BlockAnnouncesHandshake, }; diff --git a/client/network/src/service.rs b/client/network/src/service.rs index 94e374f2b8f57..090045125e0e5 100644 --- a/client/network/src/service.rs +++ b/client/network/src/service.rs @@ -29,7 +29,7 @@ use crate::{ behaviour::{self, Behaviour, BehaviourOut}, - config::Params, + config::{MultiaddrWithPeerId, Params, TransportConfig}, discovery::DiscoveryConfig, error::Error, network_state::{ @@ -68,7 +68,6 @@ use metrics::{Histogram, HistogramVec, MetricSources, Metrics}; use parking_lot::Mutex; use sc_network_common::{ - config::{MultiaddrWithPeerId, TransportConfig}, protocol::{ event::{DhtEvent, Event}, ProtocolName, diff --git a/client/network/src/service/traits.rs b/client/network/src/service/traits.rs index b4eabec5d2cd2..f0d724b82f159 100644 --- a/client/network/src/service/traits.rs +++ b/client/network/src/service/traits.rs @@ -19,6 +19,7 @@ // If you read this, you are very thorough, congratulations. use crate::{ + config::MultiaddrWithPeerId, request_responses::{IfDisconnected, RequestFailure}, service::signature::Signature, }; @@ -26,10 +27,7 @@ use crate::{ use futures::{channel::oneshot, Stream}; use libp2p::{Multiaddr, PeerId}; -use sc_network_common::{ - config::MultiaddrWithPeerId, - protocol::{event::Event, ProtocolName}, -}; +use sc_network_common::protocol::{event::Event, ProtocolName}; use sc_peerset::ReputationChange; use std::{collections::HashSet, future::Future, pin::Pin, sync::Arc}; diff --git a/client/network/sync/src/block_request_handler.rs b/client/network/sync/src/block_request_handler.rs index d126786138ef0..921efd7def622 100644 --- a/client/network/sync/src/block_request_handler.rs +++ b/client/network/sync/src/block_request_handler.rs @@ -30,8 +30,11 @@ use lru::LruCache; use prost::Message; use sc_client_api::BlockBackend; -use sc_network::request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}; -use sc_network_common::{config::ProtocolId, sync::message::BlockAttributes}; +use sc_network::{ + config::ProtocolId, + request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}, +}; +use sc_network_common::sync::message::BlockAttributes; use sp_blockchain::HeaderBackend; use sp_runtime::{ generic::BlockId, diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index c25b6732b8c7b..2be617cef6f46 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -36,11 +36,12 @@ use prometheus_endpoint::{ use sc_client_api::{BlockBackend, HeaderBackend, ProofProvider}; use sc_consensus::import_queue::ImportQueueService; use sc_network::{ - config::{NetworkConfiguration, NonDefaultSetConfig}, + config::{ + NetworkConfiguration, NonDefaultSetConfig, ProtocolId, SyncMode as SyncOperationMode, + }, utils::LruHashSet, }; use sc_network_common::{ - config::{ProtocolId, SyncMode as SyncOperationMode}, protocol::{event::Event, role::Roles, ProtocolName}, sync::{ message::{ diff --git a/client/network/sync/src/lib.rs b/client/network/sync/src/lib.rs index b32783bf2f24e..c1a44d4b26794 100644 --- a/client/network/sync/src/lib.rs +++ b/client/network/sync/src/lib.rs @@ -50,11 +50,12 @@ use sc_consensus::{ import_queue::ImportQueueService, BlockImportError, BlockImportStatus, IncomingBlock, }; use sc_network::{ - config::{NonDefaultSetConfig, SetConfig}, + config::{ + NonDefaultSetConfig, NonReservedPeerMode, NotificationHandshake, ProtocolId, SetConfig, + }, request_responses::{IfDisconnected, RequestFailure}, }; use sc_network_common::{ - config::{NonReservedPeerMode, NotificationHandshake, ProtocolId}, protocol::{role::Roles, ProtocolName}, sync::{ message::{ diff --git a/client/network/sync/src/service/mock.rs b/client/network/sync/src/service/mock.rs index 47dbc674b8ab3..fcdd740a74743 100644 --- a/client/network/sync/src/service/mock.rs +++ b/client/network/sync/src/service/mock.rs @@ -21,11 +21,12 @@ use libp2p::{Multiaddr, PeerId}; use sc_consensus::{BlockImportError, BlockImportStatus}; use sc_network::{ + config::MultiaddrWithPeerId, request_responses::{IfDisconnected, RequestFailure}, NetworkNotification, NetworkPeers, NetworkRequest, NetworkSyncForkRequest, NotificationSenderError, NotificationSenderT, }; -use sc_network_common::{config::MultiaddrWithPeerId, protocol::ProtocolName}; +use sc_network_common::protocol::ProtocolName; use sc_peerset::ReputationChange; use sp_runtime::traits::{Block as BlockT, NumberFor}; diff --git a/client/network/sync/src/state_request_handler.rs b/client/network/sync/src/state_request_handler.rs index b6729dc036a8d..0ce2c541bf92e 100644 --- a/client/network/sync/src/state_request_handler.rs +++ b/client/network/sync/src/state_request_handler.rs @@ -30,8 +30,10 @@ use lru::LruCache; use prost::Message; use sc_client_api::{BlockBackend, ProofProvider}; -use sc_network::request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}; -use sc_network_common::config::ProtocolId; +use sc_network::{ + config::ProtocolId, + request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}, +}; use sp_runtime::traits::Block as BlockT; use std::{ diff --git a/client/network/sync/src/warp_request_handler.rs b/client/network/sync/src/warp_request_handler.rs index 8936f71eb1892..7061d6485d092 100644 --- a/client/network/sync/src/warp_request_handler.rs +++ b/client/network/sync/src/warp_request_handler.rs @@ -23,13 +23,13 @@ use futures::{ }; use log::debug; -use sc_network::request_responses::{ - IncomingRequest, OutgoingResponse, ProtocolConfig as RequestResponseConfig, -}; -use sc_network_common::{ +use sc_network::{ config::ProtocolId, - sync::warp::{EncodedProof, WarpProofRequest, WarpSyncProvider}, + request_responses::{ + IncomingRequest, OutgoingResponse, ProtocolConfig as RequestResponseConfig, + }, }; +use sc_network_common::sync::warp::{EncodedProof, WarpProofRequest, WarpSyncProvider}; use sp_runtime::traits::Block as BlockT; use std::{sync::Arc, time::Duration}; diff --git a/client/network/test/src/lib.rs b/client/network/test/src/lib.rs index 42f03d02636c2..328b9bb526225 100644 --- a/client/network/test/src/lib.rs +++ b/client/network/test/src/lib.rs @@ -49,15 +49,15 @@ use sc_consensus::{ LongestChain, Verifier, }; use sc_network::{ - config::{NetworkConfiguration, NonDefaultSetConfig}, + config::{ + MultiaddrWithPeerId, NetworkConfiguration, NonDefaultSetConfig, NonReservedPeerMode, + ProtocolId, Role, SyncMode, TransportConfig, + }, request_responses::ProtocolConfig as RequestResponseConfig, Multiaddr, NetworkBlock, NetworkEventStream, NetworkService, NetworkStateInfo, NetworkSyncForkRequest, NetworkWorker, }; use sc_network_common::{ - config::{ - MultiaddrWithPeerId, NonReservedPeerMode, ProtocolId, Role, SyncMode, TransportConfig, - }, protocol::{role::Roles, ProtocolName}, sync::warp::{ AuthorityList, EncodedProof, SetId, VerificationResult, WarpSyncParams, WarpSyncProvider, diff --git a/client/network/test/src/service.rs b/client/network/test/src/service.rs index f7cc32b84ce9f..4527f05ab48d0 100644 --- a/client/network/test/src/service.rs +++ b/client/network/test/src/service.rs @@ -21,13 +21,11 @@ use libp2p::{Multiaddr, PeerId}; use sc_consensus::{ImportQueue, Link}; use sc_network::{ - config, NetworkEventStream, NetworkNotification, NetworkPeers, NetworkService, - NetworkStateInfo, NetworkWorker, -}; -use sc_network_common::{ - config::{MultiaddrWithPeerId, ProtocolId, TransportConfig}, - protocol::{event::Event, role::Roles}, + config::{self, MultiaddrWithPeerId, ProtocolId, TransportConfig}, + NetworkEventStream, NetworkNotification, NetworkPeers, NetworkService, NetworkStateInfo, + NetworkWorker, }; +use sc_network_common::protocol::{event::Event, role::Roles}; use sc_network_light::light_client_requests::handler::LightClientRequestHandler; use sc_network_sync::{ block_request_handler::BlockRequestHandler, diff --git a/client/network/transactions/src/lib.rs b/client/network/transactions/src/lib.rs index 97c4737a7e146..4ce354b1a2f44 100644 --- a/client/network/transactions/src/lib.rs +++ b/client/network/transactions/src/lib.rs @@ -35,13 +35,12 @@ use log::{debug, trace, warn}; use prometheus_endpoint::{register, Counter, PrometheusError, Registry, U64}; use sc_network::{ - config::{NonDefaultSetConfig, SetConfig}, + config::{NonDefaultSetConfig, NonReservedPeerMode, ProtocolId, SetConfig}, error, utils::{interval, LruHashSet}, NetworkEventStream, NetworkNotification, NetworkPeers, }; use sc_network_common::{ - config::{NonReservedPeerMode, ProtocolId}, protocol::{event::Event, role::ObservedRole, ProtocolName}, sync::{SyncEvent, SyncEventStream}, ExHashT, diff --git a/client/offchain/src/api.rs b/client/offchain/src/api.rs index 880d2019dd725..f7e8a8f5fbdb2 100644 --- a/client/offchain/src/api.rs +++ b/client/offchain/src/api.rs @@ -326,8 +326,8 @@ mod tests { use super::*; use libp2p::PeerId; use sc_client_db::offchain::LocalStorage; - use sc_network::{NetworkPeers, NetworkStateInfo}; - use sc_network_common::{config::MultiaddrWithPeerId, protocol::ProtocolName}; + use sc_network::{config::MultiaddrWithPeerId, NetworkPeers, NetworkStateInfo}; + use sc_network_common::protocol::ProtocolName; use sc_peerset::ReputationChange; use sp_core::offchain::{DbExternalities, Externalities}; use std::time::SystemTime; diff --git a/client/offchain/src/lib.rs b/client/offchain/src/lib.rs index 4f9a7bedf4e80..38a0442ee849d 100644 --- a/client/offchain/src/lib.rs +++ b/client/offchain/src/lib.rs @@ -246,7 +246,8 @@ mod tests { use libp2p::{Multiaddr, PeerId}; use sc_block_builder::BlockBuilderProvider as _; use sc_client_api::Backend as _; - use sc_network_common::{config::MultiaddrWithPeerId, protocol::ProtocolName}; + use sc_network::config::MultiaddrWithPeerId; + use sc_network_common::protocol::ProtocolName; use sc_peerset::ReputationChange; use sc_transaction_pool::{BasicPool, FullChainApi}; use sc_transaction_pool_api::{InPoolTransaction, TransactionPool}; diff --git a/client/rpc/src/system/tests.rs b/client/rpc/src/system/tests.rs index c0ee1e8f88005..b6bfec7736b08 100644 --- a/client/rpc/src/system/tests.rs +++ b/client/rpc/src/system/tests.rs @@ -99,7 +99,7 @@ fn api>>(sync: T) -> RpcModule> { ); }, Request::NetworkAddReservedPeer(peer, sender) => { - let _ = match sc_network_common::config::parse_str_addr(&peer) { + let _ = match sc_network::config::parse_str_addr(&peer) { Ok(_) => sender.send(Ok(())), Err(s) => sender.send(Err(error::Error::MalformattedPeerArg(s.to_string()))), diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index dea75017f6c45..a6585b28508fc 100644 --- a/client/service/src/builder.rs +++ b/client/service/src/builder.rs @@ -38,9 +38,11 @@ use sc_client_db::{Backend, DatabaseSettings}; use sc_consensus::import_queue::ImportQueue; use sc_executor::RuntimeVersionOf; use sc_keystore::LocalKeystore; -use sc_network::{NetworkEventStream, NetworkService, NetworkStateInfo, NetworkStatusProvider}; +use sc_network::{ + config::SyncMode, NetworkEventStream, NetworkService, NetworkStateInfo, NetworkStatusProvider, +}; use sc_network_bitswap::BitswapRequestHandler; -use sc_network_common::{config::SyncMode, protocol::role::Roles, sync::warp::WarpSyncParams}; +use sc_network_common::{protocol::role::Roles, sync::warp::WarpSyncParams}; use sc_network_light::light_client_requests::handler::LightClientRequestHandler; use sc_network_sync::{ block_request_handler::BlockRequestHandler, engine::SyncingEngine, diff --git a/client/service/src/config.rs b/client/service/src/config.rs index 99de079feb780..c7d98a4533436 100644 --- a/client/service/src/config.rs +++ b/client/service/src/config.rs @@ -22,19 +22,18 @@ pub use sc_client_api::execution_extensions::{ExecutionStrategies, ExecutionStra pub use sc_client_db::{BlocksPruning, Database, DatabaseSource, PruningMode}; pub use sc_executor::{WasmExecutionMethod, WasmtimeInstantiationStrategy}; pub use sc_network::{ - config::{NetworkConfiguration, NonDefaultSetConfig, Role, SetConfig}, + config::{ + MultiaddrWithPeerId, NetworkConfiguration, NodeKeyConfig, NonDefaultSetConfig, ProtocolId, + Role, SetConfig, SyncMode, TransportConfig, + }, request_responses::{ IncomingRequest, OutgoingResponse, ProtocolConfig as RequestResponseConfig, }, Multiaddr, }; -pub use sc_network_common::config::{ - MultiaddrWithPeerId, NodeKeyConfig, ProtocolId, TransportConfig, -}; use prometheus_endpoint::Registry; use sc_chain_spec::ChainSpec; -use sc_network_common::config::SyncMode; pub use sc_telemetry::TelemetryEndpoints; pub use sc_transaction_pool::Options as TransactionPoolOptions; use sp_core::crypto::SecretString; diff --git a/client/service/src/lib.rs b/client/service/src/lib.rs index 44b88f6525c9b..54f11ec25a02a 100644 --- a/client/service/src/lib.rs +++ b/client/service/src/lib.rs @@ -41,8 +41,9 @@ use futures::{channel::mpsc, pin_mut, FutureExt, StreamExt}; use jsonrpsee::{core::Error as JsonRpseeError, RpcModule}; use log::{debug, error, warn}; use sc_client_api::{blockchain::HeaderBackend, BlockBackend, BlockchainEvents, ProofProvider}; -use sc_network::{NetworkBlock, NetworkPeers, NetworkStateInfo, PeerId}; -use sc_network_common::config::MultiaddrWithPeerId; +use sc_network::{ + config::MultiaddrWithPeerId, NetworkBlock, NetworkPeers, NetworkStateInfo, PeerId, +}; use sc_network_sync::SyncingService; use sc_utils::mpsc::TracingUnboundedReceiver; use sp_blockchain::HeaderMetadata; diff --git a/client/service/test/src/lib.rs b/client/service/test/src/lib.rs index 191052fc0b57e..f80446a4d43eb 100644 --- a/client/service/test/src/lib.rs +++ b/client/service/test/src/lib.rs @@ -23,9 +23,9 @@ use log::{debug, info}; use parking_lot::Mutex; use sc_client_api::{Backend, CallExecutor}; use sc_network::{ - config::NetworkConfiguration, multiaddr, NetworkBlock, NetworkPeers, NetworkStateInfo, + config::{MultiaddrWithPeerId, NetworkConfiguration, TransportConfig}, + multiaddr, NetworkBlock, NetworkPeers, NetworkStateInfo, }; -use sc_network_common::config::{MultiaddrWithPeerId, TransportConfig}; use sc_network_sync::SyncingService; use sc_service::{ client::Client, From 45e0124f6c62c62f85f3d3c6f61d21b90bce8031 Mon Sep 17 00:00:00 2001 From: Aaro Altonen Date: Mon, 13 Mar 2023 14:44:07 +0200 Subject: [PATCH 07/10] Remove more stuff --- bin/node/cli/src/service.rs | 4 +- client/authority-discovery/src/lib.rs | 2 +- client/authority-discovery/src/worker.rs | 5 +- .../incoming_requests_handler.rs | 6 +- .../grandpa/src/communication/gossip.rs | 7 +- .../grandpa/src/communication/mod.rs | 2 +- .../grandpa/src/communication/tests.rs | 4 +- client/consensus/grandpa/src/lib.rs | 2 +- client/network-gossip/src/bridge.rs | 8 +- client/network-gossip/src/lib.rs | 6 +- client/network-gossip/src/state_machine.rs | 9 +- client/network-gossip/src/validator.rs | 2 +- client/network/bitswap/src/lib.rs | 6 +- client/network/common/src/lib.rs | 2 +- client/network/common/src/protocol.rs | 147 ------------------ .../network/common/src/{protocol => }/role.rs | 0 client/network/common/src/sync.rs | 2 +- client/network/common/src/sync/message.rs | 2 +- client/network/src/behaviour.rs | 8 +- client/network/src/config.rs | 13 +- client/network/src/error.rs | 5 +- .../{common/src/protocol => src}/event.rs | 6 +- client/network/src/lib.rs | 10 +- client/network/src/protocol.rs | 6 +- client/network/src/protocol/message.rs | 2 +- .../src/protocol/notifications/behaviour.rs | 8 +- .../src/protocol/notifications/handler.rs | 15 +- .../protocol/notifications/upgrade/collec.rs | 2 +- .../notifications/upgrade/notifications.rs | 8 +- client/network/src/request_responses.rs | 3 +- client/network/src/service.rs | 13 +- client/network/src/service/out_events.rs | 3 +- client/network/src/service/traits.rs | 3 +- client/network/src/types.rs | 144 +++++++++++++++++ client/network/sync/src/engine.rs | 4 +- client/network/sync/src/lib.rs | 5 +- client/network/sync/src/service/mock.rs | 2 +- client/network/sync/src/service/network.rs | 2 +- client/network/test/src/lib.rs | 3 +- client/network/test/src/service.rs | 3 +- client/network/transactions/src/lib.rs | 4 +- client/offchain/src/api.rs | 5 +- client/offchain/src/lib.rs | 3 +- client/service/src/builder.rs | 2 +- 44 files changed, 255 insertions(+), 243 deletions(-) delete mode 100644 client/network/common/src/protocol.rs rename client/network/common/src/{protocol => }/role.rs (100%) rename client/network/{common/src/protocol => src}/event.rs (97%) diff --git a/bin/node/cli/src/service.rs b/bin/node/cli/src/service.rs index 376bae4962bde..3c9716c08d1c0 100644 --- a/bin/node/cli/src/service.rs +++ b/bin/node/cli/src/service.rs @@ -31,8 +31,8 @@ use node_primitives::Block; use sc_client_api::BlockBackend; use sc_consensus_babe::{self, SlotProportion}; use sc_executor::NativeElseWasmExecutor; -use sc_network::{NetworkEventStream, NetworkService}; -use sc_network_common::{protocol::event::Event, sync::warp::WarpSyncParams}; +use sc_network::{event::Event, NetworkEventStream, NetworkService}; +use sc_network_common::sync::warp::WarpSyncParams; use sc_network_sync::SyncingService; use sc_service::{config::Configuration, error::Error as ServiceError, RpcHandlers, TaskManager}; use sc_telemetry::{Telemetry, TelemetryWorker}; diff --git a/client/authority-discovery/src/lib.rs b/client/authority-discovery/src/lib.rs index 579b9d8b15051..a3c6699091297 100644 --- a/client/authority-discovery/src/lib.rs +++ b/client/authority-discovery/src/lib.rs @@ -40,7 +40,7 @@ use futures::{ }; use libp2p::{Multiaddr, PeerId}; -use sc_network_common::protocol::event::DhtEvent; +use sc_network::event::DhtEvent; use sp_authority_discovery::AuthorityId; use sp_blockchain::HeaderBackend; use sp_runtime::traits::Block as BlockT; diff --git a/client/authority-discovery/src/worker.rs b/client/authority-discovery/src/worker.rs index 1213348047553..034d72902e65d 100644 --- a/client/authority-discovery/src/worker.rs +++ b/client/authority-discovery/src/worker.rs @@ -43,8 +43,9 @@ use log::{debug, error, log_enabled}; use prometheus_endpoint::{register, Counter, CounterVec, Gauge, Opts, U64}; use prost::Message; use rand::{seq::SliceRandom, thread_rng}; -use sc_network::{KademliaKey, NetworkDHTProvider, NetworkSigner, NetworkStateInfo, Signature}; -use sc_network_common::protocol::event::DhtEvent; +use sc_network::{ + event::DhtEvent, KademliaKey, NetworkDHTProvider, NetworkSigner, NetworkStateInfo, Signature, +}; use sp_api::{ApiError, ProvideRuntimeApi}; use sp_authority_discovery::{ AuthorityDiscoveryApi, AuthorityId, AuthorityPair, AuthoritySignature, diff --git a/client/consensus/beefy/src/communication/request_response/incoming_requests_handler.rs b/client/consensus/beefy/src/communication/request_response/incoming_requests_handler.rs index ab6c21a187c95..1670e99828831 100644 --- a/client/consensus/beefy/src/communication/request_response/incoming_requests_handler.rs +++ b/client/consensus/beefy/src/communication/request_response/incoming_requests_handler.rs @@ -23,8 +23,10 @@ use futures::{ }; use log::{debug, trace}; use sc_client_api::BlockBackend; -use sc_network::{config as netconfig, config::RequestResponseConfig, PeerId, ReputationChange}; -use sc_network_common::protocol::ProtocolName; +use sc_network::{ + config as netconfig, config::RequestResponseConfig, types::ProtocolName, PeerId, + ReputationChange, +}; use sp_consensus_beefy::BEEFY_ENGINE_ID; use sp_runtime::traits::Block; use std::{marker::PhantomData, sync::Arc}; diff --git a/client/consensus/grandpa/src/communication/gossip.rs b/client/consensus/grandpa/src/communication/gossip.rs index 0a3029e25477a..2c0fe3d8571e5 100644 --- a/client/consensus/grandpa/src/communication/gossip.rs +++ b/client/consensus/grandpa/src/communication/gossip.rs @@ -91,7 +91,7 @@ use parity_scale_codec::{Decode, Encode}; use prometheus_endpoint::{register, CounterVec, Opts, PrometheusError, Registry, U64}; use rand::seq::SliceRandom; use sc_network::{PeerId, ReputationChange}; -use sc_network_common::protocol::role::ObservedRole; +use sc_network_common::role::ObservedRole; use sc_network_gossip::{MessageIntent, ValidatorContext}; use sc_telemetry::{telemetry, TelemetryHandle, CONSENSUS_DEBUG}; use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; @@ -820,7 +820,7 @@ impl Inner { fn note_set(&mut self, set_id: SetId, authorities: Vec) -> MaybeMessage { let local_view = match self.local_view { ref mut x @ None => x.get_or_insert(LocalView::new(set_id, Round(1))), - Some(ref mut v) => + Some(ref mut v) => { if v.set_id == set_id { let diff_authorities = self.authorities.iter().collect::>() != authorities.iter().collect::>(); @@ -841,7 +841,8 @@ impl Inner { return None } else { v - }, + } + }, }; local_view.update_set(set_id); diff --git a/client/consensus/grandpa/src/communication/mod.rs b/client/consensus/grandpa/src/communication/mod.rs index 522c092493837..3896bc36031ba 100644 --- a/client/consensus/grandpa/src/communication/mod.rs +++ b/client/consensus/grandpa/src/communication/mod.rs @@ -74,7 +74,7 @@ pub(crate) const NEIGHBOR_REBROADCAST_PERIOD: Duration = Duration::from_secs(2 * pub mod grandpa_protocol_name { use sc_chain_spec::ChainSpec; - use sc_network_common::protocol::ProtocolName; + use sc_network::types::ProtocolName; pub(crate) const NAME: &str = "/grandpa/1"; /// Old names for the notifications protocol, used for backward compatibility. diff --git a/client/consensus/grandpa/src/communication/tests.rs b/client/consensus/grandpa/src/communication/tests.rs index 5c0d8036d5a81..f97b1f1e88181 100644 --- a/client/consensus/grandpa/src/communication/tests.rs +++ b/client/consensus/grandpa/src/communication/tests.rs @@ -27,12 +27,14 @@ use futures::prelude::*; use parity_scale_codec::Encode; use sc_network::{ config::{MultiaddrWithPeerId, Role}, + event::Event as NetworkEvent, + types::ProtocolName, Multiaddr, NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers, NetworkSyncForkRequest, NotificationSenderError, NotificationSenderT as NotificationSender, PeerId, ReputationChange, }; use sc_network_common::{ - protocol::{event::Event as NetworkEvent, role::ObservedRole, ProtocolName}, + role::ObservedRole, sync::{SyncEvent as SyncStreamEvent, SyncEventStream}, }; use sc_network_gossip::Validator; diff --git a/client/consensus/grandpa/src/lib.rs b/client/consensus/grandpa/src/lib.rs index 66bb209615635..2baa135081c55 100644 --- a/client/consensus/grandpa/src/lib.rs +++ b/client/consensus/grandpa/src/lib.rs @@ -68,7 +68,7 @@ use sc_client_api::{ StorageProvider, TransactionFor, }; use sc_consensus::BlockImport; -use sc_network_common::protocol::ProtocolName; +use sc_network::types::ProtocolName; use sc_telemetry::{telemetry, TelemetryHandle, CONSENSUS_DEBUG, CONSENSUS_INFO}; use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver}; use sp_api::ProvideRuntimeApi; diff --git a/client/network-gossip/src/bridge.rs b/client/network-gossip/src/bridge.rs index 995ebd4e5bfa3..4793d7822ddbe 100644 --- a/client/network-gossip/src/bridge.rs +++ b/client/network-gossip/src/bridge.rs @@ -21,10 +21,8 @@ use crate::{ Network, Syncing, Validator, }; -use sc_network_common::{ - protocol::{event::Event, ProtocolName}, - sync::SyncEvent, -}; +use sc_network::{event::Event, types::ProtocolName}; +use sc_network_common::sync::SyncEvent; use sc_peerset::ReputationChange; use futures::{ @@ -344,7 +342,7 @@ mod tests { config::MultiaddrWithPeerId, NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers, NotificationSenderError, NotificationSenderT as NotificationSender, }; - use sc_network_common::{protocol::role::ObservedRole, sync::SyncEventStream}; + use sc_network_common::{role::ObservedRole, sync::SyncEventStream}; use sp_runtime::{ testing::H256, traits::{Block as BlockT, NumberFor}, diff --git a/client/network-gossip/src/lib.rs b/client/network-gossip/src/lib.rs index c6a20ce8e950c..ef87dd599e010 100644 --- a/client/network-gossip/src/lib.rs +++ b/client/network-gossip/src/lib.rs @@ -68,8 +68,10 @@ pub use self::{ }; use libp2p::{multiaddr, PeerId}; -use sc_network::{NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers}; -use sc_network_common::{protocol::ProtocolName, sync::SyncEventStream}; +use sc_network::{ + types::ProtocolName, NetworkBlock, NetworkEventStream, NetworkNotification, NetworkPeers, +}; +use sc_network_common::sync::SyncEventStream; use sp_runtime::traits::{Block as BlockT, NumberFor}; use std::iter; diff --git a/client/network-gossip/src/state_machine.rs b/client/network-gossip/src/state_machine.rs index 922702242d6b0..e6d2b0e2ae4c8 100644 --- a/client/network-gossip/src/state_machine.rs +++ b/client/network-gossip/src/state_machine.rs @@ -22,7 +22,8 @@ use ahash::AHashSet; use libp2p::PeerId; use lru::LruCache; use prometheus_endpoint::{register, Counter, PrometheusError, Registry, U64}; -use sc_network_common::protocol::{role::ObservedRole, ProtocolName}; +use sc_network::types::ProtocolName; +use sc_network_common::role::ObservedRole; use sp_runtime::traits::{Block as BlockT, Hash, HashFor}; use std::{collections::HashMap, iter, num::NonZeroUsize, sync::Arc, time, time::Instant}; @@ -526,10 +527,10 @@ mod tests { use crate::multiaddr::Multiaddr; use futures::prelude::*; use sc_network::{ - config::MultiaddrWithPeerId, NetworkBlock, NetworkEventStream, NetworkNotification, - NetworkPeers, NotificationSenderError, NotificationSenderT as NotificationSender, + config::MultiaddrWithPeerId, event::Event, NetworkBlock, NetworkEventStream, + NetworkNotification, NetworkPeers, NotificationSenderError, + NotificationSenderT as NotificationSender, }; - use sc_network_common::protocol::event::Event; use sc_peerset::ReputationChange; use sp_runtime::{ testing::{Block as RawBlock, ExtrinsicWrapper, H256}, diff --git a/client/network-gossip/src/validator.rs b/client/network-gossip/src/validator.rs index 26835a5ae6687..2272efba50652 100644 --- a/client/network-gossip/src/validator.rs +++ b/client/network-gossip/src/validator.rs @@ -17,7 +17,7 @@ // along with this program. If not, see . use libp2p::PeerId; -use sc_network_common::protocol::role::ObservedRole; +use sc_network_common::role::ObservedRole; use sp_runtime::traits::Block as BlockT; /// Validates consensus messages. diff --git a/client/network/bitswap/src/lib.rs b/client/network/bitswap/src/lib.rs index dd3b9a659a458..5a7a7b51355c6 100644 --- a/client/network/bitswap/src/lib.rs +++ b/client/network/bitswap/src/lib.rs @@ -26,8 +26,10 @@ use libp2p::core::PeerId; use log::{debug, error, trace}; use prost::Message; use sc_client_api::BlockBackend; -use sc_network::request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}; -use sc_network_common::protocol::ProtocolName; +use sc_network::{ + request_responses::{IncomingRequest, OutgoingResponse, ProtocolConfig}, + types::ProtocolName, +}; use schema::bitswap::{ message::{wantlist::WantType, Block as MessageBlock, BlockPresence, BlockPresenceType}, Message as BitswapMessage, diff --git a/client/network/common/src/lib.rs b/client/network/common/src/lib.rs index 998c56d756f5b..f53590efd4c84 100644 --- a/client/network/common/src/lib.rs +++ b/client/network/common/src/lib.rs @@ -19,7 +19,7 @@ //! Common data structures of the networking layer. pub mod message; -pub mod protocol; +pub mod role; pub mod sync; /// Minimum Requirements for a Hash within Networking diff --git a/client/network/common/src/protocol.rs b/client/network/common/src/protocol.rs deleted file mode 100644 index bfeb1daf5d53b..0000000000000 --- a/client/network/common/src/protocol.rs +++ /dev/null @@ -1,147 +0,0 @@ -// This file is part of Substrate. - -// Copyright (C) Parity Technologies (UK) Ltd. -// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0 - -// This program is free software: you can redistribute it and/or modify -// it under the terms of the GNU General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. - -// This program is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU General Public License for more details. - -// You should have received a copy of the GNU General Public License -// along with this program. If not, see . - -use std::{ - borrow::Borrow, - fmt, - hash::{Hash, Hasher}, - ops::Deref, - sync::Arc, -}; - -use libp2p::core::upgrade; - -pub mod event; -pub mod role; - -/// The protocol name transmitted on the wire. -#[derive(Debug, Clone)] -pub enum ProtocolName { - /// The protocol name as a static string. - Static(&'static str), - /// The protocol name as a dynamically allocated string. - OnHeap(Arc), -} - -impl From<&'static str> for ProtocolName { - fn from(name: &'static str) -> Self { - Self::Static(name) - } -} - -impl From> for ProtocolName { - fn from(name: Arc) -> Self { - Self::OnHeap(name) - } -} - -impl From for ProtocolName { - fn from(name: String) -> Self { - Self::OnHeap(Arc::from(name)) - } -} - -impl Deref for ProtocolName { - type Target = str; - - fn deref(&self) -> &str { - match self { - Self::Static(name) => name, - Self::OnHeap(name) => &name, - } - } -} - -impl Borrow for ProtocolName { - fn borrow(&self) -> &str { - self - } -} - -impl PartialEq for ProtocolName { - fn eq(&self, other: &Self) -> bool { - (self as &str) == (other as &str) - } -} - -impl Eq for ProtocolName {} - -impl Hash for ProtocolName { - fn hash(&self, state: &mut H) { - (self as &str).hash(state) - } -} - -impl fmt::Display for ProtocolName { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - f.write_str(self) - } -} - -impl upgrade::ProtocolName for ProtocolName { - fn protocol_name(&self) -> &[u8] { - (self as &str).as_bytes() - } -} - -#[cfg(test)] -mod tests { - use super::ProtocolName; - use std::{ - borrow::Borrow, - collections::hash_map::DefaultHasher, - hash::{Hash, Hasher}, - }; - - #[test] - fn protocol_name_keys_are_equivalent_to_str_keys() { - const PROTOCOL: &'static str = "/some/protocol/1"; - let static_protocol_name = ProtocolName::from(PROTOCOL); - let on_heap_protocol_name = ProtocolName::from(String::from(PROTOCOL)); - - assert_eq!(>::borrow(&static_protocol_name), PROTOCOL); - assert_eq!(>::borrow(&on_heap_protocol_name), PROTOCOL); - assert_eq!(static_protocol_name, on_heap_protocol_name); - - assert_eq!(hash(static_protocol_name), hash(PROTOCOL)); - assert_eq!(hash(on_heap_protocol_name), hash(PROTOCOL)); - } - - #[test] - fn different_protocol_names_do_not_compare_equal() { - const PROTOCOL1: &'static str = "/some/protocol/1"; - let static_protocol_name1 = ProtocolName::from(PROTOCOL1); - let on_heap_protocol_name1 = ProtocolName::from(String::from(PROTOCOL1)); - - const PROTOCOL2: &'static str = "/some/protocol/2"; - let static_protocol_name2 = ProtocolName::from(PROTOCOL2); - let on_heap_protocol_name2 = ProtocolName::from(String::from(PROTOCOL2)); - - assert_ne!(>::borrow(&static_protocol_name1), PROTOCOL2); - assert_ne!(>::borrow(&on_heap_protocol_name1), PROTOCOL2); - assert_ne!(static_protocol_name1, static_protocol_name2); - assert_ne!(static_protocol_name1, on_heap_protocol_name2); - assert_ne!(on_heap_protocol_name1, on_heap_protocol_name2); - } - - fn hash(x: T) -> u64 { - let mut hasher = DefaultHasher::new(); - x.hash(&mut hasher); - hasher.finish() - } -} diff --git a/client/network/common/src/protocol/role.rs b/client/network/common/src/role.rs similarity index 100% rename from client/network/common/src/protocol/role.rs rename to client/network/common/src/role.rs diff --git a/client/network/common/src/sync.rs b/client/network/common/src/sync.rs index 262da6c202aa3..130f354b70050 100644 --- a/client/network/common/src/sync.rs +++ b/client/network/common/src/sync.rs @@ -22,7 +22,7 @@ pub mod message; pub mod metrics; pub mod warp; -use crate::protocol::role::Roles; +use crate::role::Roles; use futures::Stream; use libp2p::PeerId; diff --git a/client/network/common/src/sync/message.rs b/client/network/common/src/sync/message.rs index 7b8be18bda5cc..c651660c89c8d 100644 --- a/client/network/common/src/sync/message.rs +++ b/client/network/common/src/sync/message.rs @@ -19,7 +19,7 @@ //! Network packet message types. These get serialized and put into the lower level protocol //! payload. -use crate::protocol::role::Roles; +use crate::role::Roles; use bitflags::bitflags; use codec::{Decode, Encode, Error, Input, Output}; diff --git a/client/network/src/behaviour.rs b/client/network/src/behaviour.rs index cd963a044fb02..f068099928efc 100644 --- a/client/network/src/behaviour.rs +++ b/client/network/src/behaviour.rs @@ -18,9 +18,11 @@ use crate::{ discovery::{DiscoveryBehaviour, DiscoveryConfig, DiscoveryOut}, + event::DhtEvent, peer_info, protocol::{CustomMessageOutcome, NotificationsSink, Protocol}, request_responses::{self, IfDisconnected, ProtocolConfig, RequestFailure}, + types::ProtocolName, }; use bytes::Bytes; @@ -32,11 +34,7 @@ use libp2p::{ swarm::NetworkBehaviour, }; -use sc_network_common::protocol::{ - event::DhtEvent, - role::{ObservedRole, Roles}, - ProtocolName, -}; +use sc_network_common::role::{ObservedRole, Roles}; use sc_peerset::{PeersetHandle, ReputationChange}; use sp_runtime::traits::Block as BlockT; use std::{collections::HashSet, time::Duration}; diff --git a/client/network/src/config.rs b/client/network/src/config.rs index b0367550748ec..62aa110c912f9 100644 --- a/client/network/src/config.rs +++ b/client/network/src/config.rs @@ -21,18 +21,17 @@ //! The [`Params`] struct is the struct that must be passed in order to initialize the networking. //! See the documentation of [`Params`]. -pub use crate::request_responses::{ - IncomingRequest, OutgoingResponse, ProtocolConfig as RequestResponseConfig, +pub use crate::{ + request_responses::{ + IncomingRequest, OutgoingResponse, ProtocolConfig as RequestResponseConfig, + }, + types::ProtocolName, }; use codec::Encode; use libp2p::{core::PublicKey, identity::Keypair, multiaddr, Multiaddr, PeerId}; use prometheus_endpoint::Registry; -pub use sc_network_common::{ - protocol::{role::Role, ProtocolName}, - sync::warp::WarpSyncProvider, - ExHashT, -}; +pub use sc_network_common::{role::Role, sync::warp::WarpSyncProvider, ExHashT}; use zeroize::Zeroize; use std::{ diff --git a/client/network/src/error.rs b/client/network/src/error.rs index 8ba309cdb16c7..f0828fb821f35 100644 --- a/client/network/src/error.rs +++ b/client/network/src/error.rs @@ -18,13 +18,10 @@ //! Substrate network possible errors. -use crate::config::TransportConfig; +use crate::{config::TransportConfig, types::ProtocolName}; use libp2p::{Multiaddr, PeerId}; -// TODO: remove -use sc_network_common::protocol::ProtocolName; - use std::fmt; /// Result type alias for the network. diff --git a/client/network/common/src/protocol/event.rs b/client/network/src/event.rs similarity index 97% rename from client/network/common/src/protocol/event.rs rename to client/network/src/event.rs index 90c38b48c320f..3ecd8f9311429 100644 --- a/client/network/common/src/protocol/event.rs +++ b/client/network/src/event.rs @@ -19,11 +19,13 @@ //! Network event types. These are are not the part of the protocol, but rather //! events that happen on the network like DHT get/put results received. -use super::ProtocolName; -use crate::protocol::role::ObservedRole; +use crate::types::ProtocolName; + use bytes::Bytes; use libp2p::{core::PeerId, kad::record::Key}; +use sc_network_common::role::ObservedRole; + /// Events generated by DHT as a response to get_value and put_value requests. #[derive(Debug, Clone)] #[must_use] diff --git a/client/network/src/lib.rs b/client/network/src/lib.rs index 388e07a35909d..3ae290bc43d70 100644 --- a/client/network/src/lib.rs +++ b/client/network/src/lib.rs @@ -253,19 +253,18 @@ mod transport; pub mod config; pub mod error; +pub mod event; pub mod network_state; pub mod request_responses; +pub mod types; pub mod utils; +pub use event::{DhtEvent, Event}; #[doc(inline)] pub use libp2p::{multiaddr, Multiaddr, PeerId}; pub use request_responses::{IfDisconnected, RequestFailure, RequestResponseConfig}; pub use sc_network_common::{ - protocol::{ - event::{DhtEvent, Event}, - role::ObservedRole, - ProtocolName, - }, + role::ObservedRole, sync::{ warp::{WarpSyncPhase, WarpSyncProgress}, ExtendedPeerInfo, StateDownloadProgress, SyncEventStream, SyncState, SyncStatusProvider, @@ -282,6 +281,7 @@ pub use service::{ DecodingError, Keypair, NetworkService, NetworkWorker, NotificationSender, NotificationSenderReady, OutboundFailure, PublicKey, }; +pub use types::ProtocolName; pub use sc_peerset::ReputationChange; diff --git a/client/network/src/protocol.rs b/client/network/src/protocol.rs index a8ed20aacabfe..06ca02c0ca8d5 100644 --- a/client/network/src/protocol.rs +++ b/client/network/src/protocol.rs @@ -19,6 +19,7 @@ use crate::{ config::{self, NonReservedPeerMode}, error, + types::ProtocolName, }; use bytes::Bytes; @@ -33,10 +34,7 @@ use libp2p::{ }; use log::{debug, error, warn}; -use sc_network_common::{ - protocol::{role::Roles, ProtocolName}, - sync::message::BlockAnnouncesHandshake, -}; +use sc_network_common::{role::Roles, sync::message::BlockAnnouncesHandshake}; use sp_runtime::traits::Block as BlockT; use std::{ diff --git a/client/network/src/protocol/message.rs b/client/network/src/protocol/message.rs index f7981f1714951..66dca2975375f 100644 --- a/client/network/src/protocol/message.rs +++ b/client/network/src/protocol/message.rs @@ -61,7 +61,7 @@ pub mod generic { use sc_client_api::StorageProof; use sc_network_common::{ message::RequestId, - protocol::role::Roles, + role::Roles, sync::message::{ generic::{BlockRequest, BlockResponse}, BlockAnnounce, diff --git a/client/network/src/protocol/notifications/behaviour.rs b/client/network/src/protocol/notifications/behaviour.rs index 093d5846db7de..74e27fa17c602 100644 --- a/client/network/src/protocol/notifications/behaviour.rs +++ b/client/network/src/protocol/notifications/behaviour.rs @@ -16,8 +16,11 @@ // You should have received a copy of the GNU General Public License // along with this program. If not, see . -use crate::protocol::notifications::handler::{ - self, NotificationsSink, NotifsHandlerIn, NotifsHandlerOut, NotifsHandlerProto, +use crate::{ + protocol::notifications::handler::{ + self, NotificationsSink, NotifsHandlerIn, NotifsHandlerOut, NotifsHandlerProto, + }, + types::ProtocolName, }; use bytes::BytesMut; @@ -35,7 +38,6 @@ use libp2p::{ use log::{error, trace, warn}; use parking_lot::RwLock; use rand::distributions::{Distribution as _, Uniform}; -use sc_network_common::protocol::ProtocolName; use sc_peerset::DropReason; use smallvec::SmallVec; use std::{ diff --git a/client/network/src/protocol/notifications/handler.rs b/client/network/src/protocol/notifications/handler.rs index 70157386ef32e..9d8d98fd8cf27 100644 --- a/client/network/src/protocol/notifications/handler.rs +++ b/client/network/src/protocol/notifications/handler.rs @@ -57,9 +57,12 @@ //! It is illegal to send a [`NotifsHandlerIn::Open`] before a previously-emitted //! [`NotifsHandlerIn::Open`] has gotten an answer. -use crate::protocol::notifications::upgrade::{ - NotificationsIn, NotificationsInSubstream, NotificationsOut, NotificationsOutSubstream, - UpgradeCollec, +use crate::{ + protocol::notifications::upgrade::{ + NotificationsIn, NotificationsInSubstream, NotificationsOut, NotificationsOutSubstream, + UpgradeCollec, + }, + types::ProtocolName, }; use bytes::BytesMut; @@ -77,7 +80,6 @@ use libp2p::{ }; use log::error; use parking_lot::{Mutex, RwLock}; -use sc_network_common::protocol::ProtocolName; use std::{ collections::VecDeque, mem, @@ -945,8 +947,9 @@ pub mod tests { Poll::Ready(Some(NotificationsSinkMessage::Notification { message })) => Poll::Ready(Some(message)), Poll::Pending => Poll::Ready(None), - Poll::Ready(Some(NotificationsSinkMessage::ForceClose)) | Poll::Ready(None) => - panic!("sink closed"), + Poll::Ready(Some(NotificationsSinkMessage::ForceClose)) | Poll::Ready(None) => { + panic!("sink closed") + }, }) .await } diff --git a/client/network/src/protocol/notifications/upgrade/collec.rs b/client/network/src/protocol/notifications/upgrade/collec.rs index 89999ff0c7534..791821b3f75da 100644 --- a/client/network/src/protocol/notifications/upgrade/collec.rs +++ b/client/network/src/protocol/notifications/upgrade/collec.rs @@ -103,8 +103,8 @@ impl>, O, E> Future for FutWithUsize { #[cfg(test)] mod tests { use super::*; + use crate::types::ProtocolName as ProtoName; use libp2p::core::upgrade::{ProtocolName, UpgradeInfo}; - use sc_network_common::protocol::ProtocolName as ProtoName; // TODO: move to mocks mockall::mock! { diff --git a/client/network/src/protocol/notifications/upgrade/notifications.rs b/client/network/src/protocol/notifications/upgrade/notifications.rs index bfdd9ccfa7eb9..4e1c033f33b68 100644 --- a/client/network/src/protocol/notifications/upgrade/notifications.rs +++ b/client/network/src/protocol/notifications/upgrade/notifications.rs @@ -16,7 +16,6 @@ // You should have received a copy of the GNU General Public License // along with this program. If not, see . -use asynchronous_codec::Framed; /// Notifications protocol. /// /// The Substrate notifications protocol consists in the following: @@ -35,11 +34,15 @@ use asynchronous_codec::Framed; /// /// Notification substreams are unidirectional. If A opens a substream with B, then B is /// encouraged but not required to open a substream to A as well. +use crate::types::ProtocolName; + +use asynchronous_codec::Framed; use bytes::BytesMut; use futures::prelude::*; use libp2p::core::{upgrade, InboundUpgrade, OutboundUpgrade, UpgradeInfo}; use log::{error, warn}; -use sc_network_common::protocol::ProtocolName; +use unsigned_varint::codec::UviBytes; + use std::{ convert::Infallible, io, mem, @@ -47,7 +50,6 @@ use std::{ task::{Context, Poll}, vec, }; -use unsigned_varint::codec::UviBytes; /// Maximum allowed size of the two handshake messages, in bytes. const MAX_HANDSHAKE_SIZE: usize = 1024; diff --git a/client/network/src/request_responses.rs b/client/network/src/request_responses.rs index c32de5feb8eaf..171c28d2f25cd 100644 --- a/client/network/src/request_responses.rs +++ b/client/network/src/request_responses.rs @@ -34,7 +34,7 @@ //! - If provided, a ["requests processing"](ProtocolConfig::inbound_queue) channel //! is used to handle incoming requests. -use crate::ReputationChange; +use crate::{types::ProtocolName, ReputationChange}; use futures::{ channel::{mpsc, oneshot}, @@ -54,7 +54,6 @@ use libp2p::{ }, }; -use sc_network_common::protocol::ProtocolName; use sc_peerset::{PeersetHandle, BANNED_THRESHOLD}; use std::{ diff --git a/client/network/src/service.rs b/client/network/src/service.rs index 090045125e0e5..6dc00b36ceb53 100644 --- a/client/network/src/service.rs +++ b/client/network/src/service.rs @@ -32,6 +32,7 @@ use crate::{ config::{MultiaddrWithPeerId, Params, TransportConfig}, discovery::DiscoveryConfig, error::Error, + event::{DhtEvent, Event}, network_state::{ NetworkState, NotConnectedPeer as NetworkStateNotConnectedPeer, Peer as NetworkStatePeer, }, @@ -46,7 +47,9 @@ use crate::{ NotificationSenderReady as NotificationSenderReadyT, }, }, - transport, ReputationChange, + transport, + types::ProtocolName, + ReputationChange, }; use futures::{channel::oneshot, prelude::*}; @@ -67,13 +70,7 @@ use log::{debug, error, info, trace, warn}; use metrics::{Histogram, HistogramVec, MetricSources, Metrics}; use parking_lot::Mutex; -use sc_network_common::{ - protocol::{ - event::{DhtEvent, Event}, - ProtocolName, - }, - ExHashT, -}; +use sc_network_common::ExHashT; use sc_peerset::PeersetHandle; use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; use sp_blockchain::HeaderBackend; diff --git a/client/network/src/service/out_events.rs b/client/network/src/service/out_events.rs index 3771ea1643aec..398c26793fd41 100644 --- a/client/network/src/service/out_events.rs +++ b/client/network/src/service/out_events.rs @@ -31,11 +31,12 @@ //! - Send events by calling [`OutChannels::send`]. Events are cloned for each sender in the //! collection. +use crate::event::Event; + use futures::{prelude::*, ready, stream::FusedStream}; use log::error; use parking_lot::Mutex; use prometheus_endpoint::{register, CounterVec, GaugeVec, Opts, PrometheusError, Registry, U64}; -use sc_network_common::protocol::event::Event; use std::{ backtrace::Backtrace, cell::RefCell, diff --git a/client/network/src/service/traits.rs b/client/network/src/service/traits.rs index f0d724b82f159..3f9b7e552027b 100644 --- a/client/network/src/service/traits.rs +++ b/client/network/src/service/traits.rs @@ -20,14 +20,15 @@ use crate::{ config::MultiaddrWithPeerId, + event::Event, request_responses::{IfDisconnected, RequestFailure}, service::signature::Signature, + types::ProtocolName, }; use futures::{channel::oneshot, Stream}; use libp2p::{Multiaddr, PeerId}; -use sc_network_common::protocol::{event::Event, ProtocolName}; use sc_peerset::ReputationChange; use std::{collections::HashSet, future::Future, pin::Pin, sync::Arc}; diff --git a/client/network/src/types.rs b/client/network/src/types.rs index e69de29bb2d1d..ca0a4b96c06a5 100644 --- a/client/network/src/types.rs +++ b/client/network/src/types.rs @@ -0,0 +1,144 @@ +// This file is part of Substrate. + +// Copyright (C) Parity Technologies (UK) Ltd. +// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0 + +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with this program. If not, see . + +use libp2p::core::upgrade; + +use std::{ + borrow::Borrow, + fmt, + hash::{Hash, Hasher}, + ops::Deref, + sync::Arc, +}; + +/// The protocol name transmitted on the wire. +#[derive(Debug, Clone)] +pub enum ProtocolName { + /// The protocol name as a static string. + Static(&'static str), + /// The protocol name as a dynamically allocated string. + OnHeap(Arc), +} + +impl From<&'static str> for ProtocolName { + fn from(name: &'static str) -> Self { + Self::Static(name) + } +} + +impl From> for ProtocolName { + fn from(name: Arc) -> Self { + Self::OnHeap(name) + } +} + +impl From for ProtocolName { + fn from(name: String) -> Self { + Self::OnHeap(Arc::from(name)) + } +} + +impl Deref for ProtocolName { + type Target = str; + + fn deref(&self) -> &str { + match self { + Self::Static(name) => name, + Self::OnHeap(name) => &name, + } + } +} + +impl Borrow for ProtocolName { + fn borrow(&self) -> &str { + self + } +} + +impl PartialEq for ProtocolName { + fn eq(&self, other: &Self) -> bool { + (self as &str) == (other as &str) + } +} + +impl Eq for ProtocolName {} + +impl Hash for ProtocolName { + fn hash(&self, state: &mut H) { + (self as &str).hash(state) + } +} + +impl fmt::Display for ProtocolName { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.write_str(self) + } +} + +impl upgrade::ProtocolName for ProtocolName { + fn protocol_name(&self) -> &[u8] { + (self as &str).as_bytes() + } +} + +#[cfg(test)] +mod tests { + use super::ProtocolName; + use std::{ + borrow::Borrow, + collections::hash_map::DefaultHasher, + hash::{Hash, Hasher}, + }; + + #[test] + fn protocol_name_keys_are_equivalent_to_str_keys() { + const PROTOCOL: &'static str = "/some/protocol/1"; + let static_protocol_name = ProtocolName::from(PROTOCOL); + let on_heap_protocol_name = ProtocolName::from(String::from(PROTOCOL)); + + assert_eq!(>::borrow(&static_protocol_name), PROTOCOL); + assert_eq!(>::borrow(&on_heap_protocol_name), PROTOCOL); + assert_eq!(static_protocol_name, on_heap_protocol_name); + + assert_eq!(hash(static_protocol_name), hash(PROTOCOL)); + assert_eq!(hash(on_heap_protocol_name), hash(PROTOCOL)); + } + + #[test] + fn different_protocol_names_do_not_compare_equal() { + const PROTOCOL1: &'static str = "/some/protocol/1"; + let static_protocol_name1 = ProtocolName::from(PROTOCOL1); + let on_heap_protocol_name1 = ProtocolName::from(String::from(PROTOCOL1)); + + const PROTOCOL2: &'static str = "/some/protocol/2"; + let static_protocol_name2 = ProtocolName::from(PROTOCOL2); + let on_heap_protocol_name2 = ProtocolName::from(String::from(PROTOCOL2)); + + assert_ne!(>::borrow(&static_protocol_name1), PROTOCOL2); + assert_ne!(>::borrow(&on_heap_protocol_name1), PROTOCOL2); + assert_ne!(static_protocol_name1, static_protocol_name2); + assert_ne!(static_protocol_name1, on_heap_protocol_name2); + assert_ne!(on_heap_protocol_name1, on_heap_protocol_name2); + } + + fn hash(x: T) -> u64 { + let mut hasher = DefaultHasher::new(); + x.hash(&mut hasher); + hasher.finish() + } +} diff --git a/client/network/sync/src/engine.rs b/client/network/sync/src/engine.rs index 2be617cef6f46..e6e62101ffcd0 100644 --- a/client/network/sync/src/engine.rs +++ b/client/network/sync/src/engine.rs @@ -39,10 +39,12 @@ use sc_network::{ config::{ NetworkConfiguration, NonDefaultSetConfig, ProtocolId, SyncMode as SyncOperationMode, }, + event::Event, utils::LruHashSet, + ProtocolName, }; use sc_network_common::{ - protocol::{event::Event, role::Roles, ProtocolName}, + role::Roles, sync::{ message::{ generic::{BlockData, BlockResponse}, diff --git a/client/network/sync/src/lib.rs b/client/network/sync/src/lib.rs index c1a44d4b26794..45d14ffa7bb37 100644 --- a/client/network/sync/src/lib.rs +++ b/client/network/sync/src/lib.rs @@ -54,9 +54,10 @@ use sc_network::{ NonDefaultSetConfig, NonReservedPeerMode, NotificationHandshake, ProtocolId, SetConfig, }, request_responses::{IfDisconnected, RequestFailure}, + types::ProtocolName, }; use sc_network_common::{ - protocol::{role::Roles, ProtocolName}, + role::Roles, sync::{ message::{ BlockAnnounce, BlockAnnouncesHandshake, BlockAttributes, BlockData, BlockRequest, @@ -3165,7 +3166,7 @@ mod test { use futures::{executor::block_on, future::poll_fn}; use sc_block_builder::BlockBuilderProvider; use sc_network_common::{ - protocol::role::Role, + role::Role, sync::message::{BlockData, BlockState, FromBlock}, }; use sp_blockchain::HeaderBackend; diff --git a/client/network/sync/src/service/mock.rs b/client/network/sync/src/service/mock.rs index fcdd740a74743..c882633993c8b 100644 --- a/client/network/sync/src/service/mock.rs +++ b/client/network/sync/src/service/mock.rs @@ -23,10 +23,10 @@ use sc_consensus::{BlockImportError, BlockImportStatus}; use sc_network::{ config::MultiaddrWithPeerId, request_responses::{IfDisconnected, RequestFailure}, + types::ProtocolName, NetworkNotification, NetworkPeers, NetworkRequest, NetworkSyncForkRequest, NotificationSenderError, NotificationSenderT, }; -use sc_network_common::protocol::ProtocolName; use sc_peerset::ReputationChange; use sp_runtime::traits::{Block as BlockT, NumberFor}; diff --git a/client/network/sync/src/service/network.rs b/client/network/sync/src/service/network.rs index e59ba1e90d151..f87de1c4c3ecc 100644 --- a/client/network/sync/src/service/network.rs +++ b/client/network/sync/src/service/network.rs @@ -21,9 +21,9 @@ use libp2p::PeerId; use sc_network::{ request_responses::{IfDisconnected, RequestFailure}, + types::ProtocolName, NetworkNotification, NetworkPeers, NetworkRequest, }; -use sc_network_common::protocol::ProtocolName; use sc_peerset::ReputationChange; use sc_utils::mpsc::{tracing_unbounded, TracingUnboundedReceiver, TracingUnboundedSender}; diff --git a/client/network/test/src/lib.rs b/client/network/test/src/lib.rs index 328b9bb526225..75b8287b08dcf 100644 --- a/client/network/test/src/lib.rs +++ b/client/network/test/src/lib.rs @@ -54,11 +54,12 @@ use sc_network::{ ProtocolId, Role, SyncMode, TransportConfig, }, request_responses::ProtocolConfig as RequestResponseConfig, + types::ProtocolName, Multiaddr, NetworkBlock, NetworkEventStream, NetworkService, NetworkStateInfo, NetworkSyncForkRequest, NetworkWorker, }; use sc_network_common::{ - protocol::{role::Roles, ProtocolName}, + role::Roles, sync::warp::{ AuthorityList, EncodedProof, SetId, VerificationResult, WarpSyncParams, WarpSyncProvider, }, diff --git a/client/network/test/src/service.rs b/client/network/test/src/service.rs index 4527f05ab48d0..b1de2a91ebcc9 100644 --- a/client/network/test/src/service.rs +++ b/client/network/test/src/service.rs @@ -22,10 +22,11 @@ use libp2p::{Multiaddr, PeerId}; use sc_consensus::{ImportQueue, Link}; use sc_network::{ config::{self, MultiaddrWithPeerId, ProtocolId, TransportConfig}, + event::Event, NetworkEventStream, NetworkNotification, NetworkPeers, NetworkService, NetworkStateInfo, NetworkWorker, }; -use sc_network_common::protocol::{event::Event, role::Roles}; +use sc_network_common::role::Roles; use sc_network_light::light_client_requests::handler::LightClientRequestHandler; use sc_network_sync::{ block_request_handler::BlockRequestHandler, diff --git a/client/network/transactions/src/lib.rs b/client/network/transactions/src/lib.rs index 4ce354b1a2f44..381dd654b600b 100644 --- a/client/network/transactions/src/lib.rs +++ b/client/network/transactions/src/lib.rs @@ -37,11 +37,13 @@ use prometheus_endpoint::{register, Counter, PrometheusError, Registry, U64}; use sc_network::{ config::{NonDefaultSetConfig, NonReservedPeerMode, ProtocolId, SetConfig}, error, + event::Event, + types::ProtocolName, utils::{interval, LruHashSet}, NetworkEventStream, NetworkNotification, NetworkPeers, }; use sc_network_common::{ - protocol::{event::Event, role::ObservedRole, ProtocolName}, + role::ObservedRole, sync::{SyncEvent, SyncEventStream}, ExHashT, }; diff --git a/client/offchain/src/api.rs b/client/offchain/src/api.rs index f7e8a8f5fbdb2..a15f03bab6f84 100644 --- a/client/offchain/src/api.rs +++ b/client/offchain/src/api.rs @@ -326,8 +326,9 @@ mod tests { use super::*; use libp2p::PeerId; use sc_client_db::offchain::LocalStorage; - use sc_network::{config::MultiaddrWithPeerId, NetworkPeers, NetworkStateInfo}; - use sc_network_common::protocol::ProtocolName; + use sc_network::{ + config::MultiaddrWithPeerId, types::ProtocolName, NetworkPeers, NetworkStateInfo, + }; use sc_peerset::ReputationChange; use sp_core::offchain::{DbExternalities, Externalities}; use std::time::SystemTime; diff --git a/client/offchain/src/lib.rs b/client/offchain/src/lib.rs index 38a0442ee849d..677d89267e3a6 100644 --- a/client/offchain/src/lib.rs +++ b/client/offchain/src/lib.rs @@ -246,8 +246,7 @@ mod tests { use libp2p::{Multiaddr, PeerId}; use sc_block_builder::BlockBuilderProvider as _; use sc_client_api::Backend as _; - use sc_network::config::MultiaddrWithPeerId; - use sc_network_common::protocol::ProtocolName; + use sc_network::{config::MultiaddrWithPeerId, types::ProtocolName}; use sc_peerset::ReputationChange; use sc_transaction_pool::{BasicPool, FullChainApi}; use sc_transaction_pool_api::{InPoolTransaction, TransactionPool}; diff --git a/client/service/src/builder.rs b/client/service/src/builder.rs index a6585b28508fc..91ef65cf134e4 100644 --- a/client/service/src/builder.rs +++ b/client/service/src/builder.rs @@ -42,7 +42,7 @@ use sc_network::{ config::SyncMode, NetworkEventStream, NetworkService, NetworkStateInfo, NetworkStatusProvider, }; use sc_network_bitswap::BitswapRequestHandler; -use sc_network_common::{protocol::role::Roles, sync::warp::WarpSyncParams}; +use sc_network_common::{role::Roles, sync::warp::WarpSyncParams}; use sc_network_light::light_client_requests::handler::LightClientRequestHandler; use sc_network_sync::{ block_request_handler::BlockRequestHandler, engine::SyncingEngine, From 9b7b527b085e792b9683aeb0a452ce9aa6c06279 Mon Sep 17 00:00:00 2001 From: Aaro Altonen Date: Mon, 13 Mar 2023 15:46:12 +0200 Subject: [PATCH 08/10] Fix warnings --- client/network/src/config.rs | 2 +- client/network/src/request_responses.rs | 7 ++++++- client/network/src/types.rs | 2 ++ client/network/src/utils.rs | 2 ++ 4 files changed, 11 insertions(+), 2 deletions(-) diff --git a/client/network/src/config.rs b/client/network/src/config.rs index 62aa110c912f9..925a7795d290f 100644 --- a/client/network/src/config.rs +++ b/client/network/src/config.rs @@ -29,7 +29,7 @@ pub use crate::{ }; use codec::Encode; -use libp2p::{core::PublicKey, identity::Keypair, multiaddr, Multiaddr, PeerId}; +use libp2p::{identity::Keypair, multiaddr, Multiaddr, PeerId}; use prometheus_endpoint::Registry; pub use sc_network_common::{role::Role, sync::warp::WarpSyncProvider, ExHashT}; use zeroize::Zeroize; diff --git a/client/network/src/request_responses.rs b/client/network/src/request_responses.rs index 171c28d2f25cd..4dbdca26664d7 100644 --- a/client/network/src/request_responses.rs +++ b/client/network/src/request_responses.rs @@ -232,7 +232,12 @@ pub enum Event { }, /// A request protocol handler issued reputation changes for the given peer. - ReputationChanges { peer: PeerId, changes: Vec }, + ReputationChanges { + /// Peer whose reputation needs to be adjust. + peer: PeerId, + /// Reputation changes. + changes: Vec, + }, } /// Combination of a protocol name and a request id. diff --git a/client/network/src/types.rs b/client/network/src/types.rs index ca0a4b96c06a5..b0e32ae109149 100644 --- a/client/network/src/types.rs +++ b/client/network/src/types.rs @@ -16,6 +16,8 @@ // You should have received a copy of the GNU General Public License // along with this program. If not, see . +//! `sc-network` type definitions + use libp2p::core::upgrade; use std::{ diff --git a/client/network/src/utils.rs b/client/network/src/utils.rs index a89ec8ed36943..8db2cf4e7920d 100644 --- a/client/network/src/utils.rs +++ b/client/network/src/utils.rs @@ -16,6 +16,8 @@ // You should have received a copy of the GNU General Public License // along with this program. If not, see . +//! `sc-network` utilities + use futures::{stream::unfold, FutureExt, Stream, StreamExt}; use futures_timer::Delay; use linked_hash_set::LinkedHashSet; From f32d652f44917e62269fb57c14f7683269370058 Mon Sep 17 00:00:00 2001 From: Aaro Altonen <48052676+altonen@users.noreply.github.com> Date: Tue, 14 Mar 2023 09:09:59 +0200 Subject: [PATCH 09/10] Update client/network/src/request_responses.rs Co-authored-by: Dmitry Markin --- client/network/src/request_responses.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/client/network/src/request_responses.rs b/client/network/src/request_responses.rs index 4dbdca26664d7..4628b0191a2e8 100644 --- a/client/network/src/request_responses.rs +++ b/client/network/src/request_responses.rs @@ -80,7 +80,6 @@ pub enum RequestFailure { Refused, #[error("The remote replied, but the local node is no longer interested in the response.")] Obsolete, - /// Problem on the network. #[error("Problem on the network: {0}")] Network(OutboundFailure), } From 2dd8d3686a6e6fa96b23521b976c8f872a33576b Mon Sep 17 00:00:00 2001 From: Aaro Altonen Date: Tue, 14 Mar 2023 09:55:26 +0200 Subject: [PATCH 10/10] Fix cargo doc --- client/network/src/lib.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/client/network/src/lib.rs b/client/network/src/lib.rs index 3ae290bc43d70..c290f4b94db53 100644 --- a/client/network/src/lib.rs +++ b/client/network/src/lib.rs @@ -276,10 +276,10 @@ pub use service::{ KademliaKey, NetworkBlock, NetworkDHTProvider, NetworkEventStream, NetworkNotification, NetworkPeers, NetworkRequest, NetworkSigner, NetworkStateInfo, NetworkStatus, NetworkStatusProvider, NetworkSyncForkRequest, NotificationSender as NotificationSenderT, - NotificationSenderError, + NotificationSenderError, NotificationSenderReady, }, - DecodingError, Keypair, NetworkService, NetworkWorker, NotificationSender, - NotificationSenderReady, OutboundFailure, PublicKey, + DecodingError, Keypair, NetworkService, NetworkWorker, NotificationSender, OutboundFailure, + PublicKey, }; pub use types::ProtocolName;