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::{ nodes::{
node::Substream, node::Substream,
handled_node_tasks::{HandledNodesEvent, HandledNodesTasks, TaskClosedEvent}, 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} handled_node::{HandledNodeError, NodeHandler}
} }
}; };
@ -285,12 +285,13 @@ impl<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr> CollectionStream<TIn
-> ReachAttemptId -> ReachAttemptId
where where
TFut: Future<Item = (PeerId, TMuxer), Error = TReachErr> + Send + 'static, 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, TReachErr: error::Error + Send + 'static,
THandlerErr: error::Error + Send + 'static, THandlerErr: error::Error + Send + 'static,
TInEvent: Send + 'static, TInEvent: Send + 'static,
TOutEvent: 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: StreamMuxer + Send + Sync + 'static, // TODO: Send + Sync + 'static shouldn't be required
TMuxer::OutboundSubstream: Send + 'static, // TODO: shouldn't be required TMuxer::OutboundSubstream: Send + 'static, // TODO: shouldn't be required
{ {

View File

@ -59,7 +59,7 @@ mod tests;
// conditions in the user's code. See similar comments in the documentation of `NodeStream`. // conditions in the user's code. See similar comments in the documentation of `NodeStream`.
/// Implementation of `Stream` that handles a collection of nodes. /// 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 /// 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 /// 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. /// 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]>, to_spawn: SmallVec<[Box<Future<Item = (), Error = ()> + Send>; 8]>,
/// Sender to emit events to the outside. Meant to be cloned and sent to tasks. /// 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. /// 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> { fn fmt(&self, f: &mut fmt::Formatter) -> Result<(), fmt::Error> {
f.debug_list() f.debug_list()
.entries(self.tasks.keys().cloned()) .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`. /// Event that can happen on the `HandledNodesTasks`.
#[derive(Debug)] #[derive(Debug)]
pub enum HandledNodesEvent<TOutEvent, THandler, TReachErr, THandlerErr> { pub enum HandledNodesEvent<TOutEvent, TIntoHandler, TReachErr, THandlerErr> {
/// A task has been closed. /// A task has been closed.
/// ///
/// This happens once the node handler closes or an error happens. /// 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>>, result: Result<(), TaskClosedEvent<TReachErr, THandlerErr>>,
/// If the task closed before reaching the node, this contains the handler that was passed /// If the task closed before reaching the node, this contains the handler that was passed
/// to `add_reach_attempt`. /// to `add_reach_attempt`.
handler: Option<THandler>, handler: Option<TIntoHandler>,
}, },
/// A task has successfully connected to a node. /// 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)] #[derive(Debug, Copy, Clone, Hash, PartialEq, Eq, PartialOrd, Ord)]
pub struct TaskId(usize); 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. /// Creates a new empty collection.
#[inline] #[inline]
pub fn new() -> Self { 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 /// This method spawns a task dedicated to resolving this future and processing the node's
/// events. /// 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 where
TFut: Future<Item = (PeerId, TMuxer), Error = TReachErr> + Send + 'static, 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, TReachErr: error::Error + Send + 'static,
THandlerErr: error::Error + Send + 'static, THandlerErr: error::Error + Send + 'static,
TInEvent: Send + 'static, TInEvent: Send + 'static,
TOutEvent: 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: StreamMuxer + Send + Sync + 'static, // TODO: Send + Sync + 'static shouldn't be required
TMuxer::OutboundSubstream: Send + 'static, // TODO: 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. /// 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() { for to_spawn in self.to_spawn.drain() {
tokio_executor::spawn(to_spawn); 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> { impl<TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr> Stream for HandledNodesTasks<TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr> {
type Item = HandledNodesEvent<TOutEvent, THandler, TReachErr, THandlerErr>; type Item = HandledNodesEvent<TOutEvent, TIntoHandler, TReachErr, THandlerErr>;
type Error = Void; // TODO: use ! once stable type Error = Void; // TODO: use ! once stable
#[inline] #[inline]
@ -339,43 +362,45 @@ impl<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr> Stream for HandledNo
/// Message to transmit from a task to the public API. /// Message to transmit from a task to the public API.
#[derive(Debug)] #[derive(Debug)]
enum InToExtMessage<TOutEvent, THandler, TReachErr, THandlerErr> { enum InToExtMessage<TOutEvent, TIntoHandler, TReachErr, THandlerErr> {
/// A connection to a node has succeeded. /// A connection to a node has succeeded.
NodeReached(PeerId), NodeReached(PeerId),
/// The task closed. /// The task closed.
TaskClosed(Result<(), TaskClosedEvent<TReachErr, THandlerErr>>, Option<THandler>), TaskClosed(Result<(), TaskClosedEvent<TReachErr, THandlerErr>>, Option<TIntoHandler>),
/// An event from the node. /// An event from the node.
NodeEvent(TOutEvent), NodeEvent(TOutEvent),
} }
/// Implementation of `Future` that handles a single node, and all the communications between /// Implementation of `Future` that handles a single node, and all the communications between
/// the various components of the `HandledNodesTasks`. /// the various components of the `HandledNodesTasks`.
struct NodeTask<TFut, TMuxer, THandler, TInEvent, TOutEvent, TReachErr> struct NodeTask<TFut, TMuxer, TIntoHandler, TInEvent, TOutEvent, TReachErr>
where where
TMuxer: StreamMuxer, TMuxer: StreamMuxer,
THandler: NodeHandler<Substream = Substream<TMuxer>>, TIntoHandler: IntoNodeHandler,
TIntoHandler::Handler: NodeHandler<Substream = Substream<TMuxer>>,
{ {
/// Sender to transmit events to the outside. /// 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`. /// Receiving end for events sent from the main `HandledNodesTasks`.
in_events_rx: stream::Fuse<mpsc::UnboundedReceiver<TInEvent>>, in_events_rx: stream::Fuse<mpsc::UnboundedReceiver<TInEvent>>,
/// Inner state of the `NodeTask`. /// Inner state of the `NodeTask`.
inner: NodeTaskInner<TFut, TMuxer, THandler, TInEvent>, inner: NodeTaskInner<TFut, TMuxer, TIntoHandler, TInEvent>,
/// Identifier of the attempt. /// Identifier of the attempt.
id: TaskId, id: TaskId,
} }
enum NodeTaskInner<TFut, TMuxer, THandler, TInEvent> enum NodeTaskInner<TFut, TMuxer, TIntoHandler, TInEvent>
where where
TMuxer: StreamMuxer, TMuxer: StreamMuxer,
THandler: NodeHandler<Substream = Substream<TMuxer>>, TIntoHandler: IntoNodeHandler,
TIntoHandler::Handler: NodeHandler<Substream = Substream<TMuxer>>,
{ {
/// Future to resolve to connect to the node. /// Future to resolve to connect to the node.
Future { Future {
/// The future that will attempt to reach the node. /// The future that will attempt to reach the node.
future: TFut, future: TFut,
/// The handler that will be used to build the `HandledNode`. /// 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 /// 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 /// `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. /// events in `in_events_rx` because we have to detect if it gets closed.
@ -383,18 +408,19 @@ where
}, },
/// Fully functional node. /// Fully functional node.
Node(HandledNode<TMuxer, THandler>), Node(HandledNode<TMuxer, TIntoHandler::Handler>),
/// A panic happened while polling. /// A panic happened while polling.
Poisoned, Poisoned,
} }
impl<TFut, TMuxer, THandler, TInEvent, TOutEvent, TReachErr> Future for impl<TFut, TMuxer, TIntoHandler, TInEvent, TOutEvent, TReachErr> Future for
NodeTask<TFut, TMuxer, THandler, TInEvent, TOutEvent, TReachErr> NodeTask<TFut, TMuxer, TIntoHandler, TInEvent, TOutEvent, TReachErr>
where where
TMuxer: StreamMuxer, TMuxer: StreamMuxer,
TFut: Future<Item = (PeerId, TMuxer), Error = TReachErr>, 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 Item = ();
type Error = (); type Error = ();
@ -416,8 +442,8 @@ where
// Check whether dialing succeeded. // Check whether dialing succeeded.
match future.poll() { match future.poll() {
Ok(Async::Ready((peer_id, muxer))) => { Ok(Async::Ready((peer_id, muxer))) => {
let mut node = HandledNode::new(muxer, handler.into_handler(&peer_id));
let event = InToExtMessage::NodeReached(peer_id); let event = InToExtMessage::NodeReached(peer_id);
let mut node = HandledNode::new(muxer, handler);
for event in events_buffer { for event in events_buffer {
node.inject_event(event); node.inject_event(event);
} }

View File

@ -34,6 +34,7 @@ use crate::{
HandledNodeError, HandledNodeError,
NodeHandler NodeHandler
}, },
handled_node_tasks::IntoNodeHandler,
node::Substream node::Substream
}, },
nodes::listeners::{ListenersEvent, ListenersStream}, nodes::listeners::{ListenersEvent, ListenersStream},
@ -377,8 +378,9 @@ where
TTrans: Transport<Output = (PeerId, TMuxer)>, TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::Error: Send + 'static, TTrans::Error: Send + 'static,
TTrans::ListenerUpgrade: Send + 'static, TTrans::ListenerUpgrade: Send + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static, THandler: IntoNodeHandler + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary 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, THandlerErr: error::Error + Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send, TMuxer::OutboundSubstream: Send,
@ -523,8 +525,9 @@ impl<TTrans, TInEvent, TOutEvent, TMuxer, THandler, THandlerErr>
where where
TTrans: Transport + Clone, TTrans: Transport + Clone,
TMuxer: StreamMuxer, TMuxer: StreamMuxer,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static, THandler: IntoNodeHandler + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary 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, THandlerErr: error::Error + Send + 'static,
{ {
/// Creates a new node events stream. /// Creates a new node events stream.
@ -752,8 +755,9 @@ where
TMuxer::Substream: Send, TMuxer::Substream: Send,
TInEvent: Send + 'static, TInEvent: Send + 'static,
TOutEvent: Send + 'static, TOutEvent: Send + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static, THandler: IntoNodeHandler + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary 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, THandlerErr: error::Error + Send + 'static,
{ {
// Start by polling the listeners for events. // Start by polling the listeners for events.
@ -1138,8 +1142,9 @@ where
TMuxer::Substream: Send, TMuxer::Substream: Send,
TInEvent: Send + 'static, TInEvent: Send + 'static,
TOutEvent: Send + 'static, TOutEvent: Send + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static, THandler: IntoNodeHandler + Send + 'static,
THandler::OutboundOpenInfo: 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, THandlerErr: error::Error + Send + 'static,
{ {
/// If we are connected, returns the `PeerConnected`. /// If we are connected, returns the `PeerConnected`.
@ -1362,8 +1367,9 @@ where
TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send, TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send, TMuxer::Substream: Send,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static, THandler: IntoNodeHandler + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary 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, THandlerErr: error::Error + Send + 'static,
TInEvent: Send + 'static, TInEvent: Send + 'static,
TOutEvent: 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 //! > connection with a remote. In order to handle a protocol that requires knowledge of
//! > the network as a whole, see the `NetworkBehaviour` trait. //! > the network as a whole, see the `NetworkBehaviour` trait.
use crate::PeerId;
use crate::upgrade::{ use crate::upgrade::{
InboundUpgrade, InboundUpgrade,
OutboundUpgrade, OutboundUpgrade,
@ -46,7 +47,7 @@ pub use self::dummy::DummyProtocolsHandler;
pub use self::map_in::MapInEvent; pub use self::map_in::MapInEvent;
pub use self::map_out::MapOutEvent; pub use self::map_out::MapOutEvent;
pub use self::node_handler::{NodeHandlerWrapper, NodeHandlerWrapperBuilder}; pub use self::node_handler::{NodeHandlerWrapper, NodeHandlerWrapperBuilder};
pub use self::select::ProtocolsHandlerSelect; pub use self::select::{IntoProtocolsHandlerSelect, ProtocolsHandlerSelect};
mod dummy; mod dummy;
mod map_in; mod map_in;
@ -207,17 +208,19 @@ pub trait ProtocolsHandler {
where where
Self: Sized, 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. /// Builds an implementation of `NodeHandler` that handles this protocol exclusively.
/// ///
/// > **Note**: This is a shortcut for `self.into_node_handler_builder().build()`. /// > **Note**: This is a shortcut for `self.into_node_handler_builder().build()`.
#[inline] #[inline]
#[deprecated(note = "Use into_node_handler_builder instead")]
fn into_node_handler(self) -> NodeHandlerWrapper<Self> fn into_node_handler(self) -> NodeHandlerWrapper<Self>
where where
Self: Sized, Self: Sized,
{ {
#![allow(deprecated)]
self.into_node_handler_builder().build() 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. // DEALINGS IN THE SOFTWARE.
use crate::{ use crate::{
PeerId,
nodes::handled_node::{NodeHandler, NodeHandlerEndpoint, NodeHandlerEvent}, nodes::handled_node::{NodeHandler, NodeHandlerEndpoint, NodeHandlerEvent},
protocols_handler::{ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr}, nodes::handled_node_tasks::IntoNodeHandler,
protocols_handler::{ProtocolsHandler, IntoProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr},
upgrade::{ upgrade::{
self, self,
OutboundUpgrade, OutboundUpgrade,
@ -33,12 +35,9 @@ use std::time::{Duration, Instant};
use tokio_timer::{Delay, Timeout}; use tokio_timer::{Delay, Timeout};
/// Prototype for a `NodeHandlerWrapper`. /// Prototype for a `NodeHandlerWrapper`.
pub struct NodeHandlerWrapperBuilder<TProtoHandler> pub struct NodeHandlerWrapperBuilder<TIntoProtoHandler> {
where
TProtoHandler: ProtocolsHandler,
{
/// The underlying handler. /// The underlying handler.
handler: TProtoHandler, handler: TIntoProtoHandler,
/// Timeout for incoming substreams negotiation. /// Timeout for incoming substreams negotiation.
in_timeout: Duration, in_timeout: Duration,
/// Timeout for outgoing substreams negotiation. /// Timeout for outgoing substreams negotiation.
@ -47,13 +46,13 @@ where
useless_timeout: Duration, useless_timeout: Duration,
} }
impl<TProtoHandler> NodeHandlerWrapperBuilder<TProtoHandler> impl<TIntoProtoHandler> NodeHandlerWrapperBuilder<TIntoProtoHandler>
where where
TProtoHandler: ProtocolsHandler TIntoProtoHandler: IntoProtocolsHandler
{ {
/// Builds a `NodeHandlerWrapperBuilder`. /// Builds a `NodeHandlerWrapperBuilder`.
#[inline] #[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 { NodeHandlerWrapperBuilder {
handler, handler,
in_timeout, in_timeout,
@ -85,8 +84,11 @@ where
} }
/// Builds the `NodeHandlerWrapper`. /// Builds the `NodeHandlerWrapper`.
#[deprecated(note = "Pass the NodeHandlerWrapperBuilder directly")]
#[inline] #[inline]
pub fn build(self) -> NodeHandlerWrapper<TProtoHandler> { pub fn build(self) -> NodeHandlerWrapper<TIntoProtoHandler>
where TIntoProtoHandler: ProtocolsHandler
{
NodeHandlerWrapper { NodeHandlerWrapper {
handler: self.handler, handler: self.handler,
negotiating_in: Vec::new(), 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`. /// Wraps around an implementation of `ProtocolsHandler`, and implements `NodeHandler`.
// TODO: add a caching system for protocols that are supported or not // TODO: add a caching system for protocols that are supported or not
pub struct NodeHandlerWrapper<TProtoHandler> pub struct NodeHandlerWrapper<TProtoHandler>
@ -138,6 +164,7 @@ where
impl<TProtoHandler> NodeHandler for NodeHandlerWrapper<TProtoHandler> impl<TProtoHandler> NodeHandler for NodeHandlerWrapper<TProtoHandler>
where where
TProtoHandler: ProtocolsHandler, TProtoHandler: ProtocolsHandler,
// TODO: meh for Debug
<TProtoHandler::OutboundProtocol as OutboundUpgrade<<TProtoHandler as ProtocolsHandler>::Substream>>::Error: std::fmt::Debug <TProtoHandler::OutboundProtocol as OutboundUpgrade<<TProtoHandler as ProtocolsHandler>::Substream>>::Error: std::fmt::Debug
{ {
type InEvent = TProtoHandler::InEvent; type InEvent = TProtoHandler::InEvent;

View File

@ -19,9 +19,15 @@
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::{ use crate::{
PeerId,
either::EitherError, either::EitherError,
either::EitherOutput, either::EitherOutput,
protocols_handler::{ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr}, protocols_handler::{
IntoProtocolsHandler,
ProtocolsHandler,
ProtocolsHandlerEvent,
ProtocolsHandlerUpgrErr,
},
upgrade::{ upgrade::{
InboundUpgrade, InboundUpgrade,
OutboundUpgrade, OutboundUpgrade,
@ -33,6 +39,46 @@ use crate::{
use futures::prelude::*; use futures::prelude::*;
use tokio_io::{AsyncRead, AsyncWrite}; 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. /// Implementation of `ProtocolsHandler` that combines two protocols into one.
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub struct ProtocolsHandlerSelect<TProto1, TProto2> { pub struct ProtocolsHandlerSelect<TProto1, TProto2> {

View File

@ -49,7 +49,7 @@ use crate::{
node::Substream, node::Substream,
raw_swarm::{RawSwarm, RawSwarmEvent} raw_swarm::{RawSwarm, RawSwarmEvent}
}, },
protocols_handler::{NodeHandlerWrapper, ProtocolsHandler}, protocols_handler::{NodeHandlerWrapperBuilder, NodeHandlerWrapper, IntoProtocolsHandler, ProtocolsHandler},
topology::Topology, topology::Topology,
transport::TransportError, transport::TransportError,
topology::DisconnectReason, topology::DisconnectReason,
@ -67,10 +67,10 @@ where TTransport: Transport,
{ {
raw_swarm: RawSwarm< raw_swarm: RawSwarm<
TTransport, TTransport,
<<TBehaviour as NetworkBehaviour<TTopology>>::ProtocolsHandler as ProtocolsHandler>::InEvent, <<<TBehaviour as NetworkBehaviour<TTopology>>::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InEvent,
<<TBehaviour as NetworkBehaviour<TTopology>>::ProtocolsHandler as ProtocolsHandler>::OutEvent, <<<TBehaviour as NetworkBehaviour<TTopology>>::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutEvent,
NodeHandlerWrapper<TBehaviour::ProtocolsHandler>, NodeHandlerWrapperBuilder<TBehaviour::ProtocolsHandler>,
<<TBehaviour as NetworkBehaviour<TTopology>>::ProtocolsHandler as ProtocolsHandler>::Error, <<<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 /// 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::Listener: Send + 'static,
TTransport::ListenerUpgrade: Send + 'static, TTransport::ListenerUpgrade: Send + 'static,
TTransport::Dial: Send + 'static, TTransport::Dial: Send + 'static,
TBehaviour::ProtocolsHandler: ProtocolsHandler<Substream = Substream<TMuxer>> + Send + 'static, TBehaviour::ProtocolsHandler: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InEvent: Send + 'static, <TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler: ProtocolsHandler<Substream = Substream<TMuxer>> + Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutEvent: Send + 'static, <<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InEvent: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Error: Send + 'static, <<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutEvent: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary <<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::Error: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol: InboundUpgrade<Substream<TMuxer>> + Send + 'static, <<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::Info: Send + 'static, <<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol: InboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter: Send + 'static, <<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::Info: Send + 'static,
<<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static, <<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static, <<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static,
<<TBehaviour::ProtocolsHandler 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 ProtocolsHandler>::OutboundProtocol: OutboundUpgrade<Substream<TMuxer>> + Send + 'static, <<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::Info: Send + 'static, <<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter: Send + 'static, <<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::Info: Send + 'static,
<<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static, <<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static, <<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static, <<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<NodeHandlerWrapper<TBehaviour::ProtocolsHandler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary <<<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, TTopology: Topology,
{ {
/// Builds a new `Swarm`. /// Builds a new `Swarm`.
@ -145,6 +146,7 @@ where TBehaviour: NetworkBehaviour<TTopology>,
pub fn new(transport: TTransport, mut behaviour: TBehaviour, topology: TTopology) -> Self { pub fn new(transport: TTransport, mut behaviour: TBehaviour, topology: TTopology) -> Self {
let supported_protocols = behaviour let supported_protocols = behaviour
.new_handler() .new_handler()
.into_handler(topology.local_peer_id())
.listen_protocol() .listen_protocol()
.protocol_info() .protocol_info()
.into_iter() .into_iter()
@ -187,7 +189,7 @@ where TBehaviour: NetworkBehaviour<TTopology>,
#[inline] #[inline]
pub fn dial_addr(me: &mut Self, addr: Multiaddr) -> Result<(), TransportError<TTransport::Error>> { pub fn dial_addr(me: &mut Self, addr: Multiaddr) -> Result<(), TransportError<TTransport::Error>> {
let handler = me.behaviour.new_handler(); 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. /// Tries to reach the given peer using the elements in the topology.
@ -197,7 +199,7 @@ where TBehaviour: NetworkBehaviour<TTopology>,
#[inline] #[inline]
pub fn dial(me: &mut Self, peer_id: PeerId) { pub fn dial(me: &mut Self, peer_id: PeerId) {
let addrs = me.topology.addresses_of_peer(&peer_id); 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() { if let Some(peer) = me.raw_swarm.peer(peer_id).as_not_connected() {
let _ = peer.connect_iter(addrs, handler); let _ = peer.connect_iter(addrs, handler);
} }
@ -238,24 +240,25 @@ where TBehaviour: NetworkBehaviour<TTopology>,
TTransport::Listener: Send + 'static, TTransport::Listener: Send + 'static,
TTransport::ListenerUpgrade: Send + 'static, TTransport::ListenerUpgrade: Send + 'static,
TTransport::Dial: Send + 'static, TTransport::Dial: Send + 'static,
TBehaviour::ProtocolsHandler: ProtocolsHandler<Substream = Substream<TMuxer>> + Send + 'static, TBehaviour::ProtocolsHandler: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InEvent: Send + 'static, <TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler: ProtocolsHandler<Substream = Substream<TMuxer>> + Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutEvent: Send + 'static, <<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InEvent: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Error: Send + 'static, <<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutEvent: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary <<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::Error: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol: InboundUpgrade<Substream<TMuxer>> + Send + 'static, <<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static, <<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol: InboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler 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 ProtocolsHandler>::InboundProtocol as UpgradeInfo>::Info: Send + 'static, <<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter: Send + 'static, <<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::Info: Send + 'static,
<<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static, <<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade<Substream<TMuxer>> + Send + 'static, <<<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static, <<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static, <<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::Info: Send + 'static, <<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter: Send + 'static, <<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::Info: Send + 'static,
<<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter as IntoIterator>::IntoIter: Send + 'static, <<<TBehaviour::ProtocolsHandler as IntoProtocolsHandler>::Handler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::InfoIter: Send + 'static,
<NodeHandlerWrapper<TBehaviour::ProtocolsHandler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary <<<<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, TTopology: Topology,
{ {
type Item = TBehaviour::OutEvent; type Item = TBehaviour::OutEvent;
@ -291,7 +294,7 @@ where TBehaviour: NetworkBehaviour<TTopology>,
}, },
Async::Ready(RawSwarmEvent::IncomingConnection(incoming)) => { Async::Ready(RawSwarmEvent::IncomingConnection(incoming)) => {
let handler = self.behaviour.new_handler(); 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::ListenerClosed { .. }) => {},
Async::Ready(RawSwarmEvent::IncomingConnectionError { .. }) => {}, Async::Ready(RawSwarmEvent::IncomingConnectionError { .. }) => {},
@ -345,7 +348,7 @@ where TBehaviour: NetworkBehaviour<TTopology>,
/// one that handles all the behaviours at once. /// one that handles all the behaviours at once.
pub trait NetworkBehaviour<TTopology> { pub trait NetworkBehaviour<TTopology> {
/// Handler for all the protocols the network supports. /// Handler for all the protocols the network supports.
type ProtocolsHandler: ProtocolsHandler; type ProtocolsHandler: IntoProtocolsHandler;
/// Event generated by the swarm. /// Event generated by the swarm.
type OutEvent; type OutEvent;
@ -367,13 +370,13 @@ pub trait NetworkBehaviour<TTopology> {
fn inject_node_event( fn inject_node_event(
&mut self, &mut self,
peer_id: PeerId, 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. /// Polls for things that swarm should do.
/// ///
/// This API mimics the API of the `Stream` trait. /// 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 /// 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 net_behv_event_proc = quote!{::libp2p::core::swarm::NetworkBehaviourEventProcess};
let either_ident = quote!{::libp2p::core::either::EitherOutput}; let either_ident = quote!{::libp2p::core::either::EitherOutput};
let network_behaviour_action = quote!{::libp2p::core::swarm::NetworkBehaviourAction}; 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 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 peer_id = quote!{::libp2p::core::PeerId};
let connected_point = quote!{::libp2p::core::swarm::ConnectedPoint}; let connected_point = quote!{::libp2p::core::swarm::ConnectedPoint};
@ -99,10 +100,10 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream {
vec![ vec![
quote!{#ty: #trait_to_impl<#topology_generic>}, quote!{#ty: #trait_to_impl<#topology_generic>},
quote!{Self: #net_behv_event_proc<<#ty as #trait_to_impl<#topology_generic>>::OutEvent>}, 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 // 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 #into_protocols_handler>::Handler 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>::OutboundProtocol: ::libp2p::core::OutboundUpgrade<#substream_generic>},
] ]
}) })
.collect::<Vec<_>>(); .collect::<Vec<_>>();
@ -213,7 +214,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream {
let ty = &field.ty; let ty = &field.ty;
let field_info = quote!{ <#ty as #trait_to_impl<#topology_generic>>::ProtocolsHandler }; let field_info = quote!{ <#ty as #trait_to_impl<#topology_generic>>::ProtocolsHandler };
match ph_ty { 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), ref mut ev @ None => *ev = Some(field_info),
} }
} }
@ -324,7 +325,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream {
#[inline] #[inline]
fn new_handler(&mut self) -> Self::ProtocolsHandler { fn new_handler(&mut self) -> Self::ProtocolsHandler {
use #protocols_handler; use #into_protocols_handler;
#new_handler #new_handler
} }
@ -342,17 +343,17 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream {
fn inject_node_event( fn inject_node_event(
&mut self, &mut self,
peer_id: #peer_id, 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 { match event {
#(#inject_node_event_stmts),* #(#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::*; use libp2p::futures::prelude::*;
#(#poll_stmts)* #(#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 f
} }
} }