mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-01 12:11:20 +00:00
{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:
parent
861e15dabb
commit
7fc342e6c0
@ -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]
|
||||
|
||||
|
@ -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"
|
||||
|
@ -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 {}
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}));
|
||||
}
|
||||
}
|
@ -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,
|
||||
}
|
@ -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,
|
||||
})
|
||||
}
|
||||
}
|
@ -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,
|
||||
|
@ -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(_) -> _);
|
||||
}
|
@ -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(_));
|
||||
}
|
@ -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();
|
||||
}
|
@ -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>,
|
||||
|
@ -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,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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"
|
||||
|
@ -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
|
||||
}
|
||||
}
|
||||
|
@ -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]).
|
||||
|
@ -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
204
swarm/src/connection.rs
Normal 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 {}
|
@ -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)),
|
||||
}
|
||||
}
|
||||
}
|
@ -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)
|
||||
}
|
||||
|
@ -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);
|
||||
}));
|
||||
}
|
||||
}
|
@ -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())))
|
||||
}
|
||||
}
|
@ -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;
|
||||
|
@ -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
|
@ -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
|
||||
|
1328
swarm/src/lib.rs
1328
swarm/src/lib.rs
File diff suppressed because it is too large
Load Diff
@ -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};
|
||||
|
||||
|
@ -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>,
|
||||
|
Loading…
x
Reference in New Issue
Block a user