mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-18 12:31:22 +00:00
Add IntoNodeHandler and IntoProtocolsHandler traits (#848)
* Add IntoNodeHandler * Add IntoProtocolsHandler
This commit is contained in:
@ -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
|
||||||
{
|
{
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -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,
|
||||||
|
@ -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
|
||||||
|
}
|
||||||
|
}
|
||||||
|
@ -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;
|
||||||
|
@ -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> {
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
Reference in New Issue
Block a user