From 7fc342e6c0d5fb8c31309aaaf50f20e61e85e193 Mon Sep 17 00:00:00 2001 From: Max Inden Date: Sun, 13 Feb 2022 21:57:38 +0100 Subject: [PATCH] {core,swarm}: Remove Network abstraction (#2492) This commit removes the `Network` abstraction, thus managing `Listeners` and the connection `Pool` in `Swarm` directly. This is done under the assumption that noone uses the `Network` abstraction directly, but instead everyone always uses it through `Swarm`. Both `Listeners` and `Pool` are moved from `libp2p-core` into `libp2p-swarm`. Given that they are no longer exposed via `Network`, they can be treated as an implementation detail of `libp2p-swarm` and `Swarm`. This change does not include any behavioural changes. This change has the followin benefits: - Removal of `NetworkEvent`, which was mostly an isomorphism of `SwarmEvent`. - Removal of the never-directly-used `Network` abstraction. - Removal of now obsolete verbose `Peer` (`core/src/network/peer.rs`) construct. - Removal of `libp2p-core` `DialOpts`, which is a direct mapping of `libp2p-swarm` `DialOpts`. - Allowing breaking changes to the connection handling and `Swarm` API interface without a breaking change in `libp2p-core` and thus a without a breaking change in `/transport` protocols. This change enables the following potential future changes: - Removal of `NodeHandler` and `ConnectionHandler`. Thus allowing to rename `ProtocolsHandler` into `ConnectionHandler`. - Moving `NetworkBehaviour` and `ProtocolsHandler` into `libp2p-core`, having `libp2p-xxx` protocol crates only depend on `libp2p-core` and thus allowing general breaking changes to `Swarm` without breaking all `libp2p-xxx` crates. --- core/CHANGELOG.md | 5 + core/Cargo.toml | 4 +- core/src/connection.rs | 229 +-- core/src/lib.rs | 4 +- core/src/network.rs | 738 --------- core/src/network/event.rs | 300 ---- core/src/network/peer.rs | 513 ------- core/src/transport/upgrade.rs | 6 +- core/tests/concurrent_dialing.rs | 171 --- core/tests/connection_limits.rs | 220 --- core/tests/network_dial_error.rs | 272 ---- core/tests/util.rs | 62 +- misc/metrics/src/swarm.rs | 18 +- misc/multistream-select/Cargo.toml | 1 + misc/multistream-select/tests/transport.rs | 64 +- swarm/CHANGELOG.md | 4 + swarm/Cargo.toml | 11 +- swarm/src/connection.rs | 204 +++ {core => swarm}/src/connection/error.rs | 0 {core => swarm}/src/connection/handler.rs | 31 +- {core => swarm}/src/connection/listeners.rs | 73 +- {core => swarm}/src/connection/pool.rs | 225 ++- .../src/connection/pool/concurrent_dial.rs | 7 +- {core => swarm}/src/connection/pool/task.rs | 7 +- {core => swarm}/src/connection/substream.rs | 22 +- swarm/src/dial_opts.rs | 4 +- swarm/src/lib.rs | 1318 ++++++++++++----- swarm/src/protocols_handler/node_handler.rs | 10 +- transports/noise/src/lib.rs | 4 +- 29 files changed, 1324 insertions(+), 3203 deletions(-) delete mode 100644 core/src/network.rs delete mode 100644 core/src/network/event.rs delete mode 100644 core/src/network/peer.rs delete mode 100644 core/tests/concurrent_dialing.rs delete mode 100644 core/tests/connection_limits.rs delete mode 100644 core/tests/network_dial_error.rs create mode 100644 swarm/src/connection.rs rename {core => swarm}/src/connection/error.rs (100%) rename {core => swarm}/src/connection/handler.rs (78%) rename {core => swarm}/src/connection/listeners.rs (86%) rename {core => swarm}/src/connection/pool.rs (90%) rename {core => swarm}/src/connection/pool/concurrent_dial.rs (96%) rename {core => swarm}/src/connection/pool/task.rs (97%) rename {core => swarm}/src/connection/substream.rs (94%) diff --git a/core/CHANGELOG.md b/core/CHANGELOG.md index f17dcfd7c68..6e53524a6af 100644 --- a/core/CHANGELOG.md +++ b/core/CHANGELOG.md @@ -1,5 +1,9 @@ # 0.32.0 [unreleased] +- Remove `Network`. `libp2p-core` is from now on an auxiliary crate only. Users + that have previously used `Network` only, will need to use `Swarm` instead. See + [PR 2492]. + - Update to `multiaddr` `v0.14.0`. - Update to `multihash` `v0.16.0`. @@ -10,6 +14,7 @@ [PR 2456]: https://github.com/libp2p/rust-libp2p/pull/2456 [RUSTSEC-2022-0009]: https://rustsec.org/advisories/RUSTSEC-2022-0009.html +[PR 2492]: https://github.com/libp2p/rust-libp2p/pull/2492 # 0.31.0 [2022-01-27] diff --git a/core/Cargo.toml b/core/Cargo.toml index 0d5c95ea709..e9ce5235408 100644 --- a/core/Cargo.toml +++ b/core/Cargo.toml @@ -49,11 +49,11 @@ criterion = "0.3" libp2p-mplex = { path = "../muxers/mplex" } libp2p-noise = { path = "../transports/noise" } libp2p-tcp = { path = "../transports/tcp" } -serde_json = "1.0" -rmp-serde = "1.0" multihash = { version = "0.16", default-features = false, features = ["arb"] } quickcheck = "0.9.0" rand07 = { package = "rand", version = "0.7" } +rmp-serde = "1.0" +serde_json = "1.0" [build-dependencies] prost-build = "0.9" diff --git a/core/src/connection.rs b/core/src/connection.rs index 68a53d9e7e5..3a8d54d04a1 100644 --- a/core/src/connection.rs +++ b/core/src/connection.rs @@ -18,29 +18,7 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -mod error; -pub(crate) mod handler; -mod listeners; -mod substream; - -pub(crate) mod pool; - -pub use error::{ - ConnectionError, PendingConnectionError, PendingInboundConnectionError, - PendingOutboundConnectionError, -}; -pub use handler::{ConnectionHandler, ConnectionHandlerEvent, IntoConnectionHandler}; -pub use listeners::{ListenerId, ListenersEvent, ListenersStream}; -pub use pool::{ConnectionCounters, ConnectionLimits}; -pub use pool::{EstablishedConnection, EstablishedConnectionIter, PendingConnection}; -pub use substream::{Close, Substream, SubstreamEndpoint}; - use crate::multiaddr::{Multiaddr, Protocol}; -use crate::muxing::StreamMuxer; -use crate::PeerId; -use std::hash::Hash; -use std::{error::Error, fmt, pin::Pin, task::Context, task::Poll}; -use substream::{Muxing, SubstreamEvent}; /// Connection identifier. #[derive(Debug, Copy, Clone, Hash, PartialEq, Eq, PartialOrd, Ord)] @@ -53,7 +31,34 @@ impl ConnectionId { /// in test environments. There is in general no guarantee /// that all connection IDs are based on non-negative integers. pub fn new(id: usize) -> Self { - ConnectionId(id) + Self(id) + } +} + +impl std::ops::Add for ConnectionId { + type Output = Self; + + fn add(self, other: usize) -> Self { + Self(self.0 + other) + } +} + +/// The ID of a single listener. +#[derive(Copy, Clone, Debug, PartialEq, Eq, Hash, PartialOrd, Ord)] +pub struct ListenerId(u64); + +impl ListenerId { + /// Creates a `ListenerId` from a non-negative integer. + pub fn new(id: u64) -> Self { + Self(id) + } +} + +impl std::ops::Add for ListenerId { + type Output = Self; + + fn add(self, other: u64) -> Self { + Self(self.0 + other) } } @@ -236,181 +241,3 @@ impl ConnectedPoint { } } } - -/// Information about a successfully established connection. -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct Connected { - /// The connected endpoint, including network address information. - pub endpoint: ConnectedPoint, - /// Information obtained from the transport. - pub peer_id: PeerId, -} - -/// Event generated by a [`Connection`]. -#[derive(Debug, Clone)] -pub enum Event { - /// Event generated by the [`ConnectionHandler`]. - Handler(T), - /// Address of the remote has changed. - AddressChange(Multiaddr), -} - -/// A multiplexed connection to a peer with an associated `ConnectionHandler`. -pub struct Connection -where - TMuxer: StreamMuxer, - THandler: ConnectionHandler>, -{ - /// Node that handles the muxing. - muxing: substream::Muxing, - /// Handler that processes substreams. - handler: THandler, -} - -impl fmt::Debug for Connection -where - TMuxer: StreamMuxer, - THandler: ConnectionHandler> + fmt::Debug, -{ - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("Connection") - .field("muxing", &self.muxing) - .field("handler", &self.handler) - .finish() - } -} - -impl Unpin for Connection -where - TMuxer: StreamMuxer, - THandler: ConnectionHandler>, -{ -} - -impl Connection -where - TMuxer: StreamMuxer, - THandler: ConnectionHandler>, -{ - /// Builds a new `Connection` from the given substream multiplexer - /// and connection handler. - pub fn new(muxer: TMuxer, handler: THandler) -> Self { - Connection { - muxing: Muxing::new(muxer), - handler, - } - } - - /// Returns a reference to the `ConnectionHandler` - pub fn handler(&self) -> &THandler { - &self.handler - } - - /// Returns a mutable reference to the `ConnectionHandler` - pub fn handler_mut(&mut self) -> &mut THandler { - &mut self.handler - } - - /// Notifies the connection handler of an event. - pub fn inject_event(&mut self, event: THandler::InEvent) { - self.handler.inject_event(event); - } - - /// Begins an orderly shutdown of the connection, returning the connection - /// handler and a `Future` that resolves when connection shutdown is complete. - pub fn close(self) -> (THandler, Close) { - (self.handler, self.muxing.close().0) - } - - /// Polls the connection for events produced by the associated handler - /// as a result of I/O activity on the substream multiplexer. - pub fn poll( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll, ConnectionError>> { - loop { - let mut io_pending = false; - - // Perform I/O on the connection through the muxer, informing the handler - // of new substreams. - match self.muxing.poll(cx) { - Poll::Pending => io_pending = true, - Poll::Ready(Ok(SubstreamEvent::InboundSubstream { substream })) => self - .handler - .inject_substream(substream, SubstreamEndpoint::Listener), - Poll::Ready(Ok(SubstreamEvent::OutboundSubstream { - user_data, - substream, - })) => { - let endpoint = SubstreamEndpoint::Dialer(user_data); - self.handler.inject_substream(substream, endpoint) - } - Poll::Ready(Ok(SubstreamEvent::AddressChange(address))) => { - self.handler.inject_address_change(&address); - return Poll::Ready(Ok(Event::AddressChange(address))); - } - Poll::Ready(Err(err)) => return Poll::Ready(Err(ConnectionError::IO(err))), - } - - // Poll the handler for new events. - match self.handler.poll(cx) { - Poll::Pending => { - if io_pending { - return Poll::Pending; // Nothing to do - } - } - Poll::Ready(Ok(ConnectionHandlerEvent::OutboundSubstreamRequest(user_data))) => { - self.muxing.open_substream(user_data); - } - Poll::Ready(Ok(ConnectionHandlerEvent::Custom(event))) => { - return Poll::Ready(Ok(Event::Handler(event))); - } - Poll::Ready(Err(err)) => return Poll::Ready(Err(ConnectionError::Handler(err))), - } - } - } -} - -/// Borrowed information about an incoming connection currently being negotiated. -#[derive(Debug, Copy, Clone)] -pub struct IncomingInfo<'a> { - /// Local connection address. - pub local_addr: &'a Multiaddr, - /// Address used to send back data to the remote. - pub send_back_addr: &'a Multiaddr, -} - -impl<'a> IncomingInfo<'a> { - /// Builds the [`PendingPoint`] corresponding to the incoming connection. - pub fn to_pending_point(&self) -> PendingPoint { - PendingPoint::Listener { - local_addr: self.local_addr.clone(), - send_back_addr: self.send_back_addr.clone(), - } - } - /// Builds the [`ConnectedPoint`] corresponding to the incoming connection. - pub fn to_connected_point(&self) -> ConnectedPoint { - ConnectedPoint::Listener { - local_addr: self.local_addr.clone(), - send_back_addr: self.send_back_addr.clone(), - } - } -} - -/// Information about a connection limit. -#[derive(Debug, Clone)] -pub struct ConnectionLimit { - /// The maximum number of connections. - pub limit: u32, - /// The current number of connections. - pub current: u32, -} - -impl fmt::Display for ConnectionLimit { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "{}/{}", self.current, self.limit) - } -} - -/// A `ConnectionLimit` can represent an error if it has been exceeded. -impl Error for ConnectionLimit {} diff --git a/core/src/lib.rs b/core/src/lib.rs index 055374ba319..315e20bc8cd 100644 --- a/core/src/lib.rs +++ b/core/src/lib.rs @@ -61,18 +61,16 @@ pub mod connection; pub mod either; pub mod identity; pub mod muxing; -pub mod network; pub mod peer_record; pub mod signed_envelope; pub mod transport; pub mod upgrade; -pub use connection::{Connected, ConnectedPoint, Endpoint}; +pub use connection::{ConnectedPoint, Endpoint}; pub use identity::PublicKey; pub use multiaddr::Multiaddr; pub use multihash; pub use muxing::StreamMuxer; -pub use network::{DialOpts, Network}; pub use peer_id::PeerId; pub use peer_record::PeerRecord; pub use signed_envelope::SignedEnvelope; diff --git a/core/src/network.rs b/core/src/network.rs deleted file mode 100644 index 046cb9d48a0..00000000000 --- a/core/src/network.rs +++ /dev/null @@ -1,738 +0,0 @@ -// Copyright 2018 Parity Technologies (UK) Ltd. -// -// Permission is hereby granted, free of charge, to any person obtaining a -// copy of this software and associated documentation files (the "Software"), -// to deal in the Software without restriction, including without limitation -// the rights to use, copy, modify, merge, publish, distribute, sublicense, -// and/or sell copies of the Software, and to permit persons to whom the -// Software is furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS -// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING -// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER -// DEALINGS IN THE SOFTWARE. - -mod event; -pub mod peer; - -pub use crate::connection::{ConnectionCounters, ConnectionLimits, Endpoint}; -pub use event::{IncomingConnection, NetworkEvent}; -pub use peer::Peer; - -use crate::{ - connection::{ - handler::{THandlerInEvent, THandlerOutEvent}, - pool::{Pool, PoolConfig, PoolEvent}, - ConnectionHandler, ConnectionId, ConnectionLimit, IncomingInfo, IntoConnectionHandler, - ListenerId, ListenersEvent, ListenersStream, PendingPoint, Substream, - }, - muxing::StreamMuxer, - transport::{Transport, TransportError}, - Executor, Multiaddr, PeerId, -}; -use either::Either; -use multihash::Multihash; -use std::{ - convert::TryFrom as _, - error, fmt, - num::{NonZeroU8, NonZeroUsize}, - pin::Pin, - task::{Context, Poll}, -}; -use thiserror::Error; - -/// Implementation of `Stream` that handles the nodes. -pub struct Network -where - TTrans: Transport, - THandler: IntoConnectionHandler, -{ - /// The local peer ID. - local_peer_id: PeerId, - - /// Listeners for incoming connections. - listeners: ListenersStream, - - /// The nodes currently active. - pool: Pool, -} - -impl fmt::Debug for Network -where - TTrans: fmt::Debug + Transport, - THandler: fmt::Debug + ConnectionHandler, -{ - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { - f.debug_struct("ReachAttempts") - .field("local_peer_id", &self.local_peer_id) - .field("listeners", &self.listeners) - .field("peers", &self.pool) - .finish() - } -} - -impl Unpin for Network -where - TTrans: Transport, - THandler: IntoConnectionHandler, -{ -} - -impl Network -where - TTrans: Transport, - THandler: IntoConnectionHandler, -{ - /// Checks whether the network has an established connection to a peer. - pub fn is_connected(&self, peer: &PeerId) -> bool { - self.pool.is_connected(peer) - } - - fn dialing_attempts(&self, peer: PeerId) -> impl Iterator { - self.pool - .iter_pending_info() - .filter(move |(_, endpoint, peer_id)| { - matches!(endpoint, PendingPoint::Dialer { .. }) && peer_id.as_ref() == Some(&peer) - }) - .map(|(connection_id, _, _)| connection_id) - } - - /// Checks whether the network has an ongoing dialing attempt to a peer. - pub fn is_dialing(&self, peer: &PeerId) -> bool { - self.dialing_attempts(*peer).next().is_some() - } - - fn disconnect(&mut self, peer: &PeerId) { - self.pool.disconnect(peer); - } -} - -impl Network -where - TTrans: Transport + Clone + 'static, - ::Error: Send + 'static, - THandler: IntoConnectionHandler + Send + 'static, -{ - /// Creates a new node events stream. - pub fn new(transport: TTrans, local_peer_id: PeerId, config: NetworkConfig) -> Self { - Network { - local_peer_id, - listeners: ListenersStream::new(transport), - pool: Pool::new(local_peer_id, config.pool_config, config.limits), - } - } - - /// Returns the transport passed when building this object. - pub fn transport(&self) -> &TTrans { - self.listeners.transport() - } - - /// Start listening on the given multiaddress. - pub fn listen_on( - &mut self, - addr: Multiaddr, - ) -> Result> { - self.listeners.listen_on(addr) - } - - /// Remove a previously added listener. - /// - /// Returns `true` if there was a listener with this ID, `false` - /// otherwise. - pub fn remove_listener(&mut self, id: ListenerId) -> bool { - self.listeners.remove_listener(id) - } - - /// Returns an iterator that produces the list of addresses we are listening on. - pub fn listen_addrs(&self) -> impl Iterator { - self.listeners.listen_addrs() - } - - /// Maps the given `observed_addr`, representing an address of the local - /// node observed by a remote peer, onto the locally known listen addresses - /// to yield one or more addresses of the local node that may be publicly - /// reachable. - /// - /// I.e. this method incorporates the view of other peers into the listen - /// addresses seen by the local node to account for possible IP and port - /// mappings performed by intermediate network devices in an effort to - /// obtain addresses for the local peer that are also reachable for peers - /// other than the peer who reported the `observed_addr`. - /// - /// The translation is transport-specific. See [`Transport::address_translation`]. - pub fn address_translation<'a>(&'a self, observed_addr: &'a Multiaddr) -> Vec - where - THandler: 'a, - { - let transport = self.listeners.transport(); - let mut addrs: Vec<_> = self - .listen_addrs() - .filter_map(move |server| transport.address_translation(server, observed_addr)) - .collect(); - - // remove duplicates - addrs.sort_unstable(); - addrs.dedup(); - - addrs - } - - /// Returns the peer id of the local node. - pub fn local_peer_id(&self) -> &PeerId { - &self.local_peer_id - } - - /// Dial a known or unknown peer. - /// - /// The given `handler` will be used to create the - /// [`Connection`](crate::connection::Connection) upon success and the - /// connection ID is returned. - pub fn dial( - &mut self, - handler: THandler, - opts: impl Into, - ) -> Result> - where - TTrans: Transport + Send, - TTrans::Output: Send + 'static, - TTrans::Dial: Send + 'static, - TTrans::Error: Send + 'static, - TTrans::Dial: Send + 'static, - { - let opts = opts.into(); - - let (peer_id, addresses, dial_concurrency_factor_override, role_override) = match opts.0 { - // Dial a known peer. - Opts::WithPeerIdWithAddresses(WithPeerIdWithAddresses { - peer_id, - addresses, - dial_concurrency_factor_override, - role_override, - }) => ( - Some(peer_id), - Either::Left(addresses.into_iter()), - dial_concurrency_factor_override, - role_override, - ), - // Dial an unknown peer. - Opts::WithoutPeerIdWithAddress(WithoutPeerIdWithAddress { - address, - role_override, - }) => { - // If the address ultimately encapsulates an expected peer ID, dial that peer - // such that any mismatch is detected. We do not "pop off" the `P2p` protocol - // from the address, because it may be used by the `Transport`, i.e. `P2p` - // is a protocol component that can influence any transport, like `libp2p-dns`. - let peer_id = match address - .iter() - .last() - .and_then(|p| { - if let multiaddr::Protocol::P2p(ma) = p { - Some(PeerId::try_from(ma)) - } else { - None - } - }) - .transpose() - { - Ok(peer_id) => peer_id, - Err(multihash) => return Err(DialError::InvalidPeerId { handler, multihash }), - }; - - ( - peer_id, - Either::Right(std::iter::once(address)), - None, - role_override, - ) - } - }; - - self.pool.add_outgoing( - self.transport().clone(), - addresses, - peer_id, - handler, - role_override, - dial_concurrency_factor_override, - ) - } - - /// Returns information about the state of the `Network`. - pub fn info(&self) -> NetworkInfo { - let num_peers = self.pool.num_peers(); - let connection_counters = self.pool.counters().clone(); - NetworkInfo { - num_peers, - connection_counters, - } - } - - /// Returns an iterator for information on all pending incoming connections. - pub fn incoming_info(&self) -> impl Iterator> { - self.pool.iter_pending_incoming() - } - - /// Returns a list of all connected peers, i.e. peers to whom the `Network` - /// has at least one established connection. - pub fn connected_peers(&self) -> impl Iterator { - self.pool.iter_connected() - } - - /// Checks whether the network has neither an ongoing dialing attempt, - /// nor an established connection to a peer. - pub fn is_disconnected(&self, peer: &PeerId) -> bool { - !self.is_connected(peer) && !self.is_dialing(peer) - } - - /// Returns a list of all the peers to whom a new outgoing connection - /// is currently being established. - pub fn dialing_peers(&self) -> impl Iterator { - self.pool - .iter_pending_info() - .filter(|(_, endpoint, _)| matches!(endpoint, PendingPoint::Dialer { .. })) - .filter_map(|(_, _, peer)| peer.as_ref()) - } - - /// Obtains a view of a [`Peer`] with the given ID in the network. - pub fn peer(&mut self, peer_id: PeerId) -> Peer<'_, TTrans, THandler> { - Peer::new(self, peer_id) - } - - /// Accepts a pending incoming connection obtained via [`NetworkEvent::IncomingConnection`], - /// adding it to the `Network`s connection pool subject to the configured limits. - /// - /// Once the connection is established and all transport protocol upgrades - /// completed, the connection is associated with the provided `handler`. - pub fn accept( - &mut self, - IncomingConnection { - upgrade, - local_addr, - send_back_addr, - }: IncomingConnection, - handler: THandler, - ) -> Result - where - TTrans: Transport, - TTrans::Output: Send + 'static, - TTrans::Error: Send + 'static, - TTrans::ListenerUpgrade: Send + 'static, - { - self.pool.add_incoming( - upgrade, - handler, - IncomingInfo { - local_addr: &local_addr, - send_back_addr: &send_back_addr, - }, - ) - } - - /// Provides an API similar to `Stream`, except that it does not terminate. - pub fn poll<'a, TMuxer>( - &'a mut self, - cx: &mut Context<'_>, - ) -> Poll< - NetworkEvent<'a, TTrans, THandlerInEvent, THandlerOutEvent, THandler>, - > - where - TTrans: Transport, - TTrans::Error: Send + 'static, - TTrans::Dial: Send + 'static, - TTrans::ListenerUpgrade: Send + 'static, - TMuxer: StreamMuxer + Send + Sync + 'static, - TMuxer::Error: std::fmt::Debug, - TMuxer::OutboundSubstream: Send, - THandler: IntoConnectionHandler + Send + 'static, - ::Error: error::Error + Send + 'static, - ::OutboundOpenInfo: Send, - ::Error: error::Error + Send, - THandler::Handler: ConnectionHandler> + Send, - { - // Poll the listener(s) for new connections. - match ListenersStream::poll(Pin::new(&mut self.listeners), cx) { - Poll::Pending => (), - Poll::Ready(ListenersEvent::Incoming { - listener_id, - upgrade, - local_addr, - send_back_addr, - }) => { - return Poll::Ready(NetworkEvent::IncomingConnection { - listener_id, - connection: IncomingConnection { - upgrade, - local_addr, - send_back_addr, - }, - }) - } - Poll::Ready(ListenersEvent::NewAddress { - listener_id, - listen_addr, - }) => { - return Poll::Ready(NetworkEvent::NewListenerAddress { - listener_id, - listen_addr, - }) - } - Poll::Ready(ListenersEvent::AddressExpired { - listener_id, - listen_addr, - }) => { - return Poll::Ready(NetworkEvent::ExpiredListenerAddress { - listener_id, - listen_addr, - }) - } - Poll::Ready(ListenersEvent::Closed { - listener_id, - addresses, - reason, - }) => { - return Poll::Ready(NetworkEvent::ListenerClosed { - listener_id, - addresses, - reason, - }) - } - Poll::Ready(ListenersEvent::Error { listener_id, error }) => { - return Poll::Ready(NetworkEvent::ListenerError { listener_id, error }) - } - } - - // Poll the known peers. - let event = match self.pool.poll(cx) { - Poll::Pending => return Poll::Pending, - Poll::Ready(PoolEvent::ConnectionEstablished { - connection, - other_established_connection_ids, - concurrent_dial_errors, - }) => NetworkEvent::ConnectionEstablished { - connection, - other_established_connection_ids, - concurrent_dial_errors, - }, - Poll::Ready(PoolEvent::PendingOutboundConnectionError { - id: _, - error, - handler, - peer, - }) => { - if let Some(peer) = peer { - NetworkEvent::DialError { - handler, - peer_id: peer, - error, - } - } else { - NetworkEvent::UnknownPeerDialError { error, handler } - } - } - Poll::Ready(PoolEvent::PendingInboundConnectionError { - id: _, - send_back_addr, - local_addr, - error, - handler, - }) => NetworkEvent::IncomingConnectionError { - error, - handler, - send_back_addr, - local_addr, - }, - Poll::Ready(PoolEvent::ConnectionClosed { - id, - connected, - error, - remaining_established_connection_ids, - handler, - .. - }) => NetworkEvent::ConnectionClosed { - id, - connected, - remaining_established_connection_ids, - error, - handler, - }, - Poll::Ready(PoolEvent::ConnectionEvent { connection, event }) => { - NetworkEvent::ConnectionEvent { connection, event } - } - Poll::Ready(PoolEvent::AddressChange { - connection, - new_endpoint, - old_endpoint, - }) => NetworkEvent::AddressChange { - connection, - new_endpoint, - old_endpoint, - }, - }; - - Poll::Ready(event) - } -} - -/// Information about the network obtained by [`Network::info()`]. -#[derive(Clone, Debug)] -pub struct NetworkInfo { - /// The total number of connected peers. - num_peers: usize, - /// Counters of ongoing network connections. - connection_counters: ConnectionCounters, -} - -impl NetworkInfo { - /// The number of connected peers, i.e. peers with whom at least - /// one established connection exists. - pub fn num_peers(&self) -> usize { - self.num_peers - } - - /// Gets counters for ongoing network connections. - pub fn connection_counters(&self) -> &ConnectionCounters { - &self.connection_counters - } -} - -/// The (optional) configuration for a [`Network`]. -/// -/// The default configuration specifies no dedicated task executor, no -/// connection limits, a connection event buffer size of 32, and a -/// `notify_handler` buffer size of 8. -#[derive(Default)] -pub struct NetworkConfig { - /// Connection [`Pool`] configuration. - pool_config: PoolConfig, - /// The effective connection limits. - limits: ConnectionLimits, -} - -impl NetworkConfig { - /// Configures the executor to use for spawning connection background tasks. - pub fn with_executor(mut self, e: Box) -> Self { - self.pool_config.executor = Some(e); - self - } - - /// Configures the executor to use for spawning connection background tasks, - /// only if no executor has already been configured. - pub fn or_else_with_executor(mut self, f: F) -> Self - where - F: FnOnce() -> Option>, - { - self.pool_config.executor = self.pool_config.executor.or_else(f); - self - } - - /// Sets the maximum number of events sent to a connection's background task - /// that may be buffered, if the task cannot keep up with their consumption and - /// delivery to the connection handler. - /// - /// When the buffer for a particular connection is full, `notify_handler` will no - /// longer be able to deliver events to the associated `ConnectionHandler`, - /// thus exerting back-pressure on the connection and peer API. - pub fn with_notify_handler_buffer_size(mut self, n: NonZeroUsize) -> Self { - self.pool_config.task_command_buffer_size = n.get() - 1; - self - } - - /// Sets the maximum number of buffered connection events (beyond a guaranteed - /// buffer of 1 event per connection). - /// - /// When the buffer is full, the background tasks of all connections will stall. - /// In this way, the consumers of network events exert back-pressure on - /// the network connection I/O. - pub fn with_connection_event_buffer_size(mut self, n: usize) -> Self { - self.pool_config.task_event_buffer_size = n; - self - } - - /// Number of addresses concurrently dialed for a single outbound connection attempt. - pub fn with_dial_concurrency_factor(mut self, factor: NonZeroU8) -> Self { - self.pool_config.dial_concurrency_factor = factor; - self - } - - /// Sets the connection limits to enforce. - pub fn with_connection_limits(mut self, limits: ConnectionLimits) -> Self { - self.limits = limits; - self - } -} - -/// Possible (synchronous) errors when dialing a peer. -#[derive(Debug, Clone, Error)] -pub enum DialError { - /// The dialing attempt is rejected because of a connection limit. - #[error("The dialing attempt was rejected because of a connection limit: {limit}")] - ConnectionLimit { - limit: ConnectionLimit, - handler: THandler, - }, - /// The dialing attempt is rejected because the peer being dialed is the local peer. - #[error("The dialing attempt was rejected because the peer being dialed is the local peer")] - LocalPeerId { handler: THandler }, - /// The dialing attempt is rejected because the PeerId is invalid. - #[error("The dialing attempt was rejected because a valid PeerId could not be constructed from: {multihash:?}")] - InvalidPeerId { - handler: THandler, - multihash: Multihash, - }, -} - -/// Options to configure a dial to a known or unknown peer. -/// -/// Used in [`Network::dial`]. -/// -/// To construct use either of: -/// -/// - [`DialOpts::peer_id`] dialing a known peer -/// -/// - [`DialOpts::unknown_peer_id`] dialing an unknown peer -#[derive(Debug, Clone, PartialEq)] -pub struct DialOpts(pub(super) Opts); - -impl DialOpts { - /// Dial a known peer. - pub fn peer_id(peer_id: PeerId) -> WithPeerId { - WithPeerId { peer_id } - } - - /// Dial an unknown peer. - pub fn unknown_peer_id() -> WithoutPeerId { - WithoutPeerId {} - } -} - -impl From for DialOpts { - fn from(address: Multiaddr) -> Self { - DialOpts::unknown_peer_id().address(address).build() - } -} - -/// Internal options type. -/// -/// Not to be constructed manually. Use either of the below instead: -/// -/// - [`DialOpts::peer_id`] dialing a known peer -/// - [`DialOpts::unknown_peer_id`] dialing an unknown peer -#[derive(Debug, Clone, PartialEq)] -pub(super) enum Opts { - WithPeerIdWithAddresses(WithPeerIdWithAddresses), - WithoutPeerIdWithAddress(WithoutPeerIdWithAddress), -} - -#[derive(Debug, Clone, PartialEq)] -pub struct WithPeerId { - pub(crate) peer_id: PeerId, -} - -impl WithPeerId { - /// Specify a set of addresses to be used to dial the known peer. - pub fn addresses(self, addresses: Vec) -> WithPeerIdWithAddresses { - WithPeerIdWithAddresses { - peer_id: self.peer_id, - addresses, - dial_concurrency_factor_override: Default::default(), - role_override: Endpoint::Dialer, - } - } -} - -#[derive(Debug, Clone, PartialEq)] -pub struct WithPeerIdWithAddresses { - pub(crate) peer_id: PeerId, - pub(crate) addresses: Vec, - pub(crate) dial_concurrency_factor_override: Option, - pub(crate) role_override: Endpoint, -} - -impl WithPeerIdWithAddresses { - /// Override [`NetworkConfig::with_dial_concurrency_factor`]. - pub fn override_dial_concurrency_factor(mut self, factor: NonZeroU8) -> Self { - self.dial_concurrency_factor_override = Some(factor); - self - } - - /// Override role of local node on connection. I.e. execute the dial _as a - /// listener_. - /// - /// See - /// [`ConnectedPoint::Dialer`](crate::connection::ConnectedPoint::Dialer) - /// for details. - pub fn override_role(mut self, role: Endpoint) -> Self { - self.role_override = role; - self - } - - /// Build the final [`DialOpts`]. - pub fn build(self) -> DialOpts { - DialOpts(Opts::WithPeerIdWithAddresses(self)) - } -} - -#[derive(Debug, Clone, PartialEq)] -pub struct WithoutPeerId {} - -impl WithoutPeerId { - /// Specify a single address to dial the unknown peer. - pub fn address(self, address: Multiaddr) -> WithoutPeerIdWithAddress { - WithoutPeerIdWithAddress { - address, - role_override: Endpoint::Dialer, - } - } -} - -#[derive(Debug, Clone, PartialEq)] -pub struct WithoutPeerIdWithAddress { - pub(crate) address: Multiaddr, - pub(crate) role_override: Endpoint, -} - -impl WithoutPeerIdWithAddress { - /// Override role of local node on connection. I.e. execute the dial _as a - /// listener_. - /// - /// See - /// [`ConnectedPoint::Dialer`](crate::connection::ConnectedPoint::Dialer) - /// for details. - pub fn override_role(mut self, role: Endpoint) -> Self { - self.role_override = role; - self - } - - /// Build the final [`DialOpts`]. - pub fn build(self) -> DialOpts { - DialOpts(Opts::WithoutPeerIdWithAddress(self)) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use futures::future::Future; - - struct Dummy; - - impl Executor for Dummy { - fn exec(&self, _: Pin + Send>>) {} - } - - #[test] - fn set_executor() { - NetworkConfig::default() - .with_executor(Box::new(Dummy)) - .with_executor(Box::new(|f| { - async_std::task::spawn(f); - })); - } -} diff --git a/core/src/network/event.rs b/core/src/network/event.rs deleted file mode 100644 index b70cef8a888..00000000000 --- a/core/src/network/event.rs +++ /dev/null @@ -1,300 +0,0 @@ -// Copyright 2018 Parity Technologies (UK) Ltd. -// -// Permission is hereby granted, free of charge, to any person obtaining a -// copy of this software and associated documentation files (the "Software"), -// to deal in the Software without restriction, including without limitation -// the rights to use, copy, modify, merge, publish, distribute, sublicense, -// and/or sell copies of the Software, and to permit persons to whom the -// Software is furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS -// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING -// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER -// DEALINGS IN THE SOFTWARE. - -//! Network events and associated information. - -use crate::{ - connection::{ - Connected, ConnectedPoint, ConnectionError, ConnectionHandler, ConnectionId, - EstablishedConnection, IntoConnectionHandler, ListenerId, PendingInboundConnectionError, - PendingOutboundConnectionError, - }, - transport::{Transport, TransportError}, - Multiaddr, PeerId, -}; -use std::fmt; - -/// Event that can happen on the `Network`. -pub enum NetworkEvent<'a, TTrans, TInEvent, TOutEvent, THandler> -where - TTrans: Transport, - THandler: IntoConnectionHandler, -{ - /// One of the listeners gracefully closed. - ListenerClosed { - /// The listener ID that closed. - listener_id: ListenerId, - /// The addresses that the listener was listening on. - addresses: Vec, - /// Reason for the closure. Contains `Ok(())` if the stream produced `None`, or `Err` - /// if the stream produced an error. - reason: Result<(), TTrans::Error>, - }, - - /// One of the listeners reported a non-fatal error. - ListenerError { - /// The listener that errored. - listener_id: ListenerId, - /// The listener error. - error: TTrans::Error, - }, - - /// One of the listeners is now listening on an additional address. - NewListenerAddress { - /// The listener that is listening on the new address. - listener_id: ListenerId, - /// The new address the listener is now also listening on. - listen_addr: Multiaddr, - }, - - /// One of the listeners is no longer listening on some address. - ExpiredListenerAddress { - /// The listener that is no longer listening on some address. - listener_id: ListenerId, - /// The expired address. - listen_addr: Multiaddr, - }, - - /// A new connection arrived on a listener. - /// - /// To accept the connection, see [`Network::accept`](crate::Network::accept). - IncomingConnection { - /// The listener who received the connection. - listener_id: ListenerId, - /// The pending incoming connection. - connection: IncomingConnection, - }, - - /// An error happened on a connection during its initial handshake. - /// - /// This can include, for example, an error during the handshake of the encryption layer, or - /// the connection unexpectedly closed. - IncomingConnectionError { - /// Local connection address. - local_addr: Multiaddr, - /// Address used to send back data to the remote. - send_back_addr: Multiaddr, - /// The error that happened. - error: PendingInboundConnectionError, - handler: THandler, - }, - - /// A new connection to a peer has been established. - ConnectionEstablished { - /// The newly established connection. - connection: EstablishedConnection<'a, TInEvent>, - /// List of other connections to the same peer. - /// - /// Note: Does not include the connection reported through this event. - other_established_connection_ids: Vec, - /// [`Some`] when the new connection is an outgoing connection. - /// Addresses are dialed in parallel. Contains the addresses and errors - /// of dial attempts that failed before the one successful dial. - concurrent_dial_errors: Option)>>, - }, - - /// An established connection to a peer has been closed. - /// - /// A connection may close if - /// - /// * it encounters an error, which includes the connection being - /// closed by the remote. In this case `error` is `Some`. - /// * it was actively closed by [`EstablishedConnection::start_close`], - /// i.e. a successful, orderly close. In this case `error` is `None`. - /// * it was actively closed by [`super::peer::ConnectedPeer::disconnect`] or - /// [`super::peer::DialingPeer::disconnect`], i.e. dropped without an - /// orderly close. In this case `error` is `None`. - /// - ConnectionClosed { - /// The ID of the connection that encountered an error. - id: ConnectionId, - /// Information about the connection that encountered the error. - connected: Connected, - /// The error that occurred. - error: Option::Error>>, - /// List of remaining established connections to the same peer. - remaining_established_connection_ids: Vec, - handler: THandler::Handler, - }, - - /// A dialing attempt to an address of a peer failed. - DialError { - /// The number of remaining dialing attempts. - handler: THandler, - - /// Id of the peer we were trying to dial. - peer_id: PeerId, - - /// The error that happened. - error: PendingOutboundConnectionError, - }, - - /// Failed to reach a peer that we were trying to dial. - UnknownPeerDialError { - /// The error that happened. - error: PendingOutboundConnectionError, - - handler: THandler, - }, - - /// An established connection produced an event. - ConnectionEvent { - /// The connection on which the event occurred. - connection: EstablishedConnection<'a, TInEvent>, - /// Event that was produced by the node. - event: TOutEvent, - }, - - /// An established connection has changed its address. - AddressChange { - /// The connection whose address has changed. - connection: EstablishedConnection<'a, TInEvent>, - /// New endpoint of this connection. - new_endpoint: ConnectedPoint, - /// Old endpoint of this connection. - old_endpoint: ConnectedPoint, - }, -} - -impl fmt::Debug - for NetworkEvent<'_, TTrans, TInEvent, TOutEvent, THandler> -where - TInEvent: fmt::Debug, - TOutEvent: fmt::Debug, - TTrans: Transport, - TTrans::Error: fmt::Debug, - THandler: IntoConnectionHandler, - ::Error: fmt::Debug, -{ - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { - match self { - NetworkEvent::NewListenerAddress { - listener_id, - listen_addr, - } => f - .debug_struct("NewListenerAddress") - .field("listener_id", listener_id) - .field("listen_addr", listen_addr) - .finish(), - NetworkEvent::ExpiredListenerAddress { - listener_id, - listen_addr, - } => f - .debug_struct("ExpiredListenerAddress") - .field("listener_id", listener_id) - .field("listen_addr", listen_addr) - .finish(), - NetworkEvent::ListenerClosed { - listener_id, - addresses, - reason, - } => f - .debug_struct("ListenerClosed") - .field("listener_id", listener_id) - .field("addresses", addresses) - .field("reason", reason) - .finish(), - NetworkEvent::ListenerError { listener_id, error } => f - .debug_struct("ListenerError") - .field("listener_id", listener_id) - .field("error", error) - .finish(), - NetworkEvent::IncomingConnection { connection, .. } => f - .debug_struct("IncomingConnection") - .field("local_addr", &connection.local_addr) - .field("send_back_addr", &connection.send_back_addr) - .finish(), - NetworkEvent::IncomingConnectionError { - local_addr, - send_back_addr, - error, - handler: _, - } => f - .debug_struct("IncomingConnectionError") - .field("local_addr", local_addr) - .field("send_back_addr", send_back_addr) - .field("error", error) - .finish(), - NetworkEvent::ConnectionEstablished { - connection, - concurrent_dial_errors, - .. - } => f - .debug_struct("OutgoingConnectionEstablished") - .field("connection", connection) - .field("concurrent_dial_errors", concurrent_dial_errors) - .finish(), - NetworkEvent::ConnectionClosed { - id, - connected, - error, - .. - } => f - .debug_struct("ConnectionClosed") - .field("id", id) - .field("connected", connected) - .field("error", error) - .finish(), - NetworkEvent::DialError { - handler: _, - peer_id, - error, - } => f - .debug_struct("DialError") - .field("peer_id", peer_id) - .field("error", error) - .finish(), - NetworkEvent::UnknownPeerDialError { - // multiaddr, - error, - .. - } => f - .debug_struct("UnknownPeerDialError") - // .field("multiaddr", multiaddr) - .field("error", error) - .finish(), - NetworkEvent::ConnectionEvent { connection, event } => f - .debug_struct("ConnectionEvent") - .field("connection", connection) - .field("event", event) - .finish(), - NetworkEvent::AddressChange { - connection, - new_endpoint, - old_endpoint, - } => f - .debug_struct("AddressChange") - .field("connection", connection) - .field("new_endpoint", new_endpoint) - .field("old_endpoint", old_endpoint) - .finish(), - } - } -} - -/// A pending incoming connection produced by a listener. -pub struct IncomingConnection { - /// The connection upgrade. - pub(crate) upgrade: TUpgrade, - /// Local connection address. - pub local_addr: Multiaddr, - /// Address used to send back data to the remote. - pub send_back_addr: Multiaddr, -} diff --git a/core/src/network/peer.rs b/core/src/network/peer.rs deleted file mode 100644 index c797dae5665..00000000000 --- a/core/src/network/peer.rs +++ /dev/null @@ -1,513 +0,0 @@ -// Copyright 2018 Parity Technologies (UK) Ltd. -// -// Permission is hereby granted, free of charge, to any person obtaining a -// copy of this software and associated documentation files (the "Software"), -// to deal in the Software without restriction, including without limitation -// the rights to use, copy, modify, merge, publish, distribute, sublicense, -// and/or sell copies of the Software, and to permit persons to whom the -// Software is furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS -// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING -// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER -// DEALINGS IN THE SOFTWARE. - -use super::Network; -use crate::{ - connection::{ - handler::THandlerInEvent, pool::Pool, ConnectionHandler, ConnectionId, - EstablishedConnection, EstablishedConnectionIter, IntoConnectionHandler, PendingConnection, - }, - PeerId, Transport, -}; -use std::{collections::VecDeque, error, fmt}; - -/// The possible representations of a peer in a [`Network`], as -/// seen by the local node. -/// -/// > **Note**: In any state there may always be a pending incoming -/// > connection attempt from the peer, however, the remote identity -/// > of a peer is only known once a connection is fully established. -pub enum Peer<'a, TTrans, THandler> -where - TTrans: Transport, - THandler: IntoConnectionHandler, -{ - /// At least one established connection exists to the peer. - Connected(ConnectedPeer<'a, TTrans, THandler>), - - /// There is an ongoing dialing (i.e. outgoing connection) attempt - /// to the peer. There may already be other established connections - /// to the peer. - Dialing(DialingPeer<'a, TTrans, THandler>), - - /// There exists no established connection to the peer and there is - /// currently no ongoing dialing (i.e. outgoing connection) attempt - /// in progress. - Disconnected(DisconnectedPeer<'a, TTrans, THandler>), - - /// The peer represents the local node. - Local, -} - -impl<'a, TTrans, THandler> fmt::Debug for Peer<'a, TTrans, THandler> -where - TTrans: Transport, - TTrans::Error: Send + 'static, - THandler: IntoConnectionHandler, -{ - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { - match self { - Peer::Connected(p) => f.debug_struct("Connected").field("peer", &p).finish(), - Peer::Dialing(p) => f.debug_struct("Dialing").field("peer", &p).finish(), - Peer::Disconnected(p) => f.debug_struct("Disconnected").field("peer", &p).finish(), - Peer::Local => f.debug_struct("Local").finish(), - } - } -} - -impl<'a, TTrans, THandler> Peer<'a, TTrans, THandler> -where - TTrans: Transport, - TTrans::Error: Send + 'static, - THandler: IntoConnectionHandler, -{ - pub(super) fn new(network: &'a mut Network, peer_id: PeerId) -> Self { - if peer_id == network.local_peer_id { - return Peer::Local; - } - - if network.pool.is_connected(&peer_id) { - return Self::connected(network, peer_id); - } - - if network.is_dialing(&peer_id) { - return Self::dialing(network, peer_id); - } - - Self::disconnected(network, peer_id) - } - - fn disconnected(network: &'a mut Network, peer_id: PeerId) -> Self { - Peer::Disconnected(DisconnectedPeer { - _network: network, - peer_id, - }) - } - - fn connected(network: &'a mut Network, peer_id: PeerId) -> Self { - Peer::Connected(ConnectedPeer { network, peer_id }) - } - - fn dialing(network: &'a mut Network, peer_id: PeerId) -> Self { - Peer::Dialing(DialingPeer { network, peer_id }) - } -} - -impl<'a, TTrans, THandler> Peer<'a, TTrans, THandler> -where - TTrans: Transport + Clone + Send + 'static, - TTrans::Output: Send + 'static, - TTrans::Error: Send + 'static, - TTrans::Dial: Send + 'static, - THandler: IntoConnectionHandler + Send + 'static, - THandler::Handler: ConnectionHandler + Send, - ::OutboundOpenInfo: Send, - ::Error: error::Error + Send + 'static, -{ - /// Checks whether the peer is currently connected. - /// - /// Returns `true` iff [`Peer::into_connected`] returns `Some`. - pub fn is_connected(&self) -> bool { - match self { - Peer::Connected(..) => true, - Peer::Dialing(peer) => peer.is_connected(), - Peer::Disconnected(..) => false, - Peer::Local => false, - } - } - - /// Checks whether the peer is currently being dialed. - /// - /// Returns `true` iff [`Peer::into_dialing`] returns `Some`. - pub fn is_dialing(&self) -> bool { - match self { - Peer::Dialing(_) => true, - Peer::Connected(peer) => peer.is_dialing(), - Peer::Disconnected(..) => false, - Peer::Local => false, - } - } - - /// Checks whether the peer is currently disconnected. - /// - /// Returns `true` iff [`Peer::into_disconnected`] returns `Some`. - pub fn is_disconnected(&self) -> bool { - matches!(self, Peer::Disconnected(..)) - } - - /// Converts the peer into a `ConnectedPeer`, if an established connection exists. - /// - /// Succeeds if the there is at least one established connection to the peer. - pub fn into_connected(self) -> Option> { - match self { - Peer::Connected(peer) => Some(peer), - Peer::Dialing(peer) => peer.into_connected(), - Peer::Disconnected(..) => None, - Peer::Local => None, - } - } - - /// Converts the peer into a `DialingPeer`, if a dialing attempt exists. - /// - /// Succeeds if the there is at least one pending outgoing connection to the peer. - pub fn into_dialing(self) -> Option> { - match self { - Peer::Dialing(peer) => Some(peer), - Peer::Connected(peer) => peer.into_dialing(), - Peer::Disconnected(..) => None, - Peer::Local => None, - } - } - - /// Converts the peer into a `DisconnectedPeer`, if neither an established connection - /// nor a dialing attempt exists. - pub fn into_disconnected(self) -> Option> { - match self { - Peer::Disconnected(peer) => Some(peer), - _ => None, - } - } -} - -/// The representation of a peer in a [`Network`] to whom at least -/// one established connection exists. There may also be additional ongoing -/// dialing attempts to the peer. -pub struct ConnectedPeer<'a, TTrans, THandler> -where - TTrans: Transport, - THandler: IntoConnectionHandler, -{ - network: &'a mut Network, - peer_id: PeerId, -} - -impl<'a, TTrans, THandler> ConnectedPeer<'a, TTrans, THandler> -where - TTrans: Transport, - ::Error: Send + 'static, - THandler: IntoConnectionHandler, -{ - pub fn id(&self) -> &PeerId { - &self.peer_id - } - - /// Returns the `ConnectedPeer` into a `Peer`. - pub fn into_peer(self) -> Peer<'a, TTrans, THandler> { - Peer::Connected(self) - } - - /// Obtains an established connection to the peer by ID. - pub fn connection( - &mut self, - id: ConnectionId, - ) -> Option>> { - self.network.pool.get_established(id) - } - - /// The number of established connections to the peer. - pub fn num_connections(&self) -> u32 { - self.network.pool.num_peer_established(self.peer_id) - } - - /// Checks whether there is an ongoing dialing attempt to the peer. - /// - /// Returns `true` iff [`ConnectedPeer::into_dialing`] returns `Some`. - pub fn is_dialing(&self) -> bool { - self.network.is_dialing(&self.peer_id) - } - - /// Converts this peer into a [`DialingPeer`], if there is an ongoing - /// dialing attempt, `None` otherwise. - pub fn into_dialing(self) -> Option> { - if self.network.is_dialing(&self.peer_id) { - Some(DialingPeer { - network: self.network, - peer_id: self.peer_id, - }) - } else { - None - } - } - - /// Gets an iterator over all established connections to the peer. - pub fn connections( - &mut self, - ) -> EstablishedConnectionIter, THandlerInEvent> - { - self.network.pool.iter_peer_established(&self.peer_id) - } - - /// Obtains some established connection to the peer. - pub fn some_connection(&mut self) -> EstablishedConnection> { - self.connections() - .into_first() - .expect("By `Peer::new` and the definition of `ConnectedPeer`.") - } - - /// Disconnects from the peer, closing all connections. - pub fn disconnect(self) -> DisconnectedPeer<'a, TTrans, THandler> { - self.network.disconnect(&self.peer_id); - DisconnectedPeer { - _network: self.network, - peer_id: self.peer_id, - } - } -} - -impl<'a, TTrans, THandler> fmt::Debug for ConnectedPeer<'a, TTrans, THandler> -where - TTrans: Transport, - TTrans::Error: Send + 'static, - THandler: IntoConnectionHandler, -{ - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { - f.debug_struct("ConnectedPeer") - .field("peer_id", &self.peer_id) - .field( - "established", - &self - .network - .pool - .iter_peer_established_info(&self.peer_id) - .collect::>(), - ) - .field("attempts", &self.network.is_dialing(&self.peer_id)) - .finish() - } -} - -/// The representation of a peer in a [`Network`] to whom a dialing -/// attempt is ongoing. There may already exist other established -/// connections to this peer. -pub struct DialingPeer<'a, TTrans, THandler> -where - TTrans: Transport, - THandler: IntoConnectionHandler, -{ - network: &'a mut Network, - peer_id: PeerId, -} - -impl<'a, TTrans, THandler> DialingPeer<'a, TTrans, THandler> -where - TTrans: Transport, - TTrans::Error: Send + 'static, - THandler: IntoConnectionHandler, -{ - pub fn id(&self) -> &PeerId { - &self.peer_id - } - - /// Returns the `DialingPeer` into a `Peer`. - pub fn into_peer(self) -> Peer<'a, TTrans, THandler> { - Peer::Dialing(self) - } - - /// Disconnects from this peer, closing all established connections and - /// aborting all dialing attempts. - pub fn disconnect(self) -> DisconnectedPeer<'a, TTrans, THandler> { - self.network.disconnect(&self.peer_id); - DisconnectedPeer { - _network: self.network, - peer_id: self.peer_id, - } - } - - /// Checks whether there is an established connection to the peer. - /// - /// Returns `true` iff [`DialingPeer::into_connected`] returns `Some`. - pub fn is_connected(&self) -> bool { - self.network.pool.is_connected(&self.peer_id) - } - - /// Converts the peer into a `ConnectedPeer`, if an established connection exists. - pub fn into_connected(self) -> Option> { - if self.is_connected() { - Some(ConnectedPeer { - peer_id: self.peer_id, - network: self.network, - }) - } else { - None - } - } - - /// Obtains a dialing attempt to the peer by connection ID of - /// the current connection attempt. - pub fn attempt(&mut self, id: ConnectionId) -> Option> { - Some(DialingAttempt { - peer_id: self.peer_id, - inner: self.network.pool.get_outgoing(id)?, - }) - } - - /// Gets an iterator over all dialing (i.e. pending outgoing) connections to the peer. - pub fn attempts(&mut self) -> DialingAttemptIter<'_, THandler, TTrans> { - DialingAttemptIter::new(&self.peer_id, &mut self.network) - } - - /// Obtains some dialing connection to the peer. - /// - /// At least one dialing connection is guaranteed to exist on a `DialingPeer`. - pub fn some_attempt(&mut self) -> DialingAttempt<'_, THandler> { - self.attempts() - .into_first() - .expect("By `Peer::new` and the definition of `DialingPeer`.") - } -} - -impl<'a, TTrans, THandler> fmt::Debug for DialingPeer<'a, TTrans, THandler> -where - TTrans: Transport, - TTrans::Error: Send + 'static, - THandler: IntoConnectionHandler, -{ - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { - f.debug_struct("DialingPeer") - .field("peer_id", &self.peer_id) - .field( - "established", - &self - .network - .pool - .iter_peer_established_info(&self.peer_id) - .collect::>(), - ) - .finish() - } -} - -/// The representation of a peer to whom the `Network` has currently -/// neither an established connection, nor an ongoing dialing attempt -/// initiated by the local peer. -pub struct DisconnectedPeer<'a, TTrans, THandler> -where - TTrans: Transport, - THandler: IntoConnectionHandler, -{ - peer_id: PeerId, - _network: &'a mut Network, -} - -impl<'a, TTrans, THandler> fmt::Debug for DisconnectedPeer<'a, TTrans, THandler> -where - TTrans: Transport, - THandler: IntoConnectionHandler, -{ - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> { - f.debug_struct("DisconnectedPeer") - .field("peer_id", &self.peer_id) - .finish() - } -} - -impl<'a, TTrans, THandler> DisconnectedPeer<'a, TTrans, THandler> -where - TTrans: Transport, - THandler: IntoConnectionHandler, -{ - pub fn id(&self) -> &PeerId { - &self.peer_id - } - - /// Returns the `DisconnectedPeer` into a `Peer`. - pub fn into_peer(self) -> Peer<'a, TTrans, THandler> { - Peer::Disconnected(self) - } -} - -/// A [`DialingAttempt`] is a pending outgoing connection attempt to a known / -/// expected remote peer ID. -pub struct DialingAttempt<'a, THandler: IntoConnectionHandler> { - peer_id: PeerId, - /// The underlying pending connection in the `Pool`. - inner: PendingConnection<'a, THandler>, -} - -impl<'a, THandler: IntoConnectionHandler> DialingAttempt<'a, THandler> { - /// Returns the ID of the current connection attempt. - pub fn id(&self) -> ConnectionId { - self.inner.id() - } - - /// Returns the (expected) peer ID of the dialing attempt. - pub fn peer_id(&self) -> &PeerId { - &self.peer_id - } - - /// Aborts the dialing attempt. - pub fn abort(self) { - self.inner.abort(); - } -} - -/// An iterator over the ongoing dialing attempts to a peer. -pub struct DialingAttemptIter<'a, THandler: IntoConnectionHandler, TTrans: Transport> { - /// The peer whose dialing attempts are being iterated. - peer_id: &'a PeerId, - /// The underlying connection `Pool` of the `Network`. - pool: &'a mut Pool, - /// [`ConnectionId`]s of the dialing attempts of the peer. - connections: VecDeque, -} - -// Note: Ideally this would be an implementation of `Iterator`, but that -// requires GATs (cf. https://github.com/rust-lang/rust/issues/44265) and -// a different definition of `Iterator`. -impl<'a, THandler: IntoConnectionHandler, TTrans: Transport> - DialingAttemptIter<'a, THandler, TTrans> -{ - fn new(peer_id: &'a PeerId, network: &'a mut Network) -> Self { - let connections = network.dialing_attempts(*peer_id).map(|id| *id).collect(); - Self { - pool: &mut network.pool, - peer_id, - connections, - } - } - - /// Obtains the next dialing connection, if any. - #[allow(clippy::should_implement_trait)] - pub fn next(&mut self) -> Option> { - let connection_id = self.connections.pop_front()?; - - let inner = self.pool.get_outgoing(connection_id)?; - - Some(DialingAttempt { - peer_id: *self.peer_id, - inner, - }) - } - - /// Returns the first connection, if any, consuming the iterator. - pub fn into_first<'b>(mut self) -> Option> - where - 'a: 'b, - { - let connection_id = self.connections.pop_front()?; - - let inner = self.pool.get_outgoing(connection_id)?; - - Some(DialingAttempt { - peer_id: *self.peer_id, - inner, - }) - } -} diff --git a/core/src/transport/upgrade.rs b/core/src/transport/upgrade.rs index 676e0989baa..3483b709d82 100644 --- a/core/src/transport/upgrade.rs +++ b/core/src/transport/upgrade.rs @@ -46,7 +46,7 @@ use std::{ }; /// A `Builder` facilitates upgrading of a [`Transport`] for use with -/// a [`Network`]. +/// a `Swarm`. /// /// The upgrade process is defined by the following stages: /// @@ -61,11 +61,9 @@ use std::{ /// and [multiplexed](Authenticated::multiplex). /// 2. Authentication must precede the negotiation of a multiplexer. /// 3. Applying a multiplexer is the last step in the upgrade process. -/// 4. The [`Transport::Output`] conforms to the requirements of a [`Network`], +/// 4. The [`Transport::Output`] conforms to the requirements of a `Swarm`, /// namely a tuple of a [`PeerId`] (from the authentication upgrade) and a /// [`StreamMuxer`] (from the multiplexing upgrade). -/// -/// [`Network`]: crate::Network #[derive(Clone)] pub struct Builder { inner: T, diff --git a/core/tests/concurrent_dialing.rs b/core/tests/concurrent_dialing.rs deleted file mode 100644 index 2da22a5d240..00000000000 --- a/core/tests/concurrent_dialing.rs +++ /dev/null @@ -1,171 +0,0 @@ -// Copyright 2021 Protocol Labs. -// -// Permission is hereby granted, free of charge, to any person obtaining a -// copy of this software and associated documentation files (the "Software"), -// to deal in the Software without restriction, including without limitation -// the rights to use, copy, modify, merge, publish, distribute, sublicense, -// and/or sell copies of the Software, and to permit persons to whom the -// Software is furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS -// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING -// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER -// DEALINGS IN THE SOFTWARE. - -mod util; - -use futures::executor::block_on; -use futures::future::poll_fn; -use futures::ready; -use libp2p_core::{ - multiaddr::Protocol, - network::{NetworkConfig, NetworkEvent}, - ConnectedPoint, DialOpts, -}; -use quickcheck::*; -use rand07::Rng; -use std::num::NonZeroU8; -use std::task::Poll; -use util::{test_network, TestHandler}; - -#[test] -fn concurrent_dialing() { - #[derive(Clone, Debug)] - struct DialConcurrencyFactor(NonZeroU8); - - impl Arbitrary for DialConcurrencyFactor { - fn arbitrary(g: &mut G) -> Self { - Self(NonZeroU8::new(g.gen_range(1, 11)).unwrap()) - } - } - - fn prop(concurrency_factor: DialConcurrencyFactor) { - block_on(async { - let mut network_1 = test_network(NetworkConfig::default()); - let mut network_2 = test_network( - NetworkConfig::default().with_dial_concurrency_factor(concurrency_factor.0), - ); - - // Listen on `concurrency_factor + 1` addresses. - // - // `+ 1` to ensure a subset of addresses is dialed by network_2. - let num_listen_addrs = concurrency_factor.0.get() + 2; - let mut network_1_listen_addresses = Vec::new(); - for _ in 0..num_listen_addrs { - network_1.listen_on("/memory/0".parse().unwrap()).unwrap(); - - poll_fn(|cx| match ready!(network_1.poll(cx)) { - NetworkEvent::NewListenerAddress { listen_addr, .. } => { - network_1_listen_addresses.push(listen_addr); - return Poll::Ready(()); - } - _ => panic!("Expected `NewListenerAddress` event."), - }) - .await; - } - - // Have network 2 dial network 1 and wait for network 1 to receive the incoming - // connections. - network_2 - .dial( - TestHandler(), - DialOpts::peer_id(*network_1.local_peer_id()) - .addresses(network_1_listen_addresses.clone().into()) - .build(), - ) - .unwrap(); - let mut network_1_incoming_connections = Vec::new(); - for i in 0..concurrency_factor.0.get() { - poll_fn(|cx| { - match network_2.poll(cx) { - Poll::Ready(e) => panic!("Unexpected event: {:?}", e), - Poll::Pending => {} - } - - match ready!(network_1.poll(cx)) { - NetworkEvent::IncomingConnection { connection, .. } => { - assert_eq!( - connection.local_addr, network_1_listen_addresses[i as usize], - "Expect network 2 to prioritize by address order." - ); - - network_1_incoming_connections.push(connection); - return Poll::Ready(()); - } - _ => panic!("Expected `NewListenerAddress` event."), - } - }) - .await; - } - - // Have network 1 accept the incoming connection and wait for network 1 and network 2 to - // report a shared established connection. - let accepted_addr = network_1_incoming_connections[0].local_addr.clone(); - network_1 - .accept(network_1_incoming_connections.remove(0), TestHandler()) - .unwrap(); - let mut network_1_connection_established = false; - let mut network_2_connection_established = false; - poll_fn(|cx| { - match network_2.poll(cx) { - Poll::Ready(NetworkEvent::ConnectionEstablished { - connection, - concurrent_dial_errors, - .. - }) => { - match connection.endpoint() { - ConnectedPoint::Dialer { address, .. } => { - assert_eq!( - *address, - accepted_addr - .clone() - .with(Protocol::P2p((*network_1.local_peer_id()).into())) - ) - } - ConnectedPoint::Listener { .. } => panic!("Expected dialer."), - } - assert!(concurrent_dial_errors.unwrap().is_empty()); - network_2_connection_established = true; - if network_1_connection_established { - return Poll::Ready(()); - } - } - Poll::Ready(e) => panic!("Expected `ConnectionEstablished` event: {:?}.", e), - Poll::Pending => {} - } - - match ready!(network_1.poll(cx)) { - NetworkEvent::ConnectionEstablished { - connection, - concurrent_dial_errors, - .. - } => { - match connection.endpoint() { - ConnectedPoint::Listener { local_addr, .. } => { - assert_eq!(*local_addr, accepted_addr) - } - ConnectedPoint::Dialer { .. } => panic!("Expected listener."), - } - assert!(concurrent_dial_errors.is_none()); - network_1_connection_established = true; - if network_2_connection_established { - return Poll::Ready(()); - } - } - e => panic!("Expected `ConnectionEstablished` event: {:?}.", e), - } - - Poll::Pending - }) - .await; - }) - } - - QuickCheck::new().quickcheck(prop as fn(_) -> _); -} diff --git a/core/tests/connection_limits.rs b/core/tests/connection_limits.rs deleted file mode 100644 index 0d0cb3883ee..00000000000 --- a/core/tests/connection_limits.rs +++ /dev/null @@ -1,220 +0,0 @@ -// Copyright 2020 Parity Technologies (UK) Ltd. -// -// Permission is hereby granted, free of charge, to any person obtaining a -// copy of this software and associated documentation files (the "Software"), -// to deal in the Software without restriction, including without limitation -// the rights to use, copy, modify, merge, publish, distribute, sublicense, -// and/or sell copies of the Software, and to permit persons to whom the -// Software is furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS -// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING -// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER -// DEALINGS IN THE SOFTWARE. - -mod util; - -use futures::{future::poll_fn, ready}; -use libp2p_core::multiaddr::{multiaddr, Multiaddr}; -use libp2p_core::{ - connection::PendingConnectionError, - network::{ConnectionLimits, DialError, NetworkConfig, NetworkEvent}, - DialOpts, PeerId, -}; -use quickcheck::*; -use std::task::Poll; -use util::{test_network, TestHandler}; - -#[test] -fn max_outgoing() { - use rand::Rng; - - let outgoing_limit = rand::thread_rng().gen_range(1..10); - - let limits = ConnectionLimits::default().with_max_pending_outgoing(Some(outgoing_limit)); - let cfg = NetworkConfig::default().with_connection_limits(limits); - let mut network = test_network(cfg); - - let addr: Multiaddr = "/memory/1234".parse().unwrap(); - - let target = PeerId::random(); - for _ in 0..outgoing_limit { - network - .dial( - TestHandler(), - DialOpts::peer_id(target) - .addresses(vec![addr.clone()]) - .build(), - ) - .ok() - .expect("Unexpected connection limit."); - } - - match network - .dial( - TestHandler(), - DialOpts::peer_id(target) - .addresses(vec![addr.clone()]) - .build(), - ) - .expect_err("Unexpected dialing success.") - { - DialError::ConnectionLimit { limit, handler: _ } => { - assert_eq!(limit.current, outgoing_limit); - assert_eq!(limit.limit, outgoing_limit); - } - e => panic!("Unexpected error: {:?}", e), - } - - let info = network.info(); - assert_eq!(info.num_peers(), 0); - assert_eq!( - info.connection_counters().num_pending_outgoing(), - outgoing_limit - ); - - // Abort all dialing attempts. - let mut peer = network - .peer(target.clone()) - .into_dialing() - .expect("Unexpected peer state"); - - let mut attempts = peer.attempts(); - while let Some(attempt) = attempts.next() { - attempt.abort(); - } - - assert_eq!( - network.info().connection_counters().num_pending_outgoing(), - 0 - ); -} - -#[test] -fn max_established_incoming() { - use rand07::Rng; - - #[derive(Debug, Clone)] - struct Limit(u32); - - impl Arbitrary for Limit { - fn arbitrary(g: &mut G) -> Self { - Self(g.gen_range(1, 10)) - } - } - - fn config(limit: u32) -> NetworkConfig { - let limits = ConnectionLimits::default().with_max_established_incoming(Some(limit)); - NetworkConfig::default().with_connection_limits(limits) - } - - fn prop(limit: Limit) { - let limit = limit.0; - - let mut network1 = test_network(config(limit)); - let mut network2 = test_network(config(limit)); - - let _ = network1.listen_on(multiaddr![Memory(0u64)]).unwrap(); - let listen_addr = - async_std::task::block_on(poll_fn(|cx| match ready!(network1.poll(cx)) { - NetworkEvent::NewListenerAddress { listen_addr, .. } => Poll::Ready(listen_addr), - e => panic!("Unexpected network event: {:?}", e), - })); - - // Spawn and block on the dialer. - async_std::task::block_on({ - let mut n = 0; - let _ = network2.dial(TestHandler(), listen_addr.clone()).unwrap(); - - let mut expected_closed = false; - let mut network_1_established = false; - let mut network_2_established = false; - let mut network_1_limit_reached = false; - let mut network_2_limit_reached = false; - poll_fn(move |cx| { - loop { - let mut network_1_pending = false; - let mut network_2_pending = false; - - match network1.poll(cx) { - Poll::Ready(NetworkEvent::IncomingConnection { connection, .. }) => { - network1.accept(connection, TestHandler()).unwrap(); - } - Poll::Ready(NetworkEvent::ConnectionEstablished { .. }) => { - network_1_established = true; - } - Poll::Ready(NetworkEvent::IncomingConnectionError { - error: PendingConnectionError::ConnectionLimit(err), - .. - }) => { - assert_eq!(err.limit, limit); - assert_eq!(err.limit, err.current); - let info = network1.info(); - let counters = info.connection_counters(); - assert_eq!(counters.num_established_incoming(), limit); - assert_eq!(counters.num_established(), limit); - network_1_limit_reached = true; - } - Poll::Pending => { - network_1_pending = true; - } - e => panic!("Unexpected network event: {:?}", e), - } - - match network2.poll(cx) { - Poll::Ready(NetworkEvent::ConnectionEstablished { .. }) => { - network_2_established = true; - } - Poll::Ready(NetworkEvent::ConnectionClosed { .. }) => { - assert!(expected_closed); - let info = network2.info(); - let counters = info.connection_counters(); - assert_eq!(counters.num_established_outgoing(), limit); - assert_eq!(counters.num_established(), limit); - network_2_limit_reached = true; - } - Poll::Pending => { - network_2_pending = true; - } - e => panic!("Unexpected network event: {:?}", e), - } - - if network_1_pending && network_2_pending { - return Poll::Pending; - } - - if network_1_established && network_2_established { - network_1_established = false; - network_2_established = false; - - if n <= limit { - // Dial again until the limit is exceeded. - n += 1; - network2.dial(TestHandler(), listen_addr.clone()).unwrap(); - - if n == limit { - // The the next dialing attempt exceeds the limit, this - // is the connection we expected to get closed. - expected_closed = true; - } - } else { - panic!("Expect networks not to establish connections beyond the limit.") - } - } - - if network_1_limit_reached && network_2_limit_reached { - return Poll::Ready(()); - } - } - }) - }); - } - - quickcheck(prop as fn(_)); -} diff --git a/core/tests/network_dial_error.rs b/core/tests/network_dial_error.rs deleted file mode 100644 index 19b9f9db4a3..00000000000 --- a/core/tests/network_dial_error.rs +++ /dev/null @@ -1,272 +0,0 @@ -// Copyright 2019 Parity Technologies (UK) Ltd. -// -// Permission is hereby granted, free of charge, to any person obtaining a -// copy of this software and associated documentation files (the "Software"), -// to deal in the Software without restriction, including without limitation -// the rights to use, copy, modify, merge, publish, distribute, sublicense, -// and/or sell copies of the Software, and to permit persons to whom the -// Software is furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS -// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING -// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER -// DEALINGS IN THE SOFTWARE. - -mod util; - -use futures::prelude::*; -use libp2p_core::multiaddr::multiaddr; -use libp2p_core::DialOpts; -use libp2p_core::{ - connection::PendingConnectionError, - multiaddr::Protocol, - network::{NetworkConfig, NetworkEvent}, - ConnectedPoint, Endpoint, PeerId, -}; -use rand::seq::SliceRandom; -use std::{io, task::Poll}; -use util::{test_network, TestHandler}; - -#[test] -fn deny_incoming_connec() { - // Checks whether refusing an incoming connection on a swarm triggers the correct events. - - let mut swarm1 = test_network(NetworkConfig::default()); - let mut swarm2 = test_network(NetworkConfig::default()); - - swarm1.listen_on("/memory/0".parse().unwrap()).unwrap(); - - let address = futures::executor::block_on(future::poll_fn(|cx| match swarm1.poll(cx) { - Poll::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) => { - Poll::Ready(listen_addr) - } - Poll::Pending => Poll::Pending, - _ => panic!("Was expecting the listen address to be reported"), - })); - - swarm2 - .dial( - TestHandler(), - DialOpts::peer_id(*swarm1.local_peer_id()) - .addresses(vec![address.clone()]) - .build(), - ) - .unwrap(); - - futures::executor::block_on(future::poll_fn(|cx| -> Poll> { - match swarm1.poll(cx) { - Poll::Ready(NetworkEvent::IncomingConnection { connection, .. }) => drop(connection), - Poll::Ready(_) => unreachable!(), - Poll::Pending => (), - } - - match swarm2.poll(cx) { - Poll::Ready(NetworkEvent::DialError { - peer_id, - error: PendingConnectionError::Transport(errors), - handler: _, - }) => { - assert_eq!(&peer_id, swarm1.local_peer_id()); - assert_eq!( - errors.get(0).expect("One error.").0, - address.clone().with(Protocol::P2p(peer_id.into())) - ); - return Poll::Ready(Ok(())); - } - Poll::Ready(_) => unreachable!(), - Poll::Pending => (), - } - - Poll::Pending - })) - .unwrap(); -} - -#[test] -fn invalid_peer_id() { - // Checks whether dialing an address containing the wrong peer id raises an error - // for the expected peer id instead of the obtained peer id. - - let mut swarm1 = test_network(NetworkConfig::default()); - let mut swarm2 = test_network(NetworkConfig::default()); - - swarm1.listen_on("/memory/0".parse().unwrap()).unwrap(); - - let address = futures::executor::block_on(future::poll_fn(|cx| match swarm1.poll(cx) { - Poll::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) => { - Poll::Ready(listen_addr) - } - Poll::Pending => Poll::Pending, - _ => panic!("Was expecting the listen address to be reported"), - })); - - let other_id = PeerId::random(); - let other_addr = address.with(Protocol::P2p(other_id.into())); - - swarm2.dial(TestHandler(), other_addr.clone()).unwrap(); - - let (peer_id, error) = futures::executor::block_on(future::poll_fn(|cx| { - if let Poll::Ready(NetworkEvent::IncomingConnection { connection, .. }) = swarm1.poll(cx) { - swarm1.accept(connection, TestHandler()).unwrap(); - } - - match swarm2.poll(cx) { - Poll::Ready(NetworkEvent::DialError { peer_id, error, .. }) => { - Poll::Ready((peer_id, error)) - } - Poll::Ready(x) => panic!("unexpected {:?}", x), - Poll::Pending => Poll::Pending, - } - })); - assert_eq!(peer_id, other_id); - match error { - PendingConnectionError::WrongPeerId { obtained, endpoint } => { - assert_eq!(obtained, *swarm1.local_peer_id()); - assert_eq!( - endpoint, - ConnectedPoint::Dialer { - address: other_addr, - role_override: Endpoint::Dialer, - } - ); - } - x => panic!("wrong error {:?}", x), - } -} - -#[test] -fn dial_self() { - // Check whether dialing ourselves correctly fails. - // - // Dialing the same address we're listening should result in three events: - // - // - The incoming connection notification (before we know the incoming peer ID). - // - The connection error for the dialing endpoint (once we've determined that it's our own ID). - // - The connection error for the listening endpoint (once we've determined that it's our own ID). - // - // The last two can happen in any order. - - let mut swarm = test_network(NetworkConfig::default()); - swarm.listen_on("/memory/0".parse().unwrap()).unwrap(); - - let local_address = futures::executor::block_on(future::poll_fn(|cx| match swarm.poll(cx) { - Poll::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) => { - Poll::Ready(listen_addr) - } - Poll::Pending => Poll::Pending, - _ => panic!("Was expecting the listen address to be reported"), - })); - - swarm.dial(TestHandler(), local_address.clone()).unwrap(); - - let mut got_dial_err = false; - let mut got_inc_err = false; - futures::executor::block_on(future::poll_fn(|cx| -> Poll> { - loop { - match swarm.poll(cx) { - Poll::Ready(NetworkEvent::DialError { - peer_id, - error: PendingConnectionError::WrongPeerId { .. }, - .. - }) => { - assert_eq!(&peer_id, swarm.local_peer_id()); - assert!(!got_dial_err); - got_dial_err = true; - if got_inc_err { - return Poll::Ready(Ok(())); - } - } - Poll::Ready(NetworkEvent::IncomingConnectionError { local_addr, .. }) => { - assert!(!got_inc_err); - assert_eq!(local_addr, local_address); - got_inc_err = true; - if got_dial_err { - return Poll::Ready(Ok(())); - } - } - Poll::Ready(NetworkEvent::IncomingConnection { connection, .. }) => { - assert_eq!(&connection.local_addr, &local_address); - swarm.accept(connection, TestHandler()).unwrap(); - } - Poll::Ready(ev) => { - panic!("Unexpected event: {:?}", ev) - } - Poll::Pending => break Poll::Pending, - } - } - })) - .unwrap(); -} - -#[test] -fn dial_self_by_id() { - // Trying to dial self by passing the same `PeerId` shouldn't even be possible in the first - // place. - let mut swarm = test_network(NetworkConfig::default()); - let peer_id = *swarm.local_peer_id(); - assert!(swarm.peer(peer_id).into_disconnected().is_none()); -} - -#[test] -fn multiple_addresses_err() { - // Tries dialing multiple addresses, and makes sure there's one dialing error per address. - - let target = PeerId::random(); - - let mut swarm = test_network(NetworkConfig::default()); - - let mut addresses = Vec::new(); - for _ in 0..3 { - addresses.push(multiaddr![Ip4([0, 0, 0, 0]), Tcp(rand::random::())]); - } - for _ in 0..5 { - addresses.push(multiaddr![Udp(rand::random::())]); - } - addresses.shuffle(&mut rand::thread_rng()); - - swarm - .dial( - TestHandler(), - DialOpts::peer_id(target) - .addresses(addresses.clone()) - .build(), - ) - .unwrap(); - - futures::executor::block_on(future::poll_fn(|cx| -> Poll> { - loop { - match swarm.poll(cx) { - Poll::Ready(NetworkEvent::DialError { - peer_id, - // multiaddr, - error: PendingConnectionError::Transport(errors), - handler: _, - }) => { - assert_eq!(peer_id, target); - - let failed_addresses = - errors.into_iter().map(|(addr, _)| addr).collect::>(); - assert_eq!( - failed_addresses, - addresses - .clone() - .into_iter() - .map(|addr| addr.with(Protocol::P2p(target.into()))) - .collect::>() - ); - - return Poll::Ready(Ok(())); - } - Poll::Ready(_) => unreachable!(), - Poll::Pending => break Poll::Pending, - } - } - })) - .unwrap(); -} diff --git a/core/tests/util.rs b/core/tests/util.rs index 9592daca9eb..64e366ecd99 100644 --- a/core/tests/util.rs +++ b/core/tests/util.rs @@ -1,66 +1,8 @@ #![allow(dead_code)] use futures::prelude::*; -use libp2p_core::{ - connection::{ConnectionHandler, ConnectionHandlerEvent, Substream, SubstreamEndpoint}, - identity, - muxing::{StreamMuxer, StreamMuxerBox}, - network::{Network, NetworkConfig}, - transport::{self, memory::MemoryTransport}, - upgrade, Multiaddr, PeerId, Transport, -}; -use libp2p_mplex as mplex; -use libp2p_noise as noise; -use std::{io, pin::Pin, task::Context, task::Poll}; - -type TestNetwork = Network; -type TestTransport = transport::Boxed<(PeerId, StreamMuxerBox)>; - -/// Creates a new `TestNetwork` with a TCP transport. -pub fn test_network(cfg: NetworkConfig) -> TestNetwork { - let local_key = identity::Keypair::generate_ed25519(); - let local_public_key = local_key.public(); - let noise_keys = noise::Keypair::::new() - .into_authentic(&local_key) - .unwrap(); - let transport: TestTransport = MemoryTransport::default() - .upgrade(upgrade::Version::V1) - .authenticate(noise::NoiseConfig::xx(noise_keys).into_authenticated()) - .multiplex(mplex::MplexConfig::new()) - .boxed(); - - TestNetwork::new(transport, local_public_key.into(), cfg) -} - -#[derive(Debug)] -pub struct TestHandler(); - -impl ConnectionHandler for TestHandler { - type InEvent = (); - type OutEvent = (); - type Error = io::Error; - type Substream = Substream; - type OutboundOpenInfo = (); - - fn inject_substream( - &mut self, - _: Self::Substream, - _: SubstreamEndpoint, - ) { - } - - fn inject_event(&mut self, _: Self::InEvent) {} - - fn inject_address_change(&mut self, _: &Multiaddr) {} - - fn poll( - &mut self, - _: &mut Context<'_>, - ) -> Poll, Self::Error>> - { - Poll::Pending - } -} +use libp2p_core::muxing::StreamMuxer; +use std::{pin::Pin, task::Context, task::Poll}; pub struct CloseMuxer { state: CloseMuxerState, diff --git a/misc/metrics/src/swarm.rs b/misc/metrics/src/swarm.rs index 2acaa9382a4..d0fb0c664f2 100644 --- a/misc/metrics/src/swarm.rs +++ b/misc/metrics/src/swarm.rs @@ -316,29 +316,27 @@ enum PendingInboundConnectionError { ConnectionLimit, } -impl From<&libp2p_core::connection::PendingInboundConnectionError> +impl From<&libp2p_swarm::PendingInboundConnectionError> for PendingInboundConnectionError { - fn from(error: &libp2p_core::connection::PendingInboundConnectionError) -> Self { + fn from(error: &libp2p_swarm::PendingInboundConnectionError) -> Self { match error { - libp2p_core::connection::PendingInboundConnectionError::WrongPeerId { .. } => { + libp2p_swarm::PendingInboundConnectionError::WrongPeerId { .. } => { PendingInboundConnectionError::WrongPeerId } - libp2p_core::connection::PendingInboundConnectionError::ConnectionLimit(_) => { + libp2p_swarm::PendingInboundConnectionError::ConnectionLimit(_) => { PendingInboundConnectionError::ConnectionLimit } - libp2p_core::connection::PendingInboundConnectionError::Transport( + libp2p_swarm::PendingInboundConnectionError::Transport( libp2p_core::transport::TransportError::MultiaddrNotSupported(_), ) => PendingInboundConnectionError::TransportErrorMultiaddrNotSupported, - libp2p_core::connection::PendingInboundConnectionError::Transport( + libp2p_swarm::PendingInboundConnectionError::Transport( libp2p_core::transport::TransportError::Other(_), ) => PendingInboundConnectionError::TransportErrorOther, - libp2p_core::connection::PendingInboundConnectionError::Aborted => { + libp2p_swarm::PendingInboundConnectionError::Aborted => { PendingInboundConnectionError::Aborted } - libp2p_core::connection::PendingInboundConnectionError::IO(_) => { - PendingInboundConnectionError::Io - } + libp2p_swarm::PendingInboundConnectionError::IO(_) => PendingInboundConnectionError::Io, } } } diff --git a/misc/multistream-select/Cargo.toml b/misc/multistream-select/Cargo.toml index fb4e41fcb35..a3845446fb4 100644 --- a/misc/multistream-select/Cargo.toml +++ b/misc/multistream-select/Cargo.toml @@ -22,6 +22,7 @@ unsigned-varint = "0.7" async-std = "1.6.2" env_logger = "0.9" libp2p-core = { path = "../../core", default-features = false } +libp2p-swarm = { path = "../../swarm", default-features = false } libp2p-mplex = { path = "../../muxers/mplex" } libp2p-plaintext = { path = "../../transports/plaintext" } quickcheck = "0.9.0" diff --git a/misc/multistream-select/tests/transport.rs b/misc/multistream-select/tests/transport.rs index 9fa197e12e2..cd4f2f87a76 100644 --- a/misc/multistream-select/tests/transport.rs +++ b/misc/multistream-select/tests/transport.rs @@ -20,24 +20,19 @@ use futures::{channel::oneshot, prelude::*, ready}; use libp2p_core::{ - connection::{ConnectionHandler, ConnectionHandlerEvent, Substream, SubstreamEndpoint}, identity, multiaddr::Protocol, muxing::StreamMuxerBox, - network::{NetworkConfig, NetworkEvent}, transport::{self, MemoryTransport}, - upgrade, Multiaddr, Network, PeerId, Transport, + upgrade, Multiaddr, PeerId, Transport, }; use libp2p_mplex::MplexConfig; use libp2p_plaintext::PlainText2Config; +use libp2p_swarm::{DummyBehaviour, Swarm, SwarmEvent}; use rand::random; -use std::{ - io, - task::{Context, Poll}, -}; +use std::task::Poll; type TestTransport = transport::Boxed<(PeerId, StreamMuxerBox)>; -type TestNetwork = Network; fn mk_transport(up: upgrade::Version) -> (PeerId, TestTransport) { let keys = identity::Keypair::generate_ed25519(); @@ -66,19 +61,18 @@ fn transport_upgrade() { let listen_addr = Multiaddr::from(Protocol::Memory(random::())); - let mut dialer = TestNetwork::new(dialer_transport, dialer_id, NetworkConfig::default()); - let mut listener = - TestNetwork::new(listener_transport, listener_id, NetworkConfig::default()); + let mut dialer = Swarm::new(dialer_transport, DummyBehaviour::default(), dialer_id); + let mut listener = Swarm::new(listener_transport, DummyBehaviour::default(), listener_id); listener.listen_on(listen_addr).unwrap(); let (addr_sender, addr_receiver) = oneshot::channel(); let client = async move { let addr = addr_receiver.await.unwrap(); - dialer.dial(TestHandler(), addr).unwrap(); + dialer.dial(addr).unwrap(); futures::future::poll_fn(move |cx| loop { - match ready!(dialer.poll(cx)) { - NetworkEvent::ConnectionEstablished { .. } => return Poll::Ready(()), + match ready!(dialer.poll_next_unpin(cx)).unwrap() { + SwarmEvent::ConnectionEstablished { .. } => return Poll::Ready(()), _ => {} } }) @@ -87,14 +81,12 @@ fn transport_upgrade() { let mut addr_sender = Some(addr_sender); let server = futures::future::poll_fn(move |cx| loop { - match ready!(listener.poll(cx)) { - NetworkEvent::NewListenerAddress { listen_addr, .. } => { - addr_sender.take().unwrap().send(listen_addr).unwrap(); - } - NetworkEvent::IncomingConnection { connection, .. } => { - listener.accept(connection, TestHandler()).unwrap(); + match ready!(listener.poll_next_unpin(cx)).unwrap() { + SwarmEvent::NewListenAddr { address, .. } => { + addr_sender.take().unwrap().send(address).unwrap(); } - NetworkEvent::ConnectionEstablished { .. } => return Poll::Ready(()), + SwarmEvent::IncomingConnection { .. } => {} + SwarmEvent::ConnectionEstablished { .. } => return Poll::Ready(()), _ => {} } }); @@ -105,33 +97,3 @@ fn transport_upgrade() { run(upgrade::Version::V1); run(upgrade::Version::V1Lazy); } - -#[derive(Debug)] -struct TestHandler(); - -impl ConnectionHandler for TestHandler { - type InEvent = (); - type OutEvent = (); - type Error = io::Error; - type Substream = Substream; - type OutboundOpenInfo = (); - - fn inject_substream( - &mut self, - _: Self::Substream, - _: SubstreamEndpoint, - ) { - } - - fn inject_event(&mut self, _: Self::InEvent) {} - - fn inject_address_change(&mut self, _: &Multiaddr) {} - - fn poll( - &mut self, - _: &mut Context<'_>, - ) -> Poll, Self::Error>> - { - Poll::Pending - } -} diff --git a/swarm/CHANGELOG.md b/swarm/CHANGELOG.md index a5714678e5e..1d44807865e 100644 --- a/swarm/CHANGELOG.md +++ b/swarm/CHANGELOG.md @@ -1,7 +1,11 @@ # 0.34.0 [unreleased] +- Fold `libp2p-core`'s `Network` into `Swarm`. See [PR 2492]. + - Update to `libp2p-core` `v0.32.0`. +[PR 2492]: https://github.com/libp2p/rust-libp2p/pull/2492 + # 0.33.0 [2022-01-27] - Patch reporting on banned peers and their non-banned and banned connections (see [PR 2350]). diff --git a/swarm/Cargo.toml b/swarm/Cargo.toml index 184acb95a38..c8542454455 100644 --- a/swarm/Cargo.toml +++ b/swarm/Cargo.toml @@ -12,16 +12,23 @@ categories = ["network-programming", "asynchronous"] [dependencies] either = "1.6.0" +fnv = "1.0" futures = "0.3.1" +futures-timer = "3.0.2" +instant = "0.1.11" libp2p-core = { version = "0.32.0", path = "../core", default-features = false } log = "0.4" +pin-project = "1.0.0" rand = "0.7" smallvec = "1.6.1" +thiserror = "1.0" void = "1" -futures-timer = "3.0.2" -instant = "0.1.11" [dev-dependencies] +async-std = { version = "1.6.2", features = ["attributes"] } libp2p = { path = "../", default-features = false, features = ["identify", "ping", "plaintext", "yamux"] } +libp2p-mplex = { path = "../muxers/mplex" } +libp2p-noise = { path = "../transports/noise" } +libp2p-tcp = { path = "../transports/tcp" } quickcheck = "0.9.0" rand = "0.7.2" diff --git a/swarm/src/connection.rs b/swarm/src/connection.rs new file mode 100644 index 00000000000..5b7f09a8f6b --- /dev/null +++ b/swarm/src/connection.rs @@ -0,0 +1,204 @@ +// Copyright 2020 Parity Technologies (UK) Ltd. +// +// Permission is hereby granted, free of charge, to any person obtaining a +// copy of this software and associated documentation files (the "Software"), +// to deal in the Software without restriction, including without limitation +// the rights to use, copy, modify, merge, publish, distribute, sublicense, +// and/or sell copies of the Software, and to permit persons to whom the +// Software is furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS +// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +// DEALINGS IN THE SOFTWARE. + +mod error; +pub(crate) mod handler; +mod listeners; +mod substream; + +pub(crate) mod pool; + +pub use error::{ + ConnectionError, PendingConnectionError, PendingInboundConnectionError, + PendingOutboundConnectionError, +}; +pub use handler::{ConnectionHandler, ConnectionHandlerEvent, IntoConnectionHandler}; +pub use listeners::{ListenersEvent, ListenersStream}; +pub use pool::{ConnectionCounters, ConnectionLimits}; +pub use pool::{EstablishedConnection, PendingConnection}; +pub use substream::{Close, Substream, SubstreamEndpoint}; + +use libp2p_core::connection::ConnectedPoint; +use libp2p_core::multiaddr::Multiaddr; +use libp2p_core::muxing::StreamMuxer; +use libp2p_core::PeerId; +use std::{error::Error, fmt, pin::Pin, task::Context, task::Poll}; +use substream::{Muxing, SubstreamEvent}; + +/// Information about a successfully established connection. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct Connected { + /// The connected endpoint, including network address information. + pub endpoint: ConnectedPoint, + /// Information obtained from the transport. + pub peer_id: PeerId, +} + +/// Event generated by a [`Connection`]. +#[derive(Debug, Clone)] +pub enum Event { + /// Event generated by the [`ConnectionHandler`]. + Handler(T), + /// Address of the remote has changed. + AddressChange(Multiaddr), +} + +/// A multiplexed connection to a peer with an associated `ConnectionHandler`. +pub struct Connection +where + TMuxer: StreamMuxer, + THandler: ConnectionHandler>, +{ + /// Node that handles the muxing. + muxing: substream::Muxing, + /// Handler that processes substreams. + handler: THandler, +} + +impl fmt::Debug for Connection +where + TMuxer: StreamMuxer, + THandler: ConnectionHandler> + fmt::Debug, +{ + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("Connection") + .field("muxing", &self.muxing) + .field("handler", &self.handler) + .finish() + } +} + +impl Unpin for Connection +where + TMuxer: StreamMuxer, + THandler: ConnectionHandler>, +{ +} + +impl Connection +where + TMuxer: StreamMuxer, + THandler: ConnectionHandler>, +{ + /// Builds a new `Connection` from the given substream multiplexer + /// and connection handler. + pub fn new(muxer: TMuxer, handler: THandler) -> Self { + Connection { + muxing: Muxing::new(muxer), + handler, + } + } + + /// Notifies the connection handler of an event. + pub fn inject_event(&mut self, event: THandler::InEvent) { + self.handler.inject_event(event); + } + + /// Begins an orderly shutdown of the connection, returning the connection + /// handler and a `Future` that resolves when connection shutdown is complete. + pub fn close(self) -> (THandler, Close) { + (self.handler, self.muxing.close().0) + } + + /// Polls the connection for events produced by the associated handler + /// as a result of I/O activity on the substream multiplexer. + pub fn poll( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll, ConnectionError>> { + loop { + let mut io_pending = false; + + // Perform I/O on the connection through the muxer, informing the handler + // of new substreams. + match self.muxing.poll(cx) { + Poll::Pending => io_pending = true, + Poll::Ready(Ok(SubstreamEvent::InboundSubstream { substream })) => self + .handler + .inject_substream(substream, SubstreamEndpoint::Listener), + Poll::Ready(Ok(SubstreamEvent::OutboundSubstream { + user_data, + substream, + })) => { + let endpoint = SubstreamEndpoint::Dialer(user_data); + self.handler.inject_substream(substream, endpoint) + } + Poll::Ready(Ok(SubstreamEvent::AddressChange(address))) => { + self.handler.inject_address_change(&address); + return Poll::Ready(Ok(Event::AddressChange(address))); + } + Poll::Ready(Err(err)) => return Poll::Ready(Err(ConnectionError::IO(err))), + } + + // Poll the handler for new events. + match self.handler.poll(cx) { + Poll::Pending => { + if io_pending { + return Poll::Pending; // Nothing to do + } + } + Poll::Ready(Ok(ConnectionHandlerEvent::OutboundSubstreamRequest(user_data))) => { + self.muxing.open_substream(user_data); + } + Poll::Ready(Ok(ConnectionHandlerEvent::Custom(event))) => { + return Poll::Ready(Ok(Event::Handler(event))); + } + Poll::Ready(Err(err)) => return Poll::Ready(Err(ConnectionError::Handler(err))), + } + } + } +} + +/// Borrowed information about an incoming connection currently being negotiated. +#[derive(Debug, Copy, Clone)] +pub struct IncomingInfo<'a> { + /// Local connection address. + pub local_addr: &'a Multiaddr, + /// Address used to send back data to the remote. + pub send_back_addr: &'a Multiaddr, +} + +impl<'a> IncomingInfo<'a> { + /// Builds the [`ConnectedPoint`] corresponding to the incoming connection. + pub fn to_connected_point(&self) -> ConnectedPoint { + ConnectedPoint::Listener { + local_addr: self.local_addr.clone(), + send_back_addr: self.send_back_addr.clone(), + } + } +} + +/// Information about a connection limit. +#[derive(Debug, Clone)] +pub struct ConnectionLimit { + /// The maximum number of connections. + pub limit: u32, + /// The current number of connections. + pub current: u32, +} + +impl fmt::Display for ConnectionLimit { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}/{}", self.current, self.limit) + } +} + +/// A `ConnectionLimit` can represent an error if it has been exceeded. +impl Error for ConnectionLimit {} diff --git a/core/src/connection/error.rs b/swarm/src/connection/error.rs similarity index 100% rename from core/src/connection/error.rs rename to swarm/src/connection/error.rs diff --git a/core/src/connection/handler.rs b/swarm/src/connection/handler.rs similarity index 78% rename from core/src/connection/handler.rs rename to swarm/src/connection/handler.rs index 011dcc2b61e..4802b39165f 100644 --- a/core/src/connection/handler.rs +++ b/swarm/src/connection/handler.rs @@ -34,7 +34,7 @@ pub trait ConnectionHandler { /// The outbound type of events that the handler emits to the `Network` /// through [`ConnectionHandler::poll`]. /// - /// See also [`NetworkEvent::ConnectionEvent`](crate::network::NetworkEvent::ConnectionEvent). + /// See also [`PoolEvent::ConnectionEvent`](crate::connection::pool::PoolEvent::ConnectionEvent). type OutEvent: Debug + Send + 'static; /// The type of errors that the handler can produce when polled by the `Network`. type Error: Debug + Send + 'static; @@ -113,32 +113,3 @@ pub enum ConnectionHandlerEvent { /// Other event. Custom(TCustom), } - -/// Event produced by a handler. -impl ConnectionHandlerEvent { - /// If this is `OutboundSubstreamRequest`, maps the content to something else. - pub fn map_outbound_open_info(self, map: F) -> ConnectionHandlerEvent - where - F: FnOnce(TOutboundOpenInfo) -> I, - { - match self { - ConnectionHandlerEvent::OutboundSubstreamRequest(val) => { - ConnectionHandlerEvent::OutboundSubstreamRequest(map(val)) - } - ConnectionHandlerEvent::Custom(val) => ConnectionHandlerEvent::Custom(val), - } - } - - /// If this is `Custom`, maps the content to something else. - pub fn map_custom(self, map: F) -> ConnectionHandlerEvent - where - F: FnOnce(TCustom) -> I, - { - match self { - ConnectionHandlerEvent::OutboundSubstreamRequest(val) => { - ConnectionHandlerEvent::OutboundSubstreamRequest(val) - } - ConnectionHandlerEvent::Custom(val) => ConnectionHandlerEvent::Custom(map(val)), - } - } -} diff --git a/core/src/connection/listeners.rs b/swarm/src/connection/listeners.rs similarity index 86% rename from core/src/connection/listeners.rs rename to swarm/src/connection/listeners.rs index 55a371b7f4c..8b4617afdbb 100644 --- a/core/src/connection/listeners.rs +++ b/swarm/src/connection/listeners.rs @@ -25,59 +25,22 @@ use crate::{ Multiaddr, Transport, }; use futures::{prelude::*, task::Context, task::Poll}; +use libp2p_core::connection::ListenerId; use log::debug; use smallvec::SmallVec; use std::{collections::VecDeque, fmt, mem, pin::Pin}; /// Implementation of `futures::Stream` that allows listening on multiaddresses. /// -/// To start using a `ListenersStream`, create one with `new` by passing an implementation of -/// `Transport`. This `Transport` will be used to start listening, therefore you want to pass -/// a `Transport` that supports the protocols you wish you listen on. +/// To start using a [`ListenersStream`], create one with [`ListenersStream::new`] by passing an +/// implementation of [`Transport`]. This [`Transport`] will be used to start listening, therefore +/// you want to pass a [`Transport`] that supports the protocols you wish you listen on. /// -/// Then, call `ListenerStream::listen_on` for all addresses you want to start listening on. +/// Then, call [`ListenersStream::listen_on`] for all addresses you want to start listening on. /// -/// The `ListenersStream` never ends and never produces errors. If a listener errors or closes, -/// an event is generated on the stream and the listener is then dropped, but the `ListenersStream` +/// The [`ListenersStream`] never ends and never produces errors. If a listener errors or closes, an +/// event is generated on the stream and the listener is then dropped, but the [`ListenersStream`] /// itself continues. -/// -/// # Example -/// -/// ```no_run -/// use futures::prelude::*; -/// use libp2p_core::connection::{ListenersEvent, ListenersStream}; -/// -/// let mut listeners = ListenersStream::new(libp2p_tcp::TcpConfig::new()); -/// -/// // Ask the `listeners` to start listening on the given multiaddress. -/// listeners.listen_on("/ip4/0.0.0.0/tcp/0".parse().unwrap()).unwrap(); -/// -/// // The `listeners` will now generate events when polled. -/// futures::executor::block_on(async move { -/// while let Some(event) = listeners.next().await { -/// match event { -/// ListenersEvent::NewAddress { listener_id, listen_addr } => { -/// println!("Listener {:?} is listening at address {}", listener_id, listen_addr); -/// }, -/// ListenersEvent::AddressExpired { listener_id, listen_addr } => { -/// println!("Listener {:?} is no longer listening at address {}", listener_id, listen_addr); -/// }, -/// ListenersEvent::Closed { listener_id, .. } => { -/// println!("Listener {:?} has been closed", listener_id); -/// }, -/// ListenersEvent::Error { listener_id, error } => { -/// println!("Listener {:?} has experienced an error: {}", listener_id, error); -/// }, -/// ListenersEvent::Incoming { listener_id, upgrade, local_addr, .. } => { -/// println!("Listener {:?} has a new connection on {}", listener_id, local_addr); -/// // We don't do anything with the newly-opened connection, but in a real-life -/// // program you probably want to use it! -/// drop(upgrade); -/// }, -/// } -/// } -/// }) -/// ``` pub struct ListenersStream where TTrans: Transport, @@ -94,13 +57,6 @@ where pending_events: VecDeque>, } -/// The ID of a single listener. -/// -/// It is part of most [`ListenersEvent`]s and can be used to remove -/// individual listeners from the [`ListenersStream`]. -#[derive(Copy, Clone, Debug, PartialEq, Eq, Hash, PartialOrd, Ord)] -pub struct ListenerId(u64); - /// A single active listener. #[pin_project::pin_project] #[derive(Debug)] @@ -178,18 +134,7 @@ where ListenersStream { transport, listeners: VecDeque::new(), - next_id: ListenerId(1), - pending_events: VecDeque::new(), - } - } - - /// Same as `new`, but pre-allocates enough memory for the given number of - /// simultaneous listeners. - pub fn with_capacity(transport: TTrans, capacity: usize) -> Self { - ListenersStream { - transport, - listeners: VecDeque::with_capacity(capacity), - next_id: ListenerId(1), + next_id: ListenerId::new(1), pending_events: VecDeque::new(), } } @@ -211,7 +156,7 @@ where addresses: SmallVec::new(), })); let id = self.next_id; - self.next_id = ListenerId(self.next_id.0 + 1); + self.next_id = self.next_id + 1; Ok(id) } diff --git a/core/src/connection/pool.rs b/swarm/src/connection/pool.rs similarity index 90% rename from core/src/connection/pool.rs rename to swarm/src/connection/pool.rs index 32ca7aa0eaf..3cabc7b0a15 100644 --- a/core/src/connection/pool.rs +++ b/swarm/src/connection/pool.rs @@ -22,12 +22,10 @@ use crate::{ connection::{ handler::{THandlerError, THandlerInEvent, THandlerOutEvent}, - Connected, ConnectionError, ConnectionHandler, ConnectionId, ConnectionLimit, Endpoint, - IncomingInfo, IntoConnectionHandler, PendingConnectionError, PendingInboundConnectionError, - PendingOutboundConnectionError, PendingPoint, Substream, + Connected, ConnectionError, ConnectionHandler, ConnectionLimit, IncomingInfo, + IntoConnectionHandler, PendingConnectionError, PendingInboundConnectionError, + PendingOutboundConnectionError, Substream, }, - muxing::StreamMuxer, - network::DialError, transport::{Transport, TransportError}, ConnectedPoint, Executor, Multiaddr, PeerId, }; @@ -40,12 +38,13 @@ use futures::{ ready, stream::FuturesUnordered, }; -use smallvec::SmallVec; +use libp2p_core::connection::{ConnectionId, Endpoint, PendingPoint}; +use libp2p_core::muxing::StreamMuxer; use std::{ collections::{hash_map, HashMap}, convert::TryFrom as _, fmt, - num::NonZeroU8, + num::{NonZeroU8, NonZeroUsize}, pin::Pin, task::Context, task::Poll, @@ -319,7 +318,7 @@ where counters: ConnectionCounters::new(limits), established: Default::default(), pending: Default::default(), - next_connection_id: ConnectionId(0), + next_connection_id: ConnectionId::new(0), task_command_buffer_size: config.task_command_buffer_size, dial_concurrency_factor: config.dial_concurrency_factor, executor: config.executor, @@ -368,22 +367,11 @@ where } } - /// Gets a pending outgoing connection by ID. - pub fn get_outgoing(&mut self, id: ConnectionId) -> Option> { - match self.pending.entry(id) { - hash_map::Entry::Occupied(entry) => Some(PendingConnection { - entry, - counters: &mut self.counters, - }), - hash_map::Entry::Vacant(_) => None, - } - } - /// Returns true if we are connected to the given peer. /// /// This will return true only after a `NodeReached` event has been produced by `poll()`. - pub fn is_connected(&self, id: &PeerId) -> bool { - self.established.contains_key(id) + pub fn is_connected(&self, id: PeerId) -> bool { + self.established.contains_key(&id) } /// Returns the number of connected peers, i.e. those with at least one @@ -397,8 +385,8 @@ where /// All connections to the peer, whether pending or established are /// closed asap and no more events from these connections are emitted /// by the pool effective immediately. - pub fn disconnect(&mut self, peer: &PeerId) { - if let Some(conns) = self.established.get_mut(peer) { + pub fn disconnect(&mut self, peer: PeerId) { + if let Some(conns) = self.established.get_mut(&peer) { for (_, conn) in conns.iter_mut() { conn.start_close(); } @@ -408,7 +396,7 @@ where let pending_connections = self .pending .iter() - .filter(|(_, PendingConnectionInfo { peer_id, .. })| peer_id.as_ref() == Some(peer)) + .filter(|(_, PendingConnectionInfo { peer_id, .. })| peer_id.as_ref() == Some(&peer)) .map(|(id, _)| *id) .collect::>(); @@ -426,56 +414,13 @@ where } } - /// Counts the number of established connections to the given peer. - pub fn num_peer_established(&self, peer: PeerId) -> u32 { - num_peer_established(&self.established, peer) - } - /// Returns an iterator over all established connections of `peer`. - pub fn iter_peer_established<'a>( - &'a mut self, - peer: &PeerId, - ) -> EstablishedConnectionIter<'a, impl Iterator, THandlerInEvent> - { - let ids = self - .iter_peer_established_info(peer) - .map(|(id, _endpoint)| *id) - .collect::>() - .into_iter(); - - EstablishedConnectionIter { - connections: self.established.get_mut(peer), - ids, - } - } - - /// Returns an iterator for information on all pending incoming connections. - pub fn iter_pending_incoming(&self) -> impl Iterator> { - self.iter_pending_info() - .filter_map(|(_, ref endpoint, _)| match endpoint { - PendingPoint::Listener { - local_addr, - send_back_addr, - } => Some(IncomingInfo { - local_addr, - send_back_addr, - }), - PendingPoint::Dialer { .. } => None, - }) - } - - /// Returns an iterator over all connection IDs and associated endpoints - /// of established connections to `peer` known to the pool. - pub fn iter_peer_established_info( - &self, + pub fn iter_established_connections_of_peer( + &mut self, peer: &PeerId, - ) -> impl Iterator { + ) -> impl Iterator + '_ { match self.established.get(peer) { - Some(conns) => either::Either::Left( - conns - .iter() - .map(|(id, EstablishedConnectionInfo { endpoint, .. })| (id, endpoint)), - ), + Some(conns) => either::Either::Left(conns.iter().map(|(id, _)| *id)), None => either::Either::Right(std::iter::empty()), } } @@ -503,7 +448,7 @@ where fn next_connection_id(&mut self) -> ConnectionId { let connection_id = self.next_connection_id; - self.next_connection_id.0 += 1; + self.next_connection_id = self.next_connection_id + 1; connection_id } @@ -537,13 +482,13 @@ where handler: THandler, role_override: Endpoint, dial_concurrency_factor_override: Option, - ) -> Result> + ) -> Result where TTrans: Clone + Send, TTrans::Dial: Send + 'static, { if let Err(limit) = self.counters.check_max_pending_outgoing() { - return Err(DialError::ConnectionLimit { limit, handler }); + return Err((limit, handler)); }; let dial = ConcurrentDial::new( @@ -970,21 +915,6 @@ pub struct PendingConnection<'a, THandler: IntoConnectionHandler> { } impl PendingConnection<'_, THandler> { - /// Returns the local connection ID. - pub fn id(&self) -> ConnectionId { - *self.entry.key() - } - - /// Returns the (expected) identity of the remote peer, if known. - pub fn peer_id(&self) -> &Option { - &self.entry.get().peer_id - } - - /// Returns information about this endpoint of the connection. - pub fn endpoint(&self) -> &PendingPoint { - &self.entry.get().endpoint - } - /// Aborts the connection attempt, closing the connection. pub fn abort(self) { self.counters.dec_pending(&self.entry.get().endpoint); @@ -1064,55 +994,6 @@ impl EstablishedConnection<'_, TInEvent> { } } -/// An iterator over established connections in a pool. -pub struct EstablishedConnectionIter<'a, I, TInEvent> { - connections: Option<&'a mut FnvHashMap>>, - ids: I, -} - -// Note: Ideally this would be an implementation of `Iterator`, but that -// requires GATs (cf. https://github.com/rust-lang/rust/issues/44265) and -// a different definition of `Iterator`. -impl<'a, I, TInEvent> EstablishedConnectionIter<'a, I, TInEvent> -where - I: Iterator, -{ - /// Obtains the next connection, if any. - #[allow(clippy::should_implement_trait)] - pub fn next(&mut self) -> Option> { - if let (Some(id), Some(connections)) = (self.ids.next(), self.connections.as_mut()) { - Some(EstablishedConnection { - entry: connections - .entry(id) - .expect_occupied("Established entry not found in pool."), - }) - } else { - None - } - } - - /// Turns the iterator into an iterator over just the connection IDs. - pub fn into_ids(self) -> impl Iterator { - self.ids - } - - /// Returns the first connection, if any, consuming the iterator. - pub fn into_first<'b>(mut self) -> Option> - where - 'a: 'b, - { - if let (Some(id), Some(connections)) = (self.ids.next(), self.connections) { - Some(EstablishedConnection { - entry: connections - .entry(id) - .expect_occupied("Established entry not found in pool."), - }) - } else { - None - } - } -} - /// Network connection information. #[derive(Debug, Clone)] pub struct ConnectionCounters { @@ -1363,6 +1244,53 @@ impl Default for PoolConfig { } } +impl PoolConfig { + /// Configures the executor to use for spawning connection background tasks. + pub fn with_executor(mut self, e: Box) -> Self { + self.executor = Some(e); + self + } + + /// Configures the executor to use for spawning connection background tasks, + /// only if no executor has already been configured. + pub fn or_else_with_executor(mut self, f: F) -> Self + where + F: FnOnce() -> Option>, + { + self.executor = self.executor.or_else(f); + self + } + + /// Sets the maximum number of events sent to a connection's background task + /// that may be buffered, if the task cannot keep up with their consumption and + /// delivery to the connection handler. + /// + /// When the buffer for a particular connection is full, `notify_handler` will no + /// longer be able to deliver events to the associated `ConnectionHandler`, + /// thus exerting back-pressure on the connection and peer API. + pub fn with_notify_handler_buffer_size(mut self, n: NonZeroUsize) -> Self { + self.task_command_buffer_size = n.get() - 1; + self + } + + /// Sets the maximum number of buffered connection events (beyond a guaranteed + /// buffer of 1 event per connection). + /// + /// When the buffer is full, the background tasks of all connections will stall. + /// In this way, the consumers of network events exert back-pressure on + /// the network connection I/O. + pub fn with_connection_event_buffer_size(mut self, n: usize) -> Self { + self.task_event_buffer_size = n; + self + } + + /// Number of addresses concurrently dialed for a single outbound connection attempt. + pub fn with_dial_concurrency_factor(mut self, factor: NonZeroU8) -> Self { + self.dial_concurrency_factor = factor; + self + } +} + trait EntryExt<'a, K, V> { fn expect_occupied(self, msg: &'static str) -> hash_map::OccupiedEntry<'a, K, V>; } @@ -1375,3 +1303,24 @@ impl<'a, K: 'a, V: 'a> EntryExt<'a, K, V> for hash_map::Entry<'a, K, V> { } } } + +#[cfg(test)] +mod tests { + use super::*; + use futures::future::Future; + + struct Dummy; + + impl Executor for Dummy { + fn exec(&self, _: Pin + Send>>) {} + } + + #[test] + fn set_executor() { + PoolConfig::default() + .with_executor(Box::new(Dummy)) + .with_executor(Box::new(|f| { + async_std::task::spawn(f); + })); + } +} diff --git a/core/src/connection/pool/concurrent_dial.rs b/swarm/src/connection/pool/concurrent_dial.rs similarity index 96% rename from core/src/connection/pool/concurrent_dial.rs rename to swarm/src/connection/pool/concurrent_dial.rs index 6c10ca1b19d..194f18c8e5e 100644 --- a/core/src/connection/pool/concurrent_dial.rs +++ b/swarm/src/connection/pool/concurrent_dial.rs @@ -19,7 +19,6 @@ // DEALINGS IN THE SOFTWARE. use crate::{ - connection::Endpoint, transport::{Transport, TransportError}, Multiaddr, PeerId, }; @@ -28,6 +27,8 @@ use futures::{ ready, stream::{FuturesUnordered, StreamExt}, }; +use libp2p_core::connection::Endpoint; +use libp2p_core::multiaddr::Protocol; use std::{ num::NonZeroU8, pin::Pin, @@ -153,12 +154,12 @@ fn p2p_addr(peer: Option, addr: Multiaddr) -> Result return Ok(addr), }; - if let Some(multiaddr::Protocol::P2p(hash)) = addr.iter().last() { + if let Some(Protocol::P2p(hash)) = addr.iter().last() { if &hash != peer.as_ref() { return Err(addr); } Ok(addr) } else { - Ok(addr.with(multiaddr::Protocol::P2p(peer.into()))) + Ok(addr.with(Protocol::P2p(peer.into()))) } } diff --git a/core/src/connection/pool/task.rs b/swarm/src/connection/pool/task.rs similarity index 97% rename from core/src/connection/pool/task.rs rename to swarm/src/connection/pool/task.rs index 9062583fd79..1c1065d1fa5 100644 --- a/core/src/connection/pool/task.rs +++ b/swarm/src/connection/pool/task.rs @@ -26,10 +26,9 @@ use crate::{ connection::{ self, handler::{THandlerError, THandlerInEvent, THandlerOutEvent}, - ConnectionError, ConnectionHandler, ConnectionId, IntoConnectionHandler, - PendingInboundConnectionError, PendingOutboundConnectionError, Substream, + ConnectionError, ConnectionHandler, IntoConnectionHandler, PendingInboundConnectionError, + PendingOutboundConnectionError, Substream, }, - muxing::StreamMuxer, transport::{Transport, TransportError}, Multiaddr, PeerId, }; @@ -38,6 +37,8 @@ use futures::{ future::{poll_fn, Either, Future}, SinkExt, StreamExt, }; +use libp2p_core::connection::ConnectionId; +use libp2p_core::muxing::StreamMuxer; use std::pin::Pin; use void::Void; diff --git a/core/src/connection/substream.rs b/swarm/src/connection/substream.rs similarity index 94% rename from core/src/connection/substream.rs rename to swarm/src/connection/substream.rs index 399b09b9f0a..426f64d9f60 100644 --- a/core/src/connection/substream.rs +++ b/swarm/src/connection/substream.rs @@ -18,9 +18,9 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::muxing::{substream_from_ref, StreamMuxer, StreamMuxerEvent, SubstreamRef}; use futures::prelude::*; -use multiaddr::Multiaddr; +use libp2p_core::multiaddr::Multiaddr; +use libp2p_core::muxing::{substream_from_ref, StreamMuxer, StreamMuxerEvent, SubstreamRef}; use smallvec::SmallVec; use std::sync::Arc; use std::{fmt, io::Error as IoError, pin::Pin, task::Context, task::Poll}; @@ -32,24 +32,6 @@ pub enum SubstreamEndpoint { Listener, } -impl SubstreamEndpoint { - /// Returns true for `Dialer`. - pub fn is_dialer(&self) -> bool { - match self { - SubstreamEndpoint::Dialer(_) => true, - SubstreamEndpoint::Listener => false, - } - } - - /// Returns true for `Listener`. - pub fn is_listener(&self) -> bool { - match self { - SubstreamEndpoint::Dialer(_) => false, - SubstreamEndpoint::Listener => true, - } - } -} - /// Implementation of `Stream` that handles substream multiplexing. /// /// The stream will receive substreams and can be used to open new outgoing substreams. Destroying diff --git a/swarm/src/dial_opts.rs b/swarm/src/dial_opts.rs index b443d8f420b..ac6b771ced8 100644 --- a/swarm/src/dial_opts.rs +++ b/swarm/src/dial_opts.rs @@ -122,7 +122,7 @@ impl WithPeerId { } /// Override - /// [`NetworkConfig::with_dial_concurrency_factor`](libp2p_core::network::NetworkConfig::with_dial_concurrency_factor). + /// [`PoolConfig::with_dial_concurrency_factor`](crate::connection::pool::PoolConfig::with_dial_concurrency_factor). pub fn override_dial_concurrency_factor(mut self, factor: NonZeroU8) -> Self { self.dial_concurrency_factor_override = Some(factor); self @@ -196,7 +196,7 @@ impl WithPeerIdWithAddresses { } /// Override - /// [`NetworkConfig::with_dial_concurrency_factor`](libp2p_core::network::NetworkConfig::with_dial_concurrency_factor). + /// [`PoolConfig::with_dial_concurrency_factor`](crate::connection::pool::PoolConfig::with_dial_concurrency_factor). pub fn override_dial_concurrency_factor(mut self, factor: NonZeroU8) -> Self { self.dial_concurrency_factor_override = Some(factor); self diff --git a/swarm/src/lib.rs b/swarm/src/lib.rs index 533e15e6a24..2d495e8d8b1 100644 --- a/swarm/src/lib.rs +++ b/swarm/src/lib.rs @@ -53,6 +53,7 @@ //! are supported, when to open a new outbound substream, etc. //! +mod connection; mod registry; #[cfg(test)] mod test; @@ -66,6 +67,7 @@ pub use behaviour::{ CloseConnection, NetworkBehaviour, NetworkBehaviourAction, NetworkBehaviourEventProcess, NotifyHandler, PollParameters, }; +pub use connection::{PendingConnectionError, PendingInboundConnectionError}; pub use protocols_handler::{ IntoProtocolsHandler, IntoProtocolsHandlerSelect, KeepAlive, OneShotHandler, OneShotHandlerConfig, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerSelect, @@ -73,20 +75,22 @@ pub use protocols_handler::{ }; pub use registry::{AddAddressResult, AddressRecord, AddressScore}; +use crate::connection::IncomingInfo; +use crate::connection::{pool::PoolEvent, ListenersEvent, ListenersStream}; +use connection::pool::{ConnectionCounters, ConnectionLimits, Pool, PoolConfig}; +use connection::{ + ConnectionError, ConnectionHandler, ConnectionLimit, EstablishedConnection, + IntoConnectionHandler, PendingOutboundConnectionError, Substream, +}; use dial_opts::{DialOpts, PeerCondition}; +use either::Either; use futures::{executor::ThreadPoolBuilder, prelude::*, stream::FusedStream}; +use libp2p_core::connection::{ConnectionId, PendingPoint}; use libp2p_core::{ - connection::{ - ConnectedPoint, ConnectionError, ConnectionHandler, ConnectionId, ConnectionLimit, - EstablishedConnection, IntoConnectionHandler, ListenerId, PendingConnectionError, - PendingInboundConnectionError, PendingOutboundConnectionError, Substream, - }, + connection::{ConnectedPoint, ListenerId}, + multiaddr::Protocol, multihash::Multihash, muxing::StreamMuxerBox, - network::{ - self, peer::ConnectedPeer, ConnectionLimits, Network, NetworkConfig, NetworkEvent, - NetworkInfo, - }, transport::{self, TransportError}, upgrade::ProtocolName, Executor, Multiaddr, Negotiated, PeerId, Transport, @@ -95,6 +99,7 @@ use protocols_handler::{NodeHandlerWrapperBuilder, NodeHandlerWrapperError}; use registry::{AddressIntoIter, Addresses}; use smallvec::SmallVec; use std::collections::HashSet; +use std::iter; use std::num::{NonZeroU32, NonZeroU8, NonZeroUsize}; use std::{ convert::TryFrom, @@ -255,11 +260,18 @@ pub struct Swarm where TBehaviour: NetworkBehaviour, { - network: Network< - transport::Boxed<(PeerId, StreamMuxerBox)>, + /// Listeners for incoming connections. + listeners: ListenersStream>, + + /// The nodes currently active. + pool: Pool< NodeHandlerWrapperBuilder>, + transport::Boxed<(PeerId, StreamMuxerBox)>, >, + /// The local peer ID. + local_peer_id: PeerId, + /// Handles which nodes to connect to and how to handle the events sent back by the protocol /// handlers. behaviour: TBehaviour, @@ -307,9 +319,14 @@ where SwarmBuilder::new(transport, behaviour, local_peer_id).build() } - /// Returns information about the [`Network`] underlying the `Swarm`. + /// Returns information about the connections underlying the [`Swarm`]. pub fn network_info(&self) -> NetworkInfo { - self.network.info() + let num_peers = self.pool.num_peers(); + let connection_counters = self.pool.counters().clone(); + NetworkInfo { + num_peers, + connection_counters, + } } /// Starts listening on the given address. @@ -318,7 +335,7 @@ where /// Listeners report their new listening addresses as [`SwarmEvent::NewListenAddr`]. /// Depending on the underlying transport, one listener may have multiple listening addresses. pub fn listen_on(&mut self, addr: Multiaddr) -> Result> { - let id = self.network.listen_on(addr)?; + let id = self.listeners.listen_on(addr)?; self.behaviour.inject_new_listener(id); Ok(id) } @@ -328,7 +345,7 @@ where /// Returns `true` if there was a listener with this ID, `false` /// otherwise. pub fn remove_listener(&mut self, id: ListenerId) -> bool { - self.network.remove_listener(id) + self.listeners.remove_listener(id) } /// Dial a known or unknown peer. @@ -364,112 +381,151 @@ where swarm_dial_opts: DialOpts, handler: ::ProtocolsHandler, ) -> Result<(), DialError> { - let core_dial_opts = match swarm_dial_opts.0 { - // Dial a known peer. - dial_opts::Opts::WithPeerId(dial_opts::WithPeerId { - peer_id, - condition, - role_override, - dial_concurrency_factor_override, - }) - | dial_opts::Opts::WithPeerIdWithAddresses(dial_opts::WithPeerIdWithAddresses { - peer_id, - condition, - role_override, - dial_concurrency_factor_override, - .. - }) => { - // Check [`PeerCondition`] if provided. - let condition_matched = match condition { - PeerCondition::Disconnected => self.network.is_disconnected(&peer_id), - PeerCondition::NotDialing => !self.network.is_dialing(&peer_id), - PeerCondition::Always => true, - }; - if !condition_matched { - self.behaviour.inject_dial_failure( - Some(peer_id), - handler, - &DialError::DialPeerConditionFalse(condition), - ); - - return Err(DialError::DialPeerConditionFalse(condition)); - } - - // Check if peer is banned. - if self.banned_peers.contains(&peer_id) { - let error = DialError::Banned; - self.behaviour - .inject_dial_failure(Some(peer_id), handler, &error); - return Err(error); - } - - // Retrieve the addresses to dial. - let addresses = { - let mut addresses = match swarm_dial_opts.0 { - dial_opts::Opts::WithPeerId(dial_opts::WithPeerId { .. }) => { - self.behaviour.addresses_of_peer(&peer_id) - } - dial_opts::Opts::WithPeerIdWithAddresses( - dial_opts::WithPeerIdWithAddresses { - peer_id, - mut addresses, - extend_addresses_through_behaviour, - .. - }, - ) => { - if extend_addresses_through_behaviour { - addresses.extend(self.behaviour.addresses_of_peer(&peer_id)) - } - addresses - } - dial_opts::Opts::WithoutPeerIdWithAddress { .. } => { - unreachable!("Due to outer match.") + let (peer_id, addresses, dial_concurrency_factor_override, role_override) = + match swarm_dial_opts.0 { + // Dial a known peer. + dial_opts::Opts::WithPeerId(dial_opts::WithPeerId { + peer_id, + condition, + role_override, + dial_concurrency_factor_override, + }) + | dial_opts::Opts::WithPeerIdWithAddresses(dial_opts::WithPeerIdWithAddresses { + peer_id, + condition, + role_override, + dial_concurrency_factor_override, + .. + }) => { + // Check [`PeerCondition`] if provided. + let condition_matched = match condition { + PeerCondition::Disconnected => !self.is_connected(&peer_id), + PeerCondition::NotDialing => { + !self + .pool + .iter_pending_info() + .any(move |(_, endpoint, peer)| { + matches!(endpoint, PendingPoint::Dialer { .. }) + && peer.as_ref() == Some(&peer_id) + }) } + PeerCondition::Always => true, }; + if !condition_matched { + self.behaviour.inject_dial_failure( + Some(peer_id), + handler, + &DialError::DialPeerConditionFalse(condition), + ); - let mut unique_addresses = HashSet::new(); - addresses.retain(|a| { - !self.listened_addrs.contains(a) && unique_addresses.insert(a.clone()) - }); + return Err(DialError::DialPeerConditionFalse(condition)); + } - if addresses.is_empty() { - let error = DialError::NoAddresses; + // Check if peer is banned. + if self.banned_peers.contains(&peer_id) { + let error = DialError::Banned; self.behaviour .inject_dial_failure(Some(peer_id), handler, &error); return Err(error); - }; + } - addresses - }; + // Retrieve the addresses to dial. + let addresses = { + let mut addresses = match swarm_dial_opts.0 { + dial_opts::Opts::WithPeerId(dial_opts::WithPeerId { .. }) => { + self.behaviour.addresses_of_peer(&peer_id) + } + dial_opts::Opts::WithPeerIdWithAddresses( + dial_opts::WithPeerIdWithAddresses { + peer_id, + mut addresses, + extend_addresses_through_behaviour, + .. + }, + ) => { + if extend_addresses_through_behaviour { + addresses.extend(self.behaviour.addresses_of_peer(&peer_id)) + } + addresses + } + dial_opts::Opts::WithoutPeerIdWithAddress { .. } => { + unreachable!("Due to outer match.") + } + }; + + let mut unique_addresses = HashSet::new(); + addresses.retain(|a| { + !self.listened_addrs.contains(a) && unique_addresses.insert(a.clone()) + }); - let mut opts = libp2p_core::DialOpts::peer_id(peer_id) - .addresses(addresses) - .override_role(role_override); + if addresses.is_empty() { + let error = DialError::NoAddresses; + self.behaviour + .inject_dial_failure(Some(peer_id), handler, &error); + return Err(error); + }; - if let Some(f) = dial_concurrency_factor_override { - opts = opts.override_dial_concurrency_factor(f); + addresses + }; + + ( + Some(peer_id), + Either::Left(addresses.into_iter()), + dial_concurrency_factor_override, + role_override, + ) } + // Dial an unknown peer. + dial_opts::Opts::WithoutPeerIdWithAddress( + dial_opts::WithoutPeerIdWithAddress { + address, + role_override, + }, + ) => { + // If the address ultimately encapsulates an expected peer ID, dial that peer + // such that any mismatch is detected. We do not "pop off" the `P2p` protocol + // from the address, because it may be used by the `Transport`, i.e. `P2p` + // is a protocol component that can influence any transport, like `libp2p-dns`. + let peer_id = match address + .iter() + .last() + .and_then(|p| { + if let Protocol::P2p(ma) = p { + Some(PeerId::try_from(ma)) + } else { + None + } + }) + .transpose() + { + Ok(peer_id) => peer_id, + Err(multihash) => return Err(DialError::InvalidPeerId(multihash)), + }; - opts.build() - } - // Dial an unknown peer. - dial_opts::Opts::WithoutPeerIdWithAddress(dial_opts::WithoutPeerIdWithAddress { - address, - role_override, - }) => libp2p_core::DialOpts::unknown_peer_id() - .address(address) - .override_role(role_override) - .build(), - }; + ( + peer_id, + Either::Right(iter::once(address)), + None, + role_override, + ) + } + }; let handler = handler .into_node_handler_builder() .with_substream_upgrade_protocol_override(self.substream_upgrade_protocol_override); - match self.network.dial(handler, core_dial_opts).map(|_id| ()) { + match self.pool.add_outgoing( + self.listeners.transport().clone(), + addresses, + peer_id, + handler, + role_override, + dial_concurrency_factor_override, + ) { Ok(_connection_id) => Ok(()), - Err(error) => { - let (error, handler) = DialError::from_network_dial_error(error); + Err((connection_limit, handler)) => { + let error = DialError::ConnectionLimit(connection_limit); self.behaviour .inject_dial_failure(None, handler.into_protocols_handler(), &error); return Err(error); @@ -479,12 +535,12 @@ where /// Returns an iterator that produces the list of addresses we're listening on. pub fn listeners(&self) -> impl Iterator { - self.network.listen_addrs() + self.listeners.listen_addrs() } /// Returns the peer ID of the swarm passed as parameter. pub fn local_peer_id(&self) -> &PeerId { - self.network.local_peer_id() + &self.local_peer_id } /// Returns an iterator for [`AddressRecord`]s of external addresses @@ -544,13 +600,11 @@ where /// This function has no effect if the peer is already banned. pub fn ban_peer_id(&mut self, peer_id: PeerId) { if self.banned_peers.insert(peer_id) { - if let Some(peer) = self.network.peer(peer_id).into_connected() { - // Note that established connections to the now banned peer are closed but not - // added to [`Swarm::banned_peer_connections`]. They have been previously reported - // as open to the behaviour and need be reported as closed once closing the - // connection finishes. - peer.disconnect(); - } + // Note that established connections to the now banned peer are closed but not + // added to [`Swarm::banned_peer_connections`]. They have been previously reported + // as open to the behaviour and need be reported as closed once closing the + // connection finishes. + self.pool.disconnect(peer_id); } } @@ -570,22 +624,22 @@ where /// with [`ProtocolsHandler::connection_keep_alive`] or directly with /// [`ProtocolsHandlerEvent::Close`]. pub fn disconnect_peer_id(&mut self, peer_id: PeerId) -> Result<(), ()> { - if let Some(peer) = self.network.peer(peer_id).into_connected() { - peer.disconnect(); + if self.pool.is_connected(peer_id) { + self.pool.disconnect(peer_id); return Ok(()); } Err(()) } - /// Checks whether the [`Network`] has an established connection to a peer. + /// Checks whether there is an established connection to a peer. pub fn is_connected(&self, peer_id: &PeerId) -> bool { - self.network.is_connected(peer_id) + self.pool.is_connected(*peer_id) } /// Returns the currently connected peers. pub fn connected_peers(&self) -> impl Iterator { - self.network.connected_peers() + self.pool.iter_connected() } /// Returns a reference to the provided [`NetworkBehaviour`]. @@ -610,37 +664,117 @@ where let this = &mut *self; loop { - let mut network_not_ready = false; + let mut listeners_not_ready = false; + let mut connections_not_ready = false; - // First let the network make progress. - match this.network.poll(cx) { - Poll::Pending => network_not_ready = true, - Poll::Ready(NetworkEvent::ConnectionEvent { connection, event }) => { - let peer = connection.peer_id(); - let conn_id = connection.id(); - if this.banned_peer_connections.contains(&conn_id) { - log::debug!("Ignoring event from banned peer: {} {:?}.", peer, conn_id); - } else { - this.behaviour.inject_event(peer, conn_id, event); - } + // Poll the listener(s) for new connections. + match ListenersStream::poll(Pin::new(&mut this.listeners), cx) { + Poll::Pending => { + listeners_not_ready = true; } - Poll::Ready(NetworkEvent::AddressChange { - connection, - new_endpoint, - old_endpoint, + Poll::Ready(ListenersEvent::Incoming { + listener_id: _, + upgrade, + local_addr, + send_back_addr, }) => { - let peer = connection.peer_id(); - let conn_id = connection.id(); - if !this.banned_peer_connections.contains(&conn_id) { - this.behaviour.inject_address_change( - &peer, - &conn_id, - &old_endpoint, - &new_endpoint, + let handler = this + .behaviour + .new_handler() + .into_node_handler_builder() + .with_substream_upgrade_protocol_override( + this.substream_upgrade_protocol_override, ); + match this.pool.add_incoming( + upgrade, + handler, + IncomingInfo { + local_addr: &local_addr, + send_back_addr: &send_back_addr, + }, + ) { + Ok(_connection_id) => { + return Poll::Ready(SwarmEvent::IncomingConnection { + local_addr, + send_back_addr, + }); + } + Err((connection_limit, handler)) => { + this.behaviour.inject_listen_failure( + &local_addr, + &send_back_addr, + handler.into_protocols_handler(), + ); + log::warn!("Incoming connection rejected: {:?}", connection_limit); + } + }; + } + Poll::Ready(ListenersEvent::NewAddress { + listener_id, + listen_addr, + }) => { + log::debug!("Listener {:?}; New address: {:?}", listener_id, listen_addr); + if !this.listened_addrs.contains(&listen_addr) { + this.listened_addrs.push(listen_addr.clone()) + } + this.behaviour + .inject_new_listen_addr(listener_id, &listen_addr); + return Poll::Ready(SwarmEvent::NewListenAddr { + listener_id, + address: listen_addr, + }); + } + Poll::Ready(ListenersEvent::AddressExpired { + listener_id, + listen_addr, + }) => { + log::debug!( + "Listener {:?}; Expired address {:?}.", + listener_id, + listen_addr + ); + this.listened_addrs.retain(|a| a != &listen_addr); + this.behaviour + .inject_expired_listen_addr(listener_id, &listen_addr); + return Poll::Ready(SwarmEvent::ExpiredListenAddr { + listener_id, + address: listen_addr, + }); + } + Poll::Ready(ListenersEvent::Closed { + listener_id, + addresses, + reason, + }) => { + log::debug!("Listener {:?}; Closed by {:?}.", listener_id, reason); + for addr in addresses.iter() { + this.behaviour.inject_expired_listen_addr(listener_id, addr); } + this.behaviour.inject_listener_closed( + listener_id, + match &reason { + Ok(()) => Ok(()), + Err(err) => Err(err), + }, + ); + return Poll::Ready(SwarmEvent::ListenerClosed { + listener_id, + addresses, + reason, + }); + } + Poll::Ready(ListenersEvent::Error { listener_id, error }) => { + this.behaviour.inject_listener_error(listener_id, &error); + return Poll::Ready(SwarmEvent::ListenerError { listener_id, error }); } - Poll::Ready(NetworkEvent::ConnectionEstablished { + } + + // Poll the known peers. + match this.pool.poll(cx) { + Poll::Pending => { + connections_not_ready = true; + } + Poll::Ready(PoolEvent::ConnectionEstablished { connection, other_established_connection_ids, concurrent_dial_errors, @@ -651,11 +785,7 @@ where // Mark the connection for the banned peer as banned, thus withholding any // future events from the connection to the behaviour. this.banned_peer_connections.insert(connection.id()); - this.network - .peer(peer_id) - .into_connected() - .expect("the Network just notified us that we were connected; QED") - .disconnect(); + this.pool.disconnect(peer_id); return Poll::Ready(SwarmEvent::BannedPeer { peer_id, endpoint }); } else { let num_established = NonZeroU32::new( @@ -693,12 +823,57 @@ where }); } } - Poll::Ready(NetworkEvent::ConnectionClosed { + Poll::Ready(PoolEvent::PendingOutboundConnectionError { + id: _, + error, + handler, + peer, + }) => { + let error = error.into(); + + this.behaviour.inject_dial_failure( + peer, + handler.into_protocols_handler(), + &error, + ); + + if let Some(peer) = peer { + log::debug!("Connection attempt to {:?} failed with {:?}.", peer, error,); + } else { + log::debug!("Connection attempt to unknown peer failed with {:?}", error); + } + + return Poll::Ready(SwarmEvent::OutgoingConnectionError { + peer_id: peer, + error, + }); + } + Poll::Ready(PoolEvent::PendingInboundConnectionError { + id: _, + send_back_addr, + local_addr, + error, + handler, + }) => { + log::debug!("Incoming connection failed: {:?}", error); + this.behaviour.inject_listen_failure( + &local_addr, + &send_back_addr, + handler.into_protocols_handler(), + ); + return Poll::Ready(SwarmEvent::IncomingConnectionError { + local_addr, + send_back_addr, + error, + }); + } + Poll::Ready(PoolEvent::ConnectionClosed { id, connected, error, remaining_established_connection_ids, handler, + .. }) => { if let Some(error) = error.as_ref() { log::debug!( @@ -739,147 +914,32 @@ where num_established, }); } - Poll::Ready(NetworkEvent::IncomingConnection { connection, .. }) => { - let handler = this - .behaviour - .new_handler() - .into_node_handler_builder() - .with_substream_upgrade_protocol_override( - this.substream_upgrade_protocol_override, - ); - let local_addr = connection.local_addr.clone(); - let send_back_addr = connection.send_back_addr.clone(); - match this.network.accept(connection, handler) { - Ok(_connection_id) => { - return Poll::Ready(SwarmEvent::IncomingConnection { - local_addr, - send_back_addr, - }); - } - Err((connection_limit, handler)) => { - this.behaviour.inject_listen_failure( - &local_addr, - &send_back_addr, - handler.into_protocols_handler(), - ); - log::warn!("Incoming connection rejected: {:?}", connection_limit); - } - } - } - Poll::Ready(NetworkEvent::NewListenerAddress { - listener_id, - listen_addr, - }) => { - log::debug!("Listener {:?}; New address: {:?}", listener_id, listen_addr); - if !this.listened_addrs.contains(&listen_addr) { - this.listened_addrs.push(listen_addr.clone()) + Poll::Ready(PoolEvent::ConnectionEvent { connection, event }) => { + let peer = connection.peer_id(); + let conn_id = connection.id(); + if this.banned_peer_connections.contains(&conn_id) { + log::debug!("Ignoring event from banned peer: {} {:?}.", peer, conn_id); + } else { + this.behaviour.inject_event(peer, conn_id, event); } - this.behaviour - .inject_new_listen_addr(listener_id, &listen_addr); - return Poll::Ready(SwarmEvent::NewListenAddr { - listener_id, - address: listen_addr, - }); - } - Poll::Ready(NetworkEvent::ExpiredListenerAddress { - listener_id, - listen_addr, - }) => { - log::debug!( - "Listener {:?}; Expired address {:?}.", - listener_id, - listen_addr - ); - this.listened_addrs.retain(|a| a != &listen_addr); - this.behaviour - .inject_expired_listen_addr(listener_id, &listen_addr); - return Poll::Ready(SwarmEvent::ExpiredListenAddr { - listener_id, - address: listen_addr, - }); } - Poll::Ready(NetworkEvent::ListenerClosed { - listener_id, - addresses, - reason, + Poll::Ready(PoolEvent::AddressChange { + connection, + new_endpoint, + old_endpoint, }) => { - log::debug!("Listener {:?}; Closed by {:?}.", listener_id, reason); - for addr in addresses.iter() { - this.behaviour.inject_expired_listen_addr(listener_id, addr); + let peer = connection.peer_id(); + let conn_id = connection.id(); + if !this.banned_peer_connections.contains(&conn_id) { + this.behaviour.inject_address_change( + &peer, + &conn_id, + &old_endpoint, + &new_endpoint, + ); } - this.behaviour.inject_listener_closed( - listener_id, - match &reason { - Ok(()) => Ok(()), - Err(err) => Err(err), - }, - ); - return Poll::Ready(SwarmEvent::ListenerClosed { - listener_id, - addresses, - reason, - }); - } - Poll::Ready(NetworkEvent::ListenerError { listener_id, error }) => { - this.behaviour.inject_listener_error(listener_id, &error); - return Poll::Ready(SwarmEvent::ListenerError { listener_id, error }); - } - Poll::Ready(NetworkEvent::IncomingConnectionError { - local_addr, - send_back_addr, - error, - handler, - }) => { - log::debug!("Incoming connection failed: {:?}", error); - this.behaviour.inject_listen_failure( - &local_addr, - &send_back_addr, - handler.into_protocols_handler(), - ); - return Poll::Ready(SwarmEvent::IncomingConnectionError { - local_addr, - send_back_addr, - error, - }); - } - Poll::Ready(NetworkEvent::DialError { - peer_id, - error, - handler, - }) => { - let error = error.into(); - - this.behaviour.inject_dial_failure( - Some(peer_id), - handler.into_protocols_handler(), - &error, - ); - - log::debug!( - "Connection attempt to {:?} failed with {:?}.", - peer_id, - error, - ); - - return Poll::Ready(SwarmEvent::OutgoingConnectionError { - peer_id: Some(peer_id), - error, - }); } - Poll::Ready(NetworkEvent::UnknownPeerDialError { error, handler }) => { - log::debug!("Connection attempt to unknown peer failed with {:?}", error); - let error = error.into(); - this.behaviour.inject_dial_failure( - None, - handler.into_protocols_handler(), - &error, - ); - return Poll::Ready(SwarmEvent::OutgoingConnectionError { - peer_id: None, - error: error, - }); - } - } + }; // After the network had a chance to make progress, try to deliver // the pending event emitted by the behaviour in the previous iteration @@ -887,27 +947,12 @@ where // before polling the behaviour again. If the targeted peer // meanwhie disconnected, the event is discarded. if let Some((peer_id, handler, event)) = this.pending_event.take() { - if let Some(mut peer) = this.network.peer(peer_id).into_connected() { - match handler { - PendingNotifyHandler::One(conn_id) => { - if let Some(mut conn) = peer.connection(conn_id) { - if let Some(event) = notify_one(&mut conn, event, cx) { - this.pending_event = Some((peer_id, handler, event)); - if network_not_ready { - return Poll::Pending; - } else { - continue; - } - } - } - } - PendingNotifyHandler::Any(ids) => { - if let Some((event, ids)) = - notify_any::<_, _, TBehaviour>(ids, &mut peer, event, cx) - { - let handler = PendingNotifyHandler::Any(ids); + match handler { + PendingNotifyHandler::One(conn_id) => { + if let Some(mut conn) = this.pool.get_established(conn_id) { + if let Some(event) = notify_one(&mut conn, event, cx) { this.pending_event = Some((peer_id, handler, event)); - if network_not_ready { + if listeners_not_ready && connections_not_ready { return Poll::Pending; } else { continue; @@ -915,6 +960,19 @@ where } } } + PendingNotifyHandler::Any(ids) => { + if let Some((event, ids)) = + notify_any::<_, _, TBehaviour>(ids, &mut this.pool, event, cx) + { + let handler = PendingNotifyHandler::Any(ids); + this.pending_event = Some((peer_id, handler, event)); + if listeners_not_ready && connections_not_ready { + return Poll::Pending; + } else { + continue; + } + } + } } } @@ -922,7 +980,7 @@ where let behaviour_poll = { let mut parameters = SwarmPollParameters { - local_peer_id: &mut this.network.local_peer_id(), + local_peer_id: &this.local_peer_id, supported_protocols: &this.supported_protocols, listened_addrs: &this.listened_addrs, external_addrs: &this.external_addrs, @@ -931,7 +989,9 @@ where }; match behaviour_poll { - Poll::Pending if network_not_ready => return Poll::Pending, + Poll::Pending if listeners_not_ready && connections_not_ready => { + return Poll::Pending + } Poll::Pending => (), Poll::Ready(NetworkBehaviourAction::GenerateEvent(event)) => { return Poll::Ready(SwarmEvent::Behaviour(event)) @@ -948,61 +1008,83 @@ where peer_id, handler, event, - }) => { - if let Some(mut peer) = this.network.peer(peer_id).into_connected() { - match handler { - NotifyHandler::One(connection) => { - if let Some(mut conn) = peer.connection(connection) { - if let Some(event) = notify_one(&mut conn, event, cx) { - let handler = PendingNotifyHandler::One(connection); - this.pending_event = Some((peer_id, handler, event)); - if network_not_ready { - return Poll::Pending; - } else { - continue; - } - } + }) => match handler { + NotifyHandler::One(connection) => { + if let Some(mut conn) = this.pool.get_established(connection) { + if let Some(event) = notify_one(&mut conn, event, cx) { + let handler = PendingNotifyHandler::One(connection); + this.pending_event = Some((peer_id, handler, event)); + if listeners_not_ready && connections_not_ready { + return Poll::Pending; + } else { + continue; } } - NotifyHandler::Any => { - let ids = peer.connections().into_ids().collect(); - if let Some((event, ids)) = - notify_any::<_, _, TBehaviour>(ids, &mut peer, event, cx) - { - let handler = PendingNotifyHandler::Any(ids); - this.pending_event = Some((peer_id, handler, event)); - if network_not_ready { - return Poll::Pending; - } else { - continue; - } - } + } + } + NotifyHandler::Any => { + let ids = this + .pool + .iter_established_connections_of_peer(&peer_id) + .collect(); + if let Some((event, ids)) = + notify_any::<_, _, TBehaviour>(ids, &mut this.pool, event, cx) + { + let handler = PendingNotifyHandler::Any(ids); + this.pending_event = Some((peer_id, handler, event)); + if listeners_not_ready && connections_not_ready { + return Poll::Pending; + } else { + continue; } } } - } + }, Poll::Ready(NetworkBehaviourAction::ReportObservedAddr { address, score }) => { - for addr in this.network.address_translation(&address) { + // Maps the given `observed_addr`, representing an address of the local + // node observed by a remote peer, onto the locally known listen addresses + // to yield one or more addresses of the local node that may be publicly + // reachable. + // + // I.e. this method incorporates the view of other peers into the listen + // addresses seen by the local node to account for possible IP and port + // mappings performed by intermediate network devices in an effort to + // obtain addresses for the local peer that are also reachable for peers + // other than the peer who reported the `observed_addr`. + // + // The translation is transport-specific. See [`Transport::address_translation`]. + let translated_addresses = { + let transport = this.listeners.transport(); + let mut addrs: Vec<_> = this + .listeners + .listen_addrs() + .filter_map(move |server| { + transport.address_translation(server, &address) + }) + .collect(); + + // remove duplicates + addrs.sort_unstable(); + addrs.dedup(); + addrs + }; + for addr in translated_addresses { this.add_external_address(addr, score); } } Poll::Ready(NetworkBehaviourAction::CloseConnection { peer_id, connection, - }) => { - if let Some(mut peer) = this.network.peer(peer_id).into_connected() { - match connection { - CloseConnection::One(connection_id) => { - if let Some(conn) = peer.connection(connection_id) { - conn.start_close(); - } - } - CloseConnection::All => { - peer.disconnect(); - } + }) => match connection { + CloseConnection::One(connection_id) => { + if let Some(conn) = this.pool.get_established(connection_id) { + conn.start_close(); } } - } + CloseConnection::All => { + this.pool.disconnect(peer_id); + } + }, } } } @@ -1053,9 +1135,9 @@ fn notify_one<'a, THandlerInEvent>( /// /// Returns `None` if either all connections are closing or the event /// was successfully sent to a handler, in either case the event is consumed. -fn notify_any<'a, TTrans, THandler, TBehaviour>( +fn notify_any( ids: SmallVec<[ConnectionId; 10]>, - peer: &mut ConnectedPeer<'a, TTrans, THandler>, + pool: &mut Pool, event: THandlerInEvent, cx: &mut Context<'_>, ) -> Option<(THandlerInEvent, SmallVec<[ConnectionId; 10]>)> @@ -1072,7 +1154,7 @@ where let mut pending = SmallVec::new(); let mut event = Some(event); // (1) for id in ids.into_iter() { - if let Some(mut conn) = peer.connection(id) { + if let Some(mut conn) = pool.get_established(id) { match conn.poll_ready_notify_handler(cx) { Poll::Pending => pending.push(id), Poll::Ready(Err(())) => {} // connection is closing @@ -1156,13 +1238,13 @@ impl<'a> PollParameters for SwarmPollParameters<'a> { } } -/// A `SwarmBuilder` provides an API for configuring and constructing a `Swarm`, -/// including the underlying [`Network`]. +/// A [`SwarmBuilder`] provides an API for configuring and constructing a [`Swarm`]. pub struct SwarmBuilder { local_peer_id: PeerId, transport: transport::Boxed<(PeerId, StreamMuxerBox)>, behaviour: TBehaviour, - network_config: NetworkConfig, + pool_config: PoolConfig, + connection_limits: ConnectionLimits, substream_upgrade_protocol_override: Option, } @@ -1182,7 +1264,8 @@ where local_peer_id, transport, behaviour, - network_config: Default::default(), + pool_config: Default::default(), + connection_limits: Default::default(), substream_upgrade_protocol_override: None, } } @@ -1192,7 +1275,7 @@ where /// By default, unless another executor has been configured, /// [`SwarmBuilder::build`] will try to set up a `ThreadPool`. pub fn executor(mut self, e: Box) -> Self { - self.network_config = self.network_config.with_executor(e); + self.pool_config = self.pool_config.with_executor(e); self } @@ -1206,7 +1289,7 @@ where /// be sleeping more often than necessary. Increasing this value increases /// the overall memory usage. pub fn notify_handler_buffer_size(mut self, n: NonZeroUsize) -> Self { - self.network_config = self.network_config.with_notify_handler_buffer_size(n); + self.pool_config = self.pool_config.with_notify_handler_buffer_size(n); self } @@ -1234,19 +1317,19 @@ where /// event is emitted and the moment when it is received by the /// [`NetworkBehaviour`]. pub fn connection_event_buffer_size(mut self, n: usize) -> Self { - self.network_config = self.network_config.with_connection_event_buffer_size(n); + self.pool_config = self.pool_config.with_connection_event_buffer_size(n); self } /// Number of addresses concurrently dialed for a single outbound connection attempt. pub fn dial_concurrency_factor(mut self, factor: NonZeroU8) -> Self { - self.network_config = self.network_config.with_dial_concurrency_factor(factor); + self.pool_config = self.pool_config.with_dial_concurrency_factor(factor); self } /// Configures the connection limits. pub fn connection_limits(mut self, limits: ConnectionLimits) -> Self { - self.network_config = self.network_config.with_connection_limits(limits); + self.connection_limits = limits; self } @@ -1277,8 +1360,8 @@ where .collect(); // If no executor has been explicitly configured, try to set up a thread pool. - let network_cfg = - self.network_config.or_else_with_executor(|| { + let pool_config = + self.pool_config.or_else_with_executor(|| { match ThreadPoolBuilder::new() .name_prefix("libp2p-swarm-task-") .create() @@ -1291,10 +1374,10 @@ where } }); - let network = Network::new(self.transport, self.local_peer_id, network_cfg); - Swarm { - network, + local_peer_id: self.local_peer_id, + listeners: ListenersStream::new(self.transport), + pool: Pool::new(self.local_peer_id, pool_config, self.connection_limits), behaviour: self.behaviour, supported_protocols, listened_addrs: SmallVec::new(), @@ -1338,20 +1421,6 @@ pub enum DialError { Transport(Vec<(Multiaddr, TransportError)>), } -impl DialError { - fn from_network_dial_error(error: network::DialError) -> (Self, THandler) { - match error { - network::DialError::ConnectionLimit { limit, handler } => { - (DialError::ConnectionLimit(limit), handler) - } - network::DialError::LocalPeerId { handler } => (DialError::LocalPeerId, handler), - network::DialError::InvalidPeerId { handler, multihash } => { - (DialError::InvalidPeerId(multihash), handler) - } - } - } -} - impl From> for DialError { fn from(error: PendingOutboundConnectionError) -> Self { match error { @@ -1464,15 +1533,46 @@ impl NetworkBehaviour for DummyBehaviour { } } +/// Information about the connections obtained by [`Swarm::network_info()`]. +#[derive(Clone, Debug)] +pub struct NetworkInfo { + /// The total number of connected peers. + num_peers: usize, + /// Counters of ongoing network connections. + connection_counters: ConnectionCounters, +} + +impl NetworkInfo { + /// The number of connected peers, i.e. peers with whom at least + /// one established connection exists. + pub fn num_peers(&self) -> usize { + self.num_peers + } + + /// Gets counters for ongoing network connections. + pub fn connection_counters(&self) -> &ConnectionCounters { + &self.connection_counters + } +} + #[cfg(test)] mod tests { use super::*; use crate::protocols_handler::DummyProtocolsHandler; use crate::test::{CallTraceBehaviour, MockBehaviour}; - use futures::{executor, future}; + use futures::executor::block_on; + use futures::future::poll_fn; + use futures::future::Either; + use futures::{executor, future, ready}; use libp2p::core::{identity, multiaddr, transport, upgrade}; use libp2p::plaintext; use libp2p::yamux; + use libp2p_core::multiaddr::multiaddr; + use libp2p_core::transport::ListenerEvent; + use libp2p_core::Endpoint; + use quickcheck::{quickcheck, Arbitrary, Gen, QuickCheck}; + use rand::prelude::SliceRandom; + use rand::Rng; // Test execution state. // Connection => Disconnecting => Connecting. @@ -1481,7 +1581,9 @@ mod tests { Disconnecting, } - fn new_test_swarm(handler_proto: T) -> Swarm>> + fn new_test_swarm( + handler_proto: T, + ) -> SwarmBuilder>> where T: ProtocolsHandler + Clone, T::OutEvent: Clone, @@ -1497,7 +1599,7 @@ mod tests { .multiplex(yamux::YamuxConfig::default()) .boxed(); let behaviour = CallTraceBehaviour::new(MockBehaviour::new(handler_proto)); - SwarmBuilder::new(transport, behaviour, local_public_key.into()).build() + SwarmBuilder::new(transport, behaviour, local_public_key.into()) } fn swarms_connected( @@ -1551,8 +1653,8 @@ mod tests { keep_alive: KeepAlive::Yes, }; - let mut swarm1 = new_test_swarm::<_, ()>(handler_proto.clone()); - let mut swarm2 = new_test_swarm::<_, ()>(handler_proto); + let mut swarm1 = new_test_swarm::<_, ()>(handler_proto.clone()).build(); + let mut swarm2 = new_test_swarm::<_, ()>(handler_proto).build(); let addr1: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); let addr2: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); @@ -1672,8 +1774,8 @@ mod tests { keep_alive: KeepAlive::Yes, }; - let mut swarm1 = new_test_swarm::<_, ()>(handler_proto.clone()); - let mut swarm2 = new_test_swarm::<_, ()>(handler_proto); + let mut swarm1 = new_test_swarm::<_, ()>(handler_proto.clone()).build(); + let mut swarm2 = new_test_swarm::<_, ()>(handler_proto).build(); let addr1: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); let addr2: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); @@ -1741,8 +1843,8 @@ mod tests { keep_alive: KeepAlive::Yes, }; - let mut swarm1 = new_test_swarm::<_, ()>(handler_proto.clone()); - let mut swarm2 = new_test_swarm::<_, ()>(handler_proto); + let mut swarm1 = new_test_swarm::<_, ()>(handler_proto.clone()).build(); + let mut swarm2 = new_test_swarm::<_, ()>(handler_proto).build(); let addr1: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); let addr2: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); @@ -1813,8 +1915,8 @@ mod tests { keep_alive: KeepAlive::Yes, }; - let mut swarm1 = new_test_swarm::<_, ()>(handler_proto.clone()); - let mut swarm2 = new_test_swarm::<_, ()>(handler_proto); + let mut swarm1 = new_test_swarm::<_, ()>(handler_proto.clone()).build(); + let mut swarm2 = new_test_swarm::<_, ()>(handler_proto).build(); let addr1: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); let addr2: Multiaddr = multiaddr::Protocol::Memory(rand::random::()).into(); @@ -1882,4 +1984,442 @@ mod tests { } })) } + + #[test] + fn concurrent_dialing() { + #[derive(Clone, Debug)] + struct DialConcurrencyFactor(NonZeroU8); + + impl Arbitrary for DialConcurrencyFactor { + fn arbitrary(g: &mut G) -> Self { + Self(NonZeroU8::new(g.gen_range(1, 11)).unwrap()) + } + } + + fn prop(concurrency_factor: DialConcurrencyFactor) { + block_on(async { + let mut swarm = new_test_swarm::<_, ()>(DummyProtocolsHandler { + keep_alive: KeepAlive::Yes, + }) + .dial_concurrency_factor(concurrency_factor.0) + .build(); + + // Listen on `concurrency_factor + 1` addresses. + // + // `+ 2` to ensure a subset of addresses is dialed by network_2. + let num_listen_addrs = concurrency_factor.0.get() + 2; + let mut listen_addresses = Vec::new(); + let mut listeners = Vec::new(); + for _ in 0..num_listen_addrs { + let mut listener = transport::MemoryTransport {} + .listen_on("/memory/0".parse().unwrap()) + .unwrap(); + + match listener.next().await.unwrap().unwrap() { + ListenerEvent::NewAddress(address) => { + listen_addresses.push(address); + } + _ => panic!("Expected `NewListenAddr` event."), + } + + listeners.push(listener); + } + + // Have swarm dial each listener and wait for each listener to receive the incoming + // connections. + swarm + .dial( + DialOpts::peer_id(PeerId::random()) + .addresses(listen_addresses.into()) + .build(), + ) + .unwrap(); + for mut listener in listeners.into_iter() { + loop { + match futures::future::select(listener.next(), swarm.next()).await { + Either::Left((Some(Ok(ListenerEvent::Upgrade { .. })), _)) => { + break; + } + Either::Left(_) => { + panic!("Unexpected listener event.") + } + Either::Right((e, _)) => { + panic!("Expect swarm to not emit any event {:?}", e) + } + } + } + } + + match swarm.next().await.unwrap() { + SwarmEvent::OutgoingConnectionError { .. } => {} + e => panic!("Unexpected swarm event {:?}", e), + } + }) + } + + QuickCheck::new().tests(10).quickcheck(prop as fn(_) -> _); + } + + #[test] + fn max_outgoing() { + use rand::Rng; + + let outgoing_limit = rand::thread_rng().gen_range(1, 10); + + let limits = ConnectionLimits::default().with_max_pending_outgoing(Some(outgoing_limit)); + let mut network = new_test_swarm::<_, ()>(DummyProtocolsHandler { + keep_alive: KeepAlive::Yes, + }) + .connection_limits(limits) + .build(); + + let addr: Multiaddr = "/memory/1234".parse().unwrap(); + + let target = PeerId::random(); + for _ in 0..outgoing_limit { + network + .dial( + DialOpts::peer_id(target) + .addresses(vec![addr.clone()]) + .build(), + ) + .ok() + .expect("Unexpected connection limit."); + } + + match network + .dial( + DialOpts::peer_id(target) + .addresses(vec![addr.clone()]) + .build(), + ) + .expect_err("Unexpected dialing success.") + { + DialError::ConnectionLimit(limit) => { + assert_eq!(limit.current, outgoing_limit); + assert_eq!(limit.limit, outgoing_limit); + } + e => panic!("Unexpected error: {:?}", e), + } + + let info = network.network_info(); + assert_eq!(info.num_peers(), 0); + assert_eq!( + info.connection_counters().num_pending_outgoing(), + outgoing_limit + ); + } + + #[test] + fn max_established_incoming() { + use rand::Rng; + + #[derive(Debug, Clone)] + struct Limit(u32); + + impl Arbitrary for Limit { + fn arbitrary(g: &mut G) -> Self { + Self(g.gen_range(1, 10)) + } + } + + fn limits(limit: u32) -> ConnectionLimits { + ConnectionLimits::default().with_max_established_incoming(Some(limit)) + } + + fn prop(limit: Limit) { + let limit = limit.0; + + let mut network1 = new_test_swarm::<_, ()>(DummyProtocolsHandler { + keep_alive: KeepAlive::Yes, + }) + .connection_limits(limits(limit)) + .build(); + let mut network2 = new_test_swarm::<_, ()>(DummyProtocolsHandler { + keep_alive: KeepAlive::Yes, + }) + .connection_limits(limits(limit)) + .build(); + + let _ = network1.listen_on(multiaddr![Memory(0u64)]).unwrap(); + let listen_addr = async_std::task::block_on(poll_fn(|cx| { + match ready!(network1.poll_next_unpin(cx)).unwrap() { + SwarmEvent::NewListenAddr { address, .. } => Poll::Ready(address), + e => panic!("Unexpected network event: {:?}", e), + } + })); + + // Spawn and block on the dialer. + async_std::task::block_on({ + let mut n = 0; + let _ = network2.dial(listen_addr.clone()).unwrap(); + + let mut expected_closed = false; + let mut network_1_established = false; + let mut network_2_established = false; + let mut network_1_limit_reached = false; + let mut network_2_limit_reached = false; + poll_fn(move |cx| { + loop { + let mut network_1_pending = false; + let mut network_2_pending = false; + + match network1.poll_next_unpin(cx) { + Poll::Ready(Some(SwarmEvent::IncomingConnection { .. })) => {} + Poll::Ready(Some(SwarmEvent::ConnectionEstablished { .. })) => { + network_1_established = true; + } + Poll::Ready(Some(SwarmEvent::IncomingConnectionError { + error: PendingConnectionError::ConnectionLimit(err), + .. + })) => { + assert_eq!(err.limit, limit); + assert_eq!(err.limit, err.current); + let info = network1.network_info(); + let counters = info.connection_counters(); + assert_eq!(counters.num_established_incoming(), limit); + assert_eq!(counters.num_established(), limit); + network_1_limit_reached = true; + } + Poll::Pending => { + network_1_pending = true; + } + e => panic!("Unexpected network event: {:?}", e), + } + + match network2.poll_next_unpin(cx) { + Poll::Ready(Some(SwarmEvent::ConnectionEstablished { .. })) => { + network_2_established = true; + } + Poll::Ready(Some(SwarmEvent::ConnectionClosed { .. })) => { + assert!(expected_closed); + let info = network2.network_info(); + let counters = info.connection_counters(); + assert_eq!(counters.num_established_outgoing(), limit); + assert_eq!(counters.num_established(), limit); + network_2_limit_reached = true; + } + Poll::Pending => { + network_2_pending = true; + } + e => panic!("Unexpected network event: {:?}", e), + } + + if network_1_pending && network_2_pending { + return Poll::Pending; + } + + if network_1_established && network_2_established { + network_1_established = false; + network_2_established = false; + + if n <= limit { + // Dial again until the limit is exceeded. + n += 1; + network2.dial(listen_addr.clone()).unwrap(); + + if n == limit { + // The the next dialing attempt exceeds the limit, this + // is the connection we expected to get closed. + expected_closed = true; + } + } else { + panic!("Expect networks not to establish connections beyond the limit.") + } + } + + if network_1_limit_reached && network_2_limit_reached { + return Poll::Ready(()); + } + } + }) + }); + } + + quickcheck(prop as fn(_)); + } + + #[test] + fn invalid_peer_id() { + // Checks whether dialing an address containing the wrong peer id raises an error + // for the expected peer id instead of the obtained peer id. + + let mut swarm1 = new_test_swarm::<_, ()>(DummyProtocolsHandler::default()).build(); + let mut swarm2 = new_test_swarm::<_, ()>(DummyProtocolsHandler::default()).build(); + + swarm1.listen_on("/memory/0".parse().unwrap()).unwrap(); + + let address = + futures::executor::block_on(future::poll_fn(|cx| match swarm1.poll_next_unpin(cx) { + Poll::Ready(Some(SwarmEvent::NewListenAddr { address, .. })) => { + Poll::Ready(address) + } + Poll::Pending => Poll::Pending, + _ => panic!("Was expecting the listen address to be reported"), + })); + + let other_id = PeerId::random(); + let other_addr = address.with(Protocol::P2p(other_id.into())); + + swarm2.dial(other_addr.clone()).unwrap(); + + let (peer_id, error) = futures::executor::block_on(future::poll_fn(|cx| { + if let Poll::Ready(Some(SwarmEvent::IncomingConnection { .. })) = + swarm1.poll_next_unpin(cx) + {} + + match swarm2.poll_next_unpin(cx) { + Poll::Ready(Some(SwarmEvent::OutgoingConnectionError { + peer_id, error, .. + })) => Poll::Ready((peer_id, error)), + Poll::Ready(x) => panic!("unexpected {:?}", x), + Poll::Pending => Poll::Pending, + } + })); + assert_eq!(peer_id.unwrap(), other_id); + match error { + DialError::WrongPeerId { obtained, endpoint } => { + assert_eq!(obtained, *swarm1.local_peer_id()); + assert_eq!( + endpoint, + ConnectedPoint::Dialer { + address: other_addr, + role_override: Endpoint::Dialer, + } + ); + } + x => panic!("wrong error {:?}", x), + } + } + + #[test] + fn dial_self() { + // Check whether dialing ourselves correctly fails. + // + // Dialing the same address we're listening should result in three events: + // + // - The incoming connection notification (before we know the incoming peer ID). + // - The connection error for the dialing endpoint (once we've determined that it's our own ID). + // - The connection error for the listening endpoint (once we've determined that it's our own ID). + // + // The last two can happen in any order. + + let mut swarm = new_test_swarm::<_, ()>(DummyProtocolsHandler::default()).build(); + swarm.listen_on("/memory/0".parse().unwrap()).unwrap(); + + let local_address = + futures::executor::block_on(future::poll_fn(|cx| match swarm.poll_next_unpin(cx) { + Poll::Ready(Some(SwarmEvent::NewListenAddr { address, .. })) => { + Poll::Ready(address) + } + Poll::Pending => Poll::Pending, + _ => panic!("Was expecting the listen address to be reported"), + })); + + swarm.dial(local_address.clone()).unwrap(); + + let mut got_dial_err = false; + let mut got_inc_err = false; + futures::executor::block_on(future::poll_fn(|cx| -> Poll> { + loop { + match swarm.poll_next_unpin(cx) { + Poll::Ready(Some(SwarmEvent::OutgoingConnectionError { + peer_id, + error: DialError::WrongPeerId { .. }, + .. + })) => { + assert_eq!(&peer_id.unwrap(), swarm.local_peer_id()); + assert!(!got_dial_err); + got_dial_err = true; + if got_inc_err { + return Poll::Ready(Ok(())); + } + } + Poll::Ready(Some(SwarmEvent::IncomingConnectionError { + local_addr, .. + })) => { + assert!(!got_inc_err); + assert_eq!(local_addr, local_address); + got_inc_err = true; + if got_dial_err { + return Poll::Ready(Ok(())); + } + } + Poll::Ready(Some(SwarmEvent::IncomingConnection { local_addr, .. })) => { + assert_eq!(local_addr, local_address); + } + Poll::Ready(ev) => { + panic!("Unexpected event: {:?}", ev) + } + Poll::Pending => break Poll::Pending, + } + } + })) + .unwrap(); + } + + #[test] + fn dial_self_by_id() { + // Trying to dial self by passing the same `PeerId` shouldn't even be possible in the first + // place. + let swarm = new_test_swarm::<_, ()>(DummyProtocolsHandler::default()).build(); + let peer_id = *swarm.local_peer_id(); + assert!(!swarm.is_connected(&peer_id)); + } + + #[test] + fn multiple_addresses_err() { + // Tries dialing multiple addresses, and makes sure there's one dialing error per address. + + let target = PeerId::random(); + + let mut swarm = new_test_swarm::<_, ()>(DummyProtocolsHandler::default()).build(); + + let mut addresses = Vec::new(); + for _ in 0..3 { + addresses.push(multiaddr![Ip4([0, 0, 0, 0]), Tcp(rand::random::())]); + } + for _ in 0..5 { + addresses.push(multiaddr![Udp(rand::random::())]); + } + addresses.shuffle(&mut rand::thread_rng()); + + swarm + .dial( + DialOpts::peer_id(target) + .addresses(addresses.clone()) + .build(), + ) + .unwrap(); + + futures::executor::block_on(future::poll_fn(|cx| -> Poll> { + loop { + match swarm.poll_next_unpin(cx) { + Poll::Ready(Some(SwarmEvent::OutgoingConnectionError { + peer_id, + // multiaddr, + error: DialError::Transport(errors), + })) => { + assert_eq!(peer_id.unwrap(), target); + + let failed_addresses = + errors.into_iter().map(|(addr, _)| addr).collect::>(); + assert_eq!( + failed_addresses, + addresses + .clone() + .into_iter() + .map(|addr| addr.with(Protocol::P2p(target.into()))) + .collect::>() + ); + + return Poll::Ready(Ok(())); + } + Poll::Ready(_) => unreachable!(), + Poll::Pending => break Poll::Pending, + } + } + })) + .unwrap(); + } } diff --git a/swarm/src/protocols_handler/node_handler.rs b/swarm/src/protocols_handler/node_handler.rs index fa339fd9416..112ba62b11a 100644 --- a/swarm/src/protocols_handler/node_handler.rs +++ b/swarm/src/protocols_handler/node_handler.rs @@ -18,6 +18,10 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. +use crate::connection::{ + Connected, ConnectionHandler, ConnectionHandlerEvent, IntoConnectionHandler, Substream, + SubstreamEndpoint, +}; use crate::protocols_handler::{ IntoProtocolsHandler, KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr, @@ -29,13 +33,9 @@ use futures::stream::FuturesUnordered; use futures_timer::Delay; use instant::Instant; use libp2p_core::{ - connection::{ - ConnectionHandler, ConnectionHandlerEvent, IntoConnectionHandler, Substream, - SubstreamEndpoint, - }, muxing::StreamMuxerBox, upgrade::{self, InboundUpgradeApply, OutboundUpgradeApply, UpgradeError}, - Connected, Multiaddr, + Multiaddr, }; use std::{error, fmt, pin::Pin, task::Context, task::Poll, time::Duration}; diff --git a/transports/noise/src/lib.rs b/transports/noise/src/lib.rs index 68506dd77c0..f4cc85dea4a 100644 --- a/transports/noise/src/lib.rs +++ b/transports/noise/src/lib.rs @@ -83,7 +83,7 @@ pub struct NoiseConfig { impl NoiseConfig { /// Turn the `NoiseConfig` into an authenticated upgrade for use - /// with a [`Network`](libp2p_core::Network). + /// with a `Swarm`. pub fn into_authenticated(self) -> NoiseAuthenticated { NoiseAuthenticated { config: self } } @@ -351,7 +351,7 @@ where /// On success, the upgrade yields the [`PeerId`] obtained from the /// `RemoteIdentity`. The output of this upgrade is thus directly suitable /// for creating an [`authenticated`](libp2p_core::transport::upgrade::Authenticate) -/// transport for use with a [`Network`](libp2p_core::Network). +/// transport for use with a `Swarm`. #[derive(Clone)] pub struct NoiseAuthenticated { config: NoiseConfig,