{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.
This commit is contained in:
Max Inden 2022-02-13 21:57:38 +01:00 committed by GitHub
parent 861e15dabb
commit 7fc342e6c0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
29 changed files with 1332 additions and 3211 deletions

View File

@ -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]

View File

@ -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"

View File

@ -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<usize> 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<u64> 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<T> {
/// 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<TMuxer, THandler>
where
TMuxer: StreamMuxer,
THandler: ConnectionHandler<Substream = Substream<TMuxer>>,
{
/// Node that handles the muxing.
muxing: substream::Muxing<TMuxer, THandler::OutboundOpenInfo>,
/// Handler that processes substreams.
handler: THandler,
}
impl<TMuxer, THandler> fmt::Debug for Connection<TMuxer, THandler>
where
TMuxer: StreamMuxer,
THandler: ConnectionHandler<Substream = Substream<TMuxer>> + 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<TMuxer, THandler> Unpin for Connection<TMuxer, THandler>
where
TMuxer: StreamMuxer,
THandler: ConnectionHandler<Substream = Substream<TMuxer>>,
{
}
impl<TMuxer, THandler> Connection<TMuxer, THandler>
where
TMuxer: StreamMuxer,
THandler: ConnectionHandler<Substream = Substream<TMuxer>>,
{
/// 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<TMuxer>) {
(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<Result<Event<THandler::OutEvent>, ConnectionError<THandler::Error>>> {
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 {}

View File

@ -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;

View File

@ -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<TTrans, THandler>
where
TTrans: Transport,
THandler: IntoConnectionHandler,
{
/// The local peer ID.
local_peer_id: PeerId,
/// Listeners for incoming connections.
listeners: ListenersStream<TTrans>,
/// The nodes currently active.
pool: Pool<THandler, TTrans>,
}
impl<TTrans, THandler> fmt::Debug for Network<TTrans, THandler>
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<TTrans, THandler> Unpin for Network<TTrans, THandler>
where
TTrans: Transport,
THandler: IntoConnectionHandler,
{
}
impl<TTrans, THandler> Network<TTrans, THandler>
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<Item = &ConnectionId> {
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<TTrans, THandler> Network<TTrans, THandler>
where
TTrans: Transport + Clone + 'static,
<TTrans as Transport>::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<ListenerId, TransportError<TTrans::Error>> {
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<Item = &Multiaddr> {
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<Multiaddr>
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<DialOpts>,
) -> Result<ConnectionId, DialError<THandler>>
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<Item = IncomingInfo<'_>> {
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<Item = &PeerId> {
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<Item = &PeerId> {
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<TTrans::ListenerUpgrade>,
handler: THandler,
) -> Result<ConnectionId, (ConnectionLimit, THandler)>
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<THandler>, THandlerOutEvent<THandler>, THandler>,
>
where
TTrans: Transport<Output = (PeerId, TMuxer)>,
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,
<THandler::Handler as ConnectionHandler>::Error: error::Error + Send + 'static,
<THandler::Handler as ConnectionHandler>::OutboundOpenInfo: Send,
<THandler::Handler as ConnectionHandler>::Error: error::Error + Send,
THandler::Handler: ConnectionHandler<Substream = Substream<TMuxer>> + 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<dyn Executor + Send>) -> 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<F>(mut self, f: F) -> Self
where
F: FnOnce() -> Option<Box<dyn Executor + Send>>,
{
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<THandler> {
/// 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<Multiaddr> 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<Multiaddr>) -> 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<Multiaddr>,
pub(crate) dial_concurrency_factor_override: Option<NonZeroU8>,
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<Box<dyn Future<Output = ()> + Send>>) {}
}
#[test]
fn set_executor() {
NetworkConfig::default()
.with_executor(Box::new(Dummy))
.with_executor(Box::new(|f| {
async_std::task::spawn(f);
}));
}
}

View File

@ -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<Multiaddr>,
/// 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<TTrans::ListenerUpgrade>,
},
/// 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<TTrans::Error>,
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<ConnectionId>,
/// [`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<Vec<(Multiaddr, TransportError<TTrans::Error>)>>,
},
/// 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<ConnectionError<<THandler::Handler as ConnectionHandler>::Error>>,
/// List of remaining established connections to the same peer.
remaining_established_connection_ids: Vec<ConnectionId>,
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<TTrans::Error>,
},
/// Failed to reach a peer that we were trying to dial.
UnknownPeerDialError {
/// The error that happened.
error: PendingOutboundConnectionError<TTrans::Error>,
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<TTrans, TInEvent, TOutEvent, THandler> fmt::Debug
for NetworkEvent<'_, TTrans, TInEvent, TOutEvent, THandler>
where
TInEvent: fmt::Debug,
TOutEvent: fmt::Debug,
TTrans: Transport,
TTrans::Error: fmt::Debug,
THandler: IntoConnectionHandler,
<THandler::Handler as ConnectionHandler>::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<TUpgrade> {
/// 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,
}

View File

@ -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<TTrans, THandler>, 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<TTrans, THandler>, peer_id: PeerId) -> Self {
Peer::Disconnected(DisconnectedPeer {
_network: network,
peer_id,
})
}
fn connected(network: &'a mut Network<TTrans, THandler>, peer_id: PeerId) -> Self {
Peer::Connected(ConnectedPeer { network, peer_id })
}
fn dialing(network: &'a mut Network<TTrans, THandler>, 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,
<THandler::Handler as ConnectionHandler>::OutboundOpenInfo: Send,
<THandler::Handler as ConnectionHandler>::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<ConnectedPeer<'a, TTrans, THandler>> {
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<DialingPeer<'a, TTrans, THandler>> {
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<DisconnectedPeer<'a, TTrans, THandler>> {
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<TTrans, THandler>,
peer_id: PeerId,
}
impl<'a, TTrans, THandler> ConnectedPeer<'a, TTrans, THandler>
where
TTrans: Transport,
<TTrans as 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<EstablishedConnection<THandlerInEvent<THandler>>> {
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<DialingPeer<'a, TTrans, THandler>> {
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<impl Iterator<Item = ConnectionId>, THandlerInEvent<THandler>>
{
self.network.pool.iter_peer_established(&self.peer_id)
}
/// Obtains some established connection to the peer.
pub fn some_connection(&mut self) -> EstablishedConnection<THandlerInEvent<THandler>> {
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::<Vec<_>>(),
)
.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<TTrans, THandler>,
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<ConnectedPeer<'a, TTrans, THandler>> {
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<DialingAttempt<'_, THandler>> {
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::<Vec<_>>(),
)
.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<TTrans, THandler>,
}
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<THandler, TTrans>,
/// [`ConnectionId`]s of the dialing attempts of the peer.
connections: VecDeque<ConnectionId>,
}
// 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<TTrans, THandler>) -> 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<DialingAttempt<'_, THandler>> {
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<DialingAttempt<'b, THandler>>
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,
})
}
}

View File

@ -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<T> {
inner: T,

View File

@ -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: Gen>(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(_) -> _);
}

View File

@ -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: Gen>(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(_));
}

View File

@ -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<Result<(), io::Error>> {
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<Result<(), io::Error>> {
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::<u16>())]);
}
for _ in 0..5 {
addresses.push(multiaddr![Udp(rand::random::<u16>())]);
}
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<Result<(), io::Error>> {
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::<Vec<_>>();
assert_eq!(
failed_addresses,
addresses
.clone()
.into_iter()
.map(|addr| addr.with(Protocol::P2p(target.into())))
.collect::<Vec<_>>()
);
return Poll::Ready(Ok(()));
}
Poll::Ready(_) => unreachable!(),
Poll::Pending => break Poll::Pending,
}
}
}))
.unwrap();
}

View File

@ -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<TestTransport, TestHandler>;
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::<noise::X25519Spec>::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<StreamMuxerBox>;
type OutboundOpenInfo = ();
fn inject_substream(
&mut self,
_: Self::Substream,
_: SubstreamEndpoint<Self::OutboundOpenInfo>,
) {
}
fn inject_event(&mut self, _: Self::InEvent) {}
fn inject_address_change(&mut self, _: &Multiaddr) {}
fn poll(
&mut self,
_: &mut Context<'_>,
) -> Poll<Result<ConnectionHandlerEvent<Self::OutboundOpenInfo, Self::OutEvent>, Self::Error>>
{
Poll::Pending
}
}
use libp2p_core::muxing::StreamMuxer;
use std::{pin::Pin, task::Context, task::Poll};
pub struct CloseMuxer<M> {
state: CloseMuxerState<M>,

View File

@ -316,29 +316,27 @@ enum PendingInboundConnectionError {
ConnectionLimit,
}
impl<TTransErr> From<&libp2p_core::connection::PendingInboundConnectionError<TTransErr>>
impl<TTransErr> From<&libp2p_swarm::PendingInboundConnectionError<TTransErr>>
for PendingInboundConnectionError
{
fn from(error: &libp2p_core::connection::PendingInboundConnectionError<TTransErr>) -> Self {
fn from(error: &libp2p_swarm::PendingInboundConnectionError<TTransErr>) -> 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,
}
}
}

View File

@ -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"

View File

@ -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<TestTransport, TestHandler>;
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::<u64>()));
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();
match ready!(listener.poll_next_unpin(cx)).unwrap() {
SwarmEvent::NewListenAddr { address, .. } => {
addr_sender.take().unwrap().send(address).unwrap();
}
NetworkEvent::IncomingConnection { connection, .. } => {
listener.accept(connection, TestHandler()).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<StreamMuxerBox>;
type OutboundOpenInfo = ();
fn inject_substream(
&mut self,
_: Self::Substream,
_: SubstreamEndpoint<Self::OutboundOpenInfo>,
) {
}
fn inject_event(&mut self, _: Self::InEvent) {}
fn inject_address_change(&mut self, _: &Multiaddr) {}
fn poll(
&mut self,
_: &mut Context<'_>,
) -> Poll<Result<ConnectionHandlerEvent<Self::OutboundOpenInfo, Self::OutEvent>, Self::Error>>
{
Poll::Pending
}
}

View File

@ -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]).

View File

@ -12,16 +12,23 @@ categories = ["network-programming", "asynchronous"]
[dependencies]
either = "1.6.0"
fnv = "1.0"
futures = "0.3.1"
libp2p-core = { version = "0.32.0", path = "../core", default-features = false }
log = "0.4"
rand = "0.7"
smallvec = "1.6.1"
void = "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"
[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"

204
swarm/src/connection.rs Normal file
View File

@ -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<T> {
/// 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<TMuxer, THandler>
where
TMuxer: StreamMuxer,
THandler: ConnectionHandler<Substream = Substream<TMuxer>>,
{
/// Node that handles the muxing.
muxing: substream::Muxing<TMuxer, THandler::OutboundOpenInfo>,
/// Handler that processes substreams.
handler: THandler,
}
impl<TMuxer, THandler> fmt::Debug for Connection<TMuxer, THandler>
where
TMuxer: StreamMuxer,
THandler: ConnectionHandler<Substream = Substream<TMuxer>> + 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<TMuxer, THandler> Unpin for Connection<TMuxer, THandler>
where
TMuxer: StreamMuxer,
THandler: ConnectionHandler<Substream = Substream<TMuxer>>,
{
}
impl<TMuxer, THandler> Connection<TMuxer, THandler>
where
TMuxer: StreamMuxer,
THandler: ConnectionHandler<Substream = Substream<TMuxer>>,
{
/// 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<TMuxer>) {
(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<Result<Event<THandler::OutEvent>, ConnectionError<THandler::Error>>> {
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 {}

View File

@ -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<TOutboundOpenInfo, TCustom> {
/// Other event.
Custom(TCustom),
}
/// Event produced by a handler.
impl<TOutboundOpenInfo, TCustom> ConnectionHandlerEvent<TOutboundOpenInfo, TCustom> {
/// If this is `OutboundSubstreamRequest`, maps the content to something else.
pub fn map_outbound_open_info<F, I>(self, map: F) -> ConnectionHandlerEvent<I, TCustom>
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<F, I>(self, map: F) -> ConnectionHandlerEvent<TOutboundOpenInfo, I>
where
F: FnOnce(TCustom) -> I,
{
match self {
ConnectionHandlerEvent::OutboundSubstreamRequest(val) => {
ConnectionHandlerEvent::OutboundSubstreamRequest(val)
}
ConnectionHandlerEvent::Custom(val) => ConnectionHandlerEvent::Custom(map(val)),
}
}
}

View File

@ -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<TTrans>
where
TTrans: Transport,
@ -94,13 +57,6 @@ where
pending_events: VecDeque<ListenersEvent<TTrans>>,
}
/// 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)
}

View File

@ -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<PendingConnection<'_, THandler>> {
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::<Vec<_>>();
@ -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,
pub fn iter_established_connections_of_peer(
&mut self,
peer: &PeerId,
) -> EstablishedConnectionIter<'a, impl Iterator<Item = ConnectionId>, THandlerInEvent<THandler>>
{
let ids = self
.iter_peer_established_info(peer)
.map(|(id, _endpoint)| *id)
.collect::<SmallVec<[ConnectionId; 10]>>()
.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<Item = IncomingInfo<'_>> {
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,
peer: &PeerId,
) -> impl Iterator<Item = (&ConnectionId, &ConnectedPoint)> {
) -> impl Iterator<Item = ConnectionId> + '_ {
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<NonZeroU8>,
) -> Result<ConnectionId, DialError<THandler>>
) -> Result<ConnectionId, (ConnectionLimit, THandler)>
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<THandler: IntoConnectionHandler> 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<PeerId> {
&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<TInEvent> EstablishedConnection<'_, TInEvent> {
}
}
/// An iterator over established connections in a pool.
pub struct EstablishedConnectionIter<'a, I, TInEvent> {
connections: Option<&'a mut FnvHashMap<ConnectionId, EstablishedConnectionInfo<TInEvent>>>,
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<Item = ConnectionId>,
{
/// Obtains the next connection, if any.
#[allow(clippy::should_implement_trait)]
pub fn next(&mut self) -> Option<EstablishedConnection<'_, TInEvent>> {
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<Item = ConnectionId> {
self.ids
}
/// Returns the first connection, if any, consuming the iterator.
pub fn into_first<'b>(mut self) -> Option<EstablishedConnection<'b, TInEvent>>
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<dyn Executor + Send>) -> 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<F>(mut self, f: F) -> Self
where
F: FnOnce() -> Option<Box<dyn Executor + Send>>,
{
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<Box<dyn Future<Output = ()> + Send>>) {}
}
#[test]
fn set_executor() {
PoolConfig::default()
.with_executor(Box::new(Dummy))
.with_executor(Box::new(|f| {
async_std::task::spawn(f);
}));
}
}

View File

@ -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<PeerId>, addr: Multiaddr) -> Result<Multiaddr, Multiadd
None => 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())))
}
}

View File

@ -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;

View File

@ -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<TDialInfo> {
Listener,
}
impl<TDialInfo> SubstreamEndpoint<TDialInfo> {
/// 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

View File

@ -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

File diff suppressed because it is too large Load Diff

View File

@ -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};

View File

@ -83,7 +83,7 @@ pub struct NoiseConfig<P, C: Zeroize, R = ()> {
impl<H, C: Zeroize, R> NoiseConfig<H, C, R> {
/// Turn the `NoiseConfig` into an authenticated upgrade for use
/// with a [`Network`](libp2p_core::Network).
/// with a `Swarm`.
pub fn into_authenticated(self) -> NoiseAuthenticated<H, C, R> {
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<P, C: Zeroize, R> {
config: NoiseConfig<P, C, R>,