Switch to stable futures (#1196)

* Switch to stable futures

* Remove from_fn

* Fix secio

* Fix core --lib tests
This commit is contained in:
Pierre Krieger
2019-09-16 11:08:44 +02:00
committed by GitHub
parent 8c119269d6
commit 170d2d268f
105 changed files with 3193 additions and 5594 deletions

View File

@ -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
}
}

View File

@ -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)
}
}

View File

@ -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 }
}
}
}))
})
}
}

View File

@ -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)),
}
}
}

View File

@ -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
}
}

View File

@ -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
}
}

View File

@ -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
}
}