Multiple connections per peer (#1440)

* Allow multiple connections per peer in libp2p-core.

Instead of trying to enforce a single connection per peer,
which involves quite a bit of additional complexity e.g.
to prioritise simultaneously opened connections and can
have other undesirable consequences [1], we now
make multiple connections per peer a feature.

The gist of these changes is as follows:

The concept of a "node" with an implicit 1-1 correspondence
to a connection has been replaced with the "first-class"
concept of a "connection". The code from `src/nodes` has moved
(with varying degrees of modification) to `src/connection`.
A `HandledNode` has become a `Connection`, a `NodeHandler` a
`ConnectionHandler`, the `CollectionStream` was the basis for
the new `connection::Pool`, and so forth.

Conceptually, a `Network` contains a `connection::Pool` which
in turn internally employs the `connection::Manager` for
handling the background `connection::manager::Task`s, one
per connection, as before. These are all considered implementation
details. On the public API, `Peer`s are managed as before through
the `Network`, except now the API has changed with the shift of focus
to (potentially multiple) connections per peer. The `NetworkEvent`s have
accordingly also undergone changes.

The Swarm APIs remain largely unchanged, except for the fact that
`inject_replaced` is no longer called. It may now practically happen
that multiple `ProtocolsHandler`s are associated with a single
`NetworkBehaviour`, one per connection. If implementations of
`NetworkBehaviour` rely somehow on communicating with exactly
one `ProtocolsHandler`, this may cause issues, but it is unlikely.

[1]: https://github.com/paritytech/substrate/issues/4272

* Fix intra-rustdoc links.

* Update core/src/connection/pool.rs

Co-Authored-By: Max Inden <mail@max-inden.de>

* Address some review feedback and fix doc links.

* Allow responses to be sent on the same connection.

* Remove unnecessary remainders of inject_replaced.

* Update swarm/src/behaviour.rs

Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com>

* Update swarm/src/lib.rs

Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com>

* Update core/src/connection/manager.rs

Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com>

* Update core/src/connection/manager.rs

Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com>

* Update core/src/connection/pool.rs

Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com>

* Incorporate more review feedback.

* Move module declaration below imports.

* Update core/src/connection/manager.rs

Co-Authored-By: Toralf Wittner <tw@dtex.org>

* Update core/src/connection/manager.rs

Co-Authored-By: Toralf Wittner <tw@dtex.org>

* Simplify as per review.

* Fix rustoc link.

* Add try_notify_handler and simplify.

* Relocate DialingConnection and DialingAttempt.

For better visibility constraints.

* Small cleanup.

* Small cleanup. More robust EstablishedConnectionIter.

* Clarify semantics of `DialingPeer::connect`.

* Don't call inject_disconnected on InvalidPeerId.

To preserve the previous behavior and ensure calls to
`inject_disconnected` are always paired with calls to
`inject_connected`.

* Provide public ConnectionId constructor.

Mainly needed for testing purposes, e.g. in substrate.

* Move the established connection limit check to the right place.

* Clean up connection error handling.

Separate connection errors into those occuring during
connection setup or upon rejecting a newly established
connection (the `PendingConnectionError`) and those
errors occurring on previously established connections,
i.e. for which a `ConnectionEstablished` event has
been emitted by the connection pool earlier.

* Revert change in log level and clarify an invariant.

* Remove inject_replaced entirely.

* Allow notifying all connection handlers.

Thereby simplify by introducing a new enum `NotifyHandler`,
used with a single constructor `NetworkBehaviourAction::NotifyHandler`.

* Finishing touches.

Small API simplifications and code deduplication.
Some more useful debug logging.

Co-authored-by: Max Inden <mail@max-inden.de>
Co-authored-by: Pierre Krieger <pierre.krieger1708@gmail.com>
Co-authored-by: Toralf Wittner <tw@dtex.org>
This commit is contained in:
Roman Borschel
2020-03-04 13:49:25 +01:00
committed by GitHub
parent 7cbb3cf8f3
commit 8337687b3a
45 changed files with 4560 additions and 4704 deletions

View File

@ -64,7 +64,8 @@ pub use behaviour::{
NetworkBehaviour,
NetworkBehaviourAction,
NetworkBehaviourEventProcess,
PollParameters
PollParameters,
NotifyHandler
};
pub use protocols_handler::{
IntoProtocolsHandler,
@ -78,30 +79,42 @@ pub use protocols_handler::{
SubstreamProtocol
};
/// Substream for which a protocol has been chosen.
///
/// Implements the [`AsyncRead`](futures::io::AsyncRead) and
/// [`AsyncWrite`](futures::io::AsyncWrite) traits.
pub type NegotiatedSubstream = Negotiated<Substream<StreamMuxerBox>>;
use protocols_handler::{NodeHandlerWrapperBuilder, NodeHandlerWrapperError};
use futures::{prelude::*, executor::{ThreadPool, ThreadPoolBuilder}, stream::FusedStream};
use protocols_handler::NodeHandlerWrapperBuilder;
use futures::{
prelude::*,
executor::{ThreadPool, ThreadPoolBuilder},
stream::FusedStream,
};
use libp2p_core::{
Executor, Negotiated, Transport, Multiaddr, PeerId, ProtocolName,
muxing::{StreamMuxer, StreamMuxerBox},
nodes::{
ListenerId, Substream,
collection::ConnectionInfo,
network::{self, Network, NetworkEvent}
Executor,
Transport,
Multiaddr,
Negotiated,
PeerId,
connection::{
ConnectionId,
ConnectionInfo,
EstablishedConnection,
IntoConnectionHandler,
ListenerId,
Substream
},
transport::{
boxed::Boxed as BoxTransport,
TransportError,
}
transport::{TransportError, boxed::Boxed as BoxTransport},
muxing::{StreamMuxer, StreamMuxerBox},
network::{
DialError,
Network,
NetworkInfo,
NetworkEvent,
NetworkConfig,
Peer,
peer::{ConnectedPeer, PeerState},
},
upgrade::ProtocolName,
};
use registry::{Addresses, AddressIntoIter};
use smallvec::SmallVec;
use std::{error, fmt, io, ops::{Deref, DerefMut}, pin::Pin, task::{Context, Poll}};
use std::{error, fmt, hash::Hash, io, ops::{Deref, DerefMut}, pin::Pin, task::{Context, Poll}};
use std::collections::HashSet;
use upgrade::UpgradeInfoSend as _;
@ -111,10 +124,15 @@ pub type Swarm<TBehaviour, TConnInfo = PeerId> = ExpandedSwarm<
<<<TBehaviour as NetworkBehaviour>::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InEvent,
<<<TBehaviour as NetworkBehaviour>::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutEvent,
<TBehaviour as NetworkBehaviour>::ProtocolsHandler,
<<<TBehaviour as NetworkBehaviour>::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::Error,
TConnInfo,
>;
/// Substream for which a protocol has been chosen.
///
/// Implements the [`AsyncRead`](futures::io::AsyncRead) and
/// [`AsyncWrite`](futures::io::AsyncWrite) traits.
pub type NegotiatedSubstream = Negotiated<Substream<StreamMuxerBox>>;
/// Event generated by the `Swarm`.
#[derive(Debug)]
pub enum SwarmEvent<TBvEv> {
@ -143,13 +161,16 @@ pub enum SwarmEvent<TBvEv> {
}
/// Contains the state of the network, plus the way it should behave.
pub struct ExpandedSwarm<TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo = PeerId> {
pub struct ExpandedSwarm<TBehaviour, TInEvent, TOutEvent, THandler, TConnInfo = PeerId>
where
THandler: IntoProtocolsHandler,
TConnInfo: ConnectionInfo<PeerId = PeerId>,
{
network: Network<
BoxTransport<(TConnInfo, StreamMuxerBox), io::Error>,
TInEvent,
TOutEvent,
NodeHandlerWrapperBuilder<THandler>,
NodeHandlerWrapperError<THandlerErr>,
TConnInfo,
PeerId,
>,
@ -171,12 +192,17 @@ pub struct ExpandedSwarm<TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr,
/// List of nodes for which we deny any incoming connection.
banned_peers: HashSet<PeerId>,
/// Pending event message to be delivered.
send_event_to_complete: Option<(PeerId, TInEvent)>
/// Pending event to be delivered to connection handlers
/// (or dropped if the peer disconnected) before the `behaviour`
/// can be polled again.
pending_event: Option<(PeerId, PendingNotifyHandler, TInEvent)>
}
impl<TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo> Deref for
ExpandedSwarm<TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo>
impl<TBehaviour, TInEvent, TOutEvent, THandler, TConnInfo> Deref for
ExpandedSwarm<TBehaviour, TInEvent, TOutEvent, THandler, TConnInfo>
where
THandler: IntoProtocolsHandler,
TConnInfo: ConnectionInfo<PeerId = PeerId>,
{
type Target = TBehaviour;
@ -185,28 +211,33 @@ impl<TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo> Deref fo
}
}
impl<TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo> DerefMut for
ExpandedSwarm<TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo>
impl<TBehaviour, TInEvent, TOutEvent, THandler, TConnInfo> DerefMut for
ExpandedSwarm<TBehaviour, TInEvent, TOutEvent, THandler, TConnInfo>
where
THandler: IntoProtocolsHandler,
TConnInfo: ConnectionInfo<PeerId = PeerId>,
{
fn deref_mut(&mut self) -> &mut Self::Target {
&mut self.behaviour
}
}
impl<TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo> Unpin for
ExpandedSwarm<TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo>
impl<TBehaviour, TInEvent, TOutEvent, THandler, TConnInfo> Unpin for
ExpandedSwarm<TBehaviour, TInEvent, TOutEvent, THandler, TConnInfo>
where
THandler: IntoProtocolsHandler,
TConnInfo: ConnectionInfo<PeerId = PeerId>,
{
}
impl<TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo>
ExpandedSwarm<TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo>
impl<TBehaviour, TInEvent, TOutEvent, THandler, TConnInfo>
ExpandedSwarm<TBehaviour, TInEvent, TOutEvent, THandler, TConnInfo>
where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
TInEvent: Send + 'static,
TInEvent: Clone + Send + 'static,
TOutEvent: Send + 'static,
TConnInfo: ConnectionInfo<PeerId = PeerId> + fmt::Debug + Clone + Send + 'static,
THandlerErr: error::Error + Send + 'static,
THandler: IntoProtocolsHandler + Send + 'static,
THandler::Handler: ProtocolsHandler<InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr>,
THandler::Handler: ProtocolsHandler<InEvent = TInEvent, OutEvent = TOutEvent>,
{
/// Builds a new `Swarm`.
pub fn new<TTransport, TMuxer>(transport: TTransport, behaviour: TBehaviour, local_peer_id: PeerId) -> Self
@ -225,6 +256,11 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
.build()
}
/// Returns information about the [`Network`] underlying the `Swarm`.
pub fn network_info(me: &Self) -> NetworkInfo {
me.network.info()
}
/// Starts listening on the given address.
///
/// Returns an error if the address is not supported.
@ -242,9 +278,9 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
/// Tries to dial the given address.
///
/// Returns an error if the address is not supported.
pub fn dial_addr(me: &mut Self, addr: Multiaddr) -> Result<(), TransportError<io::Error>> {
pub fn dial_addr(me: &mut Self, addr: Multiaddr) -> Result<(), DialError<io::Error>> {
let handler = me.behaviour.new_handler();
me.network.dial(addr, handler.into_node_handler_builder())
me.network.dial(&addr, handler.into_node_handler_builder()).map(|_id| ())
}
/// Tries to reach the given peer using the elements in the topology.
@ -254,16 +290,19 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
pub fn dial(me: &mut Self, peer_id: PeerId) {
let addrs = me.behaviour.addresses_of_peer(&peer_id);
match me.network.peer(peer_id.clone()) {
network::Peer::NotConnected(peer) => {
let handler = me.behaviour.new_handler().into_node_handler_builder();
if peer.connect_iter(addrs, handler).is_err() {
me.behaviour.inject_dial_failure(&peer_id);
Peer::Disconnected(peer) => {
let mut addrs = addrs.into_iter();
if let Some(first) = addrs.next() {
let handler = me.behaviour.new_handler().into_node_handler_builder();
if peer.connect(first, addrs, handler).is_err() {
me.behaviour.inject_dial_failure(&peer_id);
}
}
},
network::Peer::PendingConnect(mut peer) => {
peer.append_multiaddr_attempts(addrs)
Peer::Dialing(mut peer) => {
peer.connection().add_addresses(addrs)
},
network::Peer::Connected(_) | network::Peer::LocalNode => {}
Peer::Connected(_) | Peer::Local => {}
}
}
@ -291,11 +330,12 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
me.external_addrs.add(addr)
}
/// Returns the connection info of a node, or `None` if we're not connected to it.
/// Returns the connection info for an arbitrary connection with the peer, or `None`
/// if there is no connection to that peer.
// TODO: should take &self instead of &mut self, but the API in network requires &mut
pub fn connection_info(me: &mut Self, peer_id: &PeerId) -> Option<TConnInfo> {
if let Some(mut n) = me.network.peer(peer_id.clone()).into_connected() {
Some(n.connection_info().clone())
Some(n.some_connection().info().clone())
} else {
None
}
@ -308,7 +348,7 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
pub fn ban_peer_id(me: &mut Self, peer_id: PeerId) {
me.banned_peers.insert(peer_id.clone());
if let Some(c) = me.network.peer(peer_id).into_connected() {
c.close();
c.disconnect();
}
}
@ -349,55 +389,76 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
loop {
let mut network_not_ready = false;
// First let the network make progress.
match this.network.poll(cx) {
Poll::Pending => network_not_ready = true,
Poll::Ready(NetworkEvent::NodeEvent { conn_info, event }) => {
this.behaviour.inject_node_event(conn_info.peer_id().clone(), event);
Poll::Ready(NetworkEvent::ConnectionEvent { connection, event }) => {
let peer = connection.peer_id().clone();
let connection = connection.id();
this.behaviour.inject_event(peer, connection, event);
},
Poll::Ready(NetworkEvent::Connected { conn_info, endpoint }) => {
if this.banned_peers.contains(conn_info.peer_id()) {
this.network.peer(conn_info.peer_id().clone())
Poll::Ready(NetworkEvent::ConnectionEstablished { connection, num_established }) => {
let peer = connection.peer_id().clone();
if this.banned_peers.contains(&peer) {
this.network.peer(peer)
.into_connected()
.expect("the Network just notified us that we were connected; QED")
.close();
.disconnect();
} else if num_established == 1 {
let endpoint = connection.endpoint().clone();
this.behaviour.inject_connected(peer.clone(), endpoint);
return Poll::Ready(SwarmEvent::Connected(peer));
} else {
this.behaviour.inject_connected(conn_info.peer_id().clone(), endpoint);
return Poll::Ready(SwarmEvent::Connected(conn_info.peer_id().clone()));
// For now, secondary connections are not explicitly reported to
// the behaviour. A behaviour only gets awareness of the
// connections via the events emitted from the connection handlers.
log::trace!("Secondary connection established: {:?}; Total (peer): {}.",
connection.connected(), num_established);
}
},
Poll::Ready(NetworkEvent::NodeClosed { conn_info, endpoint, error }) => {
log::trace!("Connection {:?} with endpoint {:?} closed by {:?}",
conn_info, endpoint, error);
this.behaviour.inject_disconnected(conn_info.peer_id(), endpoint);
return Poll::Ready(SwarmEvent::Disconnected(conn_info.peer_id().clone()));
},
Poll::Ready(NetworkEvent::Replaced { new_info, closed_endpoint, endpoint, .. }) => {
this.behaviour.inject_replaced(new_info.peer_id().clone(), closed_endpoint, endpoint);
Poll::Ready(NetworkEvent::ConnectionError { connected, error, num_established }) => {
log::debug!("Connection {:?} closed by {:?}", connected, error);
if num_established == 0 {
let peer = connected.peer_id().clone();
let endpoint = connected.endpoint;
this.behaviour.inject_disconnected(&peer, endpoint);
return Poll::Ready(SwarmEvent::Disconnected(peer));
}
},
Poll::Ready(NetworkEvent::IncomingConnection(incoming)) => {
let handler = this.behaviour.new_handler();
incoming.accept(handler.into_node_handler_builder());
if let Err(e) = incoming.accept(handler.into_node_handler_builder()) {
log::warn!("Incoming connection rejected: {:?}", e);
}
},
Poll::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) => {
Poll::Ready(NetworkEvent::NewListenerAddress { listener_id, listen_addr }) => {
log::debug!("Listener {:?}; New address: {:?}", listener_id, listen_addr);
if !this.listened_addrs.contains(&listen_addr) {
this.listened_addrs.push(listen_addr.clone())
}
this.behaviour.inject_new_listen_addr(&listen_addr);
return Poll::Ready(SwarmEvent::NewListenAddr(listen_addr));
}
Poll::Ready(NetworkEvent::ExpiredListenerAddress { listen_addr, .. }) => {
Poll::Ready(NetworkEvent::ExpiredListenerAddress { listener_id, listen_addr }) => {
log::debug!("Listener {:?}; Expired address {:?}.", listener_id, listen_addr);
this.listened_addrs.retain(|a| a != &listen_addr);
this.behaviour.inject_expired_listen_addr(&listen_addr);
return Poll::Ready(SwarmEvent::ExpiredListenAddr(listen_addr));
}
Poll::Ready(NetworkEvent::ListenerClosed { listener_id, .. }) =>
this.behaviour.inject_listener_closed(listener_id),
Poll::Ready(NetworkEvent::ListenerClosed { listener_id, reason }) => {
log::debug!("Listener {:?}; Closed by {:?}.", listener_id, reason);
this.behaviour.inject_listener_closed(listener_id);
}
Poll::Ready(NetworkEvent::ListenerError { listener_id, error }) =>
this.behaviour.inject_listener_error(listener_id, &error),
Poll::Ready(NetworkEvent::IncomingConnectionError { .. }) => {},
Poll::Ready(NetworkEvent::IncomingConnectionError { error, .. }) => {
log::debug!("Incoming connection failed: {:?}", error);
},
Poll::Ready(NetworkEvent::DialError { peer_id, multiaddr, error, new_state }) => {
log::debug!("Connection attempt to peer {:?} at address {:?} failed with {:?}",
peer_id, multiaddr, error);
this.behaviour.inject_addr_reach_failure(Some(&peer_id), &multiaddr, &error);
if let network::PeerState::NotConnected = new_state {
if let PeerState::Disconnected = new_state {
this.behaviour.inject_dial_failure(&peer_id);
}
return Poll::Ready(SwarmEvent::UnreachableAddr {
@ -407,6 +468,8 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
});
},
Poll::Ready(NetworkEvent::UnknownPeerDialError { multiaddr, error, .. }) => {
log::debug!("Connection attempt to address {:?} of unknown peer failed with {:?}",
multiaddr, error);
this.behaviour.inject_addr_reach_failure(None, &multiaddr, &error);
return Poll::Ready(SwarmEvent::UnreachableAddr {
peer_id: None,
@ -416,19 +479,41 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
},
}
// Try to deliver pending event.
if let Some((id, pending)) = this.send_event_to_complete.take() {
if let Some(mut peer) = this.network.peer(id.clone()).into_connected() {
match peer.poll_ready_event(cx) {
Poll::Ready(()) => peer.start_send_event(pending),
Poll::Pending => {
this.send_event_to_complete = Some((id, pending));
return Poll::Pending
},
// After the network had a chance to make progress, try to deliver
// the pending event emitted by the behaviour in the previous iteration
// to the connection handler(s). The pending event must be delivered
// before polling the behaviour again. If the targeted peer
// meanwhie disconnected, the event is discarded.
if let Some((peer_id, handler, event)) = this.pending_event.take() {
if let Some(mut peer) = this.network.peer(peer_id.clone()).into_connected() {
match handler {
PendingNotifyHandler::One(conn_id) =>
if let Some(mut conn) = peer.connection(conn_id) {
if let Some(event) = notify_one(&mut conn, event, cx) {
this.pending_event = Some((peer_id, handler, event));
return Poll::Pending
}
},
PendingNotifyHandler::Any(ids) => {
if let Some((event, ids)) = notify_any(ids, &mut peer, event, cx) {
let handler = PendingNotifyHandler::Any(ids);
this.pending_event = Some((peer_id, handler, event));
return Poll::Pending
}
}
PendingNotifyHandler::All(ids) => {
if let Some((event, ids)) = notify_all(ids, &mut peer, event, cx) {
let handler = PendingNotifyHandler::All(ids);
this.pending_event = Some((peer_id, handler, event));
return Poll::Pending
}
}
}
}
}
debug_assert!(this.pending_event.is_none());
let behaviour_poll = {
let mut parameters = SwarmPollParameters {
local_peer_id: &mut this.network.local_peer_id(),
@ -456,14 +541,34 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
return Poll::Ready(SwarmEvent::StartConnect(peer_id))
}
},
Poll::Ready(NetworkBehaviourAction::SendEvent { peer_id, event }) => {
Poll::Ready(NetworkBehaviourAction::NotifyHandler { peer_id, handler, event }) => {
if let Some(mut peer) = this.network.peer(peer_id.clone()).into_connected() {
if let Poll::Ready(()) = peer.poll_ready_event(cx) {
peer.start_send_event(event);
} else {
debug_assert!(this.send_event_to_complete.is_none());
this.send_event_to_complete = Some((peer_id, event));
return Poll::Pending;
match handler {
NotifyHandler::One(connection) => {
if let Some(mut conn) = peer.connection(connection) {
if let Some(event) = notify_one(&mut conn, event, cx) {
let handler = PendingNotifyHandler::One(connection);
this.pending_event = Some((peer_id, handler, event));
return Poll::Pending
}
}
}
NotifyHandler::Any => {
let ids = peer.connections().into_ids().collect();
if let Some((event, ids)) = notify_any(ids, &mut peer, event, cx) {
let handler = PendingNotifyHandler::Any(ids);
this.pending_event = Some((peer_id, handler, event));
return Poll::Pending
}
}
NotifyHandler::All => {
let ids = peer.connections().into_ids().collect();
if let Some((event, ids)) = notify_all(ids, &mut peer, event, cx) {
let handler = PendingNotifyHandler::All(ids);
this.pending_event = Some((peer_id, handler, event));
return Poll::Pending
}
}
}
}
},
@ -480,14 +585,156 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
}
}
impl<TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo> Stream for
ExpandedSwarm<TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo>
/// Connections to notify of a pending event.
///
/// The connection IDs to notify of an event are captured at the time
/// the behaviour emits the event, in order not to forward the event
/// to new connections which the behaviour may not have been aware of
/// at the time it issued the request for sending it.
enum PendingNotifyHandler {
One(ConnectionId),
Any(SmallVec<[ConnectionId; 10]>),
All(SmallVec<[ConnectionId; 10]>),
}
/// Notify a single connection of an event.
///
/// Returns `Some` with the given event if the connection is not currently
/// ready to receive another event, in which case the current task is
/// scheduled to be woken up.
///
/// Returns `None` if the connection is closing or the event has been
/// successfully sent, in either case the event is consumed.
fn notify_one<'a, TInEvent, TConnInfo, TPeerId>(
conn: &mut EstablishedConnection<'a, TInEvent, TConnInfo, TPeerId>,
event: TInEvent,
cx: &mut Context,
) -> Option<TInEvent>
where
TPeerId: Eq + std::hash::Hash + Clone,
TConnInfo: ConnectionInfo<PeerId = TPeerId>
{
match conn.poll_ready_notify_handler(cx) {
Poll::Pending => Some(event),
Poll::Ready(Err(())) => None, // connection is closing
Poll::Ready(Ok(())) => {
// Can now only fail if connection is closing.
let _ = conn.notify_handler(event);
None
}
}
}
/// Notify any one of a given list of connections of a peer of an event.
///
/// Returns `Some` with the given event and a new list of connections if
/// none of the given connections was able to receive the event but at
/// least one of them is not closing, in which case the current task
/// is scheduled to be woken up. The returned connections are those which
/// may still become ready to receive another event.
///
/// Returns `None` if either all connections are closing or the event
/// was successfully sent to a handler, in either case the event is consumed.
fn notify_any<'a, TTrans, TInEvent, TOutEvent, THandler, TConnInfo, TPeerId>(
ids: SmallVec<[ConnectionId; 10]>,
peer: &mut ConnectedPeer<'a, TTrans, TInEvent, TOutEvent, THandler, TConnInfo, TPeerId>,
event: TInEvent,
cx: &mut Context,
) -> Option<(TInEvent, SmallVec<[ConnectionId; 10]>)>
where
TTrans: Transport,
THandler: IntoConnectionHandler<TConnInfo>,
TPeerId: Eq + Hash + Clone,
TConnInfo: ConnectionInfo<PeerId = TPeerId>
{
let mut pending = SmallVec::new();
let mut event = Some(event); // (1)
for id in ids.into_iter() {
if let Some(mut conn) = peer.connection(id) {
match conn.poll_ready_notify_handler(cx) {
Poll::Pending => pending.push(id),
Poll::Ready(Err(())) => {} // connection is closing
Poll::Ready(Ok(())) => {
let e = event.take().expect("by (1),(2)");
if let Err(e) = conn.notify_handler(e) {
event = Some(e) // (2)
} else {
break
}
}
}
}
}
event.and_then(|e|
if !pending.is_empty() {
Some((e, pending))
} else {
None
})
}
/// Notify all of the given connections of a peer of an event.
///
/// Returns `Some` with the given event and a new list of connections if
/// at least one of the given connections is not currently able to receive the event
/// but is not closing, in which case the current task is scheduled to be woken up.
/// The returned connections are those which are not closing.
///
/// Returns `None` if all connections are either closing or the event
/// was successfully sent to all handlers whose connections are not closing,
/// in either case the event is consumed.
fn notify_all<'a, TTrans, TInEvent, TOutEvent, THandler, TConnInfo, TPeerId>(
ids: SmallVec<[ConnectionId; 10]>,
peer: &mut ConnectedPeer<'a, TTrans, TInEvent, TOutEvent, THandler, TConnInfo, TPeerId>,
event: TInEvent,
cx: &mut Context,
) -> Option<(TInEvent, SmallVec<[ConnectionId; 10]>)>
where
TTrans: Transport,
TInEvent: Clone,
THandler: IntoConnectionHandler<TConnInfo>,
TPeerId: Eq + Hash + Clone,
TConnInfo: ConnectionInfo<PeerId = TPeerId>
{
if ids.len() == 1 {
if let Some(mut conn) = peer.connection(ids[0]) {
return notify_one(&mut conn, event, cx).map(|e| (e, ids))
}
}
{
let mut pending = SmallVec::new();
for id in ids.iter() {
if let Some(mut conn) = peer.connection(*id) { // (*)
if conn.poll_ready_notify_handler(cx).is_pending() {
pending.push(*id)
}
}
}
if !pending.is_empty() {
return Some((event, pending))
}
}
for id in ids.into_iter() {
if let Some(mut conn) = peer.connection(id) {
// All connections were ready. Can now only fail due
// to a connection suddenly closing, which we ignore.
let _ = conn.notify_handler(event.clone());
}
}
None
}
impl<TBehaviour, TInEvent, TOutEvent, THandler, TConnInfo> Stream for
ExpandedSwarm<TBehaviour, TInEvent, TOutEvent, THandler, TConnInfo>
where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
THandlerErr: error::Error + Send + 'static,
THandler: IntoProtocolsHandler + Send + 'static,
TInEvent: Send + 'static,
TInEvent: Clone + Send + 'static,
TOutEvent: Send + 'static,
THandler::Handler: ProtocolsHandler<InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr>,
THandler::Handler: ProtocolsHandler<InEvent = TInEvent, OutEvent = TOutEvent>,
TConnInfo: ConnectionInfo<PeerId = PeerId> + fmt::Debug + Clone + Send + 'static,
{
type Item = TBehaviour::OutEvent;
@ -503,14 +750,13 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
}
/// the stream of behaviour events never terminates, so we can implement fused for it
impl<TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo> FusedStream for
ExpandedSwarm<TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo>
impl<TBehaviour, TInEvent, TOutEvent, THandler, TConnInfo> FusedStream for
ExpandedSwarm<TBehaviour, TInEvent, TOutEvent, THandler, TConnInfo>
where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
THandlerErr: error::Error + Send + 'static,
THandler: IntoProtocolsHandler + Send + 'static,
TInEvent: Send + 'static,
TInEvent: Clone + Send + 'static,
TOutEvent: Send + 'static,
THandler::Handler: ProtocolsHandler<InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr>,
THandler::Handler: ProtocolsHandler<InEvent = TInEvent, OutEvent = TOutEvent>,
TConnInfo: ConnectionInfo<PeerId = PeerId> + fmt::Debug + Clone + Send + 'static,
{
fn is_terminated(&self) -> bool {
@ -550,11 +796,10 @@ impl<'a> PollParameters for SwarmPollParameters<'a> {
}
pub struct SwarmBuilder<TBehaviour, TConnInfo> {
incoming_limit: Option<u32>,
executor: Option<Box<dyn Executor + Send>>,
local_peer_id: PeerId,
transport: BoxTransport<(TConnInfo, StreamMuxerBox), io::Error>,
behaviour: TBehaviour,
network: NetworkConfig,
}
impl<TBehaviour, TConnInfo> SwarmBuilder<TBehaviour, TConnInfo>
@ -579,16 +824,15 @@ where TBehaviour: NetworkBehaviour,
.boxed();
SwarmBuilder {
incoming_limit: None,
local_peer_id,
executor: None,
transport,
behaviour,
network: NetworkConfig::default(),
}
}
pub fn incoming_limit(mut self, incoming_limit: Option<u32>) -> Self {
self.incoming_limit = incoming_limit;
pub fn incoming_limit(mut self, incoming_limit: usize) -> Self {
self.network.set_pending_incoming_limit(incoming_limit);
self
}
@ -596,7 +840,7 @@ where TBehaviour: NetworkBehaviour,
///
/// By default, uses a threads pool.
pub fn executor(mut self, executor: impl Executor + Send + 'static) -> Self {
self.executor = Some(Box::new(executor));
self.network.set_executor(Box::new(executor));
self
}
@ -608,7 +852,7 @@ where TBehaviour: NetworkBehaviour,
(self.0)(f)
}
}
self.executor = Some(Box::new(SpawnImpl(executor)));
self.network.set_executor(Box::new(SpawnImpl(executor)));
self
}
@ -621,7 +865,9 @@ where TBehaviour: NetworkBehaviour,
.map(|info| info.protocol_name().to_vec())
.collect();
let executor = self.executor.or_else(|| {
// If no executor has been explicitly configured, try to set up
// a thread pool.
if self.network.executor().is_none() {
struct PoolWrapper(ThreadPool);
impl Executor for PoolWrapper {
fn exec(&self, f: Pin<Box<dyn Future<Output = ()> + Send>>) {
@ -629,18 +875,20 @@ where TBehaviour: NetworkBehaviour,
}
}
ThreadPoolBuilder::new()
if let Some(executor) = ThreadPoolBuilder::new()
.name_prefix("libp2p-task-")
.create()
.ok()
.map(|tp| Box::new(PoolWrapper(tp)) as Box<_>)
});
{
self.network.set_executor(Box::new(executor));
}
}
let network = Network::new_with_incoming_limit(
let network = Network::new(
self.transport,
self.local_peer_id,
executor,
self.incoming_limit
self.network,
);
ExpandedSwarm {
@ -650,7 +898,7 @@ where TBehaviour: NetworkBehaviour,
listened_addrs: SmallVec::new(),
external_addrs: Addresses::default(),
banned_peers: HashSet::new(),
send_event_to_complete: None
pending_event: None
}
}
}
@ -676,7 +924,7 @@ impl NetworkBehaviour for DummyBehaviour {
fn inject_disconnected(&mut self, _: &PeerId, _: libp2p_core::ConnectedPoint) {}
fn inject_node_event(&mut self, _: PeerId,
fn inject_event(&mut self, _: PeerId, _: ConnectionId,
_: <Self::ProtocolsHandler as ProtocolsHandler>::OutEvent) {}
fn poll(&mut self, _: &mut Context, _: &mut impl PollParameters) ->
@ -707,8 +955,9 @@ mod tests {
fn test_build_swarm() {
let id = get_random_id();
let transport = DummyTransport::<(PeerId, Multiplex<DummyStream>)>::new();
let swarm = SwarmBuilder::new(transport, DummyBehaviour {}, id.into())
.incoming_limit(Some(4)).build();
let behaviour = DummyBehaviour {};
let swarm = SwarmBuilder::new(transport, behaviour, id.into())
.incoming_limit(4).build();
assert_eq!(swarm.network.incoming_limit(), Some(4));
}