Add IntoNodeHandler and IntoProtocolsHandler traits (#848)

* Add IntoNodeHandler

* Add IntoProtocolsHandler
This commit is contained in:
Pierre Krieger
2019-01-14 14:22:25 +01:00
committed by GitHub
parent 60db872c31
commit bf52e9bd19
8 changed files with 267 additions and 112 deletions

View File

@ -24,7 +24,7 @@ use crate::{
nodes::{
node::Substream,
handled_node_tasks::{HandledNodesEvent, HandledNodesTasks, TaskClosedEvent},
handled_node_tasks::{Task as HandledNodesTask, TaskId},
handled_node_tasks::{IntoNodeHandler, Task as HandledNodesTask, TaskId},
handled_node::{HandledNodeError, NodeHandler}
}
};
@ -285,12 +285,13 @@ impl<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr> CollectionStream<TIn
-> ReachAttemptId
where
TFut: Future<Item = (PeerId, TMuxer), Error = TReachErr> + Send + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
THandler: IntoNodeHandler + Send + 'static,
THandler::Handler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
<THandler::Handler as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be required?
TReachErr: error::Error + Send + 'static,
THandlerErr: error::Error + Send + 'static,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be required?
TMuxer: StreamMuxer + Send + Sync + 'static, // TODO: Send + Sync + 'static shouldn't be required
TMuxer::OutboundSubstream: Send + 'static, // TODO: shouldn't be required
{
@ -459,7 +460,7 @@ impl<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr> CollectionStream<TIn
}
}
/// Reach attempt interrupt errors.
/// Reach attempt interrupt errors.
#[derive(Debug)]
pub enum InterruptError {
/// An invalid reach attempt has been used to try to interrupt. The task
@ -475,7 +476,7 @@ pub enum InterruptError {
impl fmt::Display for InterruptError {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match *self {
InterruptError::ReachAttemptNotFound =>
InterruptError::ReachAttemptNotFound =>
write!(f, "The reach attempt could not be found."),
InterruptError::AlreadyReached =>
write!(f, "The reach attempt has already completed or reached the node."),

View File

@ -59,7 +59,7 @@ mod tests;
// conditions in the user's code. See similar comments in the documentation of `NodeStream`.
/// Implementation of `Stream` that handles a collection of nodes.
pub struct HandledNodesTasks<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr> {
pub struct HandledNodesTasks<TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr> {
/// A map between active tasks to an unbounded sender, used to control the task. Closing the sender interrupts
/// the task. It is possible that we receive messages from tasks that used to be in this list
/// but no longer are, in which case we should ignore them.
@ -73,12 +73,12 @@ pub struct HandledNodesTasks<TInEvent, TOutEvent, THandler, TReachErr, THandlerE
to_spawn: SmallVec<[Box<Future<Item = (), Error = ()> + Send>; 8]>,
/// Sender to emit events to the outside. Meant to be cloned and sent to tasks.
events_tx: mpsc::UnboundedSender<(InToExtMessage<TOutEvent, THandler, TReachErr, THandlerErr>, TaskId)>,
events_tx: mpsc::UnboundedSender<(InToExtMessage<TOutEvent, TIntoHandler, TReachErr, THandlerErr>, TaskId)>,
/// Receiver side for the events.
events_rx: mpsc::UnboundedReceiver<(InToExtMessage<TOutEvent, THandler, TReachErr, THandlerErr>, TaskId)>,
events_rx: mpsc::UnboundedReceiver<(InToExtMessage<TOutEvent, TIntoHandler, TReachErr, THandlerErr>, TaskId)>,
}
impl<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr> fmt::Debug for HandledNodesTasks<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr> {
impl<TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr> fmt::Debug for HandledNodesTasks<TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr> {
fn fmt(&self, f: &mut fmt::Formatter) -> Result<(), fmt::Error> {
f.debug_list()
.entries(self.tasks.keys().cloned())
@ -121,9 +121,31 @@ where
}
}
/// Prototype for a `NodeHandler`.
pub trait IntoNodeHandler {
/// The node handler.
type Handler: NodeHandler;
/// Builds the node handler.
///
/// The `PeerId` is the id of the node the handler is going to handle.
fn into_handler(self, remote_peer_id: &PeerId) -> Self::Handler;
}
impl<T> IntoNodeHandler for T
where T: NodeHandler
{
type Handler = Self;
#[inline]
fn into_handler(self, _: &PeerId) -> Self {
self
}
}
/// Event that can happen on the `HandledNodesTasks`.
#[derive(Debug)]
pub enum HandledNodesEvent<TOutEvent, THandler, TReachErr, THandlerErr> {
pub enum HandledNodesEvent<TOutEvent, TIntoHandler, TReachErr, THandlerErr> {
/// A task has been closed.
///
/// This happens once the node handler closes or an error happens.
@ -135,7 +157,7 @@ pub enum HandledNodesEvent<TOutEvent, THandler, TReachErr, THandlerErr> {
result: Result<(), TaskClosedEvent<TReachErr, THandlerErr>>,
/// If the task closed before reaching the node, this contains the handler that was passed
/// to `add_reach_attempt`.
handler: Option<THandler>,
handler: Option<TIntoHandler>,
},
/// A task has successfully connected to a node.
@ -159,7 +181,7 @@ pub enum HandledNodesEvent<TOutEvent, THandler, TReachErr, THandlerErr> {
#[derive(Debug, Copy, Clone, Hash, PartialEq, Eq, PartialOrd, Ord)]
pub struct TaskId(usize);
impl<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr> HandledNodesTasks<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr> {
impl<TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr> HandledNodesTasks<TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr> {
/// Creates a new empty collection.
#[inline]
pub fn new() -> Self {
@ -178,15 +200,16 @@ impl<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr> HandledNodesTasks<TI
///
/// 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: TIntoHandler) -> TaskId
where
TFut: Future<Item = (PeerId, TMuxer), Error = TReachErr> + Send + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
TIntoHandler: IntoNodeHandler + Send + 'static,
TIntoHandler::Handler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
TReachErr: error::Error + Send + 'static,
THandlerErr: error::Error + Send + 'static,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be required?
<TIntoHandler::Handler as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be required?
TMuxer: StreamMuxer + Send + Sync + 'static, // TODO: Send + Sync + 'static shouldn't be required
TMuxer::OutboundSubstream: Send + 'static, // TODO: shouldn't be required
{
@ -241,7 +264,7 @@ impl<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr> HandledNodesTasks<TI
}
/// Provides an API similar to `Stream`, except that it cannot produce an error.
pub fn poll(&mut self) -> Async<HandledNodesEvent<TOutEvent, THandler, TReachErr, THandlerErr>> {
pub fn poll(&mut self) -> Async<HandledNodesEvent<TOutEvent, TIntoHandler, TReachErr, THandlerErr>> {
for to_spawn in self.to_spawn.drain() {
tokio_executor::spawn(to_spawn);
}
@ -327,8 +350,8 @@ impl<'a, TInEvent> fmt::Debug for Task<'a, TInEvent> {
}
}
impl<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr> Stream for HandledNodesTasks<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr> {
type Item = HandledNodesEvent<TOutEvent, THandler, TReachErr, THandlerErr>;
impl<TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr> Stream for HandledNodesTasks<TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr> {
type Item = HandledNodesEvent<TOutEvent, TIntoHandler, TReachErr, THandlerErr>;
type Error = Void; // TODO: use ! once stable
#[inline]
@ -339,43 +362,45 @@ impl<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr> Stream for HandledNo
/// Message to transmit from a task to the public API.
#[derive(Debug)]
enum InToExtMessage<TOutEvent, THandler, TReachErr, THandlerErr> {
enum InToExtMessage<TOutEvent, TIntoHandler, TReachErr, THandlerErr> {
/// A connection to a node has succeeded.
NodeReached(PeerId),
/// The task closed.
TaskClosed(Result<(), TaskClosedEvent<TReachErr, THandlerErr>>, Option<THandler>),
TaskClosed(Result<(), TaskClosedEvent<TReachErr, THandlerErr>>, Option<TIntoHandler>),
/// An event from the node.
NodeEvent(TOutEvent),
}
/// Implementation of `Future` that handles a single node, and all the communications between
/// the various components of the `HandledNodesTasks`.
struct NodeTask<TFut, TMuxer, THandler, TInEvent, TOutEvent, TReachErr>
struct NodeTask<TFut, TMuxer, TIntoHandler, TInEvent, TOutEvent, TReachErr>
where
TMuxer: StreamMuxer,
THandler: NodeHandler<Substream = Substream<TMuxer>>,
TIntoHandler: IntoNodeHandler,
TIntoHandler::Handler: NodeHandler<Substream = Substream<TMuxer>>,
{
/// Sender to transmit events to the outside.
events_tx: mpsc::UnboundedSender<(InToExtMessage<TOutEvent, THandler, TReachErr, THandler::Error>, TaskId)>,
events_tx: mpsc::UnboundedSender<(InToExtMessage<TOutEvent, TIntoHandler, TReachErr, <TIntoHandler::Handler as NodeHandler>::Error>, TaskId)>,
/// Receiving end for events sent from the main `HandledNodesTasks`.
in_events_rx: stream::Fuse<mpsc::UnboundedReceiver<TInEvent>>,
/// Inner state of the `NodeTask`.
inner: NodeTaskInner<TFut, TMuxer, THandler, TInEvent>,
inner: NodeTaskInner<TFut, TMuxer, TIntoHandler, TInEvent>,
/// Identifier of the attempt.
id: TaskId,
}
enum NodeTaskInner<TFut, TMuxer, THandler, TInEvent>
enum NodeTaskInner<TFut, TMuxer, TIntoHandler, TInEvent>
where
TMuxer: StreamMuxer,
THandler: NodeHandler<Substream = Substream<TMuxer>>,
TIntoHandler: IntoNodeHandler,
TIntoHandler::Handler: NodeHandler<Substream = Substream<TMuxer>>,
{
/// Future to resolve to connect to the node.
Future {
/// The future that will attempt to reach the node.
future: TFut,
/// The handler that will be used to build the `HandledNode`.
handler: THandler,
handler: TIntoHandler,
/// While we are dialing the future, we need to buffer the events received on
/// `in_events_rx` so that they get delivered once dialing succeeds. We can't simply leave
/// events in `in_events_rx` because we have to detect if it gets closed.
@ -383,18 +408,19 @@ where
},
/// Fully functional node.
Node(HandledNode<TMuxer, THandler>),
Node(HandledNode<TMuxer, TIntoHandler::Handler>),
/// A panic happened while polling.
Poisoned,
}
impl<TFut, TMuxer, THandler, TInEvent, TOutEvent, TReachErr> Future for
NodeTask<TFut, TMuxer, THandler, TInEvent, TOutEvent, TReachErr>
impl<TFut, TMuxer, TIntoHandler, TInEvent, TOutEvent, TReachErr> Future for
NodeTask<TFut, TMuxer, TIntoHandler, TInEvent, TOutEvent, TReachErr>
where
TMuxer: StreamMuxer,
TFut: Future<Item = (PeerId, TMuxer), Error = TReachErr>,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent>,
TIntoHandler: IntoNodeHandler,
TIntoHandler::Handler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent>,
{
type Item = ();
type Error = ();
@ -416,8 +442,8 @@ where
// Check whether dialing succeeded.
match future.poll() {
Ok(Async::Ready((peer_id, muxer))) => {
let mut node = HandledNode::new(muxer, handler.into_handler(&peer_id));
let event = InToExtMessage::NodeReached(peer_id);
let mut node = HandledNode::new(muxer, handler);
for event in events_buffer {
node.inject_event(event);
}

View File

@ -34,6 +34,7 @@ use crate::{
HandledNodeError,
NodeHandler
},
handled_node_tasks::IntoNodeHandler,
node::Substream
},
nodes::listeners::{ListenersEvent, ListenersStream},
@ -377,8 +378,9 @@ where
TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::Error: Send + 'static,
TTrans::ListenerUpgrade: Send + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
THandler: IntoNodeHandler + Send + 'static,
THandler::Handler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
<THandler::Handler as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
THandlerErr: error::Error + Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
@ -523,8 +525,9 @@ impl<TTrans, TInEvent, TOutEvent, TMuxer, THandler, THandlerErr>
where
TTrans: Transport + Clone,
TMuxer: StreamMuxer,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
THandler: IntoNodeHandler + Send + 'static,
THandler::Handler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
<THandler::Handler as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
THandlerErr: error::Error + Send + 'static,
{
/// Creates a new node events stream.
@ -752,8 +755,9 @@ where
TMuxer::Substream: Send,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
THandler: IntoNodeHandler + Send + 'static,
THandler::Handler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
<THandler::Handler as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
THandlerErr: error::Error + Send + 'static,
{
// Start by polling the listeners for events.
@ -1138,8 +1142,9 @@ where
TMuxer::Substream: Send,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static,
THandler: IntoNodeHandler + Send + 'static,
THandler::Handler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
<THandler::Handler as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
THandlerErr: error::Error + Send + 'static,
{
/// If we are connected, returns the `PeerConnected`.
@ -1362,8 +1367,9 @@ where
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
THandler: IntoNodeHandler + Send + 'static,
THandler::Handler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
<THandler::Handler as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
THandlerErr: error::Error + Send + 'static,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,

View File

@ -33,6 +33,7 @@
//! > connection with a remote. In order to handle a protocol that requires knowledge of
//! > the network as a whole, see the `NetworkBehaviour` trait.
use crate::PeerId;
use crate::upgrade::{
InboundUpgrade,
OutboundUpgrade,
@ -46,7 +47,7 @@ pub use self::dummy::DummyProtocolsHandler;
pub use self::map_in::MapInEvent;
pub use self::map_out::MapOutEvent;
pub use self::node_handler::{NodeHandlerWrapper, NodeHandlerWrapperBuilder};
pub use self::select::ProtocolsHandlerSelect;
pub use self::select::{IntoProtocolsHandlerSelect, ProtocolsHandlerSelect};
mod dummy;
mod map_in;
@ -207,17 +208,19 @@ pub trait ProtocolsHandler {
where
Self: Sized,
{
NodeHandlerWrapperBuilder::new(self, Duration::from_secs(10), Duration::from_secs(10), Duration::from_secs(5))
IntoProtocolsHandler::into_node_handler_builder(self)
}
/// Builds an implementation of `NodeHandler` that handles this protocol exclusively.
///
/// > **Note**: This is a shortcut for `self.into_node_handler_builder().build()`.
#[inline]
#[deprecated(note = "Use into_node_handler_builder instead")]
fn into_node_handler(self) -> NodeHandlerWrapper<Self>
where
Self: Sized,
{
#![allow(deprecated)]
self.into_node_handler_builder().build()
}
}
@ -353,3 +356,45 @@ where
}
}
}
/// Prototype for a `ProtocolsHandler`.
pub trait IntoProtocolsHandler {
/// The protocols handler.
type Handler: ProtocolsHandler;
/// Builds the protocols handler.
///
/// The `PeerId` is the id of the node the handler is going to handle.
fn into_handler(self, remote_peer_id: &PeerId) -> Self::Handler;
/// Builds an implementation of `IntoProtocolsHandler` that handles both this protocol and the
/// other one together.
#[inline]
fn select<TProto2>(self, other: TProto2) -> IntoProtocolsHandlerSelect<Self, TProto2>
where
Self: Sized,
{
IntoProtocolsHandlerSelect::new(self, other)
}
/// Creates a builder that will allow creating a `NodeHandler` that handles this protocol
/// exclusively.
#[inline]
fn into_node_handler_builder(self) -> NodeHandlerWrapperBuilder<Self>
where
Self: Sized,
{
NodeHandlerWrapperBuilder::new(self, Duration::from_secs(10), Duration::from_secs(10), Duration::from_secs(5))
}
}
impl<T> IntoProtocolsHandler for T
where T: ProtocolsHandler
{
type Handler = Self;
#[inline]
fn into_handler(self, _: &PeerId) -> Self {
self
}
}

View File

@ -19,8 +19,10 @@
// DEALINGS IN THE SOFTWARE.
use crate::{
PeerId,
nodes::handled_node::{NodeHandler, NodeHandlerEndpoint, NodeHandlerEvent},
protocols_handler::{ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr},
nodes::handled_node_tasks::IntoNodeHandler,
protocols_handler::{ProtocolsHandler, IntoProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr},
upgrade::{
self,
OutboundUpgrade,
@ -33,12 +35,9 @@ use std::time::{Duration, Instant};
use tokio_timer::{Delay, Timeout};
/// Prototype for a `NodeHandlerWrapper`.
pub struct NodeHandlerWrapperBuilder<TProtoHandler>
where
TProtoHandler: ProtocolsHandler,
{
pub struct NodeHandlerWrapperBuilder<TIntoProtoHandler> {
/// The underlying handler.
handler: TProtoHandler,
handler: TIntoProtoHandler,
/// Timeout for incoming substreams negotiation.
in_timeout: Duration,
/// Timeout for outgoing substreams negotiation.
@ -47,13 +46,13 @@ where
useless_timeout: Duration,
}
impl<TProtoHandler> NodeHandlerWrapperBuilder<TProtoHandler>
impl<TIntoProtoHandler> NodeHandlerWrapperBuilder<TIntoProtoHandler>
where
TProtoHandler: ProtocolsHandler
TIntoProtoHandler: IntoProtocolsHandler
{
/// Builds a `NodeHandlerWrapperBuilder`.
#[inline]
pub(crate) fn new(handler: TProtoHandler, in_timeout: Duration, out_timeout: Duration, useless_timeout: Duration) -> Self {
pub(crate) fn new(handler: TIntoProtoHandler, in_timeout: Duration, out_timeout: Duration, useless_timeout: Duration) -> Self {
NodeHandlerWrapperBuilder {
handler,
in_timeout,
@ -85,8 +84,11 @@ where
}
/// Builds the `NodeHandlerWrapper`.
#[deprecated(note = "Pass the NodeHandlerWrapperBuilder directly")]
#[inline]
pub fn build(self) -> NodeHandlerWrapper<TProtoHandler> {
pub fn build(self) -> NodeHandlerWrapper<TIntoProtoHandler>
where TIntoProtoHandler: ProtocolsHandler
{
NodeHandlerWrapper {
handler: self.handler,
negotiating_in: Vec::new(),
@ -101,6 +103,30 @@ where
}
}
impl<TIntoProtoHandler, TProtoHandler> IntoNodeHandler for NodeHandlerWrapperBuilder<TIntoProtoHandler>
where
TIntoProtoHandler: IntoProtocolsHandler<Handler = TProtoHandler>,
TProtoHandler: ProtocolsHandler,
// TODO: meh for Debug
<TProtoHandler::OutboundProtocol as OutboundUpgrade<<TProtoHandler as ProtocolsHandler>::Substream>>::Error: std::fmt::Debug
{
type Handler = NodeHandlerWrapper<TIntoProtoHandler::Handler>;
fn into_handler(self, remote_peer_id: &PeerId) -> Self::Handler {
NodeHandlerWrapper {
handler: self.handler.into_handler(remote_peer_id),
negotiating_in: Vec::new(),
negotiating_out: Vec::new(),
in_timeout: self.in_timeout,
out_timeout: self.out_timeout,
queued_dial_upgrades: Vec::new(),
unique_dial_upgrade_id: 0,
connection_shutdown: None,
useless_timeout: self.useless_timeout,
}
}
}
/// Wraps around an implementation of `ProtocolsHandler`, and implements `NodeHandler`.
// TODO: add a caching system for protocols that are supported or not
pub struct NodeHandlerWrapper<TProtoHandler>
@ -138,6 +164,7 @@ where
impl<TProtoHandler> NodeHandler for NodeHandlerWrapper<TProtoHandler>
where
TProtoHandler: ProtocolsHandler,
// TODO: meh for Debug
<TProtoHandler::OutboundProtocol as OutboundUpgrade<<TProtoHandler as ProtocolsHandler>::Substream>>::Error: std::fmt::Debug
{
type InEvent = TProtoHandler::InEvent;

View File

@ -19,9 +19,15 @@
// DEALINGS IN THE SOFTWARE.
use crate::{
PeerId,
either::EitherError,
either::EitherOutput,
protocols_handler::{ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr},
protocols_handler::{
IntoProtocolsHandler,
ProtocolsHandler,
ProtocolsHandlerEvent,
ProtocolsHandlerUpgrErr,
},
upgrade::{
InboundUpgrade,
OutboundUpgrade,
@ -33,6 +39,46 @@ use crate::{
use futures::prelude::*;
use tokio_io::{AsyncRead, AsyncWrite};
/// Implementation of `IntoProtocolsHandler` that combines two protocols into one.
#[derive(Debug, Clone)]
pub struct IntoProtocolsHandlerSelect<TProto1, TProto2> {
proto1: TProto1,
proto2: TProto2,
}
impl<TProto1, TProto2> IntoProtocolsHandlerSelect<TProto1, TProto2> {
/// Builds a `IntoProtocolsHandlerSelect`.
#[inline]
pub(crate) fn new(proto1: TProto1, proto2: TProto2) -> Self {
IntoProtocolsHandlerSelect {
proto1,
proto2,
}
}
}
impl<TProto1, TProto2, TSubstream> IntoProtocolsHandler for IntoProtocolsHandlerSelect<TProto1, TProto2>
where
TProto1: IntoProtocolsHandler,
TProto2: IntoProtocolsHandler,
TProto1::Handler: ProtocolsHandler<Substream = TSubstream>,
TProto2::Handler: ProtocolsHandler<Substream = TSubstream>,
TSubstream: AsyncRead + AsyncWrite,
<TProto1::Handler as ProtocolsHandler>::InboundProtocol: InboundUpgrade<TSubstream>,
<TProto2::Handler as ProtocolsHandler>::InboundProtocol: InboundUpgrade<TSubstream>,
<TProto1::Handler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade<TSubstream>,
<TProto2::Handler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade<TSubstream>
{
type Handler = ProtocolsHandlerSelect<TProto1::Handler, TProto2::Handler>;
fn into_handler(self, remote_peer_id: &PeerId) -> Self::Handler {
ProtocolsHandlerSelect {
proto1: self.proto1.into_handler(remote_peer_id),
proto2: self.proto2.into_handler(remote_peer_id),
}
}
}
/// Implementation of `ProtocolsHandler` that combines two protocols into one.
#[derive(Debug, Clone)]
pub struct ProtocolsHandlerSelect<TProto1, TProto2> {

View File

@ -26,7 +26,7 @@
//! # Initializing a Swarm
//!
//! Creating a `Swarm` requires three things:
//!
//!
//! - An implementation of the `Transport` trait. This is the type that will be used in order to
//! reach nodes on the network based on their address. See the `transport` module for more
//! information.
@ -49,7 +49,7 @@ use crate::{
node::Substream,
raw_swarm::{RawSwarm, RawSwarmEvent}
},
protocols_handler::{NodeHandlerWrapper, ProtocolsHandler},
protocols_handler::{NodeHandlerWrapperBuilder, NodeHandlerWrapper, IntoProtocolsHandler, ProtocolsHandler},
topology::Topology,
transport::TransportError,
topology::DisconnectReason,
@ -67,10 +67,10 @@ where TTransport: Transport,
{
raw_swarm: RawSwarm<
TTransport,
<<TBehaviour as NetworkBehaviour<TTopology>>::ProtocolsHandler as ProtocolsHandler>::InEvent,
<<TBehaviour as NetworkBehaviour<TTopology>>::ProtocolsHandler as ProtocolsHandler>::OutEvent,
NodeHandlerWrapper<TBehaviour::ProtocolsHandler>,
<<TBehaviour as NetworkBehaviour<TTopology>>::ProtocolsHandler as ProtocolsHandler>::Error,
<<<TBehaviour as NetworkBehaviour<TTopology>>::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InEvent,
<<<TBehaviour as NetworkBehaviour<TTopology>>::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutEvent,
NodeHandlerWrapperBuilder<TBehaviour::ProtocolsHandler>,
<<<TBehaviour as NetworkBehaviour<TTopology>>::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::Error,
>,
/// Handles which nodes to connect to and how to handle the events sent back by the protocol
@ -120,24 +120,25 @@ where TBehaviour: NetworkBehaviour<TTopology>,
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>::Error: 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>::Info: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter: Send + 'static,
<<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: 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>::Info: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter: Send + 'static,
<<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: 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
TBehaviour::ProtocolsHandler: Send + 'static,
<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler: ProtocolsHandler<Substream = Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InEvent: Send + 'static,
<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutEvent: Send + 'static,
<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::Error: Send + 'static,
<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol: InboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::Info: Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter: Send + 'static,
<<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::Info: Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter: Send + 'static,
<<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static,
<NodeHandlerWrapper<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
TTopology: Topology,
{
/// Builds a new `Swarm`.
@ -145,6 +146,7 @@ where TBehaviour: NetworkBehaviour<TTopology>,
pub fn new(transport: TTransport, mut behaviour: TBehaviour, topology: TTopology) -> Self {
let supported_protocols = behaviour
.new_handler()
.into_handler(topology.local_peer_id())
.listen_protocol()
.protocol_info()
.into_iter()
@ -187,7 +189,7 @@ where TBehaviour: NetworkBehaviour<TTopology>,
#[inline]
pub fn dial_addr(me: &mut Self, addr: Multiaddr) -> Result<(), TransportError<TTransport::Error>> {
let handler = me.behaviour.new_handler();
me.raw_swarm.dial(addr, handler.into_node_handler())
me.raw_swarm.dial(addr, handler.into_node_handler_builder())
}
/// Tries to reach the given peer using the elements in the topology.
@ -197,7 +199,7 @@ where TBehaviour: NetworkBehaviour<TTopology>,
#[inline]
pub fn dial(me: &mut Self, peer_id: PeerId) {
let addrs = me.topology.addresses_of_peer(&peer_id);
let handler = me.behaviour.new_handler().into_node_handler();
let handler = me.behaviour.new_handler().into_node_handler_builder();
if let Some(peer) = me.raw_swarm.peer(peer_id).as_not_connected() {
let _ = peer.connect_iter(addrs, handler);
}
@ -238,24 +240,25 @@ where TBehaviour: NetworkBehaviour<TTopology>,
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>::Error: 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>::Info: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter: Send + 'static,
<<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: 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>::Info: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter: Send + 'static,
<<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static,
<NodeHandlerWrapper<TBehaviour::ProtocolsHandler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
TBehaviour::ProtocolsHandler: Send + 'static,
<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler: ProtocolsHandler<Substream = Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InEvent: Send + 'static,
<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutEvent: Send + 'static,
<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::Error: Send + 'static,
<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol: InboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::Info: Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter: Send + 'static,
<<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static,
<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::Info: Send + 'static,
<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter: Send + 'static,
<<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static,
<NodeHandlerWrapper<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
TTopology: Topology,
{
type Item = TBehaviour::OutEvent;
@ -291,7 +294,7 @@ where TBehaviour: NetworkBehaviour<TTopology>,
},
Async::Ready(RawSwarmEvent::IncomingConnection(incoming)) => {
let handler = self.behaviour.new_handler();
incoming.accept(handler.into_node_handler());
incoming.accept(handler.into_node_handler_builder());
},
Async::Ready(RawSwarmEvent::ListenerClosed { .. }) => {},
Async::Ready(RawSwarmEvent::IncomingConnectionError { .. }) => {},
@ -345,7 +348,7 @@ where TBehaviour: NetworkBehaviour<TTopology>,
/// one that handles all the behaviours at once.
pub trait NetworkBehaviour<TTopology> {
/// Handler for all the protocols the network supports.
type ProtocolsHandler: ProtocolsHandler;
type ProtocolsHandler: IntoProtocolsHandler;
/// Event generated by the swarm.
type OutEvent;
@ -367,13 +370,13 @@ pub trait NetworkBehaviour<TTopology> {
fn inject_node_event(
&mut self,
peer_id: PeerId,
event: <Self::ProtocolsHandler as ProtocolsHandler>::OutEvent
event: <<Self::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutEvent
);
/// Polls for things that swarm should do.
///
/// This API mimics the API of the `Stream` trait.
fn poll(&mut self, topology: &mut PollParameters<TTopology>) -> Async<NetworkBehaviourAction<<Self::ProtocolsHandler as ProtocolsHandler>::InEvent, Self::OutEvent>>;
fn poll(&mut self, topology: &mut PollParameters<TTopology>) -> Async<NetworkBehaviourAction<<<Self::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InEvent, Self::OutEvent>>;
}
/// Used when deriving `NetworkBehaviour`. When deriving `NetworkBehaviour`, must be implemented

View File

@ -53,8 +53,9 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream {
let net_behv_event_proc = quote!{::libp2p::core::swarm::NetworkBehaviourEventProcess};
let either_ident = quote!{::libp2p::core::either::EitherOutput};
let network_behaviour_action = quote!{::libp2p::core::swarm::NetworkBehaviourAction};
let into_protocols_handler = quote!{::libp2p::core::protocols_handler::IntoProtocolsHandler};
let protocols_handler = quote!{::libp2p::core::protocols_handler::ProtocolsHandler};
let proto_select_ident = quote!{::libp2p::core::protocols_handler::ProtocolsHandlerSelect};
let into_proto_select_ident = quote!{::libp2p::core::protocols_handler::IntoProtocolsHandlerSelect};
let peer_id = quote!{::libp2p::core::PeerId};
let connected_point = quote!{::libp2p::core::swarm::ConnectedPoint};
@ -99,10 +100,10 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream {
vec![
quote!{#ty: #trait_to_impl<#topology_generic>},
quote!{Self: #net_behv_event_proc<<#ty as #trait_to_impl<#topology_generic>>::OutEvent>},
quote!{<#ty as #trait_to_impl<#topology_generic>>::ProtocolsHandler: #protocols_handler<Substream = #substream_generic>},
quote!{<<#ty as #trait_to_impl<#topology_generic>>::ProtocolsHandler as #into_protocols_handler>::Handler: #protocols_handler<Substream = #substream_generic>},
// Note: this bound is required because of https://github.com/rust-lang/rust/issues/55697
quote!{<<#ty as #trait_to_impl<#topology_generic>>::ProtocolsHandler as #protocols_handler>::InboundProtocol: ::libp2p::core::InboundUpgrade<#substream_generic>},
quote!{<<#ty as #trait_to_impl<#topology_generic>>::ProtocolsHandler as #protocols_handler>::OutboundProtocol: ::libp2p::core::OutboundUpgrade<#substream_generic>},
quote!{<<<#ty as #trait_to_impl<#topology_generic>>::ProtocolsHandler as #into_protocols_handler>::Handler as #protocols_handler>::InboundProtocol: ::libp2p::core::InboundUpgrade<#substream_generic>},
quote!{<<<#ty as #trait_to_impl<#topology_generic>>::ProtocolsHandler as #into_protocols_handler>::Handler as #protocols_handler>::OutboundProtocol: ::libp2p::core::OutboundUpgrade<#substream_generic>},
]
})
.collect::<Vec<_>>();
@ -213,7 +214,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream {
let ty = &field.ty;
let field_info = quote!{ <#ty as #trait_to_impl<#topology_generic>>::ProtocolsHandler };
match ph_ty {
Some(ev) => ph_ty = Some(quote!{ #proto_select_ident<#ev, #field_info> }),
Some(ev) => ph_ty = Some(quote!{ #into_proto_select_ident<#ev, #field_info> }),
ref mut ev @ None => *ev = Some(field_info),
}
}
@ -324,7 +325,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream {
#[inline]
fn new_handler(&mut self) -> Self::ProtocolsHandler {
use #protocols_handler;
use #into_protocols_handler;
#new_handler
}
@ -342,17 +343,17 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream {
fn inject_node_event(
&mut self,
peer_id: #peer_id,
event: <Self::ProtocolsHandler as #protocols_handler>::OutEvent
event: <<Self::ProtocolsHandler as #into_protocols_handler>::Handler as #protocols_handler>::OutEvent
) {
match event {
#(#inject_node_event_stmts),*
}
}
fn poll(&mut self, poll_params: &mut #poll_parameters) -> ::libp2p::futures::Async<#network_behaviour_action<<Self::ProtocolsHandler as #protocols_handler>::InEvent, Self::OutEvent>> {
fn poll(&mut self, poll_params: &mut #poll_parameters) -> ::libp2p::futures::Async<#network_behaviour_action<<<Self::ProtocolsHandler as #into_protocols_handler>::Handler as #protocols_handler>::InEvent, Self::OutEvent>> {
use libp2p::futures::prelude::*;
#(#poll_stmts)*
let f: ::libp2p::futures::Async<#network_behaviour_action<<Self::ProtocolsHandler as #protocols_handler>::InEvent, Self::OutEvent>> = #poll_method;
let f: ::libp2p::futures::Async<#network_behaviour_action<<<Self::ProtocolsHandler as #into_protocols_handler>::Handler as #protocols_handler>::InEvent, Self::OutEvent>> = #poll_method;
f
}
}