Replace unbounded channels with bounded ones. (#1191)

* Replace unbounded channels with bounded ones.

To remove the unbounded channels used for communicating with node tasks
an API similar to `futures::Sink` is used, i.e. sending is split into a
start and complete phase. The start phase returns `StartSend` and first
attempts to complete any pending send operations. Completing the send
means polling until `Poll::Ready(())` is returned.

In addition this PR has split the `handled_node_tasks` module into
several smaller ones (cf. `nodes::tasks`) and renamed some types:

- `nodes::handled_node_tasks::NodeTask` -> `nodes::tasks::task::Task`
- `nodes::handled_node_tasks::NodeTaskInner` -> `nodes::tasks::task::State`
- `nodes::handled_node_tasks::NodeTasks` -> `nodes::tasks::Manager`
- `nodes::handled_node_tasks::TaskClosedEvent` -> `nodes::tasks::Error`
- `nodes::handled_node_tasks::HandledNodesEvent` -> `nodes::tasks::Event`
- `nodes::handled_node_tasks::Task` -> `nodes::tasks::TaskEntry`
- `nodes::handled_node_tasks::ExtToInMessage` -> `nodes::tasks::task::ToTaskMessage`
- `nodes::handled_node_tasks::InToExtMessage` -> `nodes::tasks::task::FromTaskMessage`

* `take_over_to_complete` can be an `Option`.

Since it is always holding just a single pending message.

* `send_event_to_complete` can be an `Option`.

* Update core/src/nodes/tasks/manager.rs

Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com>

* Update core/src/nodes/tasks/manager.rs

Co-Authored-By: Pierre Krieger <pierre.krieger1708@gmail.com>

* Add comments to explain the need to flush sends ...

of take-over and event messages delivered over Sinks.
This commit is contained in:
Toralf Wittner 2019-07-09 16:47:24 +02:00 committed by Pierre Krieger
parent a0d278a479
commit 6aba7961d1
14 changed files with 1287 additions and 806 deletions

View File

@ -23,15 +23,16 @@ use crate::{
muxing::StreamMuxer, muxing::StreamMuxer,
nodes::{ nodes::{
node::Substream, node::Substream,
handled_node_tasks::{HandledNodesEvent, HandledNodesTasks, TaskClosedEvent}, handled_node::{HandledNodeError, IntoNodeHandler, NodeHandler},
handled_node_tasks::{IntoNodeHandler, Task as HandledNodesTask, TaskId, ClosedTask}, tasks::{self, ClosedTask, TaskEntry, TaskId}
handled_node::{HandledNodeError, NodeHandler}
} }
}; };
use fnv::FnvHashMap; use fnv::FnvHashMap;
use futures::prelude::*; use futures::prelude::*;
use std::{error, fmt, hash::Hash, mem}; use std::{error, fmt, hash::Hash, mem};
pub use crate::nodes::tasks::StartTakeOver;
mod tests; mod tests;
/// Implementation of `Stream` that handles a collection of nodes. /// Implementation of `Stream` that handles a collection of nodes.
@ -40,7 +41,7 @@ pub struct CollectionStream<TInEvent, TOutEvent, THandler, TReachErr, THandlerEr
/// ///
/// The user data contains the state of the task. If `Connected`, then a corresponding entry /// The user data contains the state of the task. If `Connected`, then a corresponding entry
/// must be present in `nodes`. /// must be present in `nodes`.
inner: HandledNodesTasks<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr, TaskState<TConnInfo, TUserData>, TConnInfo>, inner: tasks::Manager<TInEvent, TOutEvent, THandler, TReachErr, THandlerErr, TaskState<TConnInfo, TUserData>, TConnInfo>,
/// List of nodes, with the task id that handles this node. The corresponding entry in `tasks` /// List of nodes, with the task id that handles this node. The corresponding entry in `tasks`
/// must always be in the `Connected` state. /// must always be in the `Connected` state.
@ -310,7 +311,7 @@ where
#[inline] #[inline]
pub fn new() -> Self { pub fn new() -> Self {
CollectionStream { CollectionStream {
inner: HandledNodesTasks::new(), inner: tasks::Manager::new(),
nodes: Default::default(), nodes: Default::default(),
} }
} }
@ -357,12 +358,17 @@ where
} }
/// Sends an event to all nodes. /// Sends an event to all nodes.
#[inline] #[must_use]
pub fn broadcast_event(&mut self, event: &TInEvent) pub fn start_broadcast(&mut self, event: &TInEvent) -> AsyncSink<()>
where TInEvent: Clone, where
TInEvent: Clone
{ {
// TODO: remove the ones we're not connected to? self.inner.start_broadcast(event)
self.inner.broadcast_event(event) }
#[must_use]
pub fn complete_broadcast(&mut self) -> Async<()> {
self.inner.complete_broadcast()
} }
/// Adds an existing connection to a node to the collection. /// Adds an existing connection to a node to the collection.
@ -383,8 +389,8 @@ where
TConnInfo: Clone + Send + 'static, TConnInfo: Clone + Send + 'static,
TPeerId: Clone, TPeerId: Clone,
{ {
// Calling `HandledNodesTasks::add_connection` is the same as calling // Calling `tasks::Manager::add_connection` is the same as calling
// `HandledNodesTasks::add_reach_attempt`, except that we don't get any `NodeReached` event. // `tasks::Manager::add_reach_attempt`, except that we don't get any `NodeReached` event.
// We therefore implement this method the same way as calling `add_reach_attempt` followed // We therefore implement this method the same way as calling `add_reach_attempt` followed
// with simulating a received `NodeReached` event and accepting it. // with simulating a received `NodeReached` event and accepting it.
@ -451,29 +457,28 @@ where
}; };
match item { match item {
HandledNodesEvent::TaskClosed { task, result, handler } => { tasks::Event::TaskClosed { task, result, handler } => {
let id = task.id(); let id = task.id();
let user_data = task.into_user_data(); let user_data = task.into_user_data();
match (user_data, result, handler) { match (user_data, result, handler) {
(TaskState::Pending, TaskClosedEvent::Reach(err), Some(handler)) => { (TaskState::Pending, tasks::Error::Reach(err), Some(handler)) => {
Async::Ready(CollectionEvent::ReachError { Async::Ready(CollectionEvent::ReachError {
id: ReachAttemptId(id), id: ReachAttemptId(id),
error: err, error: err,
handler, handler,
}) })
}, },
(TaskState::Pending, TaskClosedEvent::Node(_), _) => { (TaskState::Pending, tasks::Error::Node(_), _) => {
panic!("We switch the task state to Connected once we're connected, and \ panic!("We switch the task state to Connected once we're connected, and \
a TaskClosedEvent::Node can only happen after we're \ a tasks::Error::Node can only happen after we're connected; QED");
connected; QED");
}, },
(TaskState::Pending, TaskClosedEvent::Reach(_), None) => { (TaskState::Pending, tasks::Error::Reach(_), None) => {
// TODO: this could be improved in the API of HandledNodesTasks // TODO: this could be improved in the API of tasks::Manager
panic!("The HandledNodesTasks is guaranteed to always return the handler \ panic!("The tasks::Manager is guaranteed to always return the handler \
when producing a TaskClosedEvent::Reach error"); when producing a tasks::Error::Reach error");
}, },
(TaskState::Connected(conn_info, user_data), TaskClosedEvent::Node(err), _handler) => { (TaskState::Connected(conn_info, user_data), tasks::Error::Node(err), _handler) => {
debug_assert!(_handler.is_none()); debug_assert!(_handler.is_none());
let _node_task_id = self.nodes.remove(conn_info.peer_id()); let _node_task_id = self.nodes.remove(conn_info.peer_id());
debug_assert_eq!(_node_task_id, Some(id)); debug_assert_eq!(_node_task_id, Some(id));
@ -483,13 +488,13 @@ where
user_data, user_data,
}) })
}, },
(TaskState::Connected(_, _), TaskClosedEvent::Reach(_), _) => { (TaskState::Connected(_, _), tasks::Error::Reach(_), _) => {
panic!("A TaskClosedEvent::Reach can only happen before we are connected \ panic!("A tasks::Error::Reach can only happen before we are connected \
to a node; therefore the TaskState won't be Connected; QED"); to a node; therefore the TaskState won't be Connected; QED");
}, },
} }
}, },
HandledNodesEvent::NodeReached { task, conn_info } => { tasks::Event::NodeReached { task, conn_info } => {
let id = task.id(); let id = task.id();
drop(task); drop(task);
Async::Ready(CollectionEvent::NodeReached(CollectionReachEvent { Async::Ready(CollectionEvent::NodeReached(CollectionReachEvent {
@ -498,7 +503,7 @@ where
conn_info: Some(conn_info), conn_info: Some(conn_info),
})) }))
}, },
HandledNodesEvent::NodeEvent { task, event } => { tasks::Event::NodeEvent { task, event } => {
let conn_info = match task.user_data() { let conn_info = match task.user_data() {
TaskState::Connected(conn_info, _) => conn_info.clone(), TaskState::Connected(conn_info, _) => conn_info.clone(),
_ => panic!("we can only receive NodeEvent events from a task after we \ _ => panic!("we can only receive NodeEvent events from a task after we \
@ -566,7 +571,7 @@ where
/// Access to a peer in the collection. /// Access to a peer in the collection.
pub struct PeerMut<'a, TInEvent, TUserData, TConnInfo = PeerId, TPeerId = PeerId> { pub struct PeerMut<'a, TInEvent, TUserData, TConnInfo = PeerId, TPeerId = PeerId> {
inner: HandledNodesTask<'a, TInEvent, TaskState<TConnInfo, TUserData>>, inner: TaskEntry<'a, TInEvent, TaskState<TConnInfo, TUserData>>,
nodes: &'a mut FnvHashMap<TPeerId, TaskId>, nodes: &'a mut FnvHashMap<TPeerId, TaskId>,
} }
@ -612,9 +617,13 @@ where
} }
/// Sends an event to the given node. /// Sends an event to the given node.
#[inline] pub fn start_send_event(&mut self, event: TInEvent) -> StartSend<TInEvent, ()> {
pub fn send_event(&mut self, event: TInEvent) { self.inner.start_send_event(event)
self.inner.send_event(event) }
/// Complete sending an event message initiated by `start_send_event`.
pub fn complete_send_event(&mut self) -> Poll<(), ()> {
self.inner.complete_send_event()
} }
/// Closes the connections to this node. Returns the user data. /// Closes the connections to this node. Returns the user data.
@ -639,8 +648,23 @@ where
/// The reach attempt will only be effectively cancelled once the peer (the object you're /// The reach attempt will only be effectively cancelled once the peer (the object you're
/// manipulating) has received some network activity. However no event will be ever be /// manipulating) has received some network activity. However no event will be ever be
/// generated from this reach attempt, and this takes effect immediately. /// generated from this reach attempt, and this takes effect immediately.
pub fn take_over(&mut self, id: InterruptedReachAttempt<TInEvent, TConnInfo, TUserData>) { #[must_use]
let _state = self.inner.take_over(id.inner); pub fn start_take_over(&mut self, id: InterruptedReachAttempt<TInEvent, TConnInfo, TUserData>)
debug_assert!(if let TaskState::Pending = _state { true } else { false }); -> StartTakeOver<(), InterruptedReachAttempt<TInEvent, TConnInfo, TUserData>>
{
match self.inner.start_take_over(id.inner) {
StartTakeOver::Ready(_state) => {
debug_assert!(if let TaskState::Pending = _state { true } else { false });
StartTakeOver::Ready(())
}
StartTakeOver::NotReady(inner) =>
StartTakeOver::NotReady(InterruptedReachAttempt { inner }),
StartTakeOver::Gone => StartTakeOver::Gone
}
}
/// Complete a take over initiated by `start_take_over`.
pub fn complete_take_over(&mut self) -> Poll<(), ()> {
self.inner.complete_take_over()
} }
} }

View File

@ -154,20 +154,40 @@ fn events_in_a_node_reaches_the_collection_stream() {
Ok(Async::Ready(())) Ok(Async::Ready(()))
})).expect("tokio works"); })).expect("tokio works");
let cs2 = cs.clone();
rt.block_on(future::poll_fn(move || {
if cs2.lock().start_broadcast(&InEvent::NextState).is_not_ready() {
Ok::<_, ()>(Async::NotReady)
} else {
Ok(Async::Ready(()))
}
})).unwrap();
let cs_fut = cs.clone(); let cs_fut = cs.clone();
rt.block_on(future::poll_fn(move || -> Poll<_, ()> { rt.block_on(future::poll_fn(move || -> Poll<_, ()> {
let mut cs = cs_fut.lock(); let mut cs = cs_fut.lock();
cs.broadcast_event(&InEvent::NextState); if cs.complete_broadcast().is_not_ready() {
return Ok(Async::NotReady)
}
assert_matches!(cs.poll(), Async::Ready(CollectionEvent::NodeReached(reach_ev)) => { assert_matches!(cs.poll(), Async::Ready(CollectionEvent::NodeReached(reach_ev)) => {
reach_ev.accept(()); reach_ev.accept(());
}); });
Ok(Async::Ready(())) Ok(Async::Ready(()))
})).expect("tokio works"); })).expect("tokio works");
let cs2 = cs.clone();
rt.block_on(future::poll_fn(move || {
if cs2.lock().start_broadcast(&InEvent::NextState).is_not_ready() {
Ok::<_, ()>(Async::NotReady)
} else {
Ok(Async::Ready(()))
}
})).unwrap();
let cs_fut = cs.clone(); let cs_fut = cs.clone();
rt.block_on(future::poll_fn(move || -> Poll<_, ()> { rt.block_on(future::poll_fn(move || -> Poll<_, ()> {
let mut cs = cs_fut.lock(); let mut cs = cs_fut.lock();
cs.broadcast_event(&InEvent::NextState); if cs.complete_broadcast().is_not_ready() {
return Ok(Async::NotReady)
}
assert_matches!(cs.poll(), Async::Ready(CollectionEvent::NodeEvent{peer: _, event}) => { assert_matches!(cs.poll(), Async::Ready(CollectionEvent::NodeEvent{peer: _, event}) => {
assert_matches!(event, OutEvent::Custom("init")); assert_matches!(event, OutEvent::Custom("init"));
}); });
@ -175,20 +195,40 @@ fn events_in_a_node_reaches_the_collection_stream() {
})).expect("tokio works"); })).expect("tokio works");
let cs2 = cs.clone();
rt.block_on(future::poll_fn(move || {
if cs2.lock().start_broadcast(&InEvent::NextState).is_not_ready() {
Ok::<_, ()>(Async::NotReady)
} else {
Ok(Async::Ready(()))
}
})).unwrap();
let cs_fut = cs.clone(); let cs_fut = cs.clone();
rt.block_on(future::poll_fn(move || -> Poll<_, ()> { rt.block_on(future::poll_fn(move || -> Poll<_, ()> {
let mut cs = cs_fut.lock(); let mut cs = cs_fut.lock();
cs.broadcast_event(&InEvent::NextState); if cs.complete_broadcast().is_not_ready() {
return Ok(Async::NotReady)
}
assert_matches!(cs.poll(), Async::Ready(CollectionEvent::NodeEvent{peer: _, event}) => { assert_matches!(cs.poll(), Async::Ready(CollectionEvent::NodeEvent{peer: _, event}) => {
assert_matches!(event, OutEvent::Custom("from handler 1")); assert_matches!(event, OutEvent::Custom("from handler 1"));
}); });
Ok(Async::Ready(())) Ok(Async::Ready(()))
})).expect("tokio works"); })).expect("tokio works");
let cs2 = cs.clone();
rt.block_on(future::poll_fn(move || {
if cs2.lock().start_broadcast(&InEvent::NextState).is_not_ready() {
Ok::<_, ()>(Async::NotReady)
} else {
Ok(Async::Ready(()))
}
})).unwrap();
let cs_fut = cs.clone(); let cs_fut = cs.clone();
rt.block_on(future::poll_fn(move || -> Poll<_, ()> { rt.block_on(future::poll_fn(move || -> Poll<_, ()> {
let mut cs = cs_fut.lock(); let mut cs = cs_fut.lock();
cs.broadcast_event(&InEvent::NextState); if cs.complete_broadcast().is_not_ready() {
return Ok(Async::NotReady)
}
assert_matches!(cs.poll(), Async::Ready(CollectionEvent::NodeEvent{peer: _, event}) => { assert_matches!(cs.poll(), Async::Ready(CollectionEvent::NodeEvent{peer: _, event}) => {
assert_matches!(event, OutEvent::Custom("from handler 2")); assert_matches!(event, OutEvent::Custom("from handler 2"));
}); });
@ -238,13 +278,20 @@ fn task_closed_with_error_when_task_is_connected_yields_node_error() {
let mut rt = Builder::new().core_threads(1).build().unwrap(); let mut rt = Builder::new().core_threads(1).build().unwrap();
// Kick it off // Kick it off
let cs2 = cs.clone();
rt.block_on(future::poll_fn(move || {
if cs2.lock().start_broadcast(&InEvent::NextState).is_not_ready() {
Ok::<_, ()>(Async::NotReady)
} else {
Ok(Async::Ready(()))
}
})).unwrap();
let cs_fut = cs.clone(); let cs_fut = cs.clone();
rt.block_on(future::poll_fn(move || -> Poll<_, ()> { rt.block_on(future::poll_fn(move || -> Poll<_, ()> {
let mut cs = cs_fut.lock(); let mut cs = cs_fut.lock();
assert_matches!(cs.poll(), Async::NotReady); assert_matches!(cs.poll(), Async::NotReady);
// send an event so the Handler errors in two polls // send an event so the Handler errors in two polls
cs.broadcast_event(&InEvent::NextState); Ok(cs.complete_broadcast())
Ok(Async::Ready(()))
})).expect("tokio works"); })).expect("tokio works");
// Accept the new node // Accept the new node

View File

@ -18,7 +18,7 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::muxing::StreamMuxer; use crate::{PeerId, muxing::StreamMuxer};
use crate::nodes::node::{NodeEvent, NodeStream, Substream, Close}; use crate::nodes::node::{NodeEvent, NodeStream, Substream, Close};
use futures::prelude::*; use futures::prelude::*;
use std::{error, fmt, io}; use std::{error, fmt, io};
@ -62,6 +62,29 @@ pub trait NodeHandler {
fn poll(&mut self) -> Poll<NodeHandlerEvent<Self::OutboundOpenInfo, Self::OutEvent>, Self::Error>; fn poll(&mut self) -> Poll<NodeHandlerEvent<Self::OutboundOpenInfo, Self::OutEvent>, Self::Error>;
} }
/// Prototype for a `NodeHandler`.
pub trait IntoNodeHandler<TConnInfo = PeerId> {
/// The node handler.
type Handler: NodeHandler;
/// Builds the node handler.
///
/// The `TConnInfo` is the information about the connection that the handler is going to handle.
/// This is generated by the `Transport` and typically implements the `ConnectionInfo` trait.
fn into_handler(self, remote_conn_info: &TConnInfo) -> Self::Handler;
}
impl<T, TConnInfo> IntoNodeHandler<TConnInfo> for T
where
T: NodeHandler
{
type Handler = Self;
fn into_handler(self, _: &TConnInfo) -> Self {
self
}
}
/// Endpoint for a received substream. /// Endpoint for a received substream.
#[derive(Debug, Copy, Clone, PartialEq, Eq)] #[derive(Debug, Copy, Clone, PartialEq, Eq)]
pub enum NodeHandlerEndpoint<TOutboundOpenInfo> { pub enum NodeHandlerEndpoint<TOutboundOpenInfo> {
@ -71,7 +94,6 @@ pub enum NodeHandlerEndpoint<TOutboundOpenInfo> {
impl<TOutboundOpenInfo> NodeHandlerEndpoint<TOutboundOpenInfo> { impl<TOutboundOpenInfo> NodeHandlerEndpoint<TOutboundOpenInfo> {
/// Returns true for `Dialer`. /// Returns true for `Dialer`.
#[inline]
pub fn is_dialer(&self) -> bool { pub fn is_dialer(&self) -> bool {
match self { match self {
NodeHandlerEndpoint::Dialer(_) => true, NodeHandlerEndpoint::Dialer(_) => true,
@ -80,7 +102,6 @@ impl<TOutboundOpenInfo> NodeHandlerEndpoint<TOutboundOpenInfo> {
} }
/// Returns true for `Listener`. /// Returns true for `Listener`.
#[inline]
pub fn is_listener(&self) -> bool { pub fn is_listener(&self) -> bool {
match self { match self {
NodeHandlerEndpoint::Dialer(_) => false, NodeHandlerEndpoint::Dialer(_) => false,
@ -102,7 +123,6 @@ pub enum NodeHandlerEvent<TOutboundOpenInfo, TCustom> {
/// Event produced by a handler. /// Event produced by a handler.
impl<TOutboundOpenInfo, TCustom> NodeHandlerEvent<TOutboundOpenInfo, TCustom> { impl<TOutboundOpenInfo, TCustom> NodeHandlerEvent<TOutboundOpenInfo, TCustom> {
/// If this is `OutboundSubstreamRequest`, maps the content to something else. /// If this is `OutboundSubstreamRequest`, maps the content to something else.
#[inline]
pub fn map_outbound_open_info<F, I>(self, map: F) -> NodeHandlerEvent<I, TCustom> pub fn map_outbound_open_info<F, I>(self, map: F) -> NodeHandlerEvent<I, TCustom>
where F: FnOnce(TOutboundOpenInfo) -> I where F: FnOnce(TOutboundOpenInfo) -> I
{ {
@ -115,7 +135,6 @@ impl<TOutboundOpenInfo, TCustom> NodeHandlerEvent<TOutboundOpenInfo, TCustom> {
} }
/// If this is `Custom`, maps the content to something else. /// If this is `Custom`, maps the content to something else.
#[inline]
pub fn map_custom<F, I>(self, map: F) -> NodeHandlerEvent<TOutboundOpenInfo, I> pub fn map_custom<F, I>(self, map: F) -> NodeHandlerEvent<TOutboundOpenInfo, I>
where F: FnOnce(TCustom) -> I where F: FnOnce(TCustom) -> I
{ {
@ -159,7 +178,6 @@ where
THandler: NodeHandler<Substream = Substream<TMuxer>>, THandler: NodeHandler<Substream = Substream<TMuxer>>,
{ {
/// Builds a new `HandledNode`. /// Builds a new `HandledNode`.
#[inline]
pub fn new(muxer: TMuxer, handler: THandler) -> Self { pub fn new(muxer: TMuxer, handler: THandler) -> Self {
HandledNode { HandledNode {
node: NodeStream::new(muxer), node: NodeStream::new(muxer),
@ -178,7 +196,6 @@ where
} }
/// Injects an event to the handler. Has no effect if the handler is closing. /// Injects an event to the handler. Has no effect if the handler is closing.
#[inline]
pub fn inject_event(&mut self, event: THandler::InEvent) { pub fn inject_event(&mut self, event: THandler::InEvent) {
self.handler.inject_event(event); self.handler.inject_event(event);
} }
@ -242,7 +259,8 @@ pub enum HandledNodeError<THandlerErr> {
} }
impl<THandlerErr> fmt::Display for HandledNodeError<THandlerErr> impl<THandlerErr> fmt::Display for HandledNodeError<THandlerErr>
where THandlerErr: fmt::Display where
THandlerErr: fmt::Display
{ {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self { match self {
@ -253,7 +271,8 @@ where THandlerErr: fmt::Display
} }
impl<THandlerErr> error::Error for HandledNodeError<THandlerErr> impl<THandlerErr> error::Error for HandledNodeError<THandlerErr>
where THandlerErr: error::Error + 'static where
THandlerErr: error::Error + 'static
{ {
fn source(&self) -> Option<&(dyn error::Error + 'static)> { fn source(&self) -> Option<&(dyn error::Error + 'static)> {
match self { match self {

View File

@ -1,707 +0,0 @@
// Copyright 2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use crate::{
PeerId,
muxing::StreamMuxer,
nodes::{
handled_node::{HandledNode, HandledNodeError, NodeHandler},
node::{Close, Substream}
}
};
use fnv::FnvHashMap;
use futures::{prelude::*, future::Executor, stream, sync::mpsc};
use smallvec::SmallVec;
use std::{
collections::hash_map::{Entry, OccupiedEntry},
error,
fmt,
mem
};
mod tests;
// Implementor notes
// =================
//
// This collection of nodes spawns a task for each individual node to process. This means that
// events happen on the background at the same time as the `HandledNodesTasks` is being polled.
//
// In order to make the API non-racy and avoid issues, we completely separate the state in the
// `HandledNodesTasks` from the states that the task nodes can access. They are only allowed to
// exchange messages. The state in the `HandledNodesTasks` is therefore delayed compared to the
// tasks, and is updated only when `poll()` is called.
//
// The only thing that we must be careful about is substreams, as they are "detached" from the
// state of the `HandledNodesTasks` and allowed to process in parallel. This is why there is no
// "substream closed" event being reported, as it could potentially create confusions and race
// conditions in the user's code. See similar comments in the documentation of `NodeStream`.
/// Implementation of `Stream` that handles a collection of nodes.
pub struct HandledNodesTasks<TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr, TUserData, TConnInfo = PeerId> {
/// A map between active tasks to an unbounded sender, used to control the task. Closing the sender interrupts
/// the task. It is possible that we receive messages from tasks that used to be in this list
/// but no longer are, in which case we should ignore them.
tasks: FnvHashMap<TaskId, (mpsc::UnboundedSender<ExtToInMessage<TInEvent>>, TUserData)>,
/// Identifier for the next task to spawn.
next_task_id: TaskId,
/// List of node tasks to spawn.
// TODO: stronger typing?
to_spawn: SmallVec<[Box<dyn Future<Item = (), Error = ()> + Send>; 8]>,
/// If no tokio executor is available, we move tasks to this list, and futures are polled on
/// the current thread instead.
local_spawns: Vec<Box<dyn Future<Item = (), Error = ()> + Send>>,
/// Sender to emit events to the outside. Meant to be cloned and sent to tasks.
events_tx: mpsc::UnboundedSender<(InToExtMessage<TOutEvent, TIntoHandler, TReachErr, THandlerErr, TConnInfo>, TaskId)>,
/// Receiver side for the events.
events_rx: mpsc::UnboundedReceiver<(InToExtMessage<TOutEvent, TIntoHandler, TReachErr, THandlerErr, TConnInfo>, TaskId)>,
}
impl<TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr, TUserData, TConnInfo> fmt::Debug for
HandledNodesTasks<TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr, TUserData, TConnInfo>
where
TUserData: fmt::Debug
{
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> {
f.debug_map()
.entries(self.tasks.iter().map(|(id, (_, ud))| (id, ud)))
.finish()
}
}
/// Error that can happen in a task.
#[derive(Debug)]
pub enum TaskClosedEvent<TReachErr, THandlerErr> {
/// An error happend while we were trying to reach the node.
Reach(TReachErr),
/// An error happened after the node has been reached.
Node(HandledNodeError<THandlerErr>),
}
impl<TReachErr, THandlerErr> fmt::Display for TaskClosedEvent<TReachErr, THandlerErr>
where
TReachErr: fmt::Display,
THandlerErr: fmt::Display,
{
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self {
TaskClosedEvent::Reach(err) => write!(f, "{}", err),
TaskClosedEvent::Node(err) => write!(f, "{}", err),
}
}
}
impl<TReachErr, THandlerErr> error::Error for TaskClosedEvent<TReachErr, THandlerErr>
where
TReachErr: error::Error + 'static,
THandlerErr: error::Error + 'static
{
fn source(&self) -> Option<&(dyn error::Error + 'static)> {
match self {
TaskClosedEvent::Reach(err) => Some(err),
TaskClosedEvent::Node(err) => Some(err),
}
}
}
/// Prototype for a `NodeHandler`.
pub trait IntoNodeHandler<TConnInfo = PeerId> {
/// The node handler.
type Handler: NodeHandler;
/// Builds the node handler.
///
/// The `TConnInfo` is the information about the connection that the handler is going to handle.
/// This is generated by the `Transport` and typically implements the `ConnectionInfo` trait.
fn into_handler(self, remote_conn_info: &TConnInfo) -> Self::Handler;
}
impl<T, TConnInfo> IntoNodeHandler<TConnInfo> for T
where T: NodeHandler
{
type Handler = Self;
#[inline]
fn into_handler(self, _: &TConnInfo) -> Self {
self
}
}
/// Event that can happen on the `HandledNodesTasks`.
#[derive(Debug)]
pub enum HandledNodesEvent<'a, TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr, TUserData, TConnInfo = PeerId> {
/// A task has been closed.
///
/// This happens once the node handler closes or an error happens.
// TODO: send back undelivered events?
TaskClosed {
/// The task that has been closed.
task: ClosedTask<TInEvent, TUserData>,
/// What happened.
result: TaskClosedEvent<TReachErr, THandlerErr>,
/// If the task closed before reaching the node, this contains the handler that was passed
/// to `add_reach_attempt`.
handler: Option<TIntoHandler>,
},
/// A task has successfully connected to a node.
NodeReached {
/// The task that succeeded.
task: Task<'a, TInEvent, TUserData>,
/// Identifier of the node.
conn_info: TConnInfo,
},
/// A task has produced an event.
NodeEvent {
/// The task that produced the event.
task: Task<'a, TInEvent, TUserData>,
/// The produced event.
event: TOutEvent,
},
}
/// Identifier for a future that attempts to reach a node.
#[derive(Debug, Copy, Clone, Hash, PartialEq, Eq, PartialOrd, Ord)]
pub struct TaskId(usize);
impl<TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr, TUserData, TConnInfo>
HandledNodesTasks<TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr, TUserData, TConnInfo>
{
/// Creates a new empty collection.
#[inline]
pub fn new() -> Self {
let (events_tx, events_rx) = mpsc::unbounded();
HandledNodesTasks {
tasks: Default::default(),
next_task_id: TaskId(0),
to_spawn: SmallVec::new(),
local_spawns: Vec::new(),
events_tx,
events_rx,
}
}
/// Adds to the collection a future that tries to reach a node.
///
/// This method spawns a task dedicated to resolving this future and processing the node's
/// events.
pub fn add_reach_attempt<TFut, TMuxer>(&mut self, future: TFut, user_data: TUserData, handler: TIntoHandler) -> TaskId
where
TFut: Future<Item = (TConnInfo, TMuxer), Error = TReachErr> + Send + 'static,
TIntoHandler: IntoNodeHandler<TConnInfo> + Send + 'static,
TIntoHandler::Handler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
TReachErr: error::Error + Send + 'static,
THandlerErr: error::Error + Send + 'static,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
<TIntoHandler::Handler as NodeHandler>::OutboundOpenInfo: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send + 'static,
TConnInfo: Send + 'static,
{
let task_id = self.next_task_id;
self.next_task_id.0 += 1;
let (tx, rx) = mpsc::unbounded();
self.tasks.insert(task_id, (tx, user_data));
let task = Box::new(NodeTask {
taken_over: SmallVec::new(),
inner: NodeTaskInner::Future {
future,
handler,
events_buffer: Vec::new(),
},
events_tx: self.events_tx.clone(),
in_events_rx: rx.fuse(),
id: task_id,
});
self.to_spawn.push(task);
task_id
}
/// Adds an existing connection to a node to the collection.
///
/// This method spawns a task dedicated to processing the node's events.
///
/// No `NodeReached` event will be emitted for this task, since the node has already been
/// reached.
pub fn add_connection<TMuxer, THandler>(&mut self, user_data: TUserData, muxer: TMuxer, handler: THandler) -> TaskId
where
TIntoHandler: IntoNodeHandler<TConnInfo, Handler = THandler> + Send + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
TReachErr: error::Error + Send + 'static,
THandlerErr: error::Error + Send + 'static,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
<TIntoHandler::Handler as NodeHandler>::OutboundOpenInfo: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send + 'static,
TConnInfo: Send + 'static,
{
let task_id = self.next_task_id;
self.next_task_id.0 += 1;
let (tx, rx) = mpsc::unbounded();
self.tasks.insert(task_id, (tx, user_data));
let task: NodeTask<futures::future::Empty<_, _>, _, _, _, _, _, _> = NodeTask {
taken_over: SmallVec::new(),
inner: NodeTaskInner::Node(HandledNode::new(muxer, handler)),
events_tx: self.events_tx.clone(),
in_events_rx: rx.fuse(),
id: task_id,
};
self.to_spawn.push(Box::new(task));
task_id
}
/// Sends an event to all the tasks, including the pending ones.
pub fn broadcast_event(&mut self, event: &TInEvent)
where TInEvent: Clone,
{
for (sender, _) in self.tasks.values() {
// Note: it is possible that sending an event fails if the background task has already
// finished, but the local state hasn't reflected that yet because it hasn't been
// polled. This is not an error situation.
let _ = sender.unbounded_send(ExtToInMessage::HandlerEvent(event.clone()));
}
}
/// Grants access to an object that allows controlling a task of the collection.
///
/// Returns `None` if the task id is invalid.
#[inline]
pub fn task(&mut self, id: TaskId) -> Option<Task<'_, TInEvent, TUserData>> {
match self.tasks.entry(id) {
Entry::Occupied(inner) => Some(Task { inner }),
Entry::Vacant(_) => None,
}
}
/// Returns a list of all the active tasks.
#[inline]
pub fn tasks<'a>(&'a self) -> impl Iterator<Item = TaskId> + 'a {
self.tasks.keys().cloned()
}
/// Provides an API similar to `Stream`, except that it cannot produce an error.
pub fn poll(&mut self) -> Async<HandledNodesEvent<TInEvent, TOutEvent, TIntoHandler, TReachErr, THandlerErr, TUserData, TConnInfo>> {
let (message, task_id) = match self.poll_inner() {
Async::Ready(r) => r,
Async::NotReady => return Async::NotReady,
};
Async::Ready(match message {
InToExtMessage::NodeEvent(event) => {
HandledNodesEvent::NodeEvent {
task: match self.tasks.entry(task_id) {
Entry::Occupied(inner) => Task { inner },
Entry::Vacant(_) => panic!("poll_inner only returns valid TaskIds; QED")
},
event
}
},
InToExtMessage::NodeReached(conn_info) => {
HandledNodesEvent::NodeReached {
task: match self.tasks.entry(task_id) {
Entry::Occupied(inner) => Task { inner },
Entry::Vacant(_) => panic!("poll_inner only returns valid TaskIds; QED")
},
conn_info
}
},
InToExtMessage::TaskClosed(result, handler) => {
let (channel, user_data) = self.tasks.remove(&task_id)
.expect("poll_inner only returns valid TaskIds; QED");
HandledNodesEvent::TaskClosed {
task: ClosedTask {
id: task_id,
channel,
user_data,
},
result,
handler,
}
},
})
}
/// Since non-lexical lifetimes still don't work very well in Rust at the moment, we have to
/// split `poll()` in two. This method returns an `InToExtMessage` that is guaranteed to come
/// from an alive task.
// TODO: look into merging with `poll()`
fn poll_inner(&mut self) -> Async<(InToExtMessage<TOutEvent, TIntoHandler, TReachErr, THandlerErr, TConnInfo>, TaskId)> {
for to_spawn in self.to_spawn.drain() {
// We try to use the default executor, but fall back to polling the task manually if
// no executor is available. This makes it possible to use the core in environments
// outside of tokio.
let executor = tokio_executor::DefaultExecutor::current();
if let Err(err) = executor.execute(to_spawn) {
self.local_spawns.push(err.into_future());
}
}
for n in (0..self.local_spawns.len()).rev() {
let mut task = self.local_spawns.swap_remove(n);
match task.poll() {
Ok(Async::Ready(())) => {},
Ok(Async::NotReady) => self.local_spawns.push(task),
// It would normally be desirable to either report or log when a background task
// errors. However the default tokio executor doesn't do anything in case of error,
// and therefore we mimic this behaviour by also not doing anything.
Err(()) => {}
}
}
loop {
match self.events_rx.poll() {
Ok(Async::Ready(Some((message, task_id)))) => {
// If the task id is no longer in `self.tasks`, that means that the user called
// `close()` on this task earlier. Therefore no new event should be generated
// for this task.
if self.tasks.contains_key(&task_id) {
break Async::Ready((message, task_id));
}
}
Ok(Async::NotReady) => {
break Async::NotReady;
}
Ok(Async::Ready(None)) => {
unreachable!("The sender is in self as well, therefore the receiver never \
closes.")
},
Err(()) => unreachable!("An unbounded receiver never errors"),
}
}
}
}
/// Access to a task in the collection.
pub struct Task<'a, TInEvent, TUserData> {
inner: OccupiedEntry<'a, TaskId, (mpsc::UnboundedSender<ExtToInMessage<TInEvent>>, TUserData)>,
}
impl<'a, TInEvent, TUserData> Task<'a, TInEvent, TUserData> {
/// Sends an event to the given node.
// TODO: report back on delivery
#[inline]
pub fn send_event(&mut self, event: TInEvent) {
// It is possible that the sender is closed if the background task has already finished
// but the local state hasn't been updated yet because we haven't been polled in the
// meanwhile.
let _ = self.inner.get_mut().0.unbounded_send(ExtToInMessage::HandlerEvent(event));
}
/// Returns the user data associated with the task.
pub fn user_data(&self) -> &TUserData {
&self.inner.get().1
}
/// Returns the user data associated with the task.
pub fn user_data_mut(&mut self) -> &mut TUserData {
&mut self.inner.get_mut().1
}
/// Returns the task id.
#[inline]
pub fn id(&self) -> TaskId {
*self.inner.key()
}
/// Closes the task. Returns the user data.
///
/// No further event will be generated for this task, but the connection inside the task will
/// continue to run until the `ClosedTask` is destroyed.
pub fn close(self) -> ClosedTask<TInEvent, TUserData> {
let id = *self.inner.key();
let (channel, user_data) = self.inner.remove();
ClosedTask { id, channel, user_data }
}
/// Gives ownership of a closed task. As soon as our task (`self`) has some acknowledgment from
/// the remote that its connection is alive, it will close the connection with `other`.
pub fn take_over(&mut self, other: ClosedTask<TInEvent, TUserData>) -> TUserData {
// It is possible that the sender is closed if the background task has already finished
// but the local state hasn't been updated yet because we haven't been polled in the
// meanwhile.
let _ = self.inner.get_mut().0.unbounded_send(ExtToInMessage::TakeOver(other.channel));
other.user_data
}
}
impl<'a, TInEvent, TUserData> fmt::Debug for Task<'a, TInEvent, TUserData>
where
TUserData: fmt::Debug,
{
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> {
f.debug_tuple("Task")
.field(&self.id())
.field(self.user_data())
.finish()
}
}
/// Task after it has been closed. The connection to the remote is potentially still going on, but
/// no new event for this task will be received.
pub struct ClosedTask<TInEvent, TUserData> {
/// Identifier of the task that closed. No longer corresponds to anything, but can be reported
/// to the user.
id: TaskId,
/// The channel to the task. The task will continue to work for as long as this channel is
/// alive, but events produced by it are ignored.
channel: mpsc::UnboundedSender<ExtToInMessage<TInEvent>>,
/// The data provided by the user.
user_data: TUserData,
}
impl<TInEvent, TUserData> ClosedTask<TInEvent, TUserData> {
/// Returns the task id. Note that this task is no longer part of the collection, and therefore
/// calling `task()` with this ID will fail.
#[inline]
pub fn id(&self) -> TaskId {
self.id
}
/// Returns the user data associated with the task.
pub fn user_data(&self) -> &TUserData {
&self.user_data
}
/// Returns the user data associated with the task.
pub fn user_data_mut(&mut self) -> &mut TUserData {
&mut self.user_data
}
/// Finish destroying the task and yield the user data. This closes the connection to the
/// remote.
pub fn into_user_data(self) -> TUserData {
self.user_data
}
}
impl<TInEvent, TUserData> fmt::Debug for ClosedTask<TInEvent, TUserData>
where
TUserData: fmt::Debug,
{
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> {
f.debug_tuple("ClosedTask")
.field(&self.id)
.field(&self.user_data)
.finish()
}
}
/// Message to transmit from the public API to a task.
#[derive(Debug)]
enum ExtToInMessage<TInEvent> {
/// An event to transmit to the node handler.
HandlerEvent(TInEvent),
/// When received, stores the parameter inside the task and keeps it alive until we have an
/// acknowledgment that the remote has accepted our handshake.
TakeOver(mpsc::UnboundedSender<ExtToInMessage<TInEvent>>),
}
/// Message to transmit from a task to the public API.
#[derive(Debug)]
enum InToExtMessage<TOutEvent, TIntoHandler, TReachErr, THandlerErr, TConnInfo> {
/// A connection to a node has succeeded.
NodeReached(TConnInfo),
/// The task closed.
TaskClosed(TaskClosedEvent<TReachErr, THandlerErr>, Option<TIntoHandler>),
/// An event from the node.
NodeEvent(TOutEvent),
}
/// Implementation of `Future` that handles a single node, and all the communications between
/// the various components of the `HandledNodesTasks`.
struct NodeTask<TFut, TMuxer, TIntoHandler, TInEvent, TOutEvent, TReachErr, TConnInfo>
where
TMuxer: StreamMuxer,
TIntoHandler: IntoNodeHandler<TConnInfo>,
TIntoHandler::Handler: NodeHandler<Substream = Substream<TMuxer>>,
{
/// Sender to transmit events to the outside.
events_tx: mpsc::UnboundedSender<(InToExtMessage<TOutEvent, TIntoHandler, TReachErr, <TIntoHandler::Handler as NodeHandler>::Error, TConnInfo>, TaskId)>,
/// Receiving end for events sent from the main `HandledNodesTasks`.
in_events_rx: stream::Fuse<mpsc::UnboundedReceiver<ExtToInMessage<TInEvent>>>,
/// Inner state of the `NodeTask`.
inner: NodeTaskInner<TFut, TMuxer, TIntoHandler, TInEvent, TConnInfo>,
/// Identifier of the attempt.
id: TaskId,
/// Channels to keep alive for as long as we don't have an acknowledgment from the remote.
taken_over: SmallVec<[mpsc::UnboundedSender<ExtToInMessage<TInEvent>>; 1]>,
}
enum NodeTaskInner<TFut, TMuxer, TIntoHandler, TInEvent, TConnInfo>
where
TMuxer: StreamMuxer,
TIntoHandler: IntoNodeHandler<TConnInfo>,
TIntoHandler::Handler: NodeHandler<Substream = Substream<TMuxer>>,
{
/// Future to resolve to connect to the node.
Future {
/// The future that will attempt to reach the node.
future: TFut,
/// The handler that will be used to build the `HandledNode`.
handler: TIntoHandler,
/// While we are dialing the future, we need to buffer the events received on
/// `in_events_rx` so that they get delivered once dialing succeeds. We can't simply leave
/// events in `in_events_rx` because we have to detect if it gets closed.
events_buffer: Vec<TInEvent>,
},
/// Fully functional node.
Node(HandledNode<TMuxer, TIntoHandler::Handler>),
/// Node closing.
Closing(Close<TMuxer>),
/// A panic happened while polling.
Poisoned,
}
impl<TFut, TMuxer, TIntoHandler, TInEvent, TOutEvent, TReachErr, TConnInfo> Future for
NodeTask<TFut, TMuxer, TIntoHandler, TInEvent, TOutEvent, TReachErr, TConnInfo>
where
TMuxer: StreamMuxer,
TFut: Future<Item = (TConnInfo, TMuxer), Error = TReachErr>,
TIntoHandler: IntoNodeHandler<TConnInfo>,
TIntoHandler::Handler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent>,
{
type Item = ();
type Error = ();
fn poll(&mut self) -> Poll<(), ()> {
'outer_loop: loop {
match mem::replace(&mut self.inner, NodeTaskInner::Poisoned) {
// First possibility: we are still trying to reach a node.
NodeTaskInner::Future { mut future, handler, mut events_buffer } => {
// If self.in_events_rx is closed, we stop the task.
loop {
match self.in_events_rx.poll() {
Ok(Async::Ready(None)) => return Ok(Async::Ready(())),
Ok(Async::Ready(Some(ExtToInMessage::HandlerEvent(event)))) => {
events_buffer.push(event)
},
Ok(Async::Ready(Some(ExtToInMessage::TakeOver(take_over)))) => {
self.taken_over.push(take_over);
},
Ok(Async::NotReady) => break,
Err(_) => unreachable!("An UnboundedReceiver never errors"),
}
}
// Check whether dialing succeeded.
match future.poll() {
Ok(Async::Ready((conn_info, muxer))) => {
let mut node = HandledNode::new(muxer, handler.into_handler(&conn_info));
let event = InToExtMessage::NodeReached(conn_info);
for event in events_buffer {
node.inject_event(event);
}
let _ = self.events_tx.unbounded_send((event, self.id));
self.inner = NodeTaskInner::Node(node);
}
Ok(Async::NotReady) => {
self.inner = NodeTaskInner::Future { future, handler, events_buffer };
return Ok(Async::NotReady);
},
Err(err) => {
// End the task
let event = InToExtMessage::TaskClosed(TaskClosedEvent::Reach(err), Some(handler));
let _ = self.events_tx.unbounded_send((event, self.id));
return Ok(Async::Ready(()));
}
}
},
// Second possibility: we have a node.
NodeTaskInner::Node(mut node) => {
// Start by handling commands received from the outside of the task.
if !self.in_events_rx.is_done() {
loop {
match self.in_events_rx.poll() {
Ok(Async::NotReady) => break,
Ok(Async::Ready(Some(ExtToInMessage::HandlerEvent(event)))) => {
node.inject_event(event)
},
Ok(Async::Ready(Some(ExtToInMessage::TakeOver(take_over)))) => {
self.taken_over.push(take_over);
},
Ok(Async::Ready(None)) => {
// Node closed by the external API; start closing.
self.inner = NodeTaskInner::Closing(node.close());
continue 'outer_loop;
}
Err(()) => unreachable!("An unbounded receiver never errors"),
}
}
}
// Process the node.
loop {
if !self.taken_over.is_empty() && node.is_remote_acknowledged() {
self.taken_over.clear();
}
match node.poll() {
Ok(Async::NotReady) => {
self.inner = NodeTaskInner::Node(node);
return Ok(Async::NotReady);
},
Ok(Async::Ready(event)) => {
let event = InToExtMessage::NodeEvent(event);
let _ = self.events_tx.unbounded_send((event, self.id));
}
Err(err) => {
let event = InToExtMessage::TaskClosed(TaskClosedEvent::Node(err), None);
let _ = self.events_tx.unbounded_send((event, self.id));
return Ok(Async::Ready(())); // End the task.
}
}
}
},
NodeTaskInner::Closing(mut closing) => {
match closing.poll() {
Ok(Async::Ready(())) | Err(_) => {
return Ok(Async::Ready(())); // End the task.
},
Ok(Async::NotReady) => {
self.inner = NodeTaskInner::Closing(closing);
return Ok(Async::NotReady);
}
}
},
// This happens if a previous poll has ended unexpectedly. The API of futures
// guarantees that we shouldn't be polled again.
NodeTaskInner::Poisoned => panic!("the node task panicked or errored earlier")
}
}
}
}

View File

@ -28,7 +28,7 @@
pub mod collection; pub mod collection;
pub mod handled_node; pub mod handled_node;
pub mod handled_node_tasks; pub mod tasks;
pub mod listeners; pub mod listeners;
pub mod node; pub mod node;
pub mod raw_swarm; pub mod raw_swarm;

View File

@ -28,13 +28,14 @@ use crate::{
CollectionReachEvent, CollectionReachEvent,
CollectionStream, CollectionStream,
ConnectionInfo, ConnectionInfo,
ReachAttemptId ReachAttemptId,
InterruptedReachAttempt
}, },
handled_node::{ handled_node::{
HandledNodeError, HandledNodeError,
NodeHandler NodeHandler
}, },
handled_node_tasks::IntoNodeHandler, handled_node::IntoNodeHandler,
node::Substream node::Substream
}, },
nodes::listeners::{ListenersEvent, ListenersStream}, nodes::listeners::{ListenersEvent, ListenersStream},
@ -50,6 +51,8 @@ use std::{
num::NonZeroUsize, num::NonZeroUsize,
}; };
pub use crate::nodes::collection::StartTakeOver;
mod tests; mod tests;
/// Implementation of `Stream` that handles the nodes. /// Implementation of `Stream` that handles the nodes.
@ -69,6 +72,16 @@ where
/// Max numer of incoming connections. /// Max numer of incoming connections.
incoming_limit: Option<u32>, incoming_limit: Option<u32>,
/// Unfinished take over message to be delivered.
///
/// If the pair's second element is `AsyncSink::NotReady`, the take over
/// message has yet to be sent using `PeerMut::start_take_over`.
///
/// If the pair's second element is `AsyncSink::Ready`, the take over
/// message has been sent and needs to be flushed using
/// `PeerMut::complete_take_over`.
take_over_to_complete: Option<(TPeerId, AsyncSink<InterruptedReachAttempt<TInEvent, (TConnInfo, ConnectedPoint), ()>>)>
} }
impl<TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId> fmt::Debug for impl<TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId> fmt::Debug for
@ -84,6 +97,7 @@ where
.field("active_nodes", &self.active_nodes) .field("active_nodes", &self.active_nodes)
.field("reach_attempts", &self.reach_attempts) .field("reach_attempts", &self.reach_attempts)
.field("incoming_limit", &self.incoming_limit) .field("incoming_limit", &self.incoming_limit)
.field("take_over_to_complete", &self.take_over_to_complete)
.finish() .finish()
} }
} }
@ -555,7 +569,6 @@ where
TPeerId: Eq + Hash + Clone + Send + 'static, TPeerId: Eq + Hash + Clone + Send + 'static,
{ {
/// Starts processing the incoming connection and sets the handler to use for it. /// Starts processing the incoming connection and sets the handler to use for it.
#[inline]
pub fn accept(self, handler: THandler) { pub fn accept(self, handler: THandler) {
self.accept_with_builder(|_| handler) self.accept_with_builder(|_| handler)
} }
@ -592,7 +605,6 @@ impl<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>
where TTrans: Transport where TTrans: Transport
{ {
/// Returns the `IncomingInfo` corresponding to this incoming connection. /// Returns the `IncomingInfo` corresponding to this incoming connection.
#[inline]
pub fn info(&self) -> IncomingInfo<'_> { pub fn info(&self) -> IncomingInfo<'_> {
IncomingInfo { IncomingInfo {
listen_addr: &self.listen_addr, listen_addr: &self.listen_addr,
@ -601,19 +613,16 @@ where TTrans: Transport
} }
/// Address of the listener that received the connection. /// Address of the listener that received the connection.
#[inline]
pub fn listen_addr(&self) -> &Multiaddr { pub fn listen_addr(&self) -> &Multiaddr {
&self.listen_addr &self.listen_addr
} }
/// Address used to send back data to the dialer. /// Address used to send back data to the dialer.
#[inline]
pub fn send_back_addr(&self) -> &Multiaddr { pub fn send_back_addr(&self) -> &Multiaddr {
&self.send_back_addr &self.send_back_addr
} }
/// Builds the `ConnectedPoint` corresponding to the incoming connection. /// Builds the `ConnectedPoint` corresponding to the incoming connection.
#[inline]
pub fn to_connected_point(&self) -> ConnectedPoint { pub fn to_connected_point(&self) -> ConnectedPoint {
self.info().to_connected_point() self.info().to_connected_point()
} }
@ -630,7 +639,6 @@ pub struct IncomingInfo<'a> {
impl<'a> IncomingInfo<'a> { impl<'a> IncomingInfo<'a> {
/// Builds the `ConnectedPoint` corresponding to the incoming connection. /// Builds the `ConnectedPoint` corresponding to the incoming connection.
#[inline]
pub fn to_connected_point(&self) -> ConnectedPoint { pub fn to_connected_point(&self) -> ConnectedPoint {
ConnectedPoint::Listener { ConnectedPoint::Listener {
listen_addr: self.listen_addr.clone(), listen_addr: self.listen_addr.clone(),
@ -652,7 +660,6 @@ where
TPeerId: Eq + Hash + Clone, TPeerId: Eq + Hash + Clone,
{ {
/// Creates a new node events stream. /// Creates a new node events stream.
#[inline]
pub fn new(transport: TTrans, local_peer_id: TPeerId) -> Self { pub fn new(transport: TTrans, local_peer_id: TPeerId) -> Self {
// TODO: with_capacity? // TODO: with_capacity?
RawSwarm { RawSwarm {
@ -665,11 +672,11 @@ where
connected_points: Default::default(), connected_points: Default::default(),
}, },
incoming_limit: None, incoming_limit: None,
take_over_to_complete: None
} }
} }
/// Creates a new node event stream with incoming connections limit. /// Creates a new node event stream with incoming connections limit.
#[inline]
pub fn new_with_incoming_limit(transport: TTrans, pub fn new_with_incoming_limit(transport: TTrans,
local_peer_id: TPeerId, incoming_limit: Option<u32>) -> Self local_peer_id: TPeerId, incoming_limit: Option<u32>) -> Self
{ {
@ -683,17 +690,16 @@ where
other_reach_attempts: Vec::new(), other_reach_attempts: Vec::new(),
connected_points: Default::default(), connected_points: Default::default(),
}, },
take_over_to_complete: None
} }
} }
/// Returns the transport passed when building this object. /// Returns the transport passed when building this object.
#[inline]
pub fn transport(&self) -> &TTrans { pub fn transport(&self) -> &TTrans {
self.listeners.transport() self.listeners.transport()
} }
/// Start listening on the given multiaddress. /// Start listening on the given multiaddress.
#[inline]
pub fn listen_on(&mut self, addr: Multiaddr) -> Result<(), TransportError<TTrans::Error>> { pub fn listen_on(&mut self, addr: Multiaddr) -> Result<(), TransportError<TTrans::Error>> {
self.listeners.listen_on(addr) self.listeners.listen_on(addr)
} }
@ -704,7 +710,6 @@ where
} }
/// Returns limit on incoming connections. /// Returns limit on incoming connections.
#[inline]
pub fn incoming_limit(&self) -> Option<u32> { pub fn incoming_limit(&self) -> Option<u32> {
self.incoming_limit self.incoming_limit
} }
@ -716,7 +721,6 @@ where
/// example with the identify protocol. /// example with the identify protocol.
/// ///
/// For each listener, calls `nat_traversal` with the observed address and returns the outcome. /// For each listener, calls `nat_traversal` with the observed address and returns the outcome.
#[inline]
pub fn nat_traversal<'a>(&'a self, observed_addr: &'a Multiaddr) pub fn nat_traversal<'a>(&'a self, observed_addr: &'a Multiaddr)
-> impl Iterator<Item = Multiaddr> + 'a -> impl Iterator<Item = Multiaddr> + 'a
where where
@ -729,7 +733,6 @@ where
/// Returns the peer id of the local node. /// Returns the peer id of the local node.
/// ///
/// This is the same value as was passed to `new()`. /// This is the same value as was passed to `new()`.
#[inline]
pub fn local_peer_id(&self) -> &TPeerId { pub fn local_peer_id(&self) -> &TPeerId {
&self.reach_attempts.local_peer_id &self.reach_attempts.local_peer_id
} }
@ -776,7 +779,6 @@ where
/// We don't know anything about these connections yet, so all we can do is know how many of /// We don't know anything about these connections yet, so all we can do is know how many of
/// them we have. /// them we have.
#[deprecated(note = "Use incoming_negotiated().count() instead")] #[deprecated(note = "Use incoming_negotiated().count() instead")]
#[inline]
pub fn num_incoming_negotiated(&self) -> usize { pub fn num_incoming_negotiated(&self) -> usize {
self.reach_attempts.other_reach_attempts self.reach_attempts.other_reach_attempts
.iter() .iter()
@ -786,7 +788,6 @@ where
/// Returns the list of incoming connections that are currently in the process of being /// Returns the list of incoming connections that are currently in the process of being
/// negotiated. We don't know the `PeerId` of these nodes yet. /// negotiated. We don't know the `PeerId` of these nodes yet.
#[inline]
pub fn incoming_negotiated(&self) -> impl Iterator<Item = IncomingInfo<'_>> { pub fn incoming_negotiated(&self) -> impl Iterator<Item = IncomingInfo<'_>> {
self.reach_attempts self.reach_attempts
.other_reach_attempts .other_reach_attempts
@ -801,12 +802,21 @@ where
}) })
} }
/// Sends an event to all nodes. /// Start sending an event to all nodes.
#[inline] ///
pub fn broadcast_event(&mut self, event: &TInEvent) /// Make sure to complete the broadcast with `complete_broadcast`.
where TInEvent: Clone, #[must_use]
pub fn start_broadcast(&mut self, event: &TInEvent) -> AsyncSink<()>
where
TInEvent: Clone
{ {
self.active_nodes.broadcast_event(event) self.active_nodes.start_broadcast(event)
}
/// Complete a broadcast initiated with `start_broadcast`.
#[must_use]
pub fn complete_broadcast(&mut self) -> Async<()> {
self.active_nodes.complete_broadcast()
} }
/// Returns a list of all the peers we are currently connected to. /// Returns a list of all the peers we are currently connected to.
@ -844,7 +854,6 @@ where
} }
/// Grants access to a struct that represents a peer. /// Grants access to a struct that represents a peer.
#[inline]
pub fn peer(&mut self, peer_id: TPeerId) -> Peer<'_, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId> { pub fn peer(&mut self, peer_id: TPeerId) -> Peer<'_, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId> {
if peer_id == self.reach_attempts.local_peer_id { if peer_id == self.reach_attempts.local_peer_id {
return Peer::LocalNode; return Peer::LocalNode;
@ -983,6 +992,24 @@ where
} }
} }
// Attempt to deliver any pending take over messages.
if let Some((id, interrupted)) = self.take_over_to_complete.take() {
if let Some(mut peer) = self.active_nodes.peer_mut(&id) {
if let AsyncSink::NotReady(i) = interrupted {
if let StartTakeOver::NotReady(i) = peer.start_take_over(i) {
self.take_over_to_complete = Some((id, AsyncSink::NotReady(i)))
} else if let Ok(Async::NotReady) = peer.complete_take_over() {
self.take_over_to_complete = Some((id, AsyncSink::Ready))
}
} else if let Ok(Async::NotReady) = peer.complete_take_over() {
self.take_over_to_complete = Some((id, AsyncSink::Ready))
}
}
}
if self.take_over_to_complete.is_some() {
return Async::NotReady
}
// Poll the existing nodes. // Poll the existing nodes.
let (action, out_event); let (action, out_event);
match self.active_nodes.poll() { match self.active_nodes.poll() {
@ -1035,7 +1062,15 @@ where
interrupt or when a reach attempt succeeds or errors; therefore the \ interrupt or when a reach attempt succeeds or errors; therefore the \
out_reach_attempts should always be in sync with the actual \ out_reach_attempts should always be in sync with the actual \
attempts; QED"); attempts; QED");
self.active_nodes.peer_mut(&peer_id).unwrap().take_over(interrupted); let mut peer = self.active_nodes.peer_mut(&peer_id).unwrap();
if let StartTakeOver::NotReady(i) = peer.start_take_over(interrupted) {
self.take_over_to_complete = Some((peer_id, AsyncSink::NotReady(i)));
return Async::NotReady
}
if let Ok(Async::NotReady) = peer.complete_take_over() {
self.take_over_to_complete = Some((peer_id, AsyncSink::Ready));
return Async::NotReady
}
} }
Async::Ready(out_event) Async::Ready(out_event)
@ -1207,7 +1242,6 @@ where
/// ///
/// This means that if `local` and `other` both dial each other, the connection from `local` should /// This means that if `local` and `other` both dial each other, the connection from `local` should
/// be kept and the one from `other` will be dropped. /// be kept and the one from `other` will be dropped.
#[inline]
fn has_dial_prio<TPeerId>(local: &TPeerId, other: &TPeerId) -> bool fn has_dial_prio<TPeerId>(local: &TPeerId, other: &TPeerId) -> bool
where where
TPeerId: AsRef<[u8]>, TPeerId: AsRef<[u8]>,
@ -1412,7 +1446,6 @@ where
TPeerId: Eq + Hash + Clone + Send + 'static, TPeerId: Eq + Hash + Clone + Send + 'static,
{ {
/// If we are connected, returns the `PeerConnected`. /// If we are connected, returns the `PeerConnected`.
#[inline]
pub fn into_connected(self) -> Option<PeerConnected<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>> { pub fn into_connected(self) -> Option<PeerConnected<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>> {
match self { match self {
Peer::Connected(peer) => Some(peer), Peer::Connected(peer) => Some(peer),
@ -1421,7 +1454,6 @@ where
} }
/// If a connection is pending, returns the `PeerPendingConnect`. /// If a connection is pending, returns the `PeerPendingConnect`.
#[inline]
pub fn into_pending_connect(self) -> Option<PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>> { pub fn into_pending_connect(self) -> Option<PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>> {
match self { match self {
Peer::PendingConnect(peer) => Some(peer), Peer::PendingConnect(peer) => Some(peer),
@ -1430,7 +1462,6 @@ where
} }
/// If we are not connected, returns the `PeerNotConnected`. /// If we are not connected, returns the `PeerNotConnected`.
#[inline]
pub fn into_not_connected(self) -> Option<PeerNotConnected<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>> { pub fn into_not_connected(self) -> Option<PeerNotConnected<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>> {
match self { match self {
Peer::NotConnected(peer) => Some(peer), Peer::NotConnected(peer) => Some(peer),
@ -1444,7 +1475,6 @@ where
/// the whole connection is immediately closed. /// the whole connection is immediately closed.
/// ///
/// Returns an error if we are `LocalNode`. /// Returns an error if we are `LocalNode`.
#[inline]
pub fn or_connect(self, addr: Multiaddr, handler: THandler) pub fn or_connect(self, addr: Multiaddr, handler: THandler)
-> Result<PeerPotentialConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>, Self> -> Result<PeerPotentialConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>, Self>
{ {
@ -1458,7 +1488,6 @@ where
/// the whole connection is immediately closed. /// the whole connection is immediately closed.
/// ///
/// Returns an error if we are `LocalNode`. /// Returns an error if we are `LocalNode`.
#[inline]
pub fn or_connect_with<TFn>(self, addr: TFn, handler: THandler) pub fn or_connect_with<TFn>(self, addr: TFn, handler: THandler)
-> Result<PeerPotentialConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>, Self> -> Result<PeerPotentialConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>, Self>
where where
@ -1497,7 +1526,6 @@ where
{ {
/// Closes the connection or the connection attempt. /// Closes the connection or the connection attempt.
// TODO: consider returning a `PeerNotConnected` // TODO: consider returning a `PeerNotConnected`
#[inline]
pub fn close(self) { pub fn close(self) {
match self { match self {
PeerPotentialConnect::Connected(peer) => peer.close(), PeerPotentialConnect::Connected(peer) => peer.close(),
@ -1506,7 +1534,6 @@ where
} }
/// If we are connected, returns the `PeerConnected`. /// If we are connected, returns the `PeerConnected`.
#[inline]
pub fn into_connected(self) -> Option<PeerConnected<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>> { pub fn into_connected(self) -> Option<PeerConnected<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>> {
match self { match self {
PeerPotentialConnect::Connected(peer) => Some(peer), PeerPotentialConnect::Connected(peer) => Some(peer),
@ -1515,7 +1542,6 @@ where
} }
/// If a connection is pending, returns the `PeerPendingConnect`. /// If a connection is pending, returns the `PeerPendingConnect`.
#[inline]
pub fn into_pending_connect(self) -> Option<PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>> { pub fn into_pending_connect(self) -> Option<PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>> {
match self { match self {
PeerPotentialConnect::PendingConnect(peer) => Some(peer), PeerPotentialConnect::PendingConnect(peer) => Some(peer),
@ -1583,12 +1609,18 @@ where
closed messages; QED") closed messages; QED")
} }
/// Sends an event to the node. /// Start sending an event to the node.
#[inline] pub fn start_send_event(&mut self, event: TInEvent) -> StartSend<TInEvent, ()> {
pub fn send_event(&mut self, event: TInEvent) {
self.active_nodes.peer_mut(&self.peer_id) self.active_nodes.peer_mut(&self.peer_id)
.expect("A PeerConnected is always created with a PeerId in active_nodes; QED") .expect("A PeerConnected is always created with a PeerId in active_nodes; QED")
.send_event(event) .start_send_event(event)
}
/// Complete sending an event message, initiated by `start_send_event`.
pub fn complete_send_event(&mut self) -> Poll<(), ()> {
self.active_nodes.peer_mut(&self.peer_id)
.expect("A PeerConnected is always created with a PeerId in active_nodes; QED")
.complete_send_event()
} }
} }
@ -1612,7 +1644,6 @@ where
/// Interrupt this connection attempt. /// Interrupt this connection attempt.
// TODO: consider returning a PeerNotConnected; however that is really pain in terms of // TODO: consider returning a PeerNotConnected; however that is really pain in terms of
// borrows // borrows
#[inline]
pub fn interrupt(self) { pub fn interrupt(self) {
let attempt = self.attempt.remove(); let attempt = self.attempt.remove();
if self.active_nodes.interrupt(attempt.id).is_err() { if self.active_nodes.interrupt(attempt.id).is_err() {
@ -1626,13 +1657,11 @@ where
} }
/// Returns the multiaddress we're currently trying to dial. /// Returns the multiaddress we're currently trying to dial.
#[inline]
pub fn attempted_multiaddr(&self) -> &Multiaddr { pub fn attempted_multiaddr(&self) -> &Multiaddr {
&self.attempt.get().cur_attempted &self.attempt.get().cur_attempted
} }
/// Returns a list of the multiaddresses we're going to try if the current dialing fails. /// Returns a list of the multiaddresses we're going to try if the current dialing fails.
#[inline]
pub fn pending_multiaddrs(&self) -> impl Iterator<Item = &Multiaddr> { pub fn pending_multiaddrs(&self) -> impl Iterator<Item = &Multiaddr> {
self.attempt.get().next_attempts.iter() self.attempt.get().next_attempts.iter()
} }
@ -1700,7 +1729,6 @@ where
/// ///
/// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then /// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then
/// the whole connection is immediately closed. /// the whole connection is immediately closed.
#[inline]
pub fn connect(self, addr: Multiaddr, handler: THandler) pub fn connect(self, addr: Multiaddr, handler: THandler)
-> PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId> -> PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>
where where
@ -1718,7 +1746,6 @@ where
/// ///
/// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then /// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then
/// the whole connection is immediately closed. /// the whole connection is immediately closed.
#[inline]
pub fn connect_iter<TIter>(self, addrs: TIter, handler: THandler) pub fn connect_iter<TIter>(self, addrs: TIter, handler: THandler)
-> Result<PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>, Self> -> Result<PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo, TPeerId>, Self>
where where

View File

@ -129,14 +129,24 @@ fn broadcasted_events_reach_active_nodes() {
let dial_result = swarm.dial(addr, handler); let dial_result = swarm.dial(addr, handler);
assert!(dial_result.is_ok()); assert!(dial_result.is_ok());
swarm.broadcast_event(&InEvent::NextState);
let swarm = Arc::new(Mutex::new(swarm)); let swarm = Arc::new(Mutex::new(swarm));
let mut rt = Runtime::new().unwrap(); let mut rt = Runtime::new().unwrap();
let swarm2 = swarm.clone();
rt.block_on(future::poll_fn(move || {
if swarm2.lock().start_broadcast(&InEvent::NextState).is_not_ready() {
Ok::<_, ()>(Async::NotReady)
} else {
Ok(Async::Ready(()))
}
})).unwrap();
let mut peer_id : Option<PeerId> = None; let mut peer_id : Option<PeerId> = None;
while peer_id.is_none() { while peer_id.is_none() {
let swarm_fut = swarm.clone(); let swarm_fut = swarm.clone();
peer_id = rt.block_on(future::poll_fn(move || -> Poll<Option<PeerId>, ()> { peer_id = rt.block_on(future::poll_fn(move || -> Poll<Option<PeerId>, ()> {
let mut swarm = swarm_fut.lock(); let mut swarm = swarm_fut.lock();
if swarm.complete_broadcast().is_not_ready() {
return Ok(Async::NotReady)
}
let poll_res = swarm.poll(); let poll_res = swarm.poll();
match poll_res { match poll_res {
Async::Ready(RawSwarmEvent::Connected { conn_info, .. }) => Ok(Async::Ready(Some(conn_info))), Async::Ready(RawSwarmEvent::Connected { conn_info, .. }) => Ok(Async::Ready(Some(conn_info))),
@ -331,10 +341,20 @@ fn yields_node_error_when_there_is_an_error_after_successful_connect() {
let mut keep_polling = true; let mut keep_polling = true;
while keep_polling { while keep_polling {
let swarm_fut = swarm.clone(); let swarm_fut = swarm.clone();
let swarm2 = swarm.clone();
rt.block_on(future::poll_fn(move || {
if swarm2.lock().start_broadcast(&InEvent::NextState).is_not_ready() {
Ok::<_, ()>(Async::NotReady)
} else {
Ok(Async::Ready(()))
}
})).unwrap();
keep_polling = rt.block_on(future::poll_fn(move || -> Poll<_, ()> { keep_polling = rt.block_on(future::poll_fn(move || -> Poll<_, ()> {
let mut swarm = swarm_fut.lock(); let mut swarm = swarm_fut.lock();
// Push the Handler into an error state on the next poll // Push the Handler into an error state on the next poll
swarm.broadcast_event(&InEvent::NextState); if swarm.complete_broadcast().is_not_ready() {
return Ok(Async::NotReady)
}
match swarm.poll() { match swarm.poll() {
Async::NotReady => Ok(Async::Ready(true)), Async::NotReady => Ok(Async::Ready(true)),
Async::Ready(event) => { Async::Ready(event) => {

View File

@ -0,0 +1,38 @@
use crate::nodes::handled_node::HandledNodeError;
use std::{fmt, error};
/// Error that can happen in a task.
#[derive(Debug)]
pub enum Error<R, H> {
/// An error happend while we were trying to reach the node.
Reach(R),
/// An error happened after the node has been reached.
Node(HandledNodeError<H>)
}
impl<R, H> fmt::Display for Error<R, H>
where
R: fmt::Display,
H: fmt::Display
{
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self {
Error::Reach(err) => write!(f, "reach error: {}", err),
Error::Node(err) => write!(f, "node error: {}", err)
}
}
}
impl<R, H> error::Error for Error<R, H>
where
R: error::Error + 'static,
H: error::Error + 'static
{
fn source(&self) -> Option<&(dyn error::Error + 'static)> {
match self {
Error::Reach(err) => Some(err),
Error::Node(err) => Some(err)
}
}
}

View File

@ -0,0 +1,568 @@
// Copyright 2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use crate::{
PeerId,
muxing::StreamMuxer,
nodes::{
handled_node::{HandledNode, IntoNodeHandler, NodeHandler},
node::Substream
}
};
use fnv::FnvHashMap;
use futures::{prelude::*, future::Executor, sync::mpsc};
use smallvec::SmallVec;
use std::{collections::hash_map::{Entry, OccupiedEntry}, error, fmt};
use super::{TaskId, task::{Task, FromTaskMessage, ToTaskMessage}, Error};
// Implementor notes
// =================
//
// This collection of nodes spawns a `Task` for each individual node to process.
// This means that events happen asynchronously at the same time as the
// `Manager` is being polled.
//
// In order to make the API non-racy and avoid issues, we completely separate
// the state in the `Manager` from the states that the `Task` can access.
// They are only allowed to exchange messages. The state in the `Manager` is
// therefore delayed compared to the tasks, and is updated only when `poll()`
// is called.
//
// The only thing that we must be careful about is substreams, as they are
// "detached" from the state of the `Manager` and allowed to process
// concurrently. This is why there is no "substream closed" event being
// reported, as it could potentially create confusions and race conditions in
// the user's code. See similar comments in the documentation of `NodeStream`.
//
/// Implementation of [`Stream`] that handles a collection of nodes.
pub struct Manager<I, O, H, E, HE, T, C = PeerId> {
/// Collection of managed tasks.
///
/// Closing the sender interrupts the task. It is possible that we receive
/// messages from tasks that used to be in this collection but no longer
/// are, in which case we should ignore them.
tasks: FnvHashMap<TaskId, TaskInfo<I, T>>,
/// Identifier for the next task to spawn.
next_task_id: TaskId,
/// List of node tasks to spawn.
to_spawn: SmallVec<[Box<dyn Future<Item = (), Error = ()> + Send>; 8]>,
/// If no tokio executor is available, we move tasks to this list, and futures are polled on
/// the current thread instead.
local_spawns: Vec<Box<dyn Future<Item = (), Error = ()> + Send>>,
/// Sender to emit events to the outside. Meant to be cloned and sent to tasks.
events_tx: mpsc::Sender<(FromTaskMessage<O, H, E, HE, C>, TaskId)>,
/// Receiver side for the events.
events_rx: mpsc::Receiver<(FromTaskMessage<O, H, E, HE, C>, TaskId)>
}
impl<I, O, H, E, HE, T, C> fmt::Debug for Manager<I, O, H, E, HE, T, C>
where
T: fmt::Debug
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
f.debug_map()
.entries(self.tasks.iter().map(|(id, task)| (id, &task.user_data)))
.finish()
}
}
/// Information about a running task.
///
/// Contains the sender to deliver event messages to the task,
/// the associated user data and a pending message if any,
/// meant to be delivered to the task via the sender.
struct TaskInfo<I, T> {
/// channel endpoint to send messages to the task
sender: mpsc::Sender<ToTaskMessage<I>>,
/// task associated data
user_data: T,
/// any pending event to deliver to the task
pending: Option<AsyncSink<ToTaskMessage<I>>>
}
/// Event produced by the [`Manager`].
#[derive(Debug)]
pub enum Event<'a, I, O, H, E, HE, T, C = PeerId> {
/// A task has been closed.
///
/// This happens once the node handler closes or an error happens.
TaskClosed {
/// The task that has been closed.
task: ClosedTask<I, T>,
/// What happened.
result: Error<E, HE>,
/// If the task closed before reaching the node, this contains
/// the handler that was passed to `add_reach_attempt`.
handler: Option<H>
},
/// A task has successfully connected to a node.
NodeReached {
/// The task that succeeded.
task: TaskEntry<'a, I, T>,
/// Identifier of the node.
conn_info: C
},
/// A task has produced an event.
NodeEvent {
/// The task that produced the event.
task: TaskEntry<'a, I, T>,
/// The produced event.
event: O
}
}
impl<I, O, H, E, HE, T, C> Manager<I, O, H, E, HE, T, C> {
/// Creates a new task manager.
pub fn new() -> Self {
let (tx, rx) = mpsc::channel(1);
Self {
tasks: FnvHashMap::default(),
next_task_id: TaskId(0),
to_spawn: SmallVec::new(),
local_spawns: Vec::new(),
events_tx: tx,
events_rx: rx
}
}
/// Adds to the manager a future that tries to reach a node.
///
/// This method spawns a task dedicated to resolving this future and
/// processing the node's events.
pub fn add_reach_attempt<F, M>(&mut self, future: F, user_data: T, handler: H) -> TaskId
where
F: Future<Item = (C, M), Error = E> + Send + 'static,
H: IntoNodeHandler<C> + Send + 'static,
H::Handler: NodeHandler<Substream = Substream<M>, InEvent = I, OutEvent = O, Error = HE> + Send + 'static,
E: error::Error + Send + 'static,
HE: error::Error + Send + 'static,
I: Send + 'static,
O: Send + 'static,
<H::Handler as NodeHandler>::OutboundOpenInfo: Send + 'static,
M: StreamMuxer + Send + Sync + 'static,
M::OutboundSubstream: Send + 'static,
C: Send + 'static
{
let task_id = self.next_task_id;
self.next_task_id.0 += 1;
let (tx, rx) = mpsc::channel(4);
self.tasks.insert(task_id, TaskInfo { sender: tx, user_data, pending: None });
let task = Box::new(Task::new(task_id, self.events_tx.clone(), rx, future, handler));
self.to_spawn.push(task);
task_id
}
/// Adds an existing connection to a node to the collection.
///
/// This method spawns a task dedicated to processing the node's events.
///
/// No `NodeReached` event will be emitted for this task, since the node has already been
/// reached.
pub fn add_connection<M, Handler>(&mut self, user_data: T, muxer: M, handler: Handler) -> TaskId
where
H: IntoNodeHandler<C, Handler = Handler> + Send + 'static,
Handler: NodeHandler<Substream = Substream<M>, InEvent = I, OutEvent = O, Error = HE> + Send + 'static,
E: error::Error + Send + 'static,
HE: error::Error + Send + 'static,
I: Send + 'static,
O: Send + 'static,
<H::Handler as NodeHandler>::OutboundOpenInfo: Send + 'static,
M: StreamMuxer + Send + Sync + 'static,
M::OutboundSubstream: Send + 'static,
C: Send + 'static
{
let task_id = self.next_task_id;
self.next_task_id.0 += 1;
let (tx, rx) = mpsc::channel(4);
self.tasks.insert(task_id, TaskInfo { sender: tx, user_data, pending: None });
let task: Task<futures::future::Empty<_, _>, _, _, _, _, _, _> =
Task::node(task_id, self.events_tx.clone(), rx, HandledNode::new(muxer, handler));
self.to_spawn.push(Box::new(task));
task_id
}
/// Start sending an event to all the tasks, including the pending ones.
///
/// After starting a broadcast make sure to finish it with `complete_broadcast`,
/// otherwise starting another broadcast or sending an event directly to a
/// task would overwrite the pending broadcast.
#[must_use]
pub fn start_broadcast(&mut self, event: &I) -> AsyncSink<()>
where
I: Clone
{
if self.complete_broadcast().is_not_ready() {
return AsyncSink::NotReady(())
}
for task in self.tasks.values_mut() {
let msg = ToTaskMessage::HandlerEvent(event.clone());
task.pending = Some(AsyncSink::NotReady(msg))
}
AsyncSink::Ready
}
/// Complete a started broadcast.
#[must_use]
pub fn complete_broadcast(&mut self) -> Async<()> {
let mut ready = true;
for task in self.tasks.values_mut() {
match task.pending.take() {
Some(AsyncSink::NotReady(msg)) =>
match task.sender.start_send(msg) {
Ok(AsyncSink::NotReady(msg)) => {
task.pending = Some(AsyncSink::NotReady(msg));
ready = false
}
Ok(AsyncSink::Ready) =>
if let Ok(Async::NotReady) = task.sender.poll_complete() {
task.pending = Some(AsyncSink::Ready);
ready = false
}
Err(_) => {}
}
Some(AsyncSink::Ready) =>
if let Ok(Async::NotReady) = task.sender.poll_complete() {
task.pending = Some(AsyncSink::Ready);
ready = false
}
None => {}
}
}
if ready {
Async::Ready(())
} else {
Async::NotReady
}
}
/// Grants access to an object that allows controlling a task of the collection.
///
/// Returns `None` if the task id is invalid.
pub fn task(&mut self, id: TaskId) -> Option<TaskEntry<'_, I, T>> {
match self.tasks.entry(id) {
Entry::Occupied(inner) => Some(TaskEntry { inner }),
Entry::Vacant(_) => None,
}
}
/// Returns a list of all the active tasks.
pub fn tasks<'a>(&'a self) -> impl Iterator<Item = TaskId> + 'a {
self.tasks.keys().cloned()
}
/// Provides an API similar to `Stream`, except that it cannot produce an error.
pub fn poll(&mut self) -> Async<Event<I, O, H, E, HE, T, C>> {
for to_spawn in self.to_spawn.drain() {
// We try to use the default executor, but fall back to polling the task manually if
// no executor is available. This makes it possible to use the core in environments
// outside of tokio.
let executor = tokio_executor::DefaultExecutor::current();
if let Err(err) = executor.execute(to_spawn) {
self.local_spawns.push(err.into_future())
}
}
for n in (0 .. self.local_spawns.len()).rev() {
let mut task = self.local_spawns.swap_remove(n);
match task.poll() {
Ok(Async::Ready(())) => {}
Ok(Async::NotReady) => self.local_spawns.push(task),
// It would normally be desirable to either report or log when a background task
// errors. However the default tokio executor doesn't do anything in case of error,
// and therefore we mimic this behaviour by also not doing anything.
Err(()) => {}
}
}
let (message, task_id) = loop {
match self.events_rx.poll() {
Ok(Async::Ready(Some((message, task_id)))) => {
// If the task id is no longer in `self.tasks`, that means that the user called
// `close()` on this task earlier. Therefore no new event should be generated
// for this task.
if self.tasks.contains_key(&task_id) {
break (message, task_id)
}
}
Ok(Async::NotReady) => return Async::NotReady,
Ok(Async::Ready(None)) => unreachable!("sender and receiver have same lifetime"),
Err(()) => unreachable!("An `mpsc::Receiver` does not error.")
}
};
Async::Ready(match message {
FromTaskMessage::NodeEvent(event) =>
Event::NodeEvent {
task: match self.tasks.entry(task_id) {
Entry::Occupied(inner) => TaskEntry { inner },
Entry::Vacant(_) => panic!("poll_inner only returns valid TaskIds; QED")
},
event
},
FromTaskMessage::NodeReached(conn_info) =>
Event::NodeReached {
task: match self.tasks.entry(task_id) {
Entry::Occupied(inner) => TaskEntry { inner },
Entry::Vacant(_) => panic!("poll_inner only returns valid TaskIds; QED")
},
conn_info
},
FromTaskMessage::TaskClosed(result, handler) => {
let entry = self.tasks.remove(&task_id)
.expect("poll_inner only returns valid TaskIds; QED");
Event::TaskClosed {
task: ClosedTask::new(task_id, entry.sender, entry.user_data),
result,
handler
}
}
})
}
}
/// Access to a task in the collection.
pub struct TaskEntry<'a, E, T> {
inner: OccupiedEntry<'a, TaskId, TaskInfo<E, T>>
}
impl<'a, E, T> TaskEntry<'a, E, T> {
/// Begin sending an event to the given node.
///
/// Make sure to finish the send operation with `complete_send_event`.
pub fn start_send_event(&mut self, event: E) -> StartSend<E, ()> {
let msg = ToTaskMessage::HandlerEvent(event);
if let AsyncSink::NotReady(msg) = self.start_send_event_msg(msg)? {
if let ToTaskMessage::HandlerEvent(event) = msg {
return Ok(AsyncSink::NotReady(event))
} else {
unreachable!("we tried to send an handler event, so we get one back if not ready")
}
}
Ok(AsyncSink::Ready)
}
/// Finish a send operation started with `start_send_event`.
pub fn complete_send_event(&mut self) -> Poll<(), ()> {
self.complete_send_event_msg()
}
/// Returns the user data associated with the task.
pub fn user_data(&self) -> &T {
&self.inner.get().user_data
}
/// Returns the user data associated with the task.
pub fn user_data_mut(&mut self) -> &mut T {
&mut self.inner.get_mut().user_data
}
/// Returns the task id.
pub fn id(&self) -> TaskId {
*self.inner.key()
}
/// Closes the task. Returns the user data.
///
/// No further event will be generated for this task, but the connection inside the task will
/// continue to run until the `ClosedTask` is destroyed.
pub fn close(self) -> ClosedTask<E, T> {
let id = *self.inner.key();
let task = self.inner.remove();
ClosedTask::new(id, task.sender, task.user_data)
}
/// Gives ownership of a closed task.
/// As soon as our task (`self`) has some acknowledgment from the remote
/// that its connection is alive, it will close the connection with `other`.
///
/// Make sure to complete this operation with `complete_take_over`.
#[must_use]
pub fn start_take_over(&mut self, t: ClosedTask<E, T>) -> StartTakeOver<T, ClosedTask<E, T>> {
// It is possible that the sender is closed if the background task has already finished
// but the local state hasn't been updated yet because we haven't been polled in the
// meanwhile.
let id = t.id();
match self.start_send_event_msg(ToTaskMessage::TakeOver(t.sender)) {
Ok(AsyncSink::Ready) => StartTakeOver::Ready(t.user_data),
Ok(AsyncSink::NotReady(ToTaskMessage::TakeOver(sender))) =>
StartTakeOver::NotReady(ClosedTask::new(id, sender, t.user_data)),
Ok(AsyncSink::NotReady(_)) =>
unreachable!("We tried to send a take over message, so we get one back."),
Err(()) => StartTakeOver::Gone
}
}
/// Finish take over started by `start_take_over`.
pub fn complete_take_over(&mut self) -> Poll<(), ()> {
self.complete_send_event_msg()
}
/// Begin to send a message to the task.
///
/// The API mimicks the one of [`futures::Sink`]. If this method returns
/// `Ok(AsyncSink::Ready)` drive the sending to completion with
/// `complete_send_event_msg`. If the receiving end does not longer exist,
/// i.e. the task has ended, we return this information as an error.
fn start_send_event_msg(&mut self, msg: ToTaskMessage<E>) -> StartSend<ToTaskMessage<E>, ()> {
// We first drive any pending send to completion before starting another one.
if self.complete_send_event_msg()?.is_ready() {
self.inner.get_mut().pending = Some(AsyncSink::NotReady(msg));
Ok(AsyncSink::Ready)
} else {
Ok(AsyncSink::NotReady(msg))
}
}
/// Complete event message deliver started by `start_send_event_msg`.
fn complete_send_event_msg(&mut self) -> Poll<(), ()> {
// It is possible that the sender is closed if the background task has already finished
// but the local state hasn't been updated yet because we haven't been polled in the
// meanwhile.
let task = self.inner.get_mut();
let state =
if let Some(state) = task.pending.take() {
state
} else {
return Ok(Async::Ready(()))
};
match state {
AsyncSink::NotReady(msg) =>
match task.sender.start_send(msg).map_err(|_| ())? {
AsyncSink::Ready =>
if task.sender.poll_complete().map_err(|_| ())?.is_not_ready() {
task.pending = Some(AsyncSink::Ready);
Ok(Async::NotReady)
} else {
Ok(Async::Ready(()))
}
AsyncSink::NotReady(msg) => {
task.pending = Some(AsyncSink::NotReady(msg));
Ok(Async::NotReady)
}
}
AsyncSink::Ready =>
if task.sender.poll_complete().map_err(|_| ())?.is_not_ready() {
task.pending = Some(AsyncSink::Ready);
Ok(Async::NotReady)
} else {
Ok(Async::Ready(()))
}
}
}
}
impl<E, T: fmt::Debug> fmt::Debug for TaskEntry<'_, E, T> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
f.debug_tuple("TaskEntry")
.field(&self.id())
.field(self.user_data())
.finish()
}
}
/// Result of [`TaskEntry::start_take_over`].
#[derive(Debug)]
pub enum StartTakeOver<A, B> {
/// The take over message has been enqueued.
/// Complete the take over with [`TaskEntry::complete_take_over`].
Ready(A),
/// Not ready to send the take over message to the task.
NotReady(B),
/// The task to send the take over message is no longer there.
Gone
}
/// Task after it has been closed.
///
/// The connection to the remote is potentially still going on, but no new
/// event for this task will be received.
pub struct ClosedTask<E, T> {
/// Identifier of the task that closed.
///
/// No longer corresponds to anything, but can be reported to the user.
id: TaskId,
/// The channel to the task.
///
/// The task will continue to work for as long as this channel is alive,
/// but events produced by it are ignored.
sender: mpsc::Sender<ToTaskMessage<E>>,
/// The data provided by the user.
user_data: T
}
impl<E, T> ClosedTask<E, T> {
/// Create a new `ClosedTask` value.
fn new(id: TaskId, sender: mpsc::Sender<ToTaskMessage<E>>, user_data: T) -> Self {
Self { id, sender, user_data }
}
/// Returns the task id.
///
/// Note that this task is no longer managed and therefore calling
/// `Manager::task()` with this ID will fail.
pub fn id(&self) -> TaskId {
self.id
}
/// Returns the user data associated with the task.
pub fn user_data(&self) -> &T {
&self.user_data
}
/// Returns the user data associated with the task.
pub fn user_data_mut(&mut self) -> &mut T {
&mut self.user_data
}
/// Finish destroying the task and yield the user data.
/// This closes the connection to the remote.
pub fn into_user_data(self) -> T {
self.user_data
}
}
impl<E, T: fmt::Debug> fmt::Debug for ClosedTask<E, T> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
f.debug_tuple("ClosedTask")
.field(&self.id)
.field(&self.user_data)
.finish()
}
}

View File

@ -0,0 +1,45 @@
// Copyright 2019 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
//! Management of tasks handling nodes.
//!
//! The core type is a [`task::Task`], which implements [`futures::Future`]
//! and connects and handles a node. A task receives and sends messages
//! ([`tasks::FromTaskMessage`], [`tasks::ToTaskMessage`]) to the outside.
//!
//! A set of tasks is managed by a [`Manager`] which creates tasks when a
//! node should be connected to (cf. [`Manager::add_reach_attempt`]) or
//! an existing connection to a node should be driven forward (cf.
//! [`Manager::add_connection`]). Tasks can be referred to by [`TaskId`]
//! and messages can be sent to individual tasks or all (cf.
//! [`Manager::start_broadcast`]). Messages produces by tasks can be
//! retrieved by polling the manager (cf. [`Manager::poll`]).
mod error;
mod manager;
mod task;
pub use error::Error;
pub use manager::{ClosedTask, TaskEntry, Manager, Event, StartTakeOver};
/// Task identifier.
#[derive(Debug, Copy, Clone, Hash, PartialEq, Eq, PartialOrd, Ord)]
pub struct TaskId(usize);

View File

@ -0,0 +1,367 @@
// Copyright 2019 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use crate::{
muxing::StreamMuxer,
nodes::{
handled_node::{HandledNode, IntoNodeHandler, NodeHandler},
node::{Close, Substream}
}
};
use futures::{prelude::*, stream, sync::mpsc};
use smallvec::SmallVec;
use super::{TaskId, Error};
/// Message to transmit from the public API to a task.
#[derive(Debug)]
pub enum ToTaskMessage<T> {
/// An event to transmit to the node handler.
HandlerEvent(T),
/// When received, stores the parameter inside the task and keeps it alive
/// until we have an acknowledgment that the remote has accepted our handshake.
TakeOver(mpsc::Sender<ToTaskMessage<T>>)
}
/// Message to transmit from a task to the public API.
#[derive(Debug)]
pub enum FromTaskMessage<T, H, E, HE, C> {
/// A connection to a node has succeeded.
NodeReached(C),
/// The task closed.
TaskClosed(Error<E, HE>, Option<H>),
/// An event from the node.
NodeEvent(T)
}
/// Implementation of [`Future`] that handles a single node.
pub struct Task<F, M, H, I, O, E, C>
where
M: StreamMuxer,
H: IntoNodeHandler<C>,
H::Handler: NodeHandler<Substream = Substream<M>>
{
/// The ID of this task.
id: TaskId,
/// Sender to transmit messages to the outside.
sender: mpsc::Sender<(FromTaskMessage<O, H, E, <H::Handler as NodeHandler>::Error, C>, TaskId)>,
/// Receiver of messages from the outsize.
receiver: stream::Fuse<mpsc::Receiver<ToTaskMessage<I>>>,
/// Inner state of this `Task`.
state: State<F, M, H, I, O, E, C>,
/// Channels to keep alive for as long as we don't have an acknowledgment from the remote.
taken_over: SmallVec<[mpsc::Sender<ToTaskMessage<I>>; 1]>
}
impl<F, M, H, I, O, E, C> Task<F, M, H, I, O, E, C>
where
M: StreamMuxer,
H: IntoNodeHandler<C>,
H::Handler: NodeHandler<Substream = Substream<M>>
{
/// Create a new task to connect and handle some node.
pub fn new (
i: TaskId,
s: mpsc::Sender<(FromTaskMessage<O, H, E, <H::Handler as NodeHandler>::Error, C>, TaskId)>,
r: mpsc::Receiver<ToTaskMessage<I>>,
f: F,
h: H
) -> Self {
Task {
id: i,
sender: s,
receiver: r.fuse(),
state: State::Future { future: f, handler: h, events_buffer: Vec::new() },
taken_over: SmallVec::new()
}
}
/// Create a task for an existing node we are already connected to.
pub fn node (
i: TaskId,
s: mpsc::Sender<(FromTaskMessage<O, H, E, <H::Handler as NodeHandler>::Error, C>, TaskId)>,
r: mpsc::Receiver<ToTaskMessage<I>>,
n: HandledNode<M, H::Handler>
) -> Self {
Task {
id: i,
sender: s,
receiver: r.fuse(),
state: State::Node(n),
taken_over: SmallVec::new()
}
}
}
/// State of the future.
enum State<F, M, H, I, O, E, C>
where
M: StreamMuxer,
H: IntoNodeHandler<C>,
H::Handler: NodeHandler<Substream = Substream<M>>
{
/// Future to resolve to connect to the node.
Future {
/// The future that will attempt to reach the node.
future: F,
/// The handler that will be used to build the `HandledNode`.
handler: H,
/// While we are dialing the future, we need to buffer the events received on
/// `receiver` so that they get delivered once dialing succeeds. We can't simply leave
/// events in `receiver` because we have to detect if it gets closed.
events_buffer: Vec<I>
},
/// An event should be sent to the outside world.
SendEvent {
/// The node, if available.
node: Option<HandledNode<M, H::Handler>>,
/// The actual event message to send.
event: FromTaskMessage<O, H, E, <H::Handler as NodeHandler>::Error, C>
},
/// We started sending an event, now drive the sending to completion.
///
/// The `bool` parameter determines if we transition to `State::Node`
/// afterwards or to `State::Closing` (assuming we have `Some` node,
/// otherwise the task will end).
PollComplete(Option<HandledNode<M, H::Handler>>, bool),
/// Fully functional node.
Node(HandledNode<M, H::Handler>),
/// Node closing.
Closing(Close<M>),
/// Interim state that can only be observed externally if the future
/// resolved to a value previously.
Undefined
}
impl<F, M, H, I, O, E, C> Future for Task<F, M, H, I, O, E, C>
where
M: StreamMuxer,
F: Future<Item = (C, M), Error = E>,
H: IntoNodeHandler<C>,
H::Handler: NodeHandler<Substream = Substream<M>, InEvent = I, OutEvent = O>
{
type Item = ();
type Error = ();
// NOTE: It is imperative to always consume all incoming event messages
// first in order to not prevent the outside from making progress because
// they are blocked on the channel capacity.
fn poll(&mut self) -> Poll<(), ()> {
'poll: loop {
match std::mem::replace(&mut self.state, State::Undefined) {
State::Future { mut future, handler, mut events_buffer } => {
// If self.receiver is closed, we stop the task.
loop {
match self.receiver.poll() {
Ok(Async::NotReady) => break,
Ok(Async::Ready(None)) => return Ok(Async::Ready(())),
Ok(Async::Ready(Some(ToTaskMessage::HandlerEvent(event)))) =>
events_buffer.push(event),
Ok(Async::Ready(Some(ToTaskMessage::TakeOver(take_over)))) =>
self.taken_over.push(take_over),
Err(()) => unreachable!("An `mpsc::Receiver` does not error.")
}
}
// Check if dialing succeeded.
match future.poll() {
Ok(Async::Ready((conn_info, muxer))) => {
let mut node = HandledNode::new(muxer, handler.into_handler(&conn_info));
for event in events_buffer {
node.inject_event(event)
}
self.state = State::SendEvent {
node: Some(node),
event: FromTaskMessage::NodeReached(conn_info)
}
}
Ok(Async::NotReady) => {
self.state = State::Future { future, handler, events_buffer };
return Ok(Async::NotReady)
}
Err(e) => {
let event = FromTaskMessage::TaskClosed(Error::Reach(e), Some(handler));
self.state = State::SendEvent { node: None, event }
}
}
}
State::Node(mut node) => {
// Start by handling commands received from the outside of the task.
loop {
match self.receiver.poll() {
Ok(Async::NotReady) => break,
Ok(Async::Ready(Some(ToTaskMessage::HandlerEvent(event)))) =>
node.inject_event(event),
Ok(Async::Ready(Some(ToTaskMessage::TakeOver(take_over)))) =>
self.taken_over.push(take_over),
Ok(Async::Ready(None)) => {
// Node closed by the external API; start closing.
self.state = State::Closing(node.close());
continue 'poll
}
Err(()) => unreachable!("An `mpsc::Receiver` does not error.")
}
}
// Process the node.
loop {
if !self.taken_over.is_empty() && node.is_remote_acknowledged() {
self.taken_over.clear()
}
match node.poll() {
Ok(Async::NotReady) => {
self.state = State::Node(node);
return Ok(Async::NotReady)
}
Ok(Async::Ready(event)) => {
self.state = State::SendEvent {
node: Some(node),
event: FromTaskMessage::NodeEvent(event)
};
continue 'poll
}
Err(err) => {
let event = FromTaskMessage::TaskClosed(Error::Node(err), None);
self.state = State::SendEvent { node: None, event };
continue 'poll
}
}
}
}
// Deliver an event to the outside.
State::SendEvent { mut node, event } => {
loop {
match self.receiver.poll() {
Ok(Async::NotReady) => break,
Ok(Async::Ready(Some(ToTaskMessage::HandlerEvent(event)))) =>
if let Some(ref mut n) = node {
n.inject_event(event)
}
Ok(Async::Ready(Some(ToTaskMessage::TakeOver(take_over)))) =>
self.taken_over.push(take_over),
Ok(Async::Ready(None)) =>
// Node closed by the external API; start closing.
if let Some(n) = node {
self.state = State::Closing(n.close());
continue 'poll
} else {
return Ok(Async::Ready(())) // end task
}
Err(()) => unreachable!("An `mpsc::Receiver` does not error.")
}
}
// Check if this task is about to close. We pass the flag to
// the next state so it knows what to do.
let close =
if let FromTaskMessage::TaskClosed(..) = event {
true
} else {
false
};
match self.sender.start_send((event, self.id)) {
Ok(AsyncSink::NotReady((event, _))) => {
self.state = State::SendEvent { node, event };
return Ok(Async::NotReady)
}
Ok(AsyncSink::Ready) => self.state = State::PollComplete(node, close),
Err(_) => {
if let Some(n) = node {
self.state = State::Closing(n.close());
continue 'poll
}
// We can not communicate to the outside and there is no
// node to handle, so this is the end of this task.
return Ok(Async::Ready(()))
}
}
}
// We started delivering an event, now try to complete the sending.
State::PollComplete(mut node, close) => {
loop {
match self.receiver.poll() {
Ok(Async::NotReady) => break,
Ok(Async::Ready(Some(ToTaskMessage::HandlerEvent(event)))) =>
if let Some(ref mut n) = node {
n.inject_event(event)
}
Ok(Async::Ready(Some(ToTaskMessage::TakeOver(take_over)))) =>
self.taken_over.push(take_over),
Ok(Async::Ready(None)) =>
// Node closed by the external API; start closing.
if let Some(n) = node {
self.state = State::Closing(n.close());
continue 'poll
} else {
return Ok(Async::Ready(())) // end task
}
Err(()) => unreachable!("An `mpsc::Receiver` does not error.")
}
}
match self.sender.poll_complete() {
Ok(Async::NotReady) => {
self.state = State::PollComplete(node, close);
return Ok(Async::NotReady)
}
Ok(Async::Ready(())) =>
if let Some(n) = node {
if close {
self.state = State::Closing(n.close())
} else {
self.state = State::Node(n)
}
} else {
// Since we have no node we terminate this task.
assert!(close);
return Ok(Async::Ready(()))
}
Err(_) => {
if let Some(n) = node {
self.state = State::Closing(n.close());
continue 'poll
}
// We can not communicate to the outside and there is no
// node to handle, so this is the end of this task.
return Ok(Async::Ready(()))
}
}
}
State::Closing(mut closing) =>
match closing.poll() {
Ok(Async::Ready(())) | Err(_) =>
return Ok(Async::Ready(())), // end task
Ok(Async::NotReady) => {
self.state = State::Closing(closing);
return Ok(Async::NotReady)
}
}
// This happens if a previous poll has resolved the future.
// The API contract of futures is that we should not be polled again.
State::Undefined => panic!("`Task::poll()` called after completion.")
}
}
}
}

View File

@ -29,7 +29,8 @@ use rw_stream_sink::RwStreamSink;
use std::{collections::hash_map::Entry, error, fmt, io, num::NonZeroU64}; use std::{collections::hash_map::Entry, error, fmt, io, num::NonZeroU64};
lazy_static! { lazy_static! {
static ref HUB: Mutex<FnvHashMap<NonZeroU64, mpsc::Sender<Channel<Bytes>>>> = Mutex::new(FnvHashMap::default()); static ref HUB: Mutex<FnvHashMap<NonZeroU64, mpsc::Sender<Channel<Bytes>>>> =
Mutex::new(FnvHashMap::default());
} }
/// Transport that supports `/memory/N` multiaddresses. /// Transport that supports `/memory/N` multiaddresses.
@ -184,7 +185,7 @@ impl Stream for Listener {
return Ok(Async::Ready(Some(ListenerEvent::NewAddress(self.addr.clone())))) return Ok(Async::Ready(Some(ListenerEvent::NewAddress(self.addr.clone()))))
} }
let channel = try_ready!(Ok(self.receiver.poll() let channel = try_ready!(Ok(self.receiver.poll()
.expect("An unbounded receiver never panics; QED"))); .expect("Life listeners always have a sender.")));
let channel = match channel { let channel = match channel {
Some(c) => c, Some(c) => c,
None => return Ok(Async::Ready(None)) None => return Ok(Async::Ready(None))

View File

@ -137,6 +137,16 @@ where
/// List of nodes for which we deny any incoming connection. /// List of nodes for which we deny any incoming connection.
banned_peers: HashSet<PeerId>, banned_peers: HashSet<PeerId>,
/// Pending event message to be delivered.
///
/// If the pair's second element is `AsyncSink::NotReady`, the event
/// message has yet to be sent using `PeerMut::start_send_event`.
///
/// If the pair's second element is `AsyncSink::Ready`, the event
/// message has been sent and needs to be flushed using
/// `PeerMut::complete_send_event`.
send_event_to_complete: Option<(PeerId, AsyncSink<TInEvent>)>
} }
impl<TTransport, TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo> Deref for impl<TTransport, TBehaviour, TInEvent, TOutEvent, THandler, THandlerErr, TConnInfo> Deref for
@ -379,6 +389,24 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
}, },
} }
// Try to deliver pending event.
if let Some((id, pending)) = self.send_event_to_complete.take() {
if let Some(mut peer) = self.raw_swarm.peer(id.clone()).into_connected() {
if let AsyncSink::NotReady(e) = pending {
if let Ok(a@AsyncSink::NotReady(_)) = peer.start_send_event(e) {
self.send_event_to_complete = Some((id, a))
} else if let Ok(Async::NotReady) = peer.complete_send_event() {
self.send_event_to_complete = Some((id, AsyncSink::Ready))
}
} else if let Ok(Async::NotReady) = peer.complete_send_event() {
self.send_event_to_complete = Some((id, AsyncSink::Ready))
}
}
}
if self.send_event_to_complete.is_some() {
return Ok(Async::NotReady)
}
let behaviour_poll = { let behaviour_poll = {
let mut parameters = SwarmPollParameters { let mut parameters = SwarmPollParameters {
local_peer_id: &mut self.raw_swarm.local_peer_id(), local_peer_id: &mut self.raw_swarm.local_peer_id(),
@ -406,8 +434,12 @@ where TBehaviour: NetworkBehaviour<ProtocolsHandler = THandler>,
} }
}, },
Async::Ready(NetworkBehaviourAction::SendEvent { peer_id, event }) => { Async::Ready(NetworkBehaviourAction::SendEvent { peer_id, event }) => {
if let Some(mut peer) = self.raw_swarm.peer(peer_id).into_connected() { if let Some(mut peer) = self.raw_swarm.peer(peer_id.clone()).into_connected() {
peer.send_event(event); if let Ok(a@AsyncSink::NotReady(_)) = peer.start_send_event(event) {
self.send_event_to_complete = Some((peer_id, a))
} else if let Ok(Async::NotReady) = peer.complete_send_event() {
self.send_event_to_complete = Some((peer_id, AsyncSink::Ready))
}
} }
}, },
Async::Ready(NetworkBehaviourAction::ReportObservedAddr { address }) => { Async::Ready(NetworkBehaviourAction::ReportObservedAddr { address }) => {
@ -524,6 +556,7 @@ where TBehaviour: NetworkBehaviour,
listened_addrs: SmallVec::new(), listened_addrs: SmallVec::new(),
external_addrs: Addresses::default(), external_addrs: Addresses::default(),
banned_peers: HashSet::new(), banned_peers: HashSet::new(),
send_event_to_complete: None
} }
} }
} }

View File

@ -30,8 +30,7 @@ use libp2p_core::{
ConnectedPoint, ConnectedPoint,
PeerId, PeerId,
nodes::collection::ConnectionInfo, nodes::collection::ConnectionInfo,
nodes::handled_node::{NodeHandler, NodeHandlerEndpoint, NodeHandlerEvent}, nodes::handled_node::{IntoNodeHandler, NodeHandler, NodeHandlerEndpoint, NodeHandlerEvent},
nodes::handled_node_tasks::IntoNodeHandler,
upgrade::{self, InboundUpgradeApply, OutboundUpgradeApply} upgrade::{self, InboundUpgradeApply, OutboundUpgradeApply}
}; };
use std::{error, fmt, time::Duration}; use std::{error, fmt, time::Duration};