Split ConnectionUpgrade. (#642)

Introduce `InboundUpgrade` and `OutboundUpgrade`.
This commit is contained in:
Toralf Wittner
2018-11-15 17:41:11 +01:00
committed by Pierre Krieger
parent 466385a58a
commit 2e549884ef
52 changed files with 2010 additions and 1658 deletions

View File

@ -18,15 +18,19 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use crate::{
PeerId,
muxing::StreamMuxer,
nodes::{
node::Substream,
handled_node_tasks::{HandledNodesEvent, HandledNodesTasks},
handled_node_tasks::{Task as HandledNodesTask, TaskId},
handled_node::NodeHandler
}
};
use fnv::FnvHashMap;
use futures::prelude::*;
use muxing::StreamMuxer;
use nodes::node::Substream;
use nodes::handled_node_tasks::{HandledNodesEvent, HandledNodesTasks};
use nodes::handled_node_tasks::{Task as HandledNodesTask, TaskId};
use nodes::handled_node::NodeHandler;
use std::{collections::hash_map::Entry, fmt, io, mem};
use PeerId;
// TODO: make generic over PeerId
@ -276,7 +280,8 @@ impl<TInEvent, TOutEvent, THandler> CollectionStream<TInEvent, TOutEvent, THandl
pub fn add_reach_attempt<TFut, TMuxer>(&mut self, future: TFut, handler: THandler)
-> ReachAttemptId
where
TFut: Future<Item = (PeerId, TMuxer), Error = io::Error> + Send + 'static,
TFut: Future<Item = (PeerId, TMuxer)> + Send + 'static,
TFut::Error: std::error::Error + Send + Sync + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,

View File

@ -18,18 +18,25 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use crate::{
PeerId,
muxing::StreamMuxer,
nodes::{
handled_node::{HandledNode, NodeHandler},
node::Substream
}
};
use fnv::FnvHashMap;
use futures::{prelude::*, stream, sync::mpsc};
use muxing::StreamMuxer;
use nodes::node::Substream;
use nodes::handled_node::{HandledNode, NodeHandler};
use smallvec::SmallVec;
use std::collections::hash_map::{Entry, OccupiedEntry};
use std::io::Error as IoError;
use std::{fmt, mem};
use std::{
collections::hash_map::{Entry, OccupiedEntry},
fmt,
io::{self, Error as IoError},
mem
};
use tokio_executor;
use void::Void;
use PeerId;
// TODO: make generic over PeerId
@ -135,10 +142,10 @@ impl<TInEvent, TOutEvent, THandler> HandledNodesTasks<TInEvent, TOutEvent, THand
///
/// This method spawns a task dedicated to resolving this future and processing the node's
/// events.
pub fn add_reach_attempt<TFut, TMuxer>(&mut self, future: TFut, handler: THandler)
-> TaskId
pub fn add_reach_attempt<TFut, TMuxer>(&mut self, future: TFut, handler: THandler) -> TaskId
where
TFut: Future<Item = (PeerId, TMuxer), Error = IoError> + Send + 'static,
TFut: Future<Item = (PeerId, TMuxer)> + Send + 'static,
TFut::Error: std::error::Error + Send + Sync + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
@ -349,7 +356,8 @@ impl<TFut, TMuxer, THandler, TInEvent, TOutEvent> Future for
NodeTask<TFut, TMuxer, THandler, TInEvent, TOutEvent>
where
TMuxer: StreamMuxer,
TFut: Future<Item = (PeerId, TMuxer), Error = IoError>,
TFut: Future<Item = (PeerId, TMuxer)>,
TFut::Error: std::error::Error + Send + Sync + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent>,
{
type Item = ();
@ -388,7 +396,8 @@ where
},
Err(err) => {
// End the task
let event = InToExtMessage::TaskClosed(Err(err), Some(handler));
let ioerr = IoError::new(io::ErrorKind::Other, err);
let event = InToExtMessage::TaskClosed(Err(ioerr), Some(handler));
let _ = self.events_tx.unbounded_send((event, self.id));
return Ok(Async::Ready(()));
}
@ -464,6 +473,7 @@ mod tests {
use tests::dummy_muxer::{DummyMuxer, DummyConnectionState};
use tokio::runtime::Builder;
use tokio::runtime::current_thread::Runtime;
use void::Void;
use {PeerId, PublicKey};
type TestNodeTask = NodeTask<
@ -570,7 +580,7 @@ mod tests {
let peer_id = PublicKey::Rsa((0 .. 2048).map(|_| -> u8 { random() }).collect()).into_peer_id();
let mut task_ids = Vec::new();
for _i in 0..self.task_count {
let fut = future::ok((peer_id.clone(), self.muxer.clone()));
let fut = future::ok::<_, Void>((peer_id.clone(), self.muxer.clone()));
task_ids.push(
handled_nodes.add_reach_attempt(fut, self.handler.clone())
);
@ -719,7 +729,7 @@ mod tests {
assert_eq!(handled_nodes.tasks().count(), 0);
assert_eq!(handled_nodes.to_spawn.len(), 0);
handled_nodes.add_reach_attempt( future::empty(), Handler::default() );
handled_nodes.add_reach_attempt( future::empty::<_, Void>(), Handler::default() );
assert_eq!(handled_nodes.tasks().count(), 1);
assert_eq!(handled_nodes.to_spawn.len(), 1);

View File

@ -18,15 +18,26 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use either::EitherOutput;
use crate::{
either::{EitherError, EitherOutput},
nodes::handled_node::{NodeHandler, NodeHandlerEndpoint, NodeHandlerEvent},
upgrade::{
self,
InboundUpgrade,
InboundUpgradeExt,
OutboundUpgrade,
OutboundUpgradeExt,
UpgradeInfo,
InboundUpgradeApply,
OutboundUpgradeApply,
DeniedUpgrade
}
};
use futures::prelude::*;
use nodes::handled_node::{NodeHandler, NodeHandlerEndpoint, NodeHandlerEvent};
use std::{io, marker::PhantomData, time::Duration};
use tokio_io::{AsyncRead, AsyncWrite};
use tokio_timer::Timeout;
use upgrade::{self, apply::UpgradeApplyFuture, DeniedConnectionUpgrade};
use void::Void;
use {ConnectionUpgrade, Endpoint};
/// Handler for a set of protocols for a specific connection with a remote.
///
@ -81,7 +92,9 @@ pub trait ProtocolsHandler {
/// The type of the substream that contains the raw data.
type Substream: AsyncRead + AsyncWrite;
/// The upgrade for the protocol or protocols handled by this handler.
type Protocol: ConnectionUpgrade<Self::Substream>;
type InboundProtocol: InboundUpgrade<Self::Substream>;
/// The upgrade for the protocol or protocols handled by this handler.
type OutboundProtocol: OutboundUpgrade<Self::Substream>;
/// Information about a substream. Can be sent to the handler through a `NodeHandlerEndpoint`,
/// and will be passed back in `inject_substream` or `inject_outbound_closed`.
type OutboundOpenInfo;
@ -92,15 +105,22 @@ pub trait ProtocolsHandler {
/// > context you wouldn't accept one in particular (eg. only allow one substream at
/// > a time for a given protocol). The reason is that remotes are allowed to put the
/// > list of supported protocols in a cache in order to avoid spurious queries.
fn listen_protocol(&self) -> Self::Protocol;
fn listen_protocol(&self) -> Self::InboundProtocol;
fn dialer_protocol(&self) -> Self::OutboundProtocol;
/// Injects a fully-negotiated substream in the handler.
///
/// This method is called when a substream has been successfully opened and negotiated.
fn inject_fully_negotiated(
fn inject_fully_negotiated_inbound(
&mut self,
protocol: <Self::Protocol as ConnectionUpgrade<Self::Substream>>::Output,
endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>,
protocol: <Self::InboundProtocol as InboundUpgrade<Self::Substream>>::Output
);
fn inject_fully_negotiated_outbound(
&mut self,
protocol: <Self::OutboundProtocol as OutboundUpgrade<Self::Substream>>::Output,
info: Self::OutboundOpenInfo
);
/// Injects an event coming from the outside in the handler.
@ -126,12 +146,7 @@ pub trait ProtocolsHandler {
/// > **Note**: If this handler is combined with other handlers, as soon as `poll()` returns
/// > `Ok(Async::Ready(None))`, all the other handlers will receive a call to
/// > `shutdown()` and will eventually be closed and destroyed.
fn poll(
&mut self,
) -> Poll<
Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>,
io::Error,
>;
fn poll(&mut self) -> Poll<Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>, io::Error>;
/// Adds a closure that turns the input event into something else.
#[inline]
@ -296,19 +311,32 @@ where
type InEvent = Void;
type OutEvent = Void;
type Substream = TSubstream;
type Protocol = DeniedConnectionUpgrade;
type InboundProtocol = DeniedUpgrade;
type OutboundProtocol = DeniedUpgrade;
type OutboundOpenInfo = Void;
#[inline]
fn listen_protocol(&self) -> Self::Protocol {
DeniedConnectionUpgrade
fn listen_protocol(&self) -> Self::InboundProtocol {
DeniedUpgrade
}
#[inline]
fn inject_fully_negotiated(
fn dialer_protocol(&self) -> Self::OutboundProtocol {
DeniedUpgrade
}
#[inline]
fn inject_fully_negotiated_inbound(
&mut self,
_: <Self::Protocol as ConnectionUpgrade<TSubstream>>::Output,
_: NodeHandlerEndpoint<Self::OutboundOpenInfo>,
_: <Self::InboundProtocol as InboundUpgrade<TSubstream>>::Output
) {
}
#[inline]
fn inject_fully_negotiated_outbound(
&mut self,
_: <Self::OutboundProtocol as OutboundUpgrade<TSubstream>>::Output,
_: Self::OutboundOpenInfo
) {
}
@ -330,7 +358,7 @@ where
fn poll(
&mut self,
) -> Poll<
Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>,
Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>,
io::Error,
> {
if self.shutting_down {
@ -356,21 +384,35 @@ where
type InEvent = TNewIn;
type OutEvent = TProtoHandler::OutEvent;
type Substream = TProtoHandler::Substream;
type Protocol = TProtoHandler::Protocol;
type InboundProtocol = TProtoHandler::InboundProtocol;
type OutboundProtocol = TProtoHandler::OutboundProtocol;
type OutboundOpenInfo = TProtoHandler::OutboundOpenInfo;
#[inline]
fn listen_protocol(&self) -> Self::Protocol {
fn listen_protocol(&self) -> Self::InboundProtocol {
self.inner.listen_protocol()
}
#[inline]
fn inject_fully_negotiated(
fn dialer_protocol(&self) -> Self::OutboundProtocol {
self.inner.dialer_protocol()
}
#[inline]
fn inject_fully_negotiated_inbound(
&mut self,
protocol: <Self::Protocol as ConnectionUpgrade<Self::Substream>>::Output,
endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>,
protocol: <Self::InboundProtocol as InboundUpgrade<Self::Substream>>::Output
) {
self.inner.inject_fully_negotiated(protocol, endpoint)
self.inner.inject_fully_negotiated_inbound(protocol)
}
#[inline]
fn inject_fully_negotiated_outbound(
&mut self,
protocol: <Self::OutboundProtocol as OutboundUpgrade<Self::Substream>>::Output,
info: Self::OutboundOpenInfo
) {
self.inner.inject_fully_negotiated_outbound(protocol, info)
}
#[inline]
@ -399,7 +441,7 @@ where
fn poll(
&mut self,
) -> Poll<
Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>,
Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>,
io::Error,
> {
self.inner.poll()
@ -420,21 +462,35 @@ where
type InEvent = TProtoHandler::InEvent;
type OutEvent = TNewOut;
type Substream = TProtoHandler::Substream;
type Protocol = TProtoHandler::Protocol;
type InboundProtocol = TProtoHandler::InboundProtocol;
type OutboundProtocol = TProtoHandler::OutboundProtocol;
type OutboundOpenInfo = TProtoHandler::OutboundOpenInfo;
#[inline]
fn listen_protocol(&self) -> Self::Protocol {
fn listen_protocol(&self) -> Self::InboundProtocol {
self.inner.listen_protocol()
}
#[inline]
fn inject_fully_negotiated(
fn dialer_protocol(&self) -> Self::OutboundProtocol {
self.inner.dialer_protocol()
}
#[inline]
fn inject_fully_negotiated_inbound(
&mut self,
protocol: <Self::Protocol as ConnectionUpgrade<Self::Substream>>::Output,
endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>,
protocol: <Self::InboundProtocol as InboundUpgrade<Self::Substream>>::Output
) {
self.inner.inject_fully_negotiated(protocol, endpoint)
self.inner.inject_fully_negotiated_inbound(protocol)
}
#[inline]
fn inject_fully_negotiated_outbound(
&mut self,
protocol: <Self::OutboundProtocol as OutboundUpgrade<Self::Substream>>::Output,
info: Self::OutboundOpenInfo
) {
self.inner.inject_fully_negotiated_outbound(protocol, info)
}
#[inline]
@ -461,7 +517,7 @@ where
fn poll(
&mut self,
) -> Poll<
Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>,
Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>,
io::Error,
> {
Ok(self.inner.poll()?.map(|ev| {
@ -531,12 +587,12 @@ where
handler: TProtoHandler,
/// Futures that upgrade incoming substreams.
negotiating_in:
Vec<Timeout<UpgradeApplyFuture<TProtoHandler::Substream, TProtoHandler::Protocol>>>,
Vec<Timeout<InboundUpgradeApply<TProtoHandler::Substream, TProtoHandler::InboundProtocol>>>,
/// 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<UpgradeApplyFuture<TProtoHandler::Substream, TProtoHandler::Protocol>>,
Timeout<OutboundUpgradeApply<TProtoHandler::Substream, TProtoHandler::OutboundProtocol>>,
)>,
/// Timeout for incoming substreams negotiation.
in_timeout: Duration,
@ -544,7 +600,7 @@ where
out_timeout: Duration,
/// For each outbound substream request, how to upgrade it. The first element of the tuple
/// is the unique identifier (see `unique_dial_upgrade_id`).
queued_dial_upgrades: Vec<(u64, TProtoHandler::Protocol)>,
queued_dial_upgrades: Vec<(u64, TProtoHandler::OutboundProtocol)>,
/// Unique identifier assigned to each queued dial upgrade.
unique_dial_upgrade_id: u64,
}
@ -552,7 +608,8 @@ where
impl<TProtoHandler> NodeHandler for NodeHandlerWrapper<TProtoHandler>
where
TProtoHandler: ProtocolsHandler,
<TProtoHandler::Protocol as ConnectionUpgrade<TProtoHandler::Substream>>::NamesIter: Clone,
<TProtoHandler::InboundProtocol as UpgradeInfo>::NamesIter: Clone,
<TProtoHandler::OutboundProtocol as OutboundUpgrade<<TProtoHandler as ProtocolsHandler>::Substream>>::Error: std::fmt::Debug
{
type InEvent = TProtoHandler::InEvent;
type OutEvent = TProtoHandler::OutEvent;
@ -569,7 +626,7 @@ where
match endpoint {
NodeHandlerEndpoint::Listener => {
let protocol = self.handler.listen_protocol();
let upgrade = upgrade::apply(substream, protocol, Endpoint::Listener);
let upgrade = upgrade::apply_inbound(substream, protocol);
let with_timeout = Timeout::new(upgrade, self.in_timeout);
self.negotiating_in.push(with_timeout);
}
@ -587,7 +644,7 @@ where
};
let (_, proto_upgrade) = self.queued_dial_upgrades.remove(pos);
let upgrade = upgrade::apply(substream, proto_upgrade, Endpoint::Dialer);
let upgrade = upgrade::apply_outbound(substream, proto_upgrade);
let with_timeout = Timeout::new(upgrade, self.out_timeout);
self.negotiating_out.push((user_data, with_timeout));
}
@ -630,21 +687,15 @@ where
self.handler.shutdown();
}
fn poll(
&mut self,
) -> Poll<Option<NodeHandlerEvent<Self::OutboundOpenInfo, Self::OutEvent>>, io::Error> {
fn poll(&mut self) -> Poll<Option<NodeHandlerEvent<Self::OutboundOpenInfo, Self::OutEvent>>, io::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)) => {
self.handler
.inject_fully_negotiated(upgrade, NodeHandlerEndpoint::Listener);
}
Ok(Async::NotReady) => {
self.negotiating_in.push(in_progress);
}
Ok(Async::Ready(upgrade)) =>
self.handler.inject_fully_negotiated_inbound(upgrade),
Ok(Async::NotReady) => self.negotiating_in.push(in_progress),
// TODO: return a diagnostic event?
Err(_err) => {}
}
@ -656,8 +707,7 @@ where
let (upgr_info, mut in_progress) = self.negotiating_out.swap_remove(n);
match in_progress.poll() {
Ok(Async::Ready(upgrade)) => {
let endpoint = NodeHandlerEndpoint::Dialer(upgr_info);
self.handler.inject_fully_negotiated(upgrade, endpoint);
self.handler.inject_fully_negotiated_outbound(upgrade, upgr_info);
}
Ok(Async::NotReady) => {
self.negotiating_out.push((upgr_info, in_progress));
@ -704,45 +754,138 @@ pub struct ProtocolsHandlerSelect<TProto1, TProto2> {
impl<TSubstream, TProto1, TProto2, TProto1Out, TProto2Out>
ProtocolsHandler for ProtocolsHandlerSelect<TProto1, TProto2>
where TProto1: ProtocolsHandler<Substream = TSubstream>,
TProto2: ProtocolsHandler<Substream = TSubstream>,
TSubstream: AsyncRead + AsyncWrite,
TProto1::Protocol: ConnectionUpgrade<TSubstream, Output = TProto1Out>,
TProto2::Protocol: ConnectionUpgrade<TSubstream, Output = TProto2Out>,
where
TProto1: ProtocolsHandler<Substream = TSubstream>,
TProto2: ProtocolsHandler<Substream = TSubstream>,
TSubstream: AsyncRead + AsyncWrite,
TProto1::InboundProtocol: InboundUpgrade<TSubstream, Output = TProto1Out>,
TProto2::InboundProtocol: InboundUpgrade<TSubstream, Output = TProto2Out>,
TProto1::OutboundProtocol: OutboundUpgrade<TSubstream, Output = TProto1Out>,
TProto2::OutboundProtocol: OutboundUpgrade<TSubstream, Output = TProto2Out>
{
type InEvent = EitherOutput<TProto1::InEvent, TProto2::InEvent>;
type OutEvent = EitherOutput<TProto1::OutEvent, TProto2::OutEvent>;
type Substream = TSubstream;
type Protocol = upgrade::OrUpgrade<upgrade::toggleable::Toggleable<upgrade::map::Map<TProto1::Protocol, fn(TProto1Out) -> EitherOutput<TProto1Out, TProto2Out>>>, upgrade::toggleable::Toggleable<upgrade::map::Map<TProto2::Protocol, fn(TProto2Out) -> EitherOutput<TProto1Out, TProto2Out>>>>;
type InboundProtocol =
upgrade::OrUpgrade<
upgrade::Toggleable<
upgrade::MapUpgradeErr<
upgrade::MapUpgrade<
TProto1::InboundProtocol,
fn(TProto1Out) -> EitherOutput<TProto1Out, TProto2Out>
>,
fn(<TProto1::InboundProtocol as InboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::InboundProtocol as InboundUpgrade<TSubstream>>::Error,
<TProto2::InboundProtocol as InboundUpgrade<TSubstream>>::Error
>
>
>,
upgrade::Toggleable<
upgrade::MapUpgradeErr<
upgrade::MapUpgrade<
TProto2::InboundProtocol,
fn(TProto2Out) -> EitherOutput<TProto1Out, TProto2Out>
>,
fn(<TProto2::InboundProtocol as InboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::InboundProtocol as InboundUpgrade<TSubstream>>::Error,
<TProto2::InboundProtocol as InboundUpgrade<TSubstream>>::Error
>
>
>
>;
type OutboundProtocol =
upgrade::OrUpgrade<
upgrade::Toggleable<
upgrade::MapUpgradeErr<
upgrade::MapUpgrade<
TProto1::OutboundProtocol,
fn(TProto1Out) -> EitherOutput<TProto1Out, TProto2Out>
>,
fn(<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>
>
>,
upgrade::Toggleable<
upgrade::MapUpgradeErr<
upgrade::MapUpgrade<
TProto2::OutboundProtocol,
fn(TProto2Out) -> EitherOutput<TProto1Out, TProto2Out>
>,
fn(<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>
>
>
>;
type OutboundOpenInfo = EitherOutput<TProto1::OutboundOpenInfo, TProto2::OutboundOpenInfo>;
#[inline]
fn listen_protocol(&self) -> Self::Protocol {
let proto1 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(self.proto1.listen_protocol(), EitherOutput::First));
let proto2 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(self.proto2.listen_protocol(), EitherOutput::Second));
upgrade::or(proto1, proto2)
fn listen_protocol(&self) -> Self::InboundProtocol {
let proto1 = self.proto1.listen_protocol()
.map_inbound(EitherOutput::First as fn(TProto1Out) -> EitherOutput<TProto1Out, TProto2Out>)
.map_inbound_err(EitherError::A as fn(<TProto1::InboundProtocol as InboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::InboundProtocol as InboundUpgrade<TSubstream>>::Error,
<TProto2::InboundProtocol as InboundUpgrade<TSubstream>>::Error
>);
let proto2 = self.proto2.listen_protocol()
.map_inbound(EitherOutput::Second as fn(TProto2Out) -> EitherOutput<TProto1Out, TProto2Out>)
.map_inbound_err(EitherError::B as fn(<TProto2::InboundProtocol as InboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::InboundProtocol as InboundUpgrade<TSubstream>>::Error,
<TProto2::InboundProtocol as InboundUpgrade<TSubstream>>::Error
>);
upgrade::toggleable(proto1).or_inbound(upgrade::toggleable(proto2))
}
fn inject_fully_negotiated(&mut self, protocol: <Self::Protocol as ConnectionUpgrade<TSubstream>>::Output, endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>) {
#[inline]
fn dialer_protocol(&self) -> Self::OutboundProtocol {
let proto1 = self.proto1.dialer_protocol()
.map_outbound(EitherOutput::First as fn(TProto1Out) -> EitherOutput<TProto1Out, TProto2Out>)
.map_outbound_err(EitherError::A as fn(<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>);
let proto2 = self.proto2.dialer_protocol()
.map_outbound(EitherOutput::Second as fn(TProto2Out) -> EitherOutput<TProto1Out, TProto2Out>)
.map_outbound_err(EitherError::B as fn(<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>);
upgrade::toggleable(proto1).or_outbound(upgrade::toggleable(proto2))
}
fn inject_fully_negotiated_outbound(&mut self, protocol: <Self::OutboundProtocol as OutboundUpgrade<TSubstream>>::Output, endpoint: Self::OutboundOpenInfo) {
match (protocol, endpoint) {
(EitherOutput::First(protocol), NodeHandlerEndpoint::Dialer(EitherOutput::First(info))) => {
self.proto1.inject_fully_negotiated(protocol, NodeHandlerEndpoint::Dialer(info));
},
(EitherOutput::Second(protocol), NodeHandlerEndpoint::Dialer(EitherOutput::Second(info))) => {
self.proto2.inject_fully_negotiated(protocol, NodeHandlerEndpoint::Dialer(info));
},
(EitherOutput::First(_), NodeHandlerEndpoint::Dialer(EitherOutput::Second(_))) => {
(EitherOutput::First(protocol), EitherOutput::First(info)) =>
self.proto1.inject_fully_negotiated_outbound(protocol, info),
(EitherOutput::Second(protocol), EitherOutput::Second(info)) =>
self.proto2.inject_fully_negotiated_outbound(protocol, info),
(EitherOutput::First(_), EitherOutput::Second(_)) =>
panic!("wrong API usage: the protocol doesn't match the upgrade info"),
(EitherOutput::Second(_), EitherOutput::First(_)) =>
panic!("wrong API usage: the protocol doesn't match the upgrade info")
},
(EitherOutput::Second(_), NodeHandlerEndpoint::Dialer(EitherOutput::First(_))) => {
panic!("wrong API usage: the protocol doesn't match the upgrade info")
},
(EitherOutput::First(protocol), NodeHandlerEndpoint::Listener) => {
self.proto1.inject_fully_negotiated(protocol, NodeHandlerEndpoint::Listener);
},
(EitherOutput::Second(protocol), NodeHandlerEndpoint::Listener) => {
self.proto2.inject_fully_negotiated(protocol, NodeHandlerEndpoint::Listener);
},
}
}
fn inject_fully_negotiated_inbound(&mut self, protocol: <Self::InboundProtocol as InboundUpgrade<TSubstream>>::Output) {
match protocol {
EitherOutput::First(protocol) =>
self.proto1.inject_fully_negotiated_inbound(protocol),
EitherOutput::Second(protocol) =>
self.proto2.inject_fully_negotiated_inbound(protocol)
}
}
@ -774,19 +917,33 @@ where TProto1: ProtocolsHandler<Substream = TSubstream>,
self.proto2.shutdown();
}
fn poll(&mut self) -> Poll<Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>, io::Error> {
fn poll(&mut self) -> Poll<Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>, io::Error> {
match self.proto1.poll()? {
Async::Ready(Some(ProtocolsHandlerEvent::Custom(event))) => {
return Ok(Async::Ready(Some(ProtocolsHandlerEvent::Custom(EitherOutput::First(event)))));
},
Async::Ready(Some(ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info})) => {
let upgrade = {
let proto1 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(upgrade, EitherOutput::First));
let mut proto2 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(self.proto2.listen_protocol(), EitherOutput::Second));
proto2.disable();
upgrade::or(proto1, proto2)
};
let proto1 = upgrade
.map_outbound(EitherOutput::First as fn(TProto1Out) -> EitherOutput<TProto1Out, TProto2Out>)
.map_outbound_err(EitherError::A as fn(<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>);
let proto2 = self.proto2.dialer_protocol()
.map_outbound(EitherOutput::Second as fn(TProto2Out) -> EitherOutput<TProto1Out, TProto2Out>)
.map_outbound_err(EitherError::B as fn(<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>);
let proto1 = upgrade::toggleable(proto1);
let mut proto2 = upgrade::toggleable(proto2);
proto2.disable();
proto1.or_outbound(proto2)
};
return Ok(Async::Ready(Some(ProtocolsHandlerEvent::OutboundSubstreamRequest {
upgrade,
info: EitherOutput::First(info),
@ -802,12 +959,26 @@ where TProto1: ProtocolsHandler<Substream = TSubstream>,
},
Async::Ready(Some(ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info })) => {
let upgrade = {
let mut proto1 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(self.proto1.listen_protocol(), EitherOutput::First));
proto1.disable();
let proto2 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(upgrade, EitherOutput::Second));
upgrade::or(proto1, proto2)
};
let proto1 = self.proto1.dialer_protocol()
.map_outbound(EitherOutput::First as fn(TProto1Out) -> EitherOutput<TProto1Out, TProto2Out>)
.map_outbound_err(EitherError::A as fn(<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>);
let proto2 = upgrade
.map_outbound(EitherOutput::Second as fn(TProto2Out) -> EitherOutput<TProto1Out, TProto2Out>)
.map_outbound_err(EitherError::B as fn(<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>);
let mut proto1 = upgrade::toggleable(proto1);
proto1.disable();
let proto2 = upgrade::toggleable(proto2);
proto1.or_outbound(proto2)
};
return Ok(Async::Ready(Some(ProtocolsHandlerEvent::OutboundSubstreamRequest {
upgrade,
info: EitherOutput::Second(info),

View File

@ -18,18 +18,30 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use crate::muxing::StreamMuxer;
use crate::{
Endpoint, Multiaddr, PeerId,
nodes::{
collection::{
CollectionEvent,
CollectionNodeAccept,
CollectionReachEvent,
CollectionStream,
PeerMut as CollecPeerMut,
ReachAttemptId
},
handled_node::NodeHandler,
node::Substream
},
nodes::listeners::{ListenersEvent, ListenersStream},
transport::Transport
};
use fnv::FnvHashMap;
use futures::{prelude::*, future};
use muxing::StreamMuxer;
use nodes::collection::{
CollectionEvent, CollectionNodeAccept, CollectionReachEvent, CollectionStream, PeerMut as CollecPeerMut, ReachAttemptId,
use std::{
collections::hash_map::{Entry, OccupiedEntry},
io::{Error as IoError, ErrorKind as IoErrorKind}
};
use nodes::handled_node::NodeHandler;
use nodes::listeners::{ListenersEvent, ListenersStream};
use nodes::node::Substream;
use std::collections::hash_map::{Entry, OccupiedEntry};
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use {Endpoint, Multiaddr, PeerId, Transport};
/// Implementation of `Stream` that handles the nodes.
pub struct RawSwarm<TTrans, TInEvent, TOutEvent, THandler>
@ -198,8 +210,7 @@ where TTrans: Transport
impl<'a, TTrans, TInEvent, TOutEvent, TMuxer, THandler> IncomingConnectionEvent<'a, TTrans, TInEvent, TOutEvent, THandler>
where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Dial: Send + 'static,
TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::ListenerUpgrade: Send + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
@ -318,7 +329,7 @@ impl ConnectedPoint {
impl<TTrans, TInEvent, TOutEvent, TMuxer, THandler>
RawSwarm<TTrans, TInEvent, TOutEvent, THandler>
where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans: Transport + Clone,
TMuxer: StreamMuxer,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
@ -380,7 +391,7 @@ where
/// The second parameter is the handler to use if we manage to reach a node.
pub fn dial(&mut self, addr: Multiaddr, handler: THandler) -> Result<(), Multiaddr>
where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
@ -463,10 +474,9 @@ where
///
/// It is a logic error to call this method if we already have an outgoing attempt to the
/// given peer.
fn start_dial_out(&mut self, peer_id: PeerId, handler: THandler, first: Multiaddr,
rest: Vec<Multiaddr>)
fn start_dial_out(&mut self, peer_id: PeerId, handler: THandler, first: Multiaddr, rest: Vec<Multiaddr>)
where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
@ -510,7 +520,7 @@ where
/// Provides an API similar to `Stream`, except that it cannot error.
pub fn poll(&mut self) -> Async<RawSwarmEvent<TTrans, TInEvent, TOutEvent, THandler>>
where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::Dial: Send + 'static,
TTrans::ListenerUpgrade: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
@ -524,11 +534,7 @@ where
// Start by polling the listeners for events.
match self.listeners.poll() {
Async::NotReady => (),
Async::Ready(ListenersEvent::Incoming {
upgrade,
listen_addr,
send_back_addr,
}) => {
Async::Ready(ListenersEvent::Incoming { upgrade, listen_addr, send_back_addr }) => {
let event = IncomingConnectionEvent {
upgrade,
listen_addr,
@ -536,14 +542,9 @@ where
active_nodes: &mut self.active_nodes,
other_reach_attempts: &mut self.reach_attempts.other_reach_attempts,
};
return Async::Ready(RawSwarmEvent::IncomingConnection(event));
}
Async::Ready(ListenersEvent::Closed {
listen_addr,
listener,
result,
}) => {
Async::Ready(ListenersEvent::Closed { listen_addr, listener, result }) => {
return Async::Ready(RawSwarmEvent::ListenerClosed {
listen_addr,
listener,
@ -654,7 +655,6 @@ fn handle_node_reached<'a, TTrans, TMuxer, TInEvent, TOutEvent, THandler>(
) -> (ActionItem<THandler>, RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler>)
where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send,
@ -848,10 +848,15 @@ where
impl<'a, TTrans, TMuxer, TInEvent, TOutEvent, THandler>
Peer<'a, TTrans, TInEvent, TOutEvent, THandler>
where
TTrans: Transport<Output = (PeerId, TMuxer)>,
TMuxer: StreamMuxer,
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
THandler::OutboundOpenInfo: Send + 'static,
{
/// If we are connected, returns the `PeerConnected`.
#[inline]
@ -889,19 +894,8 @@ where
/// > that we are expecting, in which case the handler will be used for this "wrong"
/// > node.
#[inline]
pub fn or_connect(
self,
addr: Multiaddr,
handler: THandler,
) -> Result<PeerPotentialConnect<'a, TInEvent, TOutEvent, THandler>, Self>
where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
pub fn or_connect(self, addr: Multiaddr, handler: THandler)
-> Result<PeerPotentialConnect<'a, TInEvent, TOutEvent, THandler>, Self>
{
self.or_connect_with(move |_| addr, handler)
}
@ -916,20 +910,10 @@ where
/// > that we are expecting, in which case the handler will be used for this "wrong"
/// > node.
#[inline]
pub fn or_connect_with<TFn>(
self,
addr: TFn,
handler: THandler,
) -> Result<PeerPotentialConnect<'a, TInEvent, TOutEvent, THandler>, Self>
pub fn or_connect_with<TFn>(self, addr: TFn, handler: THandler)
-> Result<PeerPotentialConnect<'a, TInEvent, TOutEvent, THandler>, Self>
where
TFn: FnOnce(&PeerId) -> Multiaddr,
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
{
match self {
Peer::Connected(peer) => Ok(PeerPotentialConnect::Connected(peer)),
@ -1013,7 +997,7 @@ impl<'a, TInEvent> PeerConnected<'a, TInEvent> {
.expect("We insert into connected_points whenever a connection is opened and remove \
only when a connection is closed; the underlying API is guaranteed to always \
deliver a connection closed message after it has been opened, and no two \
closed messages; qed")
closed messages; qed")
}
/// Sends an event to the node.
@ -1082,26 +1066,22 @@ where
impl<'a, TTrans, TInEvent, TOutEvent, TMuxer, THandler>
PeerNotConnected<'a, TTrans, TInEvent, TOutEvent, THandler>
where
TTrans: Transport<Output = (PeerId, TMuxer)>,
TMuxer: StreamMuxer,
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
{
/// Attempts a new connection to this node using the given multiaddress.
///
/// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then
/// the whole connection is immediately closed.
#[inline]
pub fn connect(self, addr: Multiaddr, handler: THandler) -> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self>
where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
{
pub fn connect(self, addr: Multiaddr, handler: THandler) -> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self> {
self.connect_inner(handler, addr, Vec::new())
}
@ -1114,20 +1094,10 @@ where
/// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then
/// the whole connection is immediately closed.
#[inline]
pub fn connect_iter<TIter>(
self,
addrs: TIter,
handler: THandler,
) -> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self>
pub fn connect_iter<TIter>(self, addrs: TIter, handler: THandler)
-> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self>
where
TIter: IntoIterator<Item = Multiaddr>,
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
{
let mut addrs = addrs.into_iter();
let first = match addrs.next() {
@ -1139,23 +1109,10 @@ where
}
/// Inner implementation of `connect`.
fn connect_inner(
self,
handler: THandler,
first: Multiaddr,
rest: Vec<Multiaddr>,
) -> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self>
where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
fn connect_inner(self, handler: THandler, first: Multiaddr, rest: Vec<Multiaddr>)
-> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self>
{
self.nodes.start_dial_out(self.peer_id.clone(), handler, first, rest);
Ok(PeerPendingConnect {
attempt: match self.nodes.reach_attempts.out_reach_attempts.entry(self.peer_id) {
Entry::Occupied(e) => e,

View File

@ -18,15 +18,19 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use crate::{
Transport, Multiaddr, PeerId, InboundUpgrade, OutboundUpgrade, UpgradeInfo,
muxing::StreamMuxer,
nodes::{
handled_node::NodeHandler,
node::Substream,
protocols_handler::{NodeHandlerWrapper, ProtocolsHandler},
raw_swarm::{RawSwarm, RawSwarmEvent, ConnectedPoint}
},
topology::Topology
};
use futures::prelude::*;
use muxing::StreamMuxer;
use nodes::handled_node::NodeHandler;
use nodes::node::Substream;
use nodes::protocols_handler::{NodeHandlerWrapper, ProtocolsHandler};
use nodes::raw_swarm::{RawSwarm, RawSwarmEvent, ConnectedPoint};
use std::{io, ops::{Deref, DerefMut}};
use topology::Topology;
use {ConnectionUpgrade, Multiaddr, PeerId, Transport};
use std::{fmt, io, ops::{Deref, DerefMut}};
/// Contains the state of the network, plus the way it should behave.
pub struct Swarm<TTransport, TBehaviour, TTopology>
@ -77,17 +81,23 @@ where TBehaviour: NetworkBehavior,
<TMuxer as StreamMuxer>::OutboundSubstream: Send + 'static,
<TMuxer as StreamMuxer>::Substream: Send + 'static,
TTransport: Transport<Output = (PeerId, TMuxer)> + Clone,
TTransport::Dial: Send + 'static,
TTransport::Listener: Send + 'static,
TTransport::ListenerUpgrade: Send + 'static,
TTransport::Dial: Send + 'static,
TBehaviour::ProtocolsHandler: ProtocolsHandler<Substream = Substream<TMuxer>> + Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InEvent: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutEvent: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol: ConnectionUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol as ConnectionUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol as ConnectionUpgrade<Substream<TMuxer>>>::NamesIter: Clone + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol as ConnectionUpgrade<Substream<TMuxer>>>::UpgradeIdentifier: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol: InboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::NamesIter: Clone + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::UpgradeId: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::NamesIter: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::UpgradeId: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static,
<NodeHandlerWrapper<TBehaviour::ProtocolsHandler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
TTopology: Topology,
{
@ -164,17 +174,23 @@ where TBehaviour: NetworkBehavior,
<TMuxer as StreamMuxer>::OutboundSubstream: Send + 'static,
<TMuxer as StreamMuxer>::Substream: Send + 'static,
TTransport: Transport<Output = (PeerId, TMuxer)> + Clone,
TTransport::Dial: Send + 'static,
TTransport::Listener: Send + 'static,
TTransport::ListenerUpgrade: Send + 'static,
TTransport::Dial: Send + 'static,
TBehaviour::ProtocolsHandler: ProtocolsHandler<Substream = Substream<TMuxer>> + Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InEvent: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutEvent: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol: ConnectionUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol as ConnectionUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol as ConnectionUpgrade<Substream<TMuxer>>>::NamesIter: Clone + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol as ConnectionUpgrade<Substream<TMuxer>>>::UpgradeIdentifier: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol: InboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::NamesIter: Clone + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::UpgradeId: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::NamesIter: Clone + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::UpgradeId: Send + 'static,
<NodeHandlerWrapper<TBehaviour::ProtocolsHandler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
TTopology: Topology,
{