mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-26 16:21:39 +00:00
*: Dial with handler and return handler on error and closed (#2191)
Require `NetworkBehaviourAction::{DialPeer,DialAddress}` to contain a `ProtocolsHandler`. This allows a behaviour to attach custom state to its handler. The behaviour would no longer need to track this state separately during connection establishment, thus reducing state required in a behaviour. E.g. in the case of `libp2p-kad` the behaviour can include a `GetRecord` request in its handler, or e.g. in the case of `libp2p-request-response` the behaviour can include the first request in the handler. Return `ProtocolsHandler` on connection error and close. This allows a behaviour to extract its custom state previously included in the handler on connection failure and connection closing. E.g. in the case of `libp2p-kad` the behaviour could extract the attached `GetRecord` from the handler of the failed connection and then start another connection attempt with a new handler with the same `GetRecord` or bubble up an error to the user. Co-authored-by: Thomas Eizinger <thomas@eizinger.io>
This commit is contained in:
@ -19,13 +19,17 @@
|
||||
// DEALINGS IN THE SOFTWARE.
|
||||
|
||||
use crate::protocols_handler::{IntoProtocolsHandler, ProtocolsHandler};
|
||||
use crate::{AddressRecord, AddressScore};
|
||||
use crate::{AddressRecord, AddressScore, DialError};
|
||||
use libp2p_core::{
|
||||
connection::{ConnectionId, ListenerId},
|
||||
ConnectedPoint, Multiaddr, PeerId,
|
||||
};
|
||||
use std::{error, task::Context, task::Poll};
|
||||
|
||||
/// Custom event that can be received by the [`ProtocolsHandler`].
|
||||
type THandlerInEvent<THandler> =
|
||||
<<THandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InEvent;
|
||||
|
||||
/// A behaviour for the network. Allows customizing the swarm.
|
||||
///
|
||||
/// This trait has been designed to be composable. Multiple implementations can be combined into
|
||||
@ -65,16 +69,20 @@ pub trait NetworkBehaviour: Send + 'static {
|
||||
|
||||
/// Creates a new `ProtocolsHandler` for a connection with a peer.
|
||||
///
|
||||
/// Every time an incoming connection is opened, and every time we start dialing a node, this
|
||||
/// method is called.
|
||||
/// Every time an incoming connection is opened, and every time another [`NetworkBehaviour`]
|
||||
/// emitted a dial request, this method is called.
|
||||
///
|
||||
/// The returned object is a handler for that specific connection, and will be moved to a
|
||||
/// background task dedicated to that connection.
|
||||
///
|
||||
/// 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_event`, and
|
||||
/// the behaviour can send a message to the handler by making `poll` return `SendEvent`.
|
||||
/// 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 [`NetworkBehaviour::inject_event`],
|
||||
/// and the behaviour can send a message to the handler by making [`NetworkBehaviour::poll`]
|
||||
/// return [`NetworkBehaviourAction::NotifyHandler`].
|
||||
///
|
||||
/// Note that the handler is returned to the [`NetworkBehaviour`] on connection failure and
|
||||
/// connection closing.
|
||||
fn new_handler(&mut self) -> Self::ProtocolsHandler;
|
||||
|
||||
/// Addresses that this behaviour is aware of for this specific peer, and that may allow
|
||||
@ -112,7 +120,14 @@ pub trait NetworkBehaviour: Send + 'static {
|
||||
/// A call to this method is always paired with an earlier call to
|
||||
/// `inject_connection_established` with the same peer ID, connection ID and
|
||||
/// endpoint.
|
||||
fn inject_connection_closed(&mut self, _: &PeerId, _: &ConnectionId, _: &ConnectedPoint) {}
|
||||
fn inject_connection_closed(
|
||||
&mut self,
|
||||
_: &PeerId,
|
||||
_: &ConnectionId,
|
||||
_: &ConnectedPoint,
|
||||
_: <Self::ProtocolsHandler as IntoProtocolsHandler>::Handler,
|
||||
) {
|
||||
}
|
||||
|
||||
/// Informs the behaviour that the [`ConnectedPoint`] of an existing connection has changed.
|
||||
fn inject_address_change(
|
||||
@ -153,7 +168,26 @@ pub trait NetworkBehaviour: Send + 'static {
|
||||
///
|
||||
/// The `peer_id` is guaranteed to be in a disconnected state. In other words,
|
||||
/// `inject_connected` has not been called, or `inject_disconnected` has been called since then.
|
||||
fn inject_dial_failure(&mut self, _peer_id: &PeerId) {}
|
||||
fn inject_dial_failure(
|
||||
&mut self,
|
||||
_peer_id: &PeerId,
|
||||
_handler: Self::ProtocolsHandler,
|
||||
_error: DialError,
|
||||
) {
|
||||
}
|
||||
|
||||
/// Indicates to the behaviour that an error happened on an incoming connection during its
|
||||
/// initial handshake.
|
||||
///
|
||||
/// This can include, for example, an error during the handshake of the encryption layer, or the
|
||||
/// connection unexpectedly closed.
|
||||
fn inject_listen_failure(
|
||||
&mut self,
|
||||
_local_addr: &Multiaddr,
|
||||
_send_back_addr: &Multiaddr,
|
||||
_handler: Self::ProtocolsHandler,
|
||||
) {
|
||||
}
|
||||
|
||||
/// Indicates to the behaviour that a new listener was created.
|
||||
fn inject_new_listener(&mut self, _id: ListenerId) {}
|
||||
@ -182,8 +216,11 @@ pub trait NetworkBehaviour: Send + 'static {
|
||||
///
|
||||
/// This API mimics the API of the `Stream` trait. The method may register the current task in
|
||||
/// order to wake it up at a later point in time.
|
||||
fn poll(&mut self, cx: &mut Context<'_>, params: &mut impl PollParameters)
|
||||
-> Poll<NetworkBehaviourAction<<<Self::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InEvent, Self::OutEvent>>;
|
||||
fn poll(
|
||||
&mut self,
|
||||
cx: &mut Context<'_>,
|
||||
params: &mut impl PollParameters,
|
||||
) -> Poll<NetworkBehaviourAction<Self::OutEvent, Self::ProtocolsHandler>>;
|
||||
}
|
||||
|
||||
/// Parameters passed to `poll()`, that the `NetworkBehaviour` has access to.
|
||||
@ -228,15 +265,35 @@ pub trait NetworkBehaviourEventProcess<TEvent> {
|
||||
/// in whose context it is executing.
|
||||
///
|
||||
/// [`Swarm`]: super::Swarm
|
||||
//
|
||||
// Note: `TInEvent` is needed to be able to implement
|
||||
// [`NetworkBehaviourAction::map_in`], mapping the handler `InEvent` leaving the
|
||||
// handler itself untouched.
|
||||
#[derive(Debug)]
|
||||
pub enum NetworkBehaviourAction<TInEvent, TOutEvent> {
|
||||
pub enum NetworkBehaviourAction<
|
||||
TOutEvent,
|
||||
THandler: IntoProtocolsHandler,
|
||||
TInEvent = THandlerInEvent<THandler>,
|
||||
> {
|
||||
/// Instructs the `Swarm` to return an event when it is being polled.
|
||||
GenerateEvent(TOutEvent),
|
||||
|
||||
/// Instructs the swarm to dial the given multiaddress optionally including a [`PeerId`].
|
||||
///
|
||||
/// On success, [`NetworkBehaviour::inject_connection_established`] is invoked.
|
||||
/// On failure, [`NetworkBehaviour::inject_dial_failure`] is invoked.
|
||||
///
|
||||
/// Note that the provided handler is returned to the [`NetworkBehaviour`] on connection failure
|
||||
/// and connection closing. Thus it can be used to carry state, which otherwise would have to be
|
||||
/// tracked in the [`NetworkBehaviour`] itself. E.g. a message destined to an unconnected peer
|
||||
/// can be included in the handler, and thus directly send on connection success or extracted by
|
||||
/// the [`NetworkBehaviour`] on connection failure. See [`NetworkBehaviourAction::DialPeer`] for
|
||||
/// example.
|
||||
DialAddress {
|
||||
/// The address to dial.
|
||||
address: Multiaddr,
|
||||
/// The handler to be used to handle the connection to the peer.
|
||||
handler: THandler,
|
||||
},
|
||||
|
||||
/// Instructs the swarm to dial a known `PeerId`.
|
||||
@ -247,13 +304,194 @@ pub enum NetworkBehaviourAction<TInEvent, TOutEvent> {
|
||||
/// If we were already trying to dial this node, the addresses that are not yet in the queue of
|
||||
/// addresses to try are added back to this queue.
|
||||
///
|
||||
/// On success, [`NetworkBehaviour::inject_connected`] is invoked.
|
||||
/// On success, [`NetworkBehaviour::inject_connection_established`] is invoked.
|
||||
/// On failure, [`NetworkBehaviour::inject_dial_failure`] is invoked.
|
||||
///
|
||||
/// Note that the provided handler is returned to the [`NetworkBehaviour`] on connection failure
|
||||
/// and connection closing. Thus it can be used to carry state, which otherwise would have to be
|
||||
/// tracked in the [`NetworkBehaviour`] itself. E.g. a message destined to an unconnected peer
|
||||
/// can be included in the handler, and thus directly send on connection success or extracted by
|
||||
/// the [`NetworkBehaviour`] on connection failure.
|
||||
///
|
||||
/// # Example
|
||||
///
|
||||
/// ```rust
|
||||
/// # use futures::executor::block_on;
|
||||
/// # use futures::stream::StreamExt;
|
||||
/// # use libp2p::core::connection::ConnectionId;
|
||||
/// # use libp2p::core::identity;
|
||||
/// # use libp2p::core::transport::{MemoryTransport, Transport};
|
||||
/// # use libp2p::core::upgrade::{self, DeniedUpgrade, InboundUpgrade, OutboundUpgrade};
|
||||
/// # use libp2p::core::PeerId;
|
||||
/// # use libp2p::plaintext::PlainText2Config;
|
||||
/// # use libp2p::swarm::{
|
||||
/// # DialError, DialPeerCondition, IntoProtocolsHandler, KeepAlive, NegotiatedSubstream,
|
||||
/// # NetworkBehaviour, NetworkBehaviourAction, PollParameters, ProtocolsHandler,
|
||||
/// # ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr, SubstreamProtocol, Swarm, SwarmEvent,
|
||||
/// # };
|
||||
/// # use libp2p::yamux;
|
||||
/// # use std::collections::VecDeque;
|
||||
/// # use std::task::{Context, Poll};
|
||||
/// # use void::Void;
|
||||
/// #
|
||||
/// # let local_key = identity::Keypair::generate_ed25519();
|
||||
/// # let local_public_key = local_key.public();
|
||||
/// # let local_peer_id = PeerId::from(local_public_key.clone());
|
||||
/// #
|
||||
/// # let transport = MemoryTransport::default()
|
||||
/// # .upgrade(upgrade::Version::V1)
|
||||
/// # .authenticate(PlainText2Config { local_public_key })
|
||||
/// # .multiplex(yamux::YamuxConfig::default())
|
||||
/// # .boxed();
|
||||
/// #
|
||||
/// # let mut swarm = Swarm::new(transport, MyBehaviour::default(), local_peer_id);
|
||||
/// #
|
||||
/// // Super precious message that we should better not lose.
|
||||
/// let message = PreciousMessage("My precious message".to_string());
|
||||
///
|
||||
/// // Unfortunately this peer is offline, thus sending our message to it will fail.
|
||||
/// let offline_peer = PeerId::random();
|
||||
///
|
||||
/// // Let's send it anyways. We should get it back in case connecting to the peer fails.
|
||||
/// swarm.behaviour_mut().send(offline_peer, message);
|
||||
///
|
||||
/// block_on(async {
|
||||
/// // As expected, sending failed. But great news, we got our message back.
|
||||
/// matches!(
|
||||
/// swarm.next().await.expect("Infinite stream"),
|
||||
/// SwarmEvent::Behaviour(PreciousMessage(_))
|
||||
/// );
|
||||
/// });
|
||||
///
|
||||
/// # #[derive(Default)]
|
||||
/// # struct MyBehaviour {
|
||||
/// # outbox_to_swarm: VecDeque<NetworkBehaviourAction<PreciousMessage, MyHandler>>,
|
||||
/// # }
|
||||
/// #
|
||||
/// # impl MyBehaviour {
|
||||
/// # fn send(&mut self, peer_id: PeerId, msg: PreciousMessage) {
|
||||
/// # self.outbox_to_swarm
|
||||
/// # .push_back(NetworkBehaviourAction::DialPeer {
|
||||
/// # peer_id,
|
||||
/// # condition: DialPeerCondition::Always,
|
||||
/// # handler: MyHandler { message: Some(msg) },
|
||||
/// # });
|
||||
/// # }
|
||||
/// # }
|
||||
/// #
|
||||
/// impl NetworkBehaviour for MyBehaviour {
|
||||
/// # type ProtocolsHandler = MyHandler;
|
||||
/// # type OutEvent = PreciousMessage;
|
||||
/// #
|
||||
/// # fn new_handler(&mut self) -> Self::ProtocolsHandler {
|
||||
/// # MyHandler { message: None }
|
||||
/// # }
|
||||
/// #
|
||||
/// #
|
||||
/// # fn inject_event(
|
||||
/// # &mut self,
|
||||
/// # _: PeerId,
|
||||
/// # _: ConnectionId,
|
||||
/// # _: <<Self::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutEvent,
|
||||
/// # ) {
|
||||
/// # unreachable!();
|
||||
/// # }
|
||||
/// #
|
||||
/// fn inject_dial_failure(
|
||||
/// &mut self,
|
||||
/// _: &PeerId,
|
||||
/// handler: Self::ProtocolsHandler,
|
||||
/// _: DialError,
|
||||
/// ) {
|
||||
/// // As expected, sending the message failed. But lucky us, we got the handler back, thus
|
||||
/// // the precious message is not lost and we can return it back to the user.
|
||||
/// let msg = handler.message.unwrap();
|
||||
/// self.outbox_to_swarm
|
||||
/// .push_back(NetworkBehaviourAction::GenerateEvent(msg))
|
||||
/// }
|
||||
/// #
|
||||
/// # fn poll(
|
||||
/// # &mut self,
|
||||
/// # _: &mut Context<'_>,
|
||||
/// # _: &mut impl PollParameters,
|
||||
/// # ) -> Poll<NetworkBehaviourAction<Self::OutEvent, Self::ProtocolsHandler>> {
|
||||
/// # if let Some(action) = self.outbox_to_swarm.pop_front() {
|
||||
/// # return Poll::Ready(action);
|
||||
/// # }
|
||||
/// # Poll::Pending
|
||||
/// # }
|
||||
/// }
|
||||
///
|
||||
/// # struct MyHandler {
|
||||
/// # message: Option<PreciousMessage>,
|
||||
/// # }
|
||||
/// #
|
||||
/// # impl ProtocolsHandler for MyHandler {
|
||||
/// # type InEvent = Void;
|
||||
/// # type OutEvent = Void;
|
||||
/// # type Error = Void;
|
||||
/// # type InboundProtocol = DeniedUpgrade;
|
||||
/// # type OutboundProtocol = DeniedUpgrade;
|
||||
/// # type InboundOpenInfo = ();
|
||||
/// # type OutboundOpenInfo = Void;
|
||||
/// #
|
||||
/// # fn listen_protocol(
|
||||
/// # &self,
|
||||
/// # ) -> SubstreamProtocol<Self::InboundProtocol, Self::InboundOpenInfo> {
|
||||
/// # SubstreamProtocol::new(DeniedUpgrade, ())
|
||||
/// # }
|
||||
/// #
|
||||
/// # fn inject_fully_negotiated_inbound(
|
||||
/// # &mut self,
|
||||
/// # _: <Self::InboundProtocol as InboundUpgrade<NegotiatedSubstream>>::Output,
|
||||
/// # _: Self::InboundOpenInfo,
|
||||
/// # ) {
|
||||
/// # }
|
||||
/// #
|
||||
/// # fn inject_fully_negotiated_outbound(
|
||||
/// # &mut self,
|
||||
/// # _: <Self::OutboundProtocol as OutboundUpgrade<NegotiatedSubstream>>::Output,
|
||||
/// # _: Self::OutboundOpenInfo,
|
||||
/// # ) {
|
||||
/// # }
|
||||
/// #
|
||||
/// # fn inject_event(&mut self, _event: Self::InEvent) {}
|
||||
/// #
|
||||
/// # fn inject_dial_upgrade_error(
|
||||
/// # &mut self,
|
||||
/// # _: Self::OutboundOpenInfo,
|
||||
/// # _: ProtocolsHandlerUpgrErr<Void>,
|
||||
/// # ) {
|
||||
/// # }
|
||||
/// #
|
||||
/// # fn connection_keep_alive(&self) -> KeepAlive {
|
||||
/// # KeepAlive::Yes
|
||||
/// # }
|
||||
/// #
|
||||
/// # fn poll(
|
||||
/// # &mut self,
|
||||
/// # _: &mut Context<'_>,
|
||||
/// # ) -> Poll<
|
||||
/// # ProtocolsHandlerEvent<
|
||||
/// # Self::OutboundProtocol,
|
||||
/// # Self::OutboundOpenInfo,
|
||||
/// # Self::OutEvent,
|
||||
/// # Self::Error,
|
||||
/// # >,
|
||||
/// # > {
|
||||
/// # todo!("If `Self::message.is_some()` send the message to the remote.")
|
||||
/// # }
|
||||
/// # }
|
||||
/// # #[derive(Debug, PartialEq, Eq)]
|
||||
/// # struct PreciousMessage(String);
|
||||
/// ```
|
||||
DialPeer {
|
||||
/// The peer to try reach.
|
||||
peer_id: PeerId,
|
||||
/// The condition for initiating a new dialing attempt.
|
||||
condition: DialPeerCondition,
|
||||
/// The handler to be used to handle the connection to the peer.
|
||||
handler: THandler,
|
||||
},
|
||||
|
||||
/// Instructs the `Swarm` to send an event to the handler dedicated to a
|
||||
@ -314,17 +552,28 @@ pub enum NetworkBehaviourAction<TInEvent, TOutEvent> {
|
||||
},
|
||||
}
|
||||
|
||||
impl<TInEvent, TOutEvent> NetworkBehaviourAction<TInEvent, TOutEvent> {
|
||||
impl<TOutEvent, THandler: IntoProtocolsHandler, TInEventOld>
|
||||
NetworkBehaviourAction<TOutEvent, THandler, TInEventOld>
|
||||
{
|
||||
/// Map the handler event.
|
||||
pub fn map_in<E>(self, f: impl FnOnce(TInEvent) -> E) -> NetworkBehaviourAction<E, TOutEvent> {
|
||||
pub fn map_in<TInEventNew>(
|
||||
self,
|
||||
f: impl FnOnce(TInEventOld) -> TInEventNew,
|
||||
) -> NetworkBehaviourAction<TOutEvent, THandler, TInEventNew> {
|
||||
match self {
|
||||
NetworkBehaviourAction::GenerateEvent(e) => NetworkBehaviourAction::GenerateEvent(e),
|
||||
NetworkBehaviourAction::DialAddress { address } => {
|
||||
NetworkBehaviourAction::DialAddress { address }
|
||||
}
|
||||
NetworkBehaviourAction::DialPeer { peer_id, condition } => {
|
||||
NetworkBehaviourAction::DialPeer { peer_id, condition }
|
||||
NetworkBehaviourAction::DialAddress { address, handler } => {
|
||||
NetworkBehaviourAction::DialAddress { address, handler }
|
||||
}
|
||||
NetworkBehaviourAction::DialPeer {
|
||||
peer_id,
|
||||
condition,
|
||||
handler,
|
||||
} => NetworkBehaviourAction::DialPeer {
|
||||
peer_id,
|
||||
condition,
|
||||
handler,
|
||||
},
|
||||
NetworkBehaviourAction::NotifyHandler {
|
||||
peer_id,
|
||||
handler,
|
||||
@ -346,17 +595,25 @@ impl<TInEvent, TOutEvent> NetworkBehaviourAction<TInEvent, TOutEvent> {
|
||||
},
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<TOutEvent, THandler: IntoProtocolsHandler> NetworkBehaviourAction<TOutEvent, THandler> {
|
||||
/// Map the event the swarm will return.
|
||||
pub fn map_out<E>(self, f: impl FnOnce(TOutEvent) -> E) -> NetworkBehaviourAction<TInEvent, E> {
|
||||
pub fn map_out<E>(self, f: impl FnOnce(TOutEvent) -> E) -> NetworkBehaviourAction<E, THandler> {
|
||||
match self {
|
||||
NetworkBehaviourAction::GenerateEvent(e) => NetworkBehaviourAction::GenerateEvent(f(e)),
|
||||
NetworkBehaviourAction::DialAddress { address } => {
|
||||
NetworkBehaviourAction::DialAddress { address }
|
||||
}
|
||||
NetworkBehaviourAction::DialPeer { peer_id, condition } => {
|
||||
NetworkBehaviourAction::DialPeer { peer_id, condition }
|
||||
NetworkBehaviourAction::DialAddress { address, handler } => {
|
||||
NetworkBehaviourAction::DialAddress { address, handler }
|
||||
}
|
||||
NetworkBehaviourAction::DialPeer {
|
||||
peer_id,
|
||||
condition,
|
||||
handler,
|
||||
} => NetworkBehaviourAction::DialPeer {
|
||||
peer_id,
|
||||
condition,
|
||||
handler,
|
||||
},
|
||||
NetworkBehaviourAction::NotifyHandler {
|
||||
peer_id,
|
||||
handler,
|
||||
@ -380,6 +637,60 @@ impl<TInEvent, TOutEvent> NetworkBehaviourAction<TInEvent, TOutEvent> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<TInEvent, TOutEvent, THandlerOld> NetworkBehaviourAction<TOutEvent, THandlerOld>
|
||||
where
|
||||
THandlerOld: IntoProtocolsHandler,
|
||||
<THandlerOld as IntoProtocolsHandler>::Handler: ProtocolsHandler<InEvent = TInEvent>,
|
||||
{
|
||||
/// Map the handler.
|
||||
pub fn map_handler<THandlerNew>(
|
||||
self,
|
||||
f: impl FnOnce(THandlerOld) -> THandlerNew,
|
||||
) -> NetworkBehaviourAction<TOutEvent, THandlerNew>
|
||||
where
|
||||
THandlerNew: IntoProtocolsHandler,
|
||||
<THandlerNew as IntoProtocolsHandler>::Handler: ProtocolsHandler<InEvent = TInEvent>,
|
||||
{
|
||||
match self {
|
||||
NetworkBehaviourAction::GenerateEvent(e) => NetworkBehaviourAction::GenerateEvent(e),
|
||||
NetworkBehaviourAction::DialAddress { address, handler } => {
|
||||
NetworkBehaviourAction::DialAddress {
|
||||
address,
|
||||
handler: f(handler),
|
||||
}
|
||||
}
|
||||
NetworkBehaviourAction::DialPeer {
|
||||
peer_id,
|
||||
condition,
|
||||
handler,
|
||||
} => NetworkBehaviourAction::DialPeer {
|
||||
peer_id,
|
||||
condition,
|
||||
handler: f(handler),
|
||||
},
|
||||
NetworkBehaviourAction::NotifyHandler {
|
||||
peer_id,
|
||||
handler,
|
||||
event,
|
||||
} => NetworkBehaviourAction::NotifyHandler {
|
||||
peer_id,
|
||||
handler,
|
||||
event: event,
|
||||
},
|
||||
NetworkBehaviourAction::ReportObservedAddr { address, score } => {
|
||||
NetworkBehaviourAction::ReportObservedAddr { address, score }
|
||||
}
|
||||
NetworkBehaviourAction::CloseConnection {
|
||||
peer_id,
|
||||
connection,
|
||||
} => NetworkBehaviourAction::CloseConnection {
|
||||
peer_id,
|
||||
connection,
|
||||
},
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// The options w.r.t. which connection handler to notify of an event.
|
||||
#[derive(Debug, Clone)]
|
||||
pub enum NotifyHandler {
|
||||
@ -392,7 +703,6 @@ pub enum NotifyHandler {
|
||||
/// The available conditions under which a new dialing attempt to
|
||||
/// a peer is initiated when requested by [`NetworkBehaviourAction::DialPeer`].
|
||||
#[derive(Debug, Copy, Clone)]
|
||||
#[non_exhaustive]
|
||||
pub enum DialPeerCondition {
|
||||
/// A new dialing attempt is initiated _only if_ the peer is currently
|
||||
/// considered disconnected, i.e. there is no established connection
|
||||
|
263
swarm/src/lib.rs
263
swarm/src/lib.rs
@ -82,8 +82,8 @@ use libp2p_core::{
|
||||
},
|
||||
muxing::StreamMuxerBox,
|
||||
network::{
|
||||
self, peer::ConnectedPeer, ConnectionLimits, Network, NetworkConfig, NetworkEvent,
|
||||
NetworkInfo,
|
||||
self, peer::ConnectedPeer, ConnectionLimits, DialAttemptsRemaining, Network, NetworkConfig,
|
||||
NetworkEvent, NetworkInfo,
|
||||
},
|
||||
transport::{self, TransportError},
|
||||
upgrade::ProtocolName,
|
||||
@ -331,19 +331,40 @@ where
|
||||
|
||||
/// Initiates a new dialing attempt to the given address.
|
||||
pub fn dial_addr(&mut self, addr: Multiaddr) -> Result<(), DialError> {
|
||||
let handler = self
|
||||
.behaviour
|
||||
.new_handler()
|
||||
let handler = self.behaviour.new_handler();
|
||||
self.dial_addr_with_handler(addr, handler)
|
||||
.map_err(|e| DialError::from_network_dial_error(e))
|
||||
.map_err(|(e, _)| e)
|
||||
}
|
||||
|
||||
fn dial_addr_with_handler(
|
||||
&mut self,
|
||||
addr: Multiaddr,
|
||||
handler: <TBehaviour as NetworkBehaviour>::ProtocolsHandler,
|
||||
) -> Result<(), network::DialError<NodeHandlerWrapperBuilder<THandler<TBehaviour>>>> {
|
||||
let handler = handler
|
||||
.into_node_handler_builder()
|
||||
.with_substream_upgrade_protocol_override(self.substream_upgrade_protocol_override);
|
||||
Ok(self.network.dial(&addr, handler).map(|_id| ())?)
|
||||
|
||||
self.network.dial(&addr, handler).map(|_id| ())
|
||||
}
|
||||
|
||||
/// Initiates a new dialing attempt to the given peer.
|
||||
pub fn dial(&mut self, peer_id: &PeerId) -> Result<(), DialError> {
|
||||
let handler = self.behaviour.new_handler();
|
||||
self.dial_with_handler(peer_id, handler)
|
||||
}
|
||||
|
||||
fn dial_with_handler(
|
||||
&mut self,
|
||||
peer_id: &PeerId,
|
||||
handler: <TBehaviour as NetworkBehaviour>::ProtocolsHandler,
|
||||
) -> Result<(), DialError> {
|
||||
if self.banned_peers.contains(peer_id) {
|
||||
self.behaviour.inject_dial_failure(peer_id);
|
||||
return Err(DialError::Banned);
|
||||
let error = DialError::Banned;
|
||||
self.behaviour
|
||||
.inject_dial_failure(peer_id, handler, error.clone());
|
||||
return Err(error);
|
||||
}
|
||||
|
||||
let self_listening = &self.listened_addrs;
|
||||
@ -353,31 +374,31 @@ where
|
||||
.into_iter()
|
||||
.filter(|a| !self_listening.contains(a));
|
||||
|
||||
let result = if let Some(first) = addrs.next() {
|
||||
let handler = self
|
||||
.behaviour
|
||||
.new_handler()
|
||||
.into_node_handler_builder()
|
||||
.with_substream_upgrade_protocol_override(self.substream_upgrade_protocol_override);
|
||||
self.network
|
||||
.peer(*peer_id)
|
||||
.dial(first, addrs, handler)
|
||||
.map(|_| ())
|
||||
.map_err(DialError::from)
|
||||
} else {
|
||||
Err(DialError::NoAddresses)
|
||||
let first = match addrs.next() {
|
||||
Some(first) => first,
|
||||
None => {
|
||||
let error = DialError::NoAddresses;
|
||||
self.behaviour
|
||||
.inject_dial_failure(peer_id, handler, error.clone());
|
||||
return Err(error);
|
||||
}
|
||||
};
|
||||
|
||||
if let Err(error) = &result {
|
||||
log::debug!(
|
||||
"New dialing attempt to peer {:?} failed: {:?}.",
|
||||
peer_id,
|
||||
error
|
||||
);
|
||||
self.behaviour.inject_dial_failure(&peer_id);
|
||||
let handler = handler
|
||||
.into_node_handler_builder()
|
||||
.with_substream_upgrade_protocol_override(self.substream_upgrade_protocol_override);
|
||||
match self.network.peer(*peer_id).dial(first, addrs, handler) {
|
||||
Ok(_connection_id) => Ok(()),
|
||||
Err(error) => {
|
||||
let (error, handler) = DialError::from_network_dial_error(error);
|
||||
self.behaviour.inject_dial_failure(
|
||||
&peer_id,
|
||||
handler.into_protocols_handler(),
|
||||
error.clone(),
|
||||
);
|
||||
Err(error)
|
||||
}
|
||||
}
|
||||
|
||||
result
|
||||
}
|
||||
|
||||
/// Returns an iterator that produces the list of addresses we're listening on.
|
||||
@ -568,6 +589,7 @@ where
|
||||
connected,
|
||||
error,
|
||||
num_established,
|
||||
handler,
|
||||
}) => {
|
||||
if let Some(error) = error.as_ref() {
|
||||
log::debug!("Connection {:?} closed: {:?}", connected, error);
|
||||
@ -576,8 +598,12 @@ where
|
||||
}
|
||||
let peer_id = connected.peer_id;
|
||||
let endpoint = connected.endpoint;
|
||||
this.behaviour
|
||||
.inject_connection_closed(&peer_id, &id, &endpoint);
|
||||
this.behaviour.inject_connection_closed(
|
||||
&peer_id,
|
||||
&id,
|
||||
&endpoint,
|
||||
handler.into_protocols_handler(),
|
||||
);
|
||||
if num_established == 0 {
|
||||
this.behaviour.inject_disconnected(&peer_id);
|
||||
}
|
||||
@ -668,8 +694,14 @@ where
|
||||
local_addr,
|
||||
send_back_addr,
|
||||
error,
|
||||
handler,
|
||||
}) => {
|
||||
log::debug!("Incoming connection failed: {:?}", error);
|
||||
this.behaviour.inject_listen_failure(
|
||||
&local_addr,
|
||||
&send_back_addr,
|
||||
handler.into_protocols_handler(),
|
||||
);
|
||||
return Poll::Ready(SwarmEvent::IncomingConnectionError {
|
||||
local_addr,
|
||||
send_back_addr,
|
||||
@ -682,19 +714,34 @@ where
|
||||
error,
|
||||
attempts_remaining,
|
||||
}) => {
|
||||
log::debug!(
|
||||
"Connection attempt to {:?} via {:?} failed with {:?}. Attempts remaining: {}.",
|
||||
peer_id, multiaddr, error, attempts_remaining);
|
||||
this.behaviour
|
||||
.inject_addr_reach_failure(Some(&peer_id), &multiaddr, &error);
|
||||
if attempts_remaining == 0 {
|
||||
this.behaviour.inject_dial_failure(&peer_id);
|
||||
|
||||
let num_remaining: u32;
|
||||
match attempts_remaining {
|
||||
DialAttemptsRemaining::Some(n) => {
|
||||
num_remaining = n.into();
|
||||
}
|
||||
DialAttemptsRemaining::None(handler) => {
|
||||
num_remaining = 0;
|
||||
this.behaviour.inject_dial_failure(
|
||||
&peer_id,
|
||||
handler.into_protocols_handler(),
|
||||
DialError::UnreachableAddr(multiaddr.clone()),
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
log::debug!(
|
||||
"Connection attempt to {:?} via {:?} failed with {:?}. Attempts remaining: {}.",
|
||||
peer_id, multiaddr, error, num_remaining,
|
||||
);
|
||||
|
||||
return Poll::Ready(SwarmEvent::UnreachableAddr {
|
||||
peer_id,
|
||||
address: multiaddr,
|
||||
error,
|
||||
attempts_remaining,
|
||||
attempts_remaining: num_remaining,
|
||||
});
|
||||
}
|
||||
Poll::Ready(NetworkEvent::UnknownPeerDialError {
|
||||
@ -761,44 +808,48 @@ where
|
||||
Poll::Ready(NetworkBehaviourAction::GenerateEvent(event)) => {
|
||||
return Poll::Ready(SwarmEvent::Behaviour(event))
|
||||
}
|
||||
Poll::Ready(NetworkBehaviourAction::DialAddress { address }) => {
|
||||
let _ = Swarm::dial_addr(&mut *this, address);
|
||||
Poll::Ready(NetworkBehaviourAction::DialAddress { address, handler }) => {
|
||||
let _ = Swarm::dial_addr_with_handler(&mut *this, address, handler);
|
||||
}
|
||||
Poll::Ready(NetworkBehaviourAction::DialPeer { peer_id, condition }) => {
|
||||
if this.banned_peers.contains(&peer_id) {
|
||||
this.behaviour.inject_dial_failure(&peer_id);
|
||||
Poll::Ready(NetworkBehaviourAction::DialPeer {
|
||||
peer_id,
|
||||
condition,
|
||||
handler,
|
||||
}) => {
|
||||
let condition_matched = match condition {
|
||||
DialPeerCondition::Disconnected => this.network.is_disconnected(&peer_id),
|
||||
DialPeerCondition::NotDialing => !this.network.is_dialing(&peer_id),
|
||||
DialPeerCondition::Always => true,
|
||||
};
|
||||
if condition_matched {
|
||||
if Swarm::dial_with_handler(this, &peer_id, handler).is_ok() {
|
||||
return Poll::Ready(SwarmEvent::Dialing(peer_id));
|
||||
}
|
||||
} else {
|
||||
let condition_matched = match condition {
|
||||
DialPeerCondition::Disconnected => {
|
||||
this.network.is_disconnected(&peer_id)
|
||||
}
|
||||
DialPeerCondition::NotDialing => !this.network.is_dialing(&peer_id),
|
||||
DialPeerCondition::Always => true,
|
||||
};
|
||||
if condition_matched {
|
||||
if Swarm::dial(this, &peer_id).is_ok() {
|
||||
return Poll::Ready(SwarmEvent::Dialing(peer_id));
|
||||
}
|
||||
} else {
|
||||
// Even if the condition for a _new_ dialing attempt is not met,
|
||||
// we always add any potentially new addresses of the peer to an
|
||||
// ongoing dialing attempt, if there is one.
|
||||
log::trace!(
|
||||
"Condition for new dialing attempt to {:?} not met: {:?}",
|
||||
peer_id,
|
||||
condition
|
||||
);
|
||||
let self_listening = &this.listened_addrs;
|
||||
if let Some(mut peer) = this.network.peer(peer_id).into_dialing() {
|
||||
let addrs = this.behaviour.addresses_of_peer(peer.id());
|
||||
let mut attempt = peer.some_attempt();
|
||||
for a in addrs {
|
||||
if !self_listening.contains(&a) {
|
||||
attempt.add_address(a);
|
||||
}
|
||||
// Even if the condition for a _new_ dialing attempt is not met,
|
||||
// we always add any potentially new addresses of the peer to an
|
||||
// ongoing dialing attempt, if there is one.
|
||||
log::trace!(
|
||||
"Condition for new dialing attempt to {:?} not met: {:?}",
|
||||
peer_id,
|
||||
condition
|
||||
);
|
||||
let self_listening = &this.listened_addrs;
|
||||
if let Some(mut peer) = this.network.peer(peer_id).into_dialing() {
|
||||
let addrs = this.behaviour.addresses_of_peer(peer.id());
|
||||
let mut attempt = peer.some_attempt();
|
||||
for a in addrs {
|
||||
if !self_listening.contains(&a) {
|
||||
attempt.add_address(a);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
this.behaviour.inject_dial_failure(
|
||||
&peer_id,
|
||||
handler,
|
||||
DialError::DialPeerConditionFalse(condition),
|
||||
);
|
||||
}
|
||||
}
|
||||
Poll::Ready(NetworkBehaviourAction::NotifyHandler {
|
||||
@ -1148,8 +1199,8 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
/// The possible failures of [`Swarm::dial`].
|
||||
#[derive(Debug)]
|
||||
/// The possible failures of dialing.
|
||||
#[derive(Debug, Clone)]
|
||||
pub enum DialError {
|
||||
/// The peer is currently banned.
|
||||
Banned,
|
||||
@ -1158,16 +1209,27 @@ pub enum DialError {
|
||||
ConnectionLimit(ConnectionLimit),
|
||||
/// The address given for dialing is invalid.
|
||||
InvalidAddress(Multiaddr),
|
||||
/// Tried to dial an address but it ended up being unreachaable.
|
||||
UnreachableAddr(Multiaddr),
|
||||
/// The peer being dialed is the local peer and thus the dial was aborted.
|
||||
LocalPeerId,
|
||||
/// [`NetworkBehaviour::addresses_of_peer`] returned no addresses
|
||||
/// for the peer to dial.
|
||||
NoAddresses,
|
||||
/// The provided [`DialPeerCondition`] evaluated to false and thus the dial was aborted.
|
||||
DialPeerConditionFalse(DialPeerCondition),
|
||||
}
|
||||
|
||||
impl From<network::DialError> for DialError {
|
||||
fn from(err: network::DialError) -> DialError {
|
||||
match err {
|
||||
network::DialError::ConnectionLimit(l) => DialError::ConnectionLimit(l),
|
||||
network::DialError::InvalidAddress(a) => DialError::InvalidAddress(a),
|
||||
impl DialError {
|
||||
fn from_network_dial_error<THandler>(error: network::DialError<THandler>) -> (Self, THandler) {
|
||||
match error {
|
||||
network::DialError::ConnectionLimit { limit, handler } => {
|
||||
(DialError::ConnectionLimit(limit), handler)
|
||||
}
|
||||
network::DialError::InvalidAddress { address, handler } => {
|
||||
(DialError::InvalidAddress(address), handler)
|
||||
}
|
||||
network::DialError::LocalPeerId { handler } => (DialError::LocalPeerId, handler),
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1177,8 +1239,17 @@ impl fmt::Display for DialError {
|
||||
match self {
|
||||
DialError::ConnectionLimit(err) => write!(f, "Dial error: {}", err),
|
||||
DialError::NoAddresses => write!(f, "Dial error: no addresses for peer."),
|
||||
DialError::LocalPeerId => write!(f, "Dial error: tried to dial local peer id."),
|
||||
DialError::InvalidAddress(a) => write!(f, "Dial error: invalid address: {}", a),
|
||||
DialError::UnreachableAddr(a) => write!(f, "Dial error: unreachable address: {}", a),
|
||||
DialError::Banned => write!(f, "Dial error: peer is banned."),
|
||||
DialError::DialPeerConditionFalse(c) => {
|
||||
write!(
|
||||
f,
|
||||
"Dial error: condition {:?} for dialing peer was false.",
|
||||
c
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1188,8 +1259,11 @@ impl error::Error for DialError {
|
||||
match self {
|
||||
DialError::ConnectionLimit(err) => Some(err),
|
||||
DialError::InvalidAddress(_) => None,
|
||||
DialError::UnreachableAddr(_) => None,
|
||||
DialError::LocalPeerId => None,
|
||||
DialError::NoAddresses => None,
|
||||
DialError::Banned => None,
|
||||
DialError::DialPeerConditionFalse(_) => None,
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1241,12 +1315,7 @@ impl NetworkBehaviour for DummyBehaviour {
|
||||
&mut self,
|
||||
_: &mut Context<'_>,
|
||||
_: &mut impl PollParameters,
|
||||
) -> Poll<
|
||||
NetworkBehaviourAction<
|
||||
<Self::ProtocolsHandler as ProtocolsHandler>::InEvent,
|
||||
Self::OutEvent,
|
||||
>,
|
||||
> {
|
||||
) -> Poll<NetworkBehaviourAction<Self::OutEvent, Self::ProtocolsHandler>> {
|
||||
Poll::Pending
|
||||
}
|
||||
}
|
||||
@ -1257,8 +1326,9 @@ mod tests {
|
||||
use crate::protocols_handler::DummyProtocolsHandler;
|
||||
use crate::test::{CallTraceBehaviour, MockBehaviour};
|
||||
use futures::{executor, future};
|
||||
use libp2p_core::{identity, multiaddr, transport, upgrade};
|
||||
use libp2p_noise as noise;
|
||||
use libp2p::core::{identity, multiaddr, transport, upgrade};
|
||||
use libp2p::plaintext;
|
||||
use libp2p::yamux;
|
||||
|
||||
// Test execution state.
|
||||
// Connection => Disconnecting => Connecting.
|
||||
@ -1274,17 +1344,16 @@ mod tests {
|
||||
O: Send + 'static,
|
||||
{
|
||||
let id_keys = identity::Keypair::generate_ed25519();
|
||||
let pubkey = id_keys.public();
|
||||
let noise_keys = noise::Keypair::<noise::X25519Spec>::new()
|
||||
.into_authentic(&id_keys)
|
||||
.unwrap();
|
||||
let local_public_key = id_keys.public();
|
||||
let transport = transport::MemoryTransport::default()
|
||||
.upgrade(upgrade::Version::V1)
|
||||
.authenticate(noise::NoiseConfig::xx(noise_keys).into_authenticated())
|
||||
.multiplex(libp2p_mplex::MplexConfig::new())
|
||||
.authenticate(plaintext::PlainText2Config {
|
||||
local_public_key: local_public_key.clone(),
|
||||
})
|
||||
.multiplex(yamux::YamuxConfig::default())
|
||||
.boxed();
|
||||
let behaviour = CallTraceBehaviour::new(MockBehaviour::new(handler_proto));
|
||||
SwarmBuilder::new(transport, behaviour, pubkey.into()).build()
|
||||
SwarmBuilder::new(transport, behaviour, local_public_key.into()).build()
|
||||
}
|
||||
|
||||
fn swarms_connected<TBehaviour>(
|
||||
@ -1320,17 +1389,15 @@ mod tests {
|
||||
<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutEvent: Clone
|
||||
{
|
||||
for s in &[swarm1, swarm2] {
|
||||
if s.behaviour.inject_connection_closed.len() < num_connections {
|
||||
assert_eq!(s.behaviour.inject_disconnected.len(), 0);
|
||||
} else {
|
||||
assert_eq!(s.behaviour.inject_disconnected.len(), 1);
|
||||
}
|
||||
assert_eq!(s.behaviour.inject_connection_established.len(), 0);
|
||||
assert_eq!(s.behaviour.inject_connected.len(), 0);
|
||||
}
|
||||
[swarm1, swarm2]
|
||||
.iter()
|
||||
.all(|s| s.behaviour.inject_connection_closed.len() == num_connections)
|
||||
&& [swarm1, swarm2]
|
||||
.iter()
|
||||
.all(|s| s.behaviour.inject_disconnected.len() == 1)
|
||||
}
|
||||
|
||||
/// Establishes multiple connections between two peers,
|
||||
|
@ -65,6 +65,10 @@ where
|
||||
self.substream_upgrade_protocol_override = version;
|
||||
self
|
||||
}
|
||||
|
||||
pub(crate) fn into_protocols_handler(self) -> TIntoProtoHandler {
|
||||
self.handler
|
||||
}
|
||||
}
|
||||
|
||||
impl<TIntoProtoHandler, TProtoHandler> IntoConnectionHandler
|
||||
@ -130,6 +134,12 @@ where
|
||||
substream_upgrade_protocol_override: Option<upgrade::Version>,
|
||||
}
|
||||
|
||||
impl<TProtoHandler: ProtocolsHandler> NodeHandlerWrapper<TProtoHandler> {
|
||||
pub(crate) fn into_protocols_handler(self) -> TProtoHandler {
|
||||
self.handler
|
||||
}
|
||||
}
|
||||
|
||||
struct SubstreamUpgrade<UserData, Upgrade> {
|
||||
user_data: Option<UserData>,
|
||||
timeout: Delay,
|
||||
|
@ -45,6 +45,10 @@ impl<TProto1, TProto2> IntoProtocolsHandlerSelect<TProto1, TProto2> {
|
||||
pub(crate) fn new(proto1: TProto1, proto2: TProto2) -> Self {
|
||||
IntoProtocolsHandlerSelect { proto1, proto2 }
|
||||
}
|
||||
|
||||
pub fn into_inner(self) -> (TProto1, TProto2) {
|
||||
(self.proto1, self.proto2)
|
||||
}
|
||||
}
|
||||
|
||||
impl<TProto1, TProto2> IntoProtocolsHandler for IntoProtocolsHandlerSelect<TProto1, TProto2>
|
||||
@ -87,6 +91,10 @@ impl<TProto1, TProto2> ProtocolsHandlerSelect<TProto1, TProto2> {
|
||||
pub(crate) fn new(proto1: TProto1, proto2: TProto2) -> Self {
|
||||
ProtocolsHandlerSelect { proto1, proto2 }
|
||||
}
|
||||
|
||||
pub fn into_inner(self) -> (TProto1, TProto2) {
|
||||
(self.proto1, self.proto2)
|
||||
}
|
||||
}
|
||||
|
||||
impl<TProto1, TProto2> ProtocolsHandler for ProtocolsHandlerSelect<TProto1, TProto2>
|
||||
|
@ -19,7 +19,7 @@
|
||||
// DEALINGS IN THE SOFTWARE.
|
||||
|
||||
use crate::{
|
||||
IntoProtocolsHandler, NetworkBehaviour, NetworkBehaviourAction, PollParameters,
|
||||
DialError, IntoProtocolsHandler, NetworkBehaviour, NetworkBehaviourAction, PollParameters,
|
||||
ProtocolsHandler,
|
||||
};
|
||||
use libp2p_core::{
|
||||
@ -45,7 +45,7 @@ where
|
||||
/// The next action to return from `poll`.
|
||||
///
|
||||
/// An action is only returned once.
|
||||
pub next_action: Option<NetworkBehaviourAction<THandler::InEvent, TOutEvent>>,
|
||||
pub next_action: Option<NetworkBehaviourAction<TOutEvent, THandler>>,
|
||||
}
|
||||
|
||||
impl<THandler, TOutEvent> MockBehaviour<THandler, TOutEvent>
|
||||
@ -84,7 +84,7 @@ where
|
||||
&mut self,
|
||||
_: &mut Context,
|
||||
_: &mut impl PollParameters,
|
||||
) -> Poll<NetworkBehaviourAction<THandler::InEvent, Self::OutEvent>> {
|
||||
) -> Poll<NetworkBehaviourAction<Self::OutEvent, Self::ProtocolsHandler>> {
|
||||
self.next_action.take().map_or(Poll::Pending, Poll::Ready)
|
||||
}
|
||||
}
|
||||
@ -202,10 +202,16 @@ where
|
||||
self.inner.inject_disconnected(peer);
|
||||
}
|
||||
|
||||
fn inject_connection_closed(&mut self, p: &PeerId, c: &ConnectionId, e: &ConnectedPoint) {
|
||||
fn inject_connection_closed(
|
||||
&mut self,
|
||||
p: &PeerId,
|
||||
c: &ConnectionId,
|
||||
e: &ConnectedPoint,
|
||||
handler: <Self::ProtocolsHandler as IntoProtocolsHandler>::Handler,
|
||||
) {
|
||||
self.inject_connection_closed
|
||||
.push((p.clone(), c.clone(), e.clone()));
|
||||
self.inner.inject_connection_closed(p, c, e);
|
||||
self.inner.inject_connection_closed(p, c, e, handler);
|
||||
}
|
||||
|
||||
fn inject_event(
|
||||
@ -228,9 +234,14 @@ where
|
||||
self.inner.inject_addr_reach_failure(p, a, e);
|
||||
}
|
||||
|
||||
fn inject_dial_failure(&mut self, p: &PeerId) {
|
||||
fn inject_dial_failure(
|
||||
&mut self,
|
||||
p: &PeerId,
|
||||
handler: Self::ProtocolsHandler,
|
||||
error: DialError,
|
||||
) {
|
||||
self.inject_dial_failure.push(p.clone());
|
||||
self.inner.inject_dial_failure(p);
|
||||
self.inner.inject_dial_failure(p, handler, error);
|
||||
}
|
||||
|
||||
fn inject_new_listener(&mut self, id: ListenerId) {
|
||||
@ -268,12 +279,11 @@ where
|
||||
self.inner.inject_listener_closed(l, r);
|
||||
}
|
||||
|
||||
fn poll(&mut self, cx: &mut Context, args: &mut impl PollParameters) ->
|
||||
Poll<NetworkBehaviourAction<
|
||||
<<Self::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InEvent,
|
||||
Self::OutEvent
|
||||
>>
|
||||
{
|
||||
fn poll(
|
||||
&mut self,
|
||||
cx: &mut Context,
|
||||
args: &mut impl PollParameters,
|
||||
) -> Poll<NetworkBehaviourAction<Self::OutEvent, Self::ProtocolsHandler>> {
|
||||
self.poll += 1;
|
||||
self.inner.poll(cx, args)
|
||||
}
|
||||
|
@ -24,7 +24,8 @@ use crate::protocols_handler::{
|
||||
};
|
||||
use crate::upgrade::{InboundUpgradeSend, OutboundUpgradeSend, SendWrapper};
|
||||
use crate::{
|
||||
NetworkBehaviour, NetworkBehaviourAction, NetworkBehaviourEventProcess, PollParameters,
|
||||
DialError, NetworkBehaviour, NetworkBehaviourAction, NetworkBehaviourEventProcess,
|
||||
PollParameters,
|
||||
};
|
||||
use either::Either;
|
||||
use libp2p_core::{
|
||||
@ -113,9 +114,12 @@ where
|
||||
peer_id: &PeerId,
|
||||
connection: &ConnectionId,
|
||||
endpoint: &ConnectedPoint,
|
||||
handler: <Self::ProtocolsHandler as IntoProtocolsHandler>::Handler,
|
||||
) {
|
||||
if let Some(inner) = self.inner.as_mut() {
|
||||
inner.inject_connection_closed(peer_id, connection, endpoint)
|
||||
if let Some(handler) = handler.inner {
|
||||
inner.inject_connection_closed(peer_id, connection, endpoint, handler)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -153,9 +157,29 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
fn inject_dial_failure(&mut self, peer_id: &PeerId) {
|
||||
fn inject_dial_failure(
|
||||
&mut self,
|
||||
peer_id: &PeerId,
|
||||
handler: Self::ProtocolsHandler,
|
||||
error: DialError,
|
||||
) {
|
||||
if let Some(inner) = self.inner.as_mut() {
|
||||
inner.inject_dial_failure(peer_id)
|
||||
if let Some(handler) = handler.inner {
|
||||
inner.inject_dial_failure(peer_id, handler, error)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn inject_listen_failure(
|
||||
&mut self,
|
||||
local_addr: &Multiaddr,
|
||||
send_back_addr: &Multiaddr,
|
||||
handler: Self::ProtocolsHandler,
|
||||
) {
|
||||
if let Some(inner) = self.inner.as_mut() {
|
||||
if let Some(handler) = handler.inner {
|
||||
inner.inject_listen_failure(local_addr, send_back_addr, handler)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -201,11 +225,15 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
fn poll(&mut self, cx: &mut Context<'_>, params: &mut impl PollParameters)
|
||||
-> Poll<NetworkBehaviourAction<<<Self::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InEvent, Self::OutEvent>>
|
||||
{
|
||||
fn poll(
|
||||
&mut self,
|
||||
cx: &mut Context<'_>,
|
||||
params: &mut impl PollParameters,
|
||||
) -> Poll<NetworkBehaviourAction<Self::OutEvent, Self::ProtocolsHandler>> {
|
||||
if let Some(inner) = self.inner.as_mut() {
|
||||
inner.poll(cx, params)
|
||||
inner
|
||||
.poll(cx, params)
|
||||
.map(|action| action.map_handler(|h| ToggleIntoProtoHandler { inner: Some(h) }))
|
||||
} else {
|
||||
Poll::Pending
|
||||
}
|
||||
|
Reference in New Issue
Block a user