mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-07-31 00:41:59 +00:00
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:
@@ -19,7 +19,7 @@
|
||||
// DEALINGS IN THE SOFTWARE.
|
||||
|
||||
use crate::protocols_handler::{IntoProtocolsHandler, ProtocolsHandler};
|
||||
use libp2p_core::{ConnectedPoint, Multiaddr, PeerId, nodes::ListenerId};
|
||||
use libp2p_core::{ConnectedPoint, Multiaddr, PeerId, connection::{ConnectionId, ListenerId}};
|
||||
use std::{error, task::Context, task::Poll};
|
||||
|
||||
/// A behaviour for the network. Allows customizing the swarm.
|
||||
@@ -60,7 +60,7 @@ pub trait NetworkBehaviour: Send + 'static {
|
||||
///
|
||||
/// The network behaviour (ie. the implementation of this trait) and the handlers it has
|
||||
/// spawned (ie. the objects returned by `new_handler`) can communicate by passing messages.
|
||||
/// Messages sent from the handler to the behaviour are injected with `inject_node_event`, and
|
||||
/// Messages sent from the handler to the behaviour are injected with `inject_event`, and
|
||||
/// the behaviour can send a message to the handler by making `poll` return `SendEvent`.
|
||||
fn new_handler(&mut self) -> Self::ProtocolsHandler;
|
||||
|
||||
@@ -85,27 +85,15 @@ pub trait NetworkBehaviour: Send + 'static {
|
||||
/// or may not have been processed by the handler.
|
||||
fn inject_disconnected(&mut self, peer_id: &PeerId, endpoint: ConnectedPoint);
|
||||
|
||||
/// Indicates the behaviour that we replace the connection from the node with another.
|
||||
///
|
||||
/// The handler that used to be dedicated to this node has been destroyed and replaced with a
|
||||
/// new one. Any event that has been sent to it may or may not have been processed.
|
||||
///
|
||||
/// The default implementation of this method calls `inject_disconnected` followed with
|
||||
/// `inject_connected`. This is a logically safe way to implement this behaviour. However, you
|
||||
/// may want to overwrite this method in the situations where this isn't appropriate.
|
||||
fn inject_replaced(&mut self, peer_id: PeerId, closed_endpoint: ConnectedPoint, new_endpoint: ConnectedPoint) {
|
||||
self.inject_disconnected(&peer_id, closed_endpoint);
|
||||
self.inject_connected(peer_id, new_endpoint);
|
||||
}
|
||||
|
||||
/// Informs the behaviour about an event generated by the handler dedicated to the peer identified by `peer_id`.
|
||||
/// for the behaviour.
|
||||
///
|
||||
/// The `peer_id` is guaranteed to be in a connected state. In other words, `inject_connected`
|
||||
/// has previously been called with this `PeerId`.
|
||||
fn inject_node_event(
|
||||
fn inject_event(
|
||||
&mut self,
|
||||
peer_id: PeerId,
|
||||
connection: ConnectionId,
|
||||
event: <<Self::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutEvent
|
||||
);
|
||||
|
||||
@@ -218,18 +206,27 @@ pub enum NetworkBehaviourAction<TInEvent, TOutEvent> {
|
||||
peer_id: PeerId,
|
||||
},
|
||||
|
||||
/// Instructs the `Swarm` to send a message to the handler dedicated to the connection with the peer.
|
||||
/// Instructs the `Swarm` to send an event to the handler dedicated to a
|
||||
/// connection with a peer.
|
||||
///
|
||||
/// If the `Swarm` is connected to the peer, the message is delivered to the remote's
|
||||
/// protocol handler. If there is no connection to the peer, the message is ignored.
|
||||
/// To ensure delivery, the `NetworkBehaviour` must keep track of connected peers.
|
||||
/// If the `Swarm` is connected to the peer, the message is delivered to the
|
||||
/// `ProtocolsHandler` instance identified by the peer ID and connection ID.
|
||||
///
|
||||
/// If the specified connection no longer exists, the event is silently dropped.
|
||||
///
|
||||
/// Typically the connection ID given is the same as the one passed to
|
||||
/// [`NetworkBehaviour::inject_event`], i.e. whenever the behaviour wishes to
|
||||
/// respond to a request on the same connection (and possibly the same
|
||||
/// substream, as per the implementation of `ProtocolsHandler`).
|
||||
///
|
||||
/// Note that even if the peer is currently connected, connections can get closed
|
||||
/// at any time and thus the message may not reach its destination.
|
||||
SendEvent {
|
||||
/// The peer to which to send the message.
|
||||
/// at any time and thus the event may not reach a handler.
|
||||
NotifyHandler {
|
||||
/// The peer for whom a `ProtocolsHandler` should be notified.
|
||||
peer_id: PeerId,
|
||||
/// The message to send.
|
||||
/// The ID of the connection whose `ProtocolsHandler` to notify.
|
||||
handler: NotifyHandler,
|
||||
/// The event to send.
|
||||
event: TInEvent,
|
||||
},
|
||||
|
||||
@@ -244,3 +241,15 @@ pub enum NetworkBehaviourAction<TInEvent, TOutEvent> {
|
||||
address: Multiaddr,
|
||||
},
|
||||
}
|
||||
|
||||
/// The options w.r.t. which connection handlers to notify of an event.
|
||||
#[derive(Debug, Clone)]
|
||||
pub enum NotifyHandler {
|
||||
/// Notify a particular connection handler.
|
||||
One(ConnectionId),
|
||||
/// Notify an arbitrary connection handler.
|
||||
Any,
|
||||
/// Notify all connection handlers.
|
||||
All
|
||||
}
|
||||
|
||||
|
479
swarm/src/lib.rs
479
swarm/src/lib.rs
@@ -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));
|
||||
}
|
||||
|
||||
|
@@ -220,19 +220,6 @@ pub trait ProtocolsHandler: Send + 'static {
|
||||
{
|
||||
IntoProtocolsHandler::into_node_handler_builder(self)
|
||||
}
|
||||
|
||||
/// Builds an implementation of `NodeHandler` that handles this protocol exclusively.
|
||||
///
|
||||
/// > **Note**: This is a shortcut for `self.into_node_handler_builder().build()`.
|
||||
#[inline]
|
||||
#[deprecated(note = "Use into_node_handler_builder instead")]
|
||||
fn into_node_handler(self) -> NodeHandlerWrapper<Self>
|
||||
where
|
||||
Self: Sized,
|
||||
{
|
||||
#![allow(deprecated)]
|
||||
self.into_node_handler_builder().build()
|
||||
}
|
||||
}
|
||||
|
||||
/// Configuration of inbound or outbound substream protocol(s)
|
||||
|
@@ -29,12 +29,17 @@ use crate::protocols_handler::{
|
||||
|
||||
use futures::prelude::*;
|
||||
use libp2p_core::{
|
||||
ConnectedPoint,
|
||||
PeerId,
|
||||
ConnectionInfo,
|
||||
Connected,
|
||||
connection::{
|
||||
ConnectionHandler,
|
||||
ConnectionHandlerEvent,
|
||||
IntoConnectionHandler,
|
||||
Substream,
|
||||
SubstreamEndpoint,
|
||||
},
|
||||
muxing::StreamMuxerBox,
|
||||
nodes::Substream,
|
||||
nodes::collection::ConnectionInfo,
|
||||
nodes::handled_node::{IntoNodeHandler, NodeHandler, NodeHandlerEndpoint, NodeHandlerEvent},
|
||||
upgrade::{self, InboundUpgradeApply, OutboundUpgradeApply}
|
||||
};
|
||||
use std::{error, fmt, pin::Pin, task::Context, task::Poll, time::Duration};
|
||||
@@ -51,31 +56,14 @@ where
|
||||
TIntoProtoHandler: IntoProtocolsHandler
|
||||
{
|
||||
/// Builds a `NodeHandlerWrapperBuilder`.
|
||||
#[inline]
|
||||
pub(crate) fn new(handler: TIntoProtoHandler) -> Self {
|
||||
NodeHandlerWrapperBuilder {
|
||||
handler,
|
||||
}
|
||||
}
|
||||
|
||||
/// Builds the `NodeHandlerWrapper`.
|
||||
#[deprecated(note = "Pass the NodeHandlerWrapperBuilder directly")]
|
||||
#[inline]
|
||||
pub fn build(self) -> NodeHandlerWrapper<TIntoProtoHandler>
|
||||
where TIntoProtoHandler: ProtocolsHandler
|
||||
{
|
||||
NodeHandlerWrapper {
|
||||
handler: self.handler,
|
||||
negotiating_in: Vec::new(),
|
||||
negotiating_out: Vec::new(),
|
||||
queued_dial_upgrades: Vec::new(),
|
||||
unique_dial_upgrade_id: 0,
|
||||
shutdown: Shutdown::None,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<TIntoProtoHandler, TProtoHandler, TConnInfo> IntoNodeHandler<(TConnInfo, ConnectedPoint)>
|
||||
impl<TIntoProtoHandler, TProtoHandler, TConnInfo> IntoConnectionHandler<TConnInfo>
|
||||
for NodeHandlerWrapperBuilder<TIntoProtoHandler>
|
||||
where
|
||||
TIntoProtoHandler: IntoProtocolsHandler<Handler = TProtoHandler>,
|
||||
@@ -84,9 +72,9 @@ where
|
||||
{
|
||||
type Handler = NodeHandlerWrapper<TIntoProtoHandler::Handler>;
|
||||
|
||||
fn into_handler(self, remote_info: &(TConnInfo, ConnectedPoint)) -> Self::Handler {
|
||||
fn into_handler(self, connected: &Connected<TConnInfo>) -> Self::Handler {
|
||||
NodeHandlerWrapper {
|
||||
handler: self.handler.into_handler(&remote_info.0.peer_id(), &remote_info.1),
|
||||
handler: self.handler.into_handler(connected.peer_id(), &connected.endpoint),
|
||||
negotiating_in: Vec::new(),
|
||||
negotiating_out: Vec::new(),
|
||||
queued_dial_upgrades: Vec::new(),
|
||||
@@ -96,6 +84,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
// A `ConnectionHandler` for an underlying `ProtocolsHandler`.
|
||||
/// Wraps around an implementation of `ProtocolsHandler`, and implements `NodeHandler`.
|
||||
// TODO: add a caching system for protocols that are supported or not
|
||||
pub struct NodeHandlerWrapper<TProtoHandler>
|
||||
@@ -181,7 +170,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<TProtoHandler> NodeHandler for NodeHandlerWrapper<TProtoHandler>
|
||||
impl<TProtoHandler> ConnectionHandler for NodeHandlerWrapper<TProtoHandler>
|
||||
where
|
||||
TProtoHandler: ProtocolsHandler,
|
||||
{
|
||||
@@ -196,17 +185,17 @@ where
|
||||
fn inject_substream(
|
||||
&mut self,
|
||||
substream: Self::Substream,
|
||||
endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>,
|
||||
endpoint: SubstreamEndpoint<Self::OutboundOpenInfo>,
|
||||
) {
|
||||
match endpoint {
|
||||
NodeHandlerEndpoint::Listener => {
|
||||
SubstreamEndpoint::Listener => {
|
||||
let protocol = self.handler.listen_protocol();
|
||||
let timeout = protocol.timeout().clone();
|
||||
let upgrade = upgrade::apply_inbound(substream, SendWrapper(protocol.into_upgrade().1));
|
||||
let timeout = Delay::new(timeout);
|
||||
self.negotiating_in.push((upgrade, timeout));
|
||||
}
|
||||
NodeHandlerEndpoint::Dialer((upgrade_id, user_data, timeout)) => {
|
||||
SubstreamEndpoint::Dialer((upgrade_id, user_data, timeout)) => {
|
||||
let pos = match self
|
||||
.queued_dial_upgrades
|
||||
.iter()
|
||||
@@ -227,12 +216,13 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn inject_event(&mut self, event: Self::InEvent) {
|
||||
self.handler.inject_event(event);
|
||||
}
|
||||
|
||||
fn poll(&mut self, cx: &mut Context) -> Poll<Result<NodeHandlerEvent<Self::OutboundOpenInfo, Self::OutEvent>, Self::Error>> {
|
||||
fn poll(&mut self, cx: &mut Context) -> Poll<
|
||||
Result<ConnectionHandlerEvent<Self::OutboundOpenInfo, Self::OutEvent>, Self::Error>
|
||||
> {
|
||||
// Continue negotiation of newly-opened substreams on the listening side.
|
||||
// We remove each element from `negotiating_in` one by one and add them back if not ready.
|
||||
for n in (0..self.negotiating_in.len()).rev() {
|
||||
@@ -300,7 +290,7 @@ where
|
||||
|
||||
match poll_result {
|
||||
Poll::Ready(ProtocolsHandlerEvent::Custom(event)) => {
|
||||
return Poll::Ready(Ok(NodeHandlerEvent::Custom(event)));
|
||||
return Poll::Ready(Ok(ConnectionHandlerEvent::Custom(event)));
|
||||
}
|
||||
Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
|
||||
protocol,
|
||||
@@ -312,7 +302,7 @@ where
|
||||
let (version, upgrade) = protocol.into_upgrade();
|
||||
self.queued_dial_upgrades.push((id, (version, SendWrapper(upgrade))));
|
||||
return Poll::Ready(Ok(
|
||||
NodeHandlerEvent::OutboundSubstreamRequest((id, info, timeout)),
|
||||
ConnectionHandlerEvent::OutboundSubstreamRequest((id, info, timeout)),
|
||||
));
|
||||
}
|
||||
Poll::Ready(ProtocolsHandlerEvent::Close(err)) => return Poll::Ready(Err(err.into())),
|
||||
|
@@ -33,6 +33,7 @@ use libp2p_core::{
|
||||
ConnectedPoint,
|
||||
PeerId,
|
||||
Multiaddr,
|
||||
connection::ConnectionId,
|
||||
either::EitherOutput,
|
||||
upgrade::{DeniedUpgrade, EitherUpgrade}
|
||||
};
|
||||
@@ -87,19 +88,14 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
fn inject_replaced(&mut self, peer_id: PeerId, closed_endpoint: ConnectedPoint, new_endpoint: ConnectedPoint) {
|
||||
if let Some(inner) = self.inner.as_mut() {
|
||||
inner.inject_replaced(peer_id, closed_endpoint, new_endpoint)
|
||||
}
|
||||
}
|
||||
|
||||
fn inject_node_event(
|
||||
fn inject_event(
|
||||
&mut self,
|
||||
peer_id: PeerId,
|
||||
connection: ConnectionId,
|
||||
event: <<Self::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutEvent
|
||||
) {
|
||||
if let Some(inner) = self.inner.as_mut() {
|
||||
inner.inject_node_event(peer_id, event);
|
||||
inner.inject_event(peer_id, connection, event);
|
||||
}
|
||||
}
|
||||
|
||||
|
Reference in New Issue
Block a user