mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-05-29 02:31:20 +00:00
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:
parent
a0d278a479
commit
6aba7961d1
@ -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()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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
|
||||||
|
@ -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 {
|
||||||
|
@ -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")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
@ -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;
|
||||||
|
@ -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
|
||||||
|
@ -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) => {
|
||||||
|
38
core/src/nodes/tasks/error.rs
Normal file
38
core/src/nodes/tasks/error.rs
Normal 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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
568
core/src/nodes/tasks/manager.rs
Normal file
568
core/src/nodes/tasks/manager.rs
Normal 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()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
45
core/src/nodes/tasks/mod.rs
Normal file
45
core/src/nodes/tasks/mod.rs
Normal 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);
|
||||||
|
|
367
core/src/nodes/tasks/task.rs
Normal file
367
core/src/nodes/tasks/task.rs
Normal 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.")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -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))
|
||||||
|
@ -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
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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};
|
||||||
|
Loading…
x
Reference in New Issue
Block a user