mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-25 15:51:34 +00:00
Switch to stable futures (#1196)
* Switch to stable futures * Remove from_fn * Fix secio * Fix core --lib tests
This commit is contained in:
@ -20,8 +20,7 @@
|
||||
|
||||
use crate::protocols_handler::{IntoProtocolsHandler, ProtocolsHandler};
|
||||
use libp2p_core::{ConnectedPoint, Multiaddr, PeerId, nodes::ListenerId};
|
||||
use futures::prelude::*;
|
||||
use std::error;
|
||||
use std::{error, task::Context, task::Poll};
|
||||
|
||||
/// A behaviour for the network. Allows customizing the swarm.
|
||||
///
|
||||
@ -133,8 +132,8 @@ pub trait NetworkBehaviour {
|
||||
///
|
||||
/// 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, params: &mut impl PollParameters)
|
||||
-> Async<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::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InEvent, Self::OutEvent>>;
|
||||
}
|
||||
|
||||
/// Parameters passed to `poll()`, that the `NetworkBehaviour` has access to.
|
||||
|
194
swarm/src/lib.rs
194
swarm/src/lib.rs
@ -93,7 +93,7 @@ use libp2p_core::{
|
||||
};
|
||||
use registry::{Addresses, AddressIntoIter};
|
||||
use smallvec::SmallVec;
|
||||
use std::{error, fmt, io, ops::{Deref, DerefMut}};
|
||||
use std::{error, fmt, io, ops::{Deref, DerefMut}, pin::Pin, task::{Context, Poll}};
|
||||
use std::collections::HashSet;
|
||||
|
||||
/// Contains the state of the network, plus the way it should behave.
|
||||
@ -140,14 +140,7 @@ where
|
||||
banned_peers: HashSet<PeerId>,
|
||||
|
||||
/// Pending event message to be delivered.
|
||||
///
|
||||
/// If the pair's second element is `AsyncSink::NotReady`, the event
|
||||
/// message has yet to be sent using `PeerMut::start_send_event`.
|
||||
///
|
||||
/// If the pair's second element is `AsyncSink::Ready`, the event
|
||||
/// message has been sent and needs to be flushed using
|
||||
/// `PeerMut::complete_send_event`.
|
||||
send_event_to_complete: Option<(PeerId, AsyncSink<TInEvent>)>
|
||||
send_event_to_complete: Option<(PeerId, TInEvent)>
|
||||
}
|
||||
|
||||
impl<TTransport, TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo> Deref for
|
||||
@ -172,6 +165,13 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<TTransport, TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo> Unpin for
|
||||
ExpandedSwarm<TTransport, TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo>
|
||||
where
|
||||
TTransport: Transport,
|
||||
{
|
||||
}
|
||||
|
||||
impl<TTransport, TBehaviour, TMuxer, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo>
|
||||
ExpandedSwarm<TTransport, TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo>
|
||||
where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
|
||||
@ -180,9 +180,9 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
|
||||
<TMuxer as StreamMuxer>::Substream: Send + 'static,
|
||||
TTransport: Transport<Output = (TConnInfo, TMuxer)> + Clone,
|
||||
TTransport::Error: Send + 'static,
|
||||
TTransport::Listener: Send + 'static,
|
||||
TTransport::ListenerUpgrade: Send + 'static,
|
||||
TTransport::Dial: Send + 'static,
|
||||
TTransport::Listener: Unpin + Send + 'static,
|
||||
TTransport::ListenerUpgrade: Unpin + Send + 'static,
|
||||
TTransport::Dial: Unpin + Send + 'static,
|
||||
THandlerErr: error::Error,
|
||||
THandler: IntoProtocolsHandler + Send + 'static,
|
||||
<THandler as IntoProtocolsHandler>::Handler: ProtocolsHandler<InEvent = TInEvent, OutEvent = TOutEvent, Substream = Substream<TMuxer>, Error = THandlerErr> + Send + 'static,
|
||||
@ -315,9 +315,9 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
|
||||
<TMuxer as StreamMuxer>::Substream: Send + 'static,
|
||||
TTransport: Transport<Output = (TConnInfo, TMuxer)> + Clone,
|
||||
TTransport::Error: Send + 'static,
|
||||
TTransport::Listener: Send + 'static,
|
||||
TTransport::ListenerUpgrade: Send + 'static,
|
||||
TTransport::Dial: Send + 'static,
|
||||
TTransport::Listener: Unpin + Send + 'static,
|
||||
TTransport::ListenerUpgrade: Unpin + Send + 'static,
|
||||
TTransport::Dial: Unpin + Send + 'static,
|
||||
THandlerErr: error::Error,
|
||||
THandler: IntoProtocolsHandler + Send + 'static,
|
||||
<THandler as IntoProtocolsHandler>::Handler: ProtocolsHandler<InEvent = TInEvent, OutEvent = TOutEvent, Substream = Substream<TMuxer>, Error = THandlerErr> + Send + 'static,
|
||||
@ -340,123 +340,122 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
|
||||
<NodeHandlerWrapper<<THandler as IntoProtocolsHandler>::Handler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
|
||||
TConnInfo: ConnectionInfo<PeerId = PeerId> + fmt::Debug + Clone + Send + 'static,
|
||||
{
|
||||
type Item = TBehaviour::OutEvent;
|
||||
type Error = io::Error;
|
||||
type Item = Result<TBehaviour::OutEvent, io::Error>;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Option<Self::Item>> {
|
||||
// We use a `this` variable to solve borrowing issues.
|
||||
let this = &mut *self;
|
||||
|
||||
fn poll(&mut self) -> Poll<Option<Self::Item>, io::Error> {
|
||||
loop {
|
||||
let mut network_not_ready = false;
|
||||
|
||||
match self.network.poll() {
|
||||
Async::NotReady => network_not_ready = true,
|
||||
Async::Ready(NetworkEvent::NodeEvent { conn_info, event }) => {
|
||||
self.behaviour.inject_node_event(conn_info.peer_id().clone(), event);
|
||||
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);
|
||||
},
|
||||
Async::Ready(NetworkEvent::Connected { conn_info, endpoint }) => {
|
||||
if self.banned_peers.contains(conn_info.peer_id()) {
|
||||
self.network.peer(conn_info.peer_id().clone())
|
||||
Poll::Ready(NetworkEvent::Connected { conn_info, endpoint }) => {
|
||||
if this.banned_peers.contains(conn_info.peer_id()) {
|
||||
this.network.peer(conn_info.peer_id().clone())
|
||||
.into_connected()
|
||||
.expect("the Network just notified us that we were connected; QED")
|
||||
.close();
|
||||
} else {
|
||||
self.behaviour.inject_connected(conn_info.peer_id().clone(), endpoint);
|
||||
this.behaviour.inject_connected(conn_info.peer_id().clone(), endpoint);
|
||||
}
|
||||
},
|
||||
Async::Ready(NetworkEvent::NodeClosed { conn_info, endpoint, .. }) => {
|
||||
self.behaviour.inject_disconnected(conn_info.peer_id(), endpoint);
|
||||
Poll::Ready(NetworkEvent::NodeClosed { conn_info, endpoint, .. }) => {
|
||||
this.behaviour.inject_disconnected(conn_info.peer_id(), endpoint);
|
||||
},
|
||||
Async::Ready(NetworkEvent::Replaced { new_info, closed_endpoint, endpoint, .. }) => {
|
||||
self.behaviour.inject_replaced(new_info.peer_id().clone(), closed_endpoint, endpoint);
|
||||
Poll::Ready(NetworkEvent::Replaced { new_info, closed_endpoint, endpoint, .. }) => {
|
||||
this.behaviour.inject_replaced(new_info.peer_id().clone(), closed_endpoint, endpoint);
|
||||
},
|
||||
Async::Ready(NetworkEvent::IncomingConnection(incoming)) => {
|
||||
let handler = self.behaviour.new_handler();
|
||||
Poll::Ready(NetworkEvent::IncomingConnection(incoming)) => {
|
||||
let handler = this.behaviour.new_handler();
|
||||
incoming.accept(handler.into_node_handler_builder());
|
||||
},
|
||||
Async::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) => {
|
||||
if !self.listened_addrs.contains(&listen_addr) {
|
||||
self.listened_addrs.push(listen_addr.clone())
|
||||
Poll::Ready(NetworkEvent::NewListenerAddress { listen_addr, .. }) => {
|
||||
if !this.listened_addrs.contains(&listen_addr) {
|
||||
this.listened_addrs.push(listen_addr.clone())
|
||||
}
|
||||
self.behaviour.inject_new_listen_addr(&listen_addr);
|
||||
this.behaviour.inject_new_listen_addr(&listen_addr);
|
||||
}
|
||||
Async::Ready(NetworkEvent::ExpiredListenerAddress { listen_addr, .. }) => {
|
||||
self.listened_addrs.retain(|a| a != &listen_addr);
|
||||
self.behaviour.inject_expired_listen_addr(&listen_addr);
|
||||
Poll::Ready(NetworkEvent::ExpiredListenerAddress { listen_addr, .. }) => {
|
||||
this.listened_addrs.retain(|a| a != &listen_addr);
|
||||
this.behaviour.inject_expired_listen_addr(&listen_addr);
|
||||
}
|
||||
Async::Ready(NetworkEvent::ListenerClosed { listener_id, .. }) =>
|
||||
self.behaviour.inject_listener_closed(listener_id),
|
||||
Async::Ready(NetworkEvent::ListenerError { listener_id, error }) =>
|
||||
self.behaviour.inject_listener_error(listener_id, &error),
|
||||
Async::Ready(NetworkEvent::IncomingConnectionError { .. }) => {},
|
||||
Async::Ready(NetworkEvent::DialError { peer_id, multiaddr, error, new_state }) => {
|
||||
self.behaviour.inject_addr_reach_failure(Some(&peer_id), &multiaddr, &error);
|
||||
Poll::Ready(NetworkEvent::ListenerClosed { listener_id, .. }) =>
|
||||
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::DialError { peer_id, multiaddr, error, new_state }) => {
|
||||
this.behaviour.inject_addr_reach_failure(Some(&peer_id), &multiaddr, &error);
|
||||
if let network::PeerState::NotConnected = new_state {
|
||||
self.behaviour.inject_dial_failure(&peer_id);
|
||||
this.behaviour.inject_dial_failure(&peer_id);
|
||||
}
|
||||
},
|
||||
Async::Ready(NetworkEvent::UnknownPeerDialError { multiaddr, error, .. }) => {
|
||||
self.behaviour.inject_addr_reach_failure(None, &multiaddr, &error);
|
||||
Poll::Ready(NetworkEvent::UnknownPeerDialError { multiaddr, error, .. }) => {
|
||||
this.behaviour.inject_addr_reach_failure(None, &multiaddr, &error);
|
||||
},
|
||||
}
|
||||
|
||||
// Try to deliver pending event.
|
||||
if let Some((id, pending)) = self.send_event_to_complete.take() {
|
||||
if let Some(mut peer) = self.network.peer(id.clone()).into_connected() {
|
||||
if let AsyncSink::NotReady(e) = pending {
|
||||
if let Ok(a@AsyncSink::NotReady(_)) = peer.start_send_event(e) {
|
||||
self.send_event_to_complete = Some((id, a))
|
||||
} else if let Ok(Async::NotReady) = peer.complete_send_event() {
|
||||
self.send_event_to_complete = Some((id, AsyncSink::Ready))
|
||||
}
|
||||
} else if let Ok(Async::NotReady) = peer.complete_send_event() {
|
||||
self.send_event_to_complete = Some((id, AsyncSink::Ready))
|
||||
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
|
||||
},
|
||||
}
|
||||
}
|
||||
}
|
||||
if self.send_event_to_complete.is_some() {
|
||||
return Ok(Async::NotReady)
|
||||
}
|
||||
|
||||
let behaviour_poll = {
|
||||
let mut parameters = SwarmPollParameters {
|
||||
local_peer_id: &mut self.network.local_peer_id(),
|
||||
supported_protocols: &self.supported_protocols,
|
||||
listened_addrs: &self.listened_addrs,
|
||||
external_addrs: &self.external_addrs
|
||||
local_peer_id: &mut this.network.local_peer_id(),
|
||||
supported_protocols: &this.supported_protocols,
|
||||
listened_addrs: &this.listened_addrs,
|
||||
external_addrs: &this.external_addrs
|
||||
};
|
||||
self.behaviour.poll(&mut parameters)
|
||||
this.behaviour.poll(cx, &mut parameters)
|
||||
};
|
||||
|
||||
match behaviour_poll {
|
||||
Async::NotReady if network_not_ready => return Ok(Async::NotReady),
|
||||
Async::NotReady => (),
|
||||
Async::Ready(NetworkBehaviourAction::GenerateEvent(event)) => {
|
||||
return Ok(Async::Ready(Some(event)))
|
||||
Poll::Pending if network_not_ready => return Poll::Pending,
|
||||
Poll::Pending => (),
|
||||
Poll::Ready(NetworkBehaviourAction::GenerateEvent(event)) => {
|
||||
return Poll::Ready(Some(Ok(event)))
|
||||
},
|
||||
Async::Ready(NetworkBehaviourAction::DialAddress { address }) => {
|
||||
let _ = ExpandedSwarm::dial_addr(self, address);
|
||||
Poll::Ready(NetworkBehaviourAction::DialAddress { address }) => {
|
||||
let _ = ExpandedSwarm::dial_addr(&mut *this, address);
|
||||
},
|
||||
Async::Ready(NetworkBehaviourAction::DialPeer { peer_id }) => {
|
||||
if self.banned_peers.contains(&peer_id) {
|
||||
self.behaviour.inject_dial_failure(&peer_id);
|
||||
Poll::Ready(NetworkBehaviourAction::DialPeer { peer_id }) => {
|
||||
if this.banned_peers.contains(&peer_id) {
|
||||
this.behaviour.inject_dial_failure(&peer_id);
|
||||
} else {
|
||||
ExpandedSwarm::dial(self, peer_id);
|
||||
ExpandedSwarm::dial(&mut *this, peer_id);
|
||||
}
|
||||
},
|
||||
Async::Ready(NetworkBehaviourAction::SendEvent { peer_id, event }) => {
|
||||
if let Some(mut peer) = self.network.peer(peer_id.clone()).into_connected() {
|
||||
if let Ok(a@AsyncSink::NotReady(_)) = peer.start_send_event(event) {
|
||||
self.send_event_to_complete = Some((peer_id, a))
|
||||
} else if let Ok(Async::NotReady) = peer.complete_send_event() {
|
||||
self.send_event_to_complete = Some((peer_id, AsyncSink::Ready))
|
||||
Poll::Ready(NetworkBehaviourAction::SendEvent { peer_id, 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;
|
||||
}
|
||||
}
|
||||
},
|
||||
Async::Ready(NetworkBehaviourAction::ReportObservedAddr { address }) => {
|
||||
for addr in self.network.address_translation(&address) {
|
||||
if self.external_addrs.iter().all(|a| *a != addr) {
|
||||
self.behaviour.inject_new_external_addr(&addr);
|
||||
Poll::Ready(NetworkBehaviourAction::ReportObservedAddr { address }) => {
|
||||
for addr in this.network.address_translation(&address) {
|
||||
if this.external_addrs.iter().all(|a| *a != addr) {
|
||||
this.behaviour.inject_new_external_addr(&addr);
|
||||
}
|
||||
self.external_addrs.add(addr)
|
||||
this.external_addrs.add(addr)
|
||||
}
|
||||
},
|
||||
}
|
||||
@ -509,9 +508,9 @@ where TBehaviour: NetworkBehaviour,
|
||||
<TMuxer as StreamMuxer>::Substream: Send + 'static,
|
||||
TTransport: Transport<Output = (TConnInfo, TMuxer)> + Clone,
|
||||
TTransport::Error: Send + 'static,
|
||||
TTransport::Listener: Send + 'static,
|
||||
TTransport::ListenerUpgrade: Send + 'static,
|
||||
TTransport::Dial: Send + 'static,
|
||||
TTransport::Listener: Unpin + Send + 'static,
|
||||
TTransport::ListenerUpgrade: Unpin + Send + 'static,
|
||||
TTransport::Dial: Unpin + Send + 'static,
|
||||
<TBehaviour as NetworkBehaviour>::ProtocolsHandler: Send + 'static,
|
||||
<<TBehaviour as NetworkBehaviour>::ProtocolsHandler as IntoProtocolsHandler>::Handler: ProtocolsHandler<Substream = Substream<TMuxer>> + Send + 'static,
|
||||
<<<TBehaviour as NetworkBehaviour>::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InEvent: Send + 'static,
|
||||
@ -584,8 +583,7 @@ mod tests {
|
||||
};
|
||||
use libp2p_mplex::Multiplex;
|
||||
use futures::prelude::*;
|
||||
use std::marker::PhantomData;
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use std::{marker::PhantomData, task::Context, task::Poll};
|
||||
use void::Void;
|
||||
|
||||
#[derive(Clone)]
|
||||
@ -593,11 +591,9 @@ mod tests {
|
||||
marker: PhantomData<TSubstream>,
|
||||
}
|
||||
|
||||
trait TSubstream: AsyncRead + AsyncWrite {}
|
||||
|
||||
impl<TSubstream> NetworkBehaviour
|
||||
for DummyBehaviour<TSubstream>
|
||||
where TSubstream: AsyncRead + AsyncWrite
|
||||
where TSubstream: AsyncRead + AsyncWrite + Unpin
|
||||
{
|
||||
type ProtocolsHandler = DummyProtocolsHandler<TSubstream>;
|
||||
type OutEvent = Void;
|
||||
@ -617,11 +613,11 @@ mod tests {
|
||||
fn inject_node_event(&mut self, _: PeerId,
|
||||
_: <Self::ProtocolsHandler as ProtocolsHandler>::OutEvent) {}
|
||||
|
||||
fn poll(&mut self, _: &mut impl PollParameters) ->
|
||||
Async<NetworkBehaviourAction<<Self::ProtocolsHandler as
|
||||
fn poll(&mut self, _: &mut Context, _: &mut impl PollParameters) ->
|
||||
Poll<NetworkBehaviourAction<<Self::ProtocolsHandler as
|
||||
ProtocolsHandler>::InEvent, Self::OutEvent>>
|
||||
{
|
||||
Async::NotReady
|
||||
Poll::Pending
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -27,8 +27,7 @@ use crate::protocols_handler::{
|
||||
};
|
||||
use futures::prelude::*;
|
||||
use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade, DeniedUpgrade};
|
||||
use std::marker::PhantomData;
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use std::{marker::PhantomData, task::Context, task::Poll};
|
||||
use void::Void;
|
||||
|
||||
/// Implementation of `ProtocolsHandler` that doesn't handle anything.
|
||||
@ -47,7 +46,7 @@ impl<TSubstream> Default for DummyProtocolsHandler<TSubstream> {
|
||||
|
||||
impl<TSubstream> ProtocolsHandler for DummyProtocolsHandler<TSubstream>
|
||||
where
|
||||
TSubstream: AsyncRead + AsyncWrite,
|
||||
TSubstream: AsyncRead + AsyncWrite + Unpin,
|
||||
{
|
||||
type InEvent = Void;
|
||||
type OutEvent = Void;
|
||||
@ -89,10 +88,10 @@ where
|
||||
#[inline]
|
||||
fn poll(
|
||||
&mut self,
|
||||
_: &mut Context,
|
||||
) -> Poll<
|
||||
ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>,
|
||||
Void,
|
||||
ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent, Self::Error>,
|
||||
> {
|
||||
Ok(Async::NotReady)
|
||||
Poll::Pending
|
||||
}
|
||||
}
|
||||
|
@ -25,9 +25,8 @@ use crate::protocols_handler::{
|
||||
ProtocolsHandlerEvent,
|
||||
ProtocolsHandlerUpgrErr
|
||||
};
|
||||
use futures::prelude::*;
|
||||
use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade};
|
||||
use std::marker::PhantomData;
|
||||
use std::{marker::PhantomData, task::Context, task::Poll};
|
||||
|
||||
/// Wrapper around a protocol handler that turns the input event into something else.
|
||||
pub struct MapInEvent<TProtoHandler, TNewIn, TMap> {
|
||||
@ -103,10 +102,10 @@ where
|
||||
#[inline]
|
||||
fn poll(
|
||||
&mut self,
|
||||
cx: &mut Context,
|
||||
) -> Poll<
|
||||
ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>,
|
||||
Self::Error,
|
||||
ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent, Self::Error>,
|
||||
> {
|
||||
self.inner.poll()
|
||||
self.inner.poll(cx)
|
||||
}
|
||||
}
|
||||
|
@ -25,8 +25,8 @@ use crate::protocols_handler::{
|
||||
ProtocolsHandlerEvent,
|
||||
ProtocolsHandlerUpgrErr
|
||||
};
|
||||
use futures::prelude::*;
|
||||
use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade};
|
||||
use std::task::{Context, Poll};
|
||||
|
||||
/// Wrapper around a protocol handler that turns the output event into something else.
|
||||
pub struct MapOutEvent<TProtoHandler, TMap> {
|
||||
@ -98,17 +98,18 @@ where
|
||||
#[inline]
|
||||
fn poll(
|
||||
&mut self,
|
||||
cx: &mut Context,
|
||||
) -> Poll<
|
||||
ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>,
|
||||
Self::Error,
|
||||
ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent, Self::Error>,
|
||||
> {
|
||||
Ok(self.inner.poll()?.map(|ev| {
|
||||
self.inner.poll(cx).map(|ev| {
|
||||
match ev {
|
||||
ProtocolsHandlerEvent::Custom(ev) => ProtocolsHandlerEvent::Custom((self.map)(ev)),
|
||||
ProtocolsHandlerEvent::Close(err) => ProtocolsHandlerEvent::Close(err),
|
||||
ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info } => {
|
||||
ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info }
|
||||
}
|
||||
}
|
||||
}))
|
||||
})
|
||||
}
|
||||
}
|
||||
|
@ -50,8 +50,7 @@ use libp2p_core::{
|
||||
PeerId,
|
||||
upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeError},
|
||||
};
|
||||
use std::{cmp::Ordering, error, fmt, time::Duration};
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use std::{cmp::Ordering, error, fmt, task::Context, task::Poll, time::Duration};
|
||||
use wasm_timer::Instant;
|
||||
|
||||
pub use dummy::DummyProtocolsHandler;
|
||||
@ -101,7 +100,7 @@ pub trait ProtocolsHandler {
|
||||
/// The type of errors returned by [`ProtocolsHandler::poll`].
|
||||
type Error: error::Error;
|
||||
/// The type of substreams on which the protocol(s) are negotiated.
|
||||
type Substream: AsyncRead + AsyncWrite;
|
||||
type Substream: AsyncRead + AsyncWrite + Unpin;
|
||||
/// The inbound upgrade for the protocol(s) used by the handler.
|
||||
type InboundProtocol: InboundUpgrade<Self::Substream>;
|
||||
/// The outbound upgrade for the protocol(s) used by the handler.
|
||||
@ -171,9 +170,8 @@ pub trait ProtocolsHandler {
|
||||
/// Should behave like `Stream::poll()`.
|
||||
///
|
||||
/// Returning an error will close the connection to the remote.
|
||||
fn poll(&mut self) -> Poll<
|
||||
ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>,
|
||||
Self::Error
|
||||
fn poll(&mut self, cx: &mut Context) -> Poll<
|
||||
ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent, Self::Error>
|
||||
>;
|
||||
|
||||
/// Adds a closure that turns the input event into something else.
|
||||
@ -300,7 +298,7 @@ impl<TUpgrade> From<TUpgrade> for SubstreamProtocol<TUpgrade> {
|
||||
|
||||
/// Event produced by a handler.
|
||||
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
|
||||
pub enum ProtocolsHandlerEvent<TConnectionUpgrade, TOutboundOpenInfo, TCustom> {
|
||||
pub enum ProtocolsHandlerEvent<TConnectionUpgrade, TOutboundOpenInfo, TCustom, TErr> {
|
||||
/// Request a new outbound substream to be opened with the remote.
|
||||
OutboundSubstreamRequest {
|
||||
/// The protocol(s) to apply on the substream.
|
||||
@ -309,13 +307,16 @@ pub enum ProtocolsHandlerEvent<TConnectionUpgrade, TOutboundOpenInfo, TCustom> {
|
||||
info: TOutboundOpenInfo,
|
||||
},
|
||||
|
||||
/// Close the connection for the given reason.
|
||||
Close(TErr),
|
||||
|
||||
/// Other event.
|
||||
Custom(TCustom),
|
||||
}
|
||||
|
||||
/// Event produced by a handler.
|
||||
impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom>
|
||||
ProtocolsHandlerEvent<TConnectionUpgrade, TOutboundOpenInfo, TCustom>
|
||||
impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom, TErr>
|
||||
ProtocolsHandlerEvent<TConnectionUpgrade, TOutboundOpenInfo, TCustom, TErr>
|
||||
{
|
||||
/// If this is an `OutboundSubstreamRequest`, maps the `info` member from a
|
||||
/// `TOutboundOpenInfo` to something else.
|
||||
@ -323,7 +324,7 @@ impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom>
|
||||
pub fn map_outbound_open_info<F, I>(
|
||||
self,
|
||||
map: F,
|
||||
) -> ProtocolsHandlerEvent<TConnectionUpgrade, I, TCustom>
|
||||
) -> ProtocolsHandlerEvent<TConnectionUpgrade, I, TCustom, TErr>
|
||||
where
|
||||
F: FnOnce(TOutboundOpenInfo) -> I,
|
||||
{
|
||||
@ -335,6 +336,7 @@ impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom>
|
||||
}
|
||||
}
|
||||
ProtocolsHandlerEvent::Custom(val) => ProtocolsHandlerEvent::Custom(val),
|
||||
ProtocolsHandlerEvent::Close(val) => ProtocolsHandlerEvent::Close(val),
|
||||
}
|
||||
}
|
||||
|
||||
@ -344,7 +346,7 @@ impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom>
|
||||
pub fn map_protocol<F, I>(
|
||||
self,
|
||||
map: F,
|
||||
) -> ProtocolsHandlerEvent<I, TOutboundOpenInfo, TCustom>
|
||||
) -> ProtocolsHandlerEvent<I, TOutboundOpenInfo, TCustom, TErr>
|
||||
where
|
||||
F: FnOnce(TConnectionUpgrade) -> I,
|
||||
{
|
||||
@ -356,6 +358,7 @@ impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom>
|
||||
}
|
||||
}
|
||||
ProtocolsHandlerEvent::Custom(val) => ProtocolsHandlerEvent::Custom(val),
|
||||
ProtocolsHandlerEvent::Close(val) => ProtocolsHandlerEvent::Close(val),
|
||||
}
|
||||
}
|
||||
|
||||
@ -364,7 +367,7 @@ impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom>
|
||||
pub fn map_custom<F, I>(
|
||||
self,
|
||||
map: F,
|
||||
) -> ProtocolsHandlerEvent<TConnectionUpgrade, TOutboundOpenInfo, I>
|
||||
) -> ProtocolsHandlerEvent<TConnectionUpgrade, TOutboundOpenInfo, I, TErr>
|
||||
where
|
||||
F: FnOnce(TCustom) -> I,
|
||||
{
|
||||
@ -373,6 +376,25 @@ impl<TConnectionUpgrade, TOutboundOpenInfo, TCustom>
|
||||
ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info }
|
||||
}
|
||||
ProtocolsHandlerEvent::Custom(val) => ProtocolsHandlerEvent::Custom(map(val)),
|
||||
ProtocolsHandlerEvent::Close(val) => ProtocolsHandlerEvent::Close(val),
|
||||
}
|
||||
}
|
||||
|
||||
/// If this is a `Close` event, maps the content to something else.
|
||||
#[inline]
|
||||
pub fn map_close<F, I>(
|
||||
self,
|
||||
map: F,
|
||||
) -> ProtocolsHandlerEvent<TConnectionUpgrade, TOutboundOpenInfo, TCustom, I>
|
||||
where
|
||||
F: FnOnce(TErr) -> I,
|
||||
{
|
||||
match self {
|
||||
ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info } => {
|
||||
ProtocolsHandlerEvent::OutboundSubstreamRequest { protocol, info }
|
||||
}
|
||||
ProtocolsHandlerEvent::Custom(val) => ProtocolsHandlerEvent::Custom(val),
|
||||
ProtocolsHandlerEvent::Close(val) => ProtocolsHandlerEvent::Close(map(val)),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -33,8 +33,8 @@ use libp2p_core::{
|
||||
nodes::handled_node::{IntoNodeHandler, NodeHandler, NodeHandlerEndpoint, NodeHandlerEvent},
|
||||
upgrade::{self, InboundUpgradeApply, OutboundUpgradeApply}
|
||||
};
|
||||
use std::{error, fmt, time::Duration};
|
||||
use wasm_timer::{Delay, Timeout};
|
||||
use std::{error, fmt, pin::Pin, task::Context, task::Poll, time::Duration};
|
||||
use wasm_timer::{Delay, Instant};
|
||||
|
||||
/// Prototype for a `NodeHandlerWrapper`.
|
||||
pub struct NodeHandlerWrapperBuilder<TIntoProtoHandler> {
|
||||
@ -102,12 +102,13 @@ where
|
||||
handler: TProtoHandler,
|
||||
/// Futures that upgrade incoming substreams.
|
||||
negotiating_in:
|
||||
Vec<Timeout<InboundUpgradeApply<TProtoHandler::Substream, TProtoHandler::InboundProtocol>>>,
|
||||
Vec<(InboundUpgradeApply<TProtoHandler::Substream, TProtoHandler::InboundProtocol>, Delay)>,
|
||||
/// Futures that upgrade outgoing substreams. The first element of the tuple is the userdata
|
||||
/// to pass back once successfully opened.
|
||||
negotiating_out: Vec<(
|
||||
TProtoHandler::OutboundOpenInfo,
|
||||
Timeout<OutboundUpgradeApply<TProtoHandler::Substream, TProtoHandler::OutboundProtocol>>,
|
||||
OutboundUpgradeApply<TProtoHandler::Substream, TProtoHandler::OutboundProtocol>,
|
||||
Delay,
|
||||
)>,
|
||||
/// For each outbound substream request, how to upgrade it. The first element of the tuple
|
||||
/// is the unique identifier (see `unique_dial_upgrade_id`).
|
||||
@ -133,7 +134,7 @@ enum Shutdown {
|
||||
/// A shut down is planned as soon as possible.
|
||||
Asap,
|
||||
/// A shut down is planned for when a `Delay` has elapsed.
|
||||
Later(Delay)
|
||||
Later(Delay, Instant)
|
||||
}
|
||||
|
||||
/// Error generated by the `NodeHandlerWrapper`.
|
||||
@ -198,8 +199,8 @@ where
|
||||
let protocol = self.handler.listen_protocol();
|
||||
let timeout = protocol.timeout().clone();
|
||||
let upgrade = upgrade::apply_inbound(substream, protocol.into_upgrade());
|
||||
let with_timeout = Timeout::new(upgrade, timeout);
|
||||
self.negotiating_in.push(with_timeout);
|
||||
let timeout = Delay::new(timeout);
|
||||
self.negotiating_in.push((upgrade, timeout));
|
||||
}
|
||||
NodeHandlerEndpoint::Dialer((upgrade_id, user_data, timeout)) => {
|
||||
let pos = match self
|
||||
@ -216,8 +217,8 @@ where
|
||||
|
||||
let (_, proto_upgrade) = self.queued_dial_upgrades.remove(pos);
|
||||
let upgrade = upgrade::apply_outbound(substream, proto_upgrade);
|
||||
let with_timeout = Timeout::new(upgrade, timeout);
|
||||
self.negotiating_out.push((user_data, with_timeout));
|
||||
let timeout = Delay::new(timeout);
|
||||
self.negotiating_out.push((user_data, upgrade, timeout));
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -227,44 +228,50 @@ where
|
||||
self.handler.inject_event(event);
|
||||
}
|
||||
|
||||
fn poll(&mut self) -> Poll<NodeHandlerEvent<Self::OutboundOpenInfo, Self::OutEvent>, Self::Error> {
|
||||
fn poll(&mut self, cx: &mut Context) -> Poll<Result<NodeHandlerEvent<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() {
|
||||
let mut in_progress = self.negotiating_in.swap_remove(n);
|
||||
match in_progress.poll() {
|
||||
Ok(Async::Ready(upgrade)) =>
|
||||
let (mut in_progress, mut timeout) = self.negotiating_in.swap_remove(n);
|
||||
match Future::poll(Pin::new(&mut timeout), cx) {
|
||||
Poll::Ready(_) => continue,
|
||||
Poll::Pending => {},
|
||||
}
|
||||
match Future::poll(Pin::new(&mut in_progress), cx) {
|
||||
Poll::Ready(Ok(upgrade)) =>
|
||||
self.handler.inject_fully_negotiated_inbound(upgrade),
|
||||
Ok(Async::NotReady) => self.negotiating_in.push(in_progress),
|
||||
Poll::Pending => self.negotiating_in.push((in_progress, timeout)),
|
||||
// TODO: return a diagnostic event?
|
||||
Err(_err) => {}
|
||||
Poll::Ready(Err(_err)) => {}
|
||||
}
|
||||
}
|
||||
|
||||
// Continue negotiation of newly-opened substreams.
|
||||
// We remove each element from `negotiating_out` one by one and add them back if not ready.
|
||||
for n in (0..self.negotiating_out.len()).rev() {
|
||||
let (upgr_info, mut in_progress) = self.negotiating_out.swap_remove(n);
|
||||
match in_progress.poll() {
|
||||
Ok(Async::Ready(upgrade)) => {
|
||||
let (upgr_info, mut in_progress, mut timeout) = self.negotiating_out.swap_remove(n);
|
||||
match Future::poll(Pin::new(&mut timeout), cx) {
|
||||
Poll::Ready(Ok(_)) => {
|
||||
let err = ProtocolsHandlerUpgrErr::Timeout;
|
||||
self.handler.inject_dial_upgrade_error(upgr_info, err);
|
||||
continue;
|
||||
},
|
||||
Poll::Ready(Err(_)) => {
|
||||
let err = ProtocolsHandlerUpgrErr::Timer;
|
||||
self.handler.inject_dial_upgrade_error(upgr_info, err);
|
||||
continue;
|
||||
},
|
||||
Poll::Pending => {},
|
||||
}
|
||||
match Future::poll(Pin::new(&mut in_progress), cx) {
|
||||
Poll::Ready(Ok(upgrade)) => {
|
||||
self.handler.inject_fully_negotiated_outbound(upgrade, upgr_info);
|
||||
}
|
||||
Ok(Async::NotReady) => {
|
||||
self.negotiating_out.push((upgr_info, in_progress));
|
||||
Poll::Pending => {
|
||||
self.negotiating_out.push((upgr_info, in_progress, timeout));
|
||||
}
|
||||
Err(err) => {
|
||||
let err = if err.is_elapsed() {
|
||||
ProtocolsHandlerUpgrErr::Timeout
|
||||
} else if err.is_timer() {
|
||||
ProtocolsHandlerUpgrErr::Timer
|
||||
} else {
|
||||
debug_assert!(err.is_inner());
|
||||
let err = err.into_inner().expect("Timeout error is one of {elapsed, \
|
||||
timer, inner}; is_elapsed and is_timer are both false; error is \
|
||||
inner; QED");
|
||||
ProtocolsHandlerUpgrErr::Upgrade(err)
|
||||
};
|
||||
|
||||
Poll::Ready(Err(err)) => {
|
||||
let err = ProtocolsHandlerUpgrErr::Upgrade(err);
|
||||
self.handler.inject_dial_upgrade_error(upgr_info, err);
|
||||
}
|
||||
}
|
||||
@ -272,25 +279,26 @@ where
|
||||
|
||||
// Poll the handler at the end so that we see the consequences of the method
|
||||
// calls on `self.handler`.
|
||||
let poll_result = self.handler.poll()?;
|
||||
let poll_result = self.handler.poll(cx);
|
||||
|
||||
// Ask the handler whether it wants the connection (and the handler itself)
|
||||
// to be kept alive, which determines the planned shutdown, if any.
|
||||
match (&mut self.shutdown, self.handler.connection_keep_alive()) {
|
||||
(Shutdown::Later(d), KeepAlive::Until(t)) =>
|
||||
if d.deadline() != t {
|
||||
d.reset(t)
|
||||
(Shutdown::Later(timer, deadline), KeepAlive::Until(t)) =>
|
||||
if *deadline != t {
|
||||
*deadline = t;
|
||||
timer.reset_at(t)
|
||||
},
|
||||
(_, KeepAlive::Until(t)) => self.shutdown = Shutdown::Later(Delay::new(t)),
|
||||
(_, KeepAlive::Until(t)) => self.shutdown = Shutdown::Later(Delay::new_at(t), t),
|
||||
(_, KeepAlive::No) => self.shutdown = Shutdown::Asap,
|
||||
(_, KeepAlive::Yes) => self.shutdown = Shutdown::None
|
||||
};
|
||||
|
||||
match poll_result {
|
||||
Async::Ready(ProtocolsHandlerEvent::Custom(event)) => {
|
||||
return Ok(Async::Ready(NodeHandlerEvent::Custom(event)));
|
||||
Poll::Ready(ProtocolsHandlerEvent::Custom(event)) => {
|
||||
return Poll::Ready(Ok(NodeHandlerEvent::Custom(event)));
|
||||
}
|
||||
Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
|
||||
Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
|
||||
protocol,
|
||||
info,
|
||||
}) => {
|
||||
@ -298,11 +306,12 @@ where
|
||||
let timeout = protocol.timeout().clone();
|
||||
self.unique_dial_upgrade_id += 1;
|
||||
self.queued_dial_upgrades.push((id, protocol.into_upgrade()));
|
||||
return Ok(Async::Ready(
|
||||
return Poll::Ready(Ok(
|
||||
NodeHandlerEvent::OutboundSubstreamRequest((id, info, timeout)),
|
||||
));
|
||||
}
|
||||
Async::NotReady => (),
|
||||
Poll::Ready(ProtocolsHandlerEvent::Close(err)) => return Poll::Ready(Err(err.into())),
|
||||
Poll::Pending => (),
|
||||
};
|
||||
|
||||
// Check if the connection (and handler) should be shut down.
|
||||
@ -310,15 +319,14 @@ where
|
||||
if self.negotiating_in.is_empty() && self.negotiating_out.is_empty() {
|
||||
match self.shutdown {
|
||||
Shutdown::None => {},
|
||||
Shutdown::Asap => return Err(NodeHandlerWrapperError::UselessTimeout),
|
||||
Shutdown::Later(ref mut delay) => match delay.poll() {
|
||||
Ok(Async::Ready(_)) | Err(_) =>
|
||||
return Err(NodeHandlerWrapperError::UselessTimeout),
|
||||
Ok(Async::NotReady) => {}
|
||||
Shutdown::Asap => return Poll::Ready(Err(NodeHandlerWrapperError::UselessTimeout)),
|
||||
Shutdown::Later(ref mut delay, _) => match Future::poll(Pin::new(delay), cx) {
|
||||
Poll::Ready(_) => return Poll::Ready(Err(NodeHandlerWrapperError::UselessTimeout)),
|
||||
Poll::Pending => {}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(Async::NotReady)
|
||||
Poll::Pending
|
||||
}
|
||||
}
|
||||
|
@ -28,8 +28,7 @@ use crate::protocols_handler::{
|
||||
use futures::prelude::*;
|
||||
use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade};
|
||||
use smallvec::SmallVec;
|
||||
use std::{error, marker::PhantomData, time::Duration};
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use std::{error, marker::PhantomData, task::Context, task::Poll, time::Duration};
|
||||
use wasm_timer::Instant;
|
||||
|
||||
/// Implementation of `ProtocolsHandler` that opens a new substream for each individual message.
|
||||
@ -132,7 +131,7 @@ where
|
||||
impl<TSubstream, TInProto, TOutProto, TOutEvent> ProtocolsHandler
|
||||
for OneShotHandler<TSubstream, TInProto, TOutProto, TOutEvent>
|
||||
where
|
||||
TSubstream: AsyncRead + AsyncWrite,
|
||||
TSubstream: AsyncRead + AsyncWrite + Unpin,
|
||||
TInProto: InboundUpgrade<TSubstream>,
|
||||
TOutProto: OutboundUpgrade<TSubstream>,
|
||||
TInProto::Output: Into<TOutEvent>,
|
||||
@ -208,18 +207,18 @@ where
|
||||
|
||||
fn poll(
|
||||
&mut self,
|
||||
_: &mut Context,
|
||||
) -> Poll<
|
||||
ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>,
|
||||
Self::Error,
|
||||
ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent, Self::Error>,
|
||||
> {
|
||||
if let Some(err) = self.pending_error.take() {
|
||||
return Err(err);
|
||||
return Poll::Ready(ProtocolsHandlerEvent::Close(err));
|
||||
}
|
||||
|
||||
if !self.events_out.is_empty() {
|
||||
return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(
|
||||
return Poll::Ready(ProtocolsHandlerEvent::Custom(
|
||||
self.events_out.remove(0),
|
||||
)));
|
||||
));
|
||||
} else {
|
||||
self.events_out.shrink_to_fit();
|
||||
}
|
||||
@ -227,17 +226,17 @@ where
|
||||
if !self.dial_queue.is_empty() {
|
||||
if self.dial_negotiated < self.max_dial_negotiated {
|
||||
self.dial_negotiated += 1;
|
||||
return Ok(Async::Ready(
|
||||
return Poll::Ready(
|
||||
ProtocolsHandlerEvent::OutboundSubstreamRequest {
|
||||
protocol: SubstreamProtocol::new(self.dial_queue.remove(0)),
|
||||
info: (),
|
||||
},
|
||||
));
|
||||
);
|
||||
}
|
||||
} else {
|
||||
self.dial_queue.shrink_to_fit();
|
||||
}
|
||||
|
||||
Ok(Async::NotReady)
|
||||
Poll::Pending
|
||||
}
|
||||
}
|
||||
|
@ -33,8 +33,7 @@ use libp2p_core::{
|
||||
either::{EitherError, EitherOutput},
|
||||
upgrade::{InboundUpgrade, OutboundUpgrade, EitherUpgrade, SelectUpgrade, UpgradeError}
|
||||
};
|
||||
use std::cmp;
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use std::{cmp, task::Context, task::Poll};
|
||||
|
||||
/// Implementation of `IntoProtocolsHandler` that combines two protocols into one.
|
||||
#[derive(Debug, Clone)]
|
||||
@ -62,7 +61,7 @@ where
|
||||
TProto2: IntoProtocolsHandler,
|
||||
TProto1::Handler: ProtocolsHandler<Substream = TSubstream>,
|
||||
TProto2::Handler: ProtocolsHandler<Substream = TSubstream>,
|
||||
TSubstream: AsyncRead + AsyncWrite,
|
||||
TSubstream: AsyncRead + AsyncWrite + Unpin,
|
||||
<TProto1::Handler as ProtocolsHandler>::InboundProtocol: InboundUpgrade<TSubstream>,
|
||||
<TProto2::Handler as ProtocolsHandler>::InboundProtocol: InboundUpgrade<TSubstream>,
|
||||
<TProto1::Handler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade<TSubstream>,
|
||||
@ -107,7 +106,7 @@ impl<TSubstream, TProto1, TProto2>
|
||||
where
|
||||
TProto1: ProtocolsHandler<Substream = TSubstream>,
|
||||
TProto2: ProtocolsHandler<Substream = TSubstream>,
|
||||
TSubstream: AsyncRead + AsyncWrite,
|
||||
TSubstream: AsyncRead + AsyncWrite + Unpin,
|
||||
TProto1::InboundProtocol: InboundUpgrade<TSubstream>,
|
||||
TProto2::InboundProtocol: InboundUpgrade<TSubstream>,
|
||||
TProto1::OutboundProtocol: OutboundUpgrade<TSubstream>,
|
||||
@ -201,40 +200,46 @@ where
|
||||
cmp::max(self.proto1.connection_keep_alive(), self.proto2.connection_keep_alive())
|
||||
}
|
||||
|
||||
fn poll(&mut self) -> Poll<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>, Self::Error> {
|
||||
fn poll(&mut self, cx: &mut Context) -> Poll<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent, Self::Error>> {
|
||||
|
||||
match self.proto1.poll().map_err(EitherError::A)? {
|
||||
Async::Ready(ProtocolsHandlerEvent::Custom(event)) => {
|
||||
return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(EitherOutput::First(event))));
|
||||
match self.proto1.poll(cx) {
|
||||
Poll::Ready(ProtocolsHandlerEvent::Custom(event)) => {
|
||||
return Poll::Ready(ProtocolsHandlerEvent::Custom(EitherOutput::First(event)));
|
||||
},
|
||||
Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
|
||||
Poll::Ready(ProtocolsHandlerEvent::Close(event)) => {
|
||||
return Poll::Ready(ProtocolsHandlerEvent::Close(EitherError::A(event)));
|
||||
},
|
||||
Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
|
||||
protocol,
|
||||
info,
|
||||
}) => {
|
||||
return Ok(Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
|
||||
return Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
|
||||
protocol: protocol.map_upgrade(EitherUpgrade::A),
|
||||
info: EitherOutput::First(info),
|
||||
}));
|
||||
});
|
||||
},
|
||||
Async::NotReady => ()
|
||||
Poll::Pending => ()
|
||||
};
|
||||
|
||||
match self.proto2.poll().map_err(EitherError::B)? {
|
||||
Async::Ready(ProtocolsHandlerEvent::Custom(event)) => {
|
||||
return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(EitherOutput::Second(event))));
|
||||
match self.proto2.poll(cx) {
|
||||
Poll::Ready(ProtocolsHandlerEvent::Custom(event)) => {
|
||||
return Poll::Ready(ProtocolsHandlerEvent::Custom(EitherOutput::Second(event)));
|
||||
},
|
||||
Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
|
||||
Poll::Ready(ProtocolsHandlerEvent::Close(event)) => {
|
||||
return Poll::Ready(ProtocolsHandlerEvent::Close(EitherError::B(event)));
|
||||
},
|
||||
Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
|
||||
protocol,
|
||||
info,
|
||||
}) => {
|
||||
return Ok(Async::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
|
||||
return Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest {
|
||||
protocol: protocol.map_upgrade(EitherUpgrade::B),
|
||||
info: EitherOutput::Second(info),
|
||||
}));
|
||||
});
|
||||
},
|
||||
Async::NotReady => ()
|
||||
Poll::Pending => ()
|
||||
};
|
||||
|
||||
Ok(Async::NotReady)
|
||||
Poll::Pending
|
||||
}
|
||||
}
|
||||
|
@ -34,8 +34,7 @@ use libp2p_core::{
|
||||
either::EitherOutput,
|
||||
upgrade::{InboundUpgrade, OutboundUpgrade, DeniedUpgrade, EitherUpgrade}
|
||||
};
|
||||
use futures::prelude::*;
|
||||
use std::error;
|
||||
use std::{error, task::Context, task::Poll};
|
||||
|
||||
/// Implementation of `NetworkBehaviour` that can be either in the disabled or enabled state.
|
||||
///
|
||||
@ -132,13 +131,13 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
fn poll(&mut self, params: &mut impl PollParameters)
|
||||
-> Async<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::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InEvent, Self::OutEvent>>
|
||||
{
|
||||
if let Some(inner) = self.inner.as_mut() {
|
||||
inner.poll(params)
|
||||
inner.poll(cx, params)
|
||||
} else {
|
||||
Async::NotReady
|
||||
Poll::Pending
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -244,14 +243,14 @@ where
|
||||
|
||||
fn poll(
|
||||
&mut self,
|
||||
cx: &mut Context,
|
||||
) -> Poll<
|
||||
ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>,
|
||||
Self::Error,
|
||||
ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent, Self::Error>
|
||||
> {
|
||||
if let Some(inner) = self.inner.as_mut() {
|
||||
inner.poll()
|
||||
inner.poll(cx)
|
||||
} else {
|
||||
Ok(Async::NotReady)
|
||||
Poll::Pending
|
||||
}
|
||||
}
|
||||
}
|
||||
|
Reference in New Issue
Block a user