From 5d1c54cc1054f40a8b5b69194605e81a4a16a86b Mon Sep 17 00:00:00 2001 From: Pierre Krieger Date: Wed, 17 Oct 2018 10:17:40 +0100 Subject: [PATCH 1/2] New core (#568) * New core * Fix lifetime requirements * Remove identify transport * Address &mut & ref ref mut * Fix whitespaces --- core/src/either.rs | 97 ++-- core/src/lib.rs | 4 +- core/src/nodes/collection.rs | 95 ++-- core/src/nodes/handled_node.rs | 70 +-- core/src/nodes/handled_node_tasks.rs | 87 ++-- core/src/nodes/listeners.rs | 50 +- core/src/nodes/mod.rs | 9 +- core/src/nodes/node.rs | 136 +---- core/src/nodes/{swarm.rs => raw_swarm.rs} | 572 ++++++++++++---------- core/src/tests/dummy_transport.rs | 9 +- core/src/transport/and_then.rs | 74 +-- core/src/transport/boxed.rs | 122 +---- core/src/transport/choice.rs | 46 +- core/src/transport/denied.rs | 19 +- core/src/transport/interruptible.rs | 16 +- core/src/transport/map.rs | 42 +- core/src/transport/map_err.rs | 95 +--- core/src/transport/map_err_dial.rs | 20 +- core/src/transport/memory.rs | 18 +- core/src/transport/mod.rs | 55 +-- core/src/transport/upgrade.rs | 92 +--- core/src/upgrade/apply.rs | 37 +- core/src/upgrade/choice.rs | 54 +- core/src/upgrade/denied.rs | 8 +- core/src/upgrade/loop_upg.rs | 20 +- core/src/upgrade/map.rs | 12 +- core/src/upgrade/mod.rs | 2 - core/src/upgrade/plaintext.rs | 9 +- core/src/upgrade/toggleable.rs | 10 +- core/src/upgrade/traits.rs | 7 +- muxers/mplex/src/lib.rs | 9 +- muxers/mplex/tests/two_peers.rs | 10 +- muxers/yamux/src/lib.rs | 11 +- protocols/floodsub/src/lib.rs | 19 +- protocols/identify/Cargo.toml | 2 + protocols/identify/src/lib.rs | 6 +- protocols/identify/src/protocol.rs | 31 +- protocols/kad/src/kad_server.rs | 15 +- protocols/kad/src/protocol.rs | 13 +- protocols/ping/src/lib.rs | 27 +- protocols/secio/src/lib.rs | 11 +- src/lib.rs | 1 - src/simple.rs | 10 +- transports/dns/src/lib.rs | 8 +- transports/ratelimit/src/lib.rs | 18 +- transports/relay/src/protocol.rs | 37 +- transports/relay/src/transport.rs | 17 +- transports/tcp/src/lib.rs | 83 ++-- transports/timeout/src/lib.rs | 55 +-- transports/uds/src/lib.rs | 17 +- transports/websocket/src/browser.rs | 12 +- transports/websocket/src/desktop.rs | 44 +- 52 files changed, 850 insertions(+), 1493 deletions(-) rename core/src/nodes/{swarm.rs => raw_swarm.rs} (66%) diff --git a/core/src/either.rs b/core/src/either.rs index 30a8b18df49..0566dbd36d9 100644 --- a/core/src/either.rs +++ b/core/src/either.rs @@ -18,10 +18,11 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use futures::{prelude::*, future}; +use futures::prelude::*; use muxing::{Shutdown, StreamMuxer}; use std::io::{Error as IoError, Read, Write}; use tokio_io::{AsyncRead, AsyncWrite}; +use Multiaddr; /// Implements `AsyncRead` and `AsyncWrite` and dispatches all method calls to /// either `First` or `Second`. @@ -39,8 +40,8 @@ where #[inline] unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { match self { - &EitherOutput::First(ref a) => a.prepare_uninitialized_buffer(buf), - &EitherOutput::Second(ref b) => b.prepare_uninitialized_buffer(buf), + EitherOutput::First(a) => a.prepare_uninitialized_buffer(buf), + EitherOutput::Second(b) => b.prepare_uninitialized_buffer(buf), } } } @@ -53,8 +54,8 @@ where #[inline] fn read(&mut self, buf: &mut [u8]) -> Result { match self { - &mut EitherOutput::First(ref mut a) => a.read(buf), - &mut EitherOutput::Second(ref mut b) => b.read(buf), + EitherOutput::First(a) => a.read(buf), + EitherOutput::Second(b) => b.read(buf), } } } @@ -67,8 +68,8 @@ where #[inline] fn shutdown(&mut self) -> Poll<(), IoError> { match self { - &mut EitherOutput::First(ref mut a) => a.shutdown(), - &mut EitherOutput::Second(ref mut b) => b.shutdown(), + EitherOutput::First(a) => a.shutdown(), + EitherOutput::Second(b) => b.shutdown(), } } } @@ -81,16 +82,16 @@ where #[inline] fn write(&mut self, buf: &[u8]) -> Result { match self { - &mut EitherOutput::First(ref mut a) => a.write(buf), - &mut EitherOutput::Second(ref mut b) => b.write(buf), + EitherOutput::First(a) => a.write(buf), + EitherOutput::Second(b) => b.write(buf), } } #[inline] fn flush(&mut self) -> Result<(), IoError> { match self { - &mut EitherOutput::First(ref mut a) => a.flush(), - &mut EitherOutput::Second(ref mut b) => b.flush(), + EitherOutput::First(a) => a.flush(), + EitherOutput::Second(b) => b.flush(), } } } @@ -104,16 +105,16 @@ where type OutboundSubstream = EitherOutbound; fn poll_inbound(&self) -> Poll, IoError> { - match *self { - EitherOutput::First(ref inner) => inner.poll_inbound().map(|p| p.map(|o| o.map(EitherOutput::First))), - EitherOutput::Second(ref inner) => inner.poll_inbound().map(|p| p.map(|o| o.map(EitherOutput::Second))), + match self { + EitherOutput::First(inner) => inner.poll_inbound().map(|p| p.map(|o| o.map(EitherOutput::First))), + EitherOutput::Second(inner) => inner.poll_inbound().map(|p| p.map(|o| o.map(EitherOutput::Second))), } } fn open_outbound(&self) -> Self::OutboundSubstream { - match *self { - EitherOutput::First(ref inner) => EitherOutbound::A(inner.open_outbound()), - EitherOutput::Second(ref inner) => EitherOutbound::B(inner.open_outbound()), + match self { + EitherOutput::First(inner) => EitherOutbound::A(inner.open_outbound()), + EitherOutput::Second(inner) => EitherOutbound::B(inner.open_outbound()), } } @@ -130,14 +131,14 @@ where } fn destroy_outbound(&self, substream: Self::OutboundSubstream) { - match *self { - EitherOutput::First(ref inner) => { + match self { + EitherOutput::First(inner) => { match substream { EitherOutbound::A(substream) => inner.destroy_outbound(substream), _ => panic!("Wrong API usage") } }, - EitherOutput::Second(ref inner) => { + EitherOutput::Second(inner) => { match substream { EitherOutbound::B(substream) => inner.destroy_outbound(substream), _ => panic!("Wrong API usage") @@ -195,14 +196,14 @@ where } fn destroy_substream(&self, substream: Self::Substream) { - match *self { - EitherOutput::First(ref inner) => { + match self { + EitherOutput::First(inner) => { match substream { EitherOutput::First(substream) => inner.destroy_substream(substream), _ => panic!("Wrong API usage") } }, - EitherOutput::Second(ref inner) => { + EitherOutput::Second(inner) => { match substream { EitherOutput::Second(substream) => inner.destroy_substream(substream), _ => panic!("Wrong API usage") @@ -212,16 +213,16 @@ where } fn shutdown(&self, kind: Shutdown) -> Poll<(), IoError> { - match *self { - EitherOutput::First(ref inner) => inner.shutdown(kind), - EitherOutput::Second(ref inner) => inner.shutdown(kind) + match self { + EitherOutput::First(inner) => inner.shutdown(kind), + EitherOutput::Second(inner) => inner.shutdown(kind) } } fn flush_all(&self) -> Poll<(), IoError> { - match *self { - EitherOutput::First(ref inner) => inner.flush_all(), - EitherOutput::Second(ref inner) => inner.flush_all() + match self { + EitherOutput::First(inner) => inner.flush_all(), + EitherOutput::Second(inner) => inner.flush_all() } } } @@ -243,52 +244,44 @@ pub enum EitherListenStream { impl Stream for EitherListenStream where - AStream: Stream, - BStream: Stream, + AStream: Stream, + BStream: Stream, { - type Item = EitherListenUpgrade; + type Item = (EitherFuture, Multiaddr); type Error = IoError; #[inline] fn poll(&mut self) -> Poll, Self::Error> { match self { - &mut EitherListenStream::First(ref mut a) => a.poll() - .map(|i| i.map(|v| v.map(EitherListenUpgrade::First))), - &mut EitherListenStream::Second(ref mut a) => a.poll() - .map(|i| i.map(|v| v.map(EitherListenUpgrade::Second))), + EitherListenStream::First(a) => a.poll() + .map(|i| (i.map(|v| (v.map(|(o, addr)| (EitherFuture::First(o), addr)))))), + EitherListenStream::Second(a) => a.poll() + .map(|i| (i.map(|v| (v.map(|(o, addr)| (EitherFuture::Second(o), addr)))))), } } } -// TODO: This type is needed because of the lack of `impl Trait` in stable Rust. -// If Rust had impl Trait we could use the Either enum from the futures crate and add some -// modifiers to it. This custom enum is a combination of Either and these modifiers. +/// Implements `Future` and dispatches all method calls to either `First` or `Second`. #[derive(Debug, Copy, Clone)] #[must_use = "futures do nothing unless polled"] -pub enum EitherListenUpgrade { +pub enum EitherFuture { First(A), Second(B), } -impl Future for EitherListenUpgrade +impl Future for EitherFuture where - A: Future, - B: Future, + AFuture: Future, + BFuture: Future, { - type Item = (EitherOutput, future::Either); + type Item = EitherOutput; type Error = IoError; #[inline] fn poll(&mut self) -> Poll { match self { - &mut EitherListenUpgrade::First(ref mut a) => { - let (item, addr) = try_ready!(a.poll()); - Ok(Async::Ready((EitherOutput::First(item), future::Either::A(addr)))) - } - &mut EitherListenUpgrade::Second(ref mut b) => { - let (item, addr) = try_ready!(b.poll()); - Ok(Async::Ready((EitherOutput::Second(item), future::Either::B(addr)))) - } + EitherFuture::First(a) => a.poll().map(|v| v.map(EitherOutput::First)), + EitherFuture::Second(a) => a.poll().map(|v| v.map(EitherOutput::Second)), } } } diff --git a/core/src/lib.rs b/core/src/lib.rs index 00292c3ab4a..798405bba50 100644 --- a/core/src/lib.rs +++ b/core/src/lib.rs @@ -142,7 +142,7 @@ //! // TODO: right now the only available protocol is ping, but we want to replace it with //! // something that is more simple to use //! .dial("127.0.0.1:12345".parse::().unwrap()).unwrap_or_else(|_| panic!()) -//! .and_then(|(out, _)| { +//! .and_then(|out| { //! match out { //! PingOutput::Ponger(processing) => Box::new(processing) as Box>, //! PingOutput::Pinger(mut pinger) => { @@ -220,5 +220,5 @@ pub use self::multiaddr::Multiaddr; pub use self::muxing::StreamMuxer; pub use self::peer_id::PeerId; pub use self::public_key::PublicKey; -pub use self::transport::{MuxedTransport, Transport}; +pub use self::transport::Transport; pub use self::upgrade::{ConnectionUpgrade, Endpoint}; diff --git a/core/src/nodes/collection.rs b/core/src/nodes/collection.rs index 90384314fa3..fc17f43aaf8 100644 --- a/core/src/nodes/collection.rs +++ b/core/src/nodes/collection.rs @@ -25,16 +25,15 @@ use nodes::node::Substream; use nodes::handled_node_tasks::{HandledNodesEvent, HandledNodesTasks}; use nodes::handled_node_tasks::{Task as HandledNodesTask, TaskId}; use nodes::handled_node::NodeHandler; -use std::{collections::hash_map::Entry, fmt, mem}; -use std::io::{Error as IoError, ErrorKind as IoErrorKind}; -use {Multiaddr, PeerId}; +use std::{collections::hash_map::Entry, fmt, io, mem}; +use PeerId; // TODO: make generic over PeerId /// Implementation of `Stream` that handles a collection of nodes. -pub struct CollectionStream { +pub struct CollectionStream { /// Object that handles the tasks. - inner: HandledNodesTasks, + inner: HandledNodesTasks, /// List of nodes, with the task id that handles this node. The corresponding entry in `tasks` /// must always be in the `Connected` state. nodes: FnvHashMap, @@ -43,7 +42,7 @@ pub struct CollectionStream { tasks: FnvHashMap, } -impl fmt::Debug for CollectionStream { +impl fmt::Debug for CollectionStream { fn fmt(&self, f: &mut fmt::Formatter) -> Result<(), fmt::Error> { let mut list = f.debug_list(); for (id, task) in &self.tasks { @@ -70,10 +69,10 @@ enum TaskState { } /// Event that can happen on the `CollectionStream`. -pub enum CollectionEvent<'a, TInEvent:'a , TOutEvent: 'a> { +pub enum CollectionEvent<'a, TInEvent:'a , TOutEvent: 'a, THandler: 'a> { /// A connection to a node has succeeded. You must use the provided event in order to accept /// the connection. - NodeReached(CollectionReachEvent<'a, TInEvent, TOutEvent>), + NodeReached(CollectionReachEvent<'a, TInEvent, TOutEvent, THandler>), /// A connection to a node has been closed. /// @@ -85,11 +84,13 @@ pub enum CollectionEvent<'a, TInEvent:'a , TOutEvent: 'a> { }, /// A connection to a node has errored. + /// + /// Can only happen after a node has been successfully reached. NodeError { /// Identifier of the node. peer_id: PeerId, /// The error that happened. - error: IoError, + error: io::Error, }, /// An error happened on the future that was trying to reach a node. @@ -97,7 +98,9 @@ pub enum CollectionEvent<'a, TInEvent:'a , TOutEvent: 'a> { /// Identifier of the reach attempt that failed. id: ReachAttemptId, /// Error that happened on the future. - error: IoError, + error: io::Error, + /// The handler that was passed to `add_reach_attempt`. + handler: THandler, }, /// A node has produced an event. @@ -109,7 +112,7 @@ pub enum CollectionEvent<'a, TInEvent:'a , TOutEvent: 'a> { }, } -impl<'a, TInEvent, TOutEvent> fmt::Debug for CollectionEvent<'a, TInEvent, TOutEvent> +impl<'a, TInEvent, TOutEvent, THandler> fmt::Debug for CollectionEvent<'a, TInEvent, TOutEvent, THandler> where TOutEvent: fmt::Debug { fn fmt(&self, f: &mut fmt::Formatter) -> Result<(), fmt::Error> { @@ -130,7 +133,7 @@ where TOutEvent: fmt::Debug .field("error", error) .finish() }, - CollectionEvent::ReachError { ref id, ref error } => { + CollectionEvent::ReachError { ref id, ref error, .. } => { f.debug_struct("CollectionEvent::ReachError") .field("id", id) .field("error", error) @@ -148,16 +151,16 @@ where TOutEvent: fmt::Debug /// Event that happens when we reach a node. #[must_use = "The node reached event is used to accept the newly-opened connection"] -pub struct CollectionReachEvent<'a, TInEvent: 'a, TOutEvent: 'a> { +pub struct CollectionReachEvent<'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a> { /// Peer id we connected to. peer_id: PeerId, /// The task id that reached the node. id: TaskId, /// The `CollectionStream` we are referencing. - parent: &'a mut CollectionStream, + parent: &'a mut CollectionStream, } -impl<'a, TInEvent, TOutEvent> CollectionReachEvent<'a, TInEvent, TOutEvent> { +impl<'a, TInEvent, TOutEvent, THandler> CollectionReachEvent<'a, TInEvent, TOutEvent, THandler> { /// Returns the peer id the node that has been reached. #[inline] pub fn peer_id(&self) -> &PeerId { @@ -220,7 +223,7 @@ impl<'a, TInEvent, TOutEvent> CollectionReachEvent<'a, TInEvent, TOutEvent> { } } -impl<'a, TInEvent, TOutEvent> fmt::Debug for CollectionReachEvent<'a, TInEvent, TOutEvent> { +impl<'a, TInEvent, TOutEvent, THandler> fmt::Debug for CollectionReachEvent<'a, TInEvent, TOutEvent, THandler> { fn fmt(&self, f: &mut fmt::Formatter) -> Result<(), fmt::Error> { f.debug_struct("CollectionReachEvent") .field("peer_id", &self.peer_id) @@ -229,7 +232,7 @@ impl<'a, TInEvent, TOutEvent> fmt::Debug for CollectionReachEvent<'a, TInEvent, } } -impl<'a, TInEvent, TOutEvent> Drop for CollectionReachEvent<'a, TInEvent, TOutEvent> { +impl<'a, TInEvent, TOutEvent, THandler> Drop for CollectionReachEvent<'a, TInEvent, TOutEvent, THandler> { fn drop(&mut self) { let task_state = self.parent.tasks.remove(&self.id); debug_assert!(if let Some(TaskState::Pending) = task_state { true } else { false }); @@ -255,7 +258,7 @@ pub enum CollectionNodeAccept { #[derive(Debug, Copy, Clone, Hash, PartialEq, Eq, PartialOrd, Ord)] pub struct ReachAttemptId(TaskId); -impl CollectionStream { +impl CollectionStream { /// Creates a new empty collection. #[inline] pub fn new() -> Self { @@ -270,12 +273,11 @@ impl CollectionStream { /// /// This method spawns a task dedicated to resolving this future and processing the node's /// events. - pub fn add_reach_attempt(&mut self, future: TFut, handler: THandler) + pub fn add_reach_attempt(&mut self, future: TFut, handler: THandler) -> ReachAttemptId where - TFut: Future + Send + 'static, - TAddrFut: Future + Send + 'static, - THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, + TFut: Future + Send + 'static, + THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, TInEvent: Send + 'static, TOutEvent: Send + 'static, THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be required? @@ -362,44 +364,44 @@ impl CollectionStream { /// > **Note**: we use a regular `poll` method instead of implementing `Stream` in order to /// > remove the `Err` variant, but also because we want the `CollectionStream` to stay /// > borrowed if necessary. - pub fn poll(&mut self) -> Async>> { + pub fn poll(&mut self) -> Async> { let item = match self.inner.poll() { Async::Ready(item) => item, Async::NotReady => return Async::NotReady, }; match item { - Some(HandledNodesEvent::TaskClosed { id, result }) => { - match (self.tasks.remove(&id), result) { - (Some(TaskState::Pending), Err(err)) => { - Async::Ready(Some(CollectionEvent::ReachError { + HandledNodesEvent::TaskClosed { id, result, handler } => { + match (self.tasks.remove(&id), result, handler) { + (Some(TaskState::Pending), Err(err), Some(handler)) => { + Async::Ready(CollectionEvent::ReachError { id: ReachAttemptId(id), error: err, - })) + handler, + }) }, - (Some(TaskState::Pending), Ok(())) => { + (Some(TaskState::Pending), _, _) => { // TODO: this variant shouldn't happen ; prove this - Async::Ready(Some(CollectionEvent::ReachError { - id: ReachAttemptId(id), - error: IoError::new(IoErrorKind::Other, "couldn't reach the node"), - })) + panic!() }, - (Some(TaskState::Connected(peer_id)), Ok(())) => { + (Some(TaskState::Connected(peer_id)), Ok(()), _handler) => { + debug_assert!(_handler.is_none()); let _node_task_id = self.nodes.remove(&peer_id); debug_assert_eq!(_node_task_id, Some(id)); - Async::Ready(Some(CollectionEvent::NodeClosed { + Async::Ready(CollectionEvent::NodeClosed { peer_id, - })) + }) }, - (Some(TaskState::Connected(peer_id)), Err(err)) => { + (Some(TaskState::Connected(peer_id)), Err(err), _handler) => { + debug_assert!(_handler.is_none()); let _node_task_id = self.nodes.remove(&peer_id); debug_assert_eq!(_node_task_id, Some(id)); - Async::Ready(Some(CollectionEvent::NodeError { + Async::Ready(CollectionEvent::NodeError { peer_id, error: err, - })) + }) }, - (None, _) => { + (None, _, _) => { panic!("self.tasks is always kept in sync with the tasks in self.inner ; \ when we add a task in self.inner we add a corresponding entry in \ self.tasks, and remove the entry only when the task is closed ; \ @@ -407,14 +409,14 @@ impl CollectionStream { }, } }, - Some(HandledNodesEvent::NodeReached { id, peer_id }) => { - Async::Ready(Some(CollectionEvent::NodeReached(CollectionReachEvent { + HandledNodesEvent::NodeReached { id, peer_id } => { + Async::Ready(CollectionEvent::NodeReached(CollectionReachEvent { parent: self, id, peer_id, - }))) + })) }, - Some(HandledNodesEvent::NodeEvent { id, event }) => { + HandledNodesEvent::NodeEvent { id, event } => { let peer_id = match self.tasks.get(&id) { Some(TaskState::Connected(peer_id)) => peer_id.clone(), _ => panic!("we can only receive NodeEvent events from a task after we \ @@ -423,12 +425,11 @@ impl CollectionStream { self.tasks is switched to the Connected state ; qed"), }; - Async::Ready(Some(CollectionEvent::NodeEvent { + Async::Ready(CollectionEvent::NodeEvent { peer_id, event, - })) + }) } - None => Async::Ready(None), } } } diff --git a/core/src/nodes/handled_node.rs b/core/src/nodes/handled_node.rs index 456d782718d..cc638ab6f26 100644 --- a/core/src/nodes/handled_node.rs +++ b/core/src/nodes/handled_node.rs @@ -22,20 +22,18 @@ use muxing::StreamMuxer; use nodes::node::{NodeEvent, NodeStream, Substream}; use futures::{prelude::*, stream::Fuse}; use std::io::Error as IoError; -use Multiaddr; /// Handler for the substreams of a node. -/// -/// > Note: When implementing the various methods, don't forget that you have to register the -/// > task that was the latest to poll and notify it. // TODO: right now it is possible for a node handler to be built, then shut down right after if we // realize we dialed the wrong peer for example ; this could be surprising and should either // be documented or changed (favouring the "documented" right now) -pub trait NodeHandler { +pub trait NodeHandler { /// Custom event that can be received from the outside. type InEvent; /// Custom event that can be produced by the handler and that will be returned by the swarm. type OutEvent; + /// The type of the substream containing the data. + type Substream; /// Information about a substream. Can be sent to the handler through a `NodeHandlerEndpoint`, /// and will be passed back in `inject_substream` or `inject_outbound_closed`. type OutboundOpenInfo; @@ -43,7 +41,7 @@ pub trait NodeHandler { /// Sends a new substream to the handler. /// /// The handler is responsible for upgrading the substream to whatever protocol it wants. - fn inject_substream(&mut self, substream: TSubstream, endpoint: NodeHandlerEndpoint); + fn inject_substream(&mut self, substream: Self::Substream, endpoint: NodeHandlerEndpoint); /// Indicates to the handler that the inbound part of the muxer has been closed, and that /// therefore no more inbound substream will be produced. @@ -53,9 +51,6 @@ pub trait NodeHandler { /// part of the muxer has been closed. fn inject_outbound_closed(&mut self, user_data: Self::OutboundOpenInfo); - /// Indicates to the handler that the multiaddr future has resolved. - fn inject_multiaddr(&mut self, multiaddr: Result); - /// Injects an event coming from the outside into the handler. fn inject_event(&mut self, event: Self::InEvent); @@ -78,6 +73,26 @@ pub enum NodeHandlerEndpoint { Listener, } +impl NodeHandlerEndpoint { + /// Returns true for `Dialer`. + #[inline] + pub fn is_dialer(&self) -> bool { + match self { + NodeHandlerEndpoint::Dialer(_) => true, + NodeHandlerEndpoint::Listener => false, + } + } + + /// Returns true for `Listener`. + #[inline] + pub fn is_listener(&self) -> bool { + match self { + NodeHandlerEndpoint::Dialer(_) => false, + NodeHandlerEndpoint::Listener => true, + } + } +} + /// Event produced by a handler. #[derive(Debug, Copy, Clone, PartialEq, Eq)] pub enum NodeHandlerEvent { @@ -119,30 +134,29 @@ impl NodeHandlerEvent { /// A node combined with an implementation of `NodeHandler`. // TODO: impl Debug -pub struct HandledNode +pub struct HandledNode where TMuxer: StreamMuxer, - THandler: NodeHandler>, + THandler: NodeHandler>, { /// Node that handles the muxing. - node: Fuse>, + node: Fuse>, /// Handler that processes substreams. handler: THandler, // True, if the node is shutting down. is_shutting_down: bool } -impl HandledNode +impl HandledNode where TMuxer: StreamMuxer, - THandler: NodeHandler>, - TAddrFut: Future, + THandler: NodeHandler>, { /// Builds a new `HandledNode`. #[inline] - pub fn new(muxer: TMuxer, multiaddr_future: TAddrFut, handler: THandler) -> Self { + pub fn new(muxer: TMuxer, handler: THandler) -> Self { HandledNode { - node: NodeStream::new(muxer, multiaddr_future).fuse(), + node: NodeStream::new(muxer).fuse(), handler, is_shutting_down: false } @@ -192,11 +206,10 @@ where } } -impl Stream for HandledNode +impl Stream for HandledNode where TMuxer: StreamMuxer, - THandler: NodeHandler>, - TAddrFut: Future, + THandler: NodeHandler>, { type Item = THandler::OutEvent; type Error = IoError; @@ -220,9 +233,6 @@ where self.handler.shutdown() } } - Async::Ready(Some(NodeEvent::Multiaddr(result))) => { - self.handler.inject_multiaddr(result) - } Async::Ready(Some(NodeEvent::OutboundClosed { user_data })) => { self.handler.inject_outbound_closed(user_data) } @@ -263,8 +273,8 @@ where #[cfg(test)] mod tests { use super::*; - use futures::future; use muxing::{StreamMuxer, Shutdown}; + use std::marker::PhantomData; use tokio::runtime::current_thread; // TODO: move somewhere? this could be useful as a dummy @@ -288,15 +298,17 @@ mod tests { #[test] fn proper_shutdown() { // Test that `shutdown()` is properly called on the handler once a node stops. - struct Handler { + struct Handler { did_substream_attempt: bool, inbound_closed: bool, substream_attempt_cancelled: bool, shutdown_called: bool, + marker: PhantomData, }; - impl NodeHandler for Handler { + impl NodeHandler for Handler { type InEvent = (); type OutEvent = (); + type Substream = T; type OutboundOpenInfo = (); fn inject_substream(&mut self, _: T, _: NodeHandlerEndpoint<()>) { panic!() } fn inject_inbound_closed(&mut self) { @@ -307,7 +319,6 @@ mod tests { assert!(!self.substream_attempt_cancelled); self.substream_attempt_cancelled = true; } - fn inject_multiaddr(&mut self, _: Result) {} fn inject_event(&mut self, _: Self::InEvent) { panic!() } fn shutdown(&mut self) { assert!(self.inbound_closed); @@ -325,17 +336,18 @@ mod tests { } } } - impl Drop for Handler { + impl Drop for Handler { fn drop(&mut self) { assert!(self.shutdown_called); } } - let handled = HandledNode::new(InstaCloseMuxer, future::empty(), Handler { + let handled = HandledNode::new(InstaCloseMuxer, Handler { did_substream_attempt: false, inbound_closed: false, substream_attempt_cancelled: false, shutdown_called: false, + marker: PhantomData, }); current_thread::Runtime::new().unwrap().block_on(handled.for_each(|_| Ok(()))).unwrap(); diff --git a/core/src/nodes/handled_node_tasks.rs b/core/src/nodes/handled_node_tasks.rs index 55457205b84..611330d2857 100644 --- a/core/src/nodes/handled_node_tasks.rs +++ b/core/src/nodes/handled_node_tasks.rs @@ -29,7 +29,7 @@ use std::io::Error as IoError; use std::{fmt, mem}; use tokio_executor; use void::Void; -use {Multiaddr, PeerId}; +use PeerId; // TODO: make generic over PeerId @@ -51,7 +51,7 @@ use {Multiaddr, PeerId}; /// Implementation of `Stream` that handles a collection of nodes. // TODO: implement Debug -pub struct HandledNodesTasks { +pub struct HandledNodesTasks { /// For each active task, a sender allowing to transmit messages. 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. @@ -64,12 +64,12 @@ pub struct HandledNodesTasks { to_spawn: SmallVec<[Box + Send>; 8]>, /// Sender to emit events to the outside. Meant to be cloned and sent to tasks. - events_tx: mpsc::UnboundedSender<(InToExtMessage, TaskId)>, + events_tx: mpsc::UnboundedSender<(InToExtMessage, TaskId)>, /// Receiver side for the events. - events_rx: mpsc::UnboundedReceiver<(InToExtMessage, TaskId)>, + events_rx: mpsc::UnboundedReceiver<(InToExtMessage, TaskId)>, } -impl fmt::Debug for HandledNodesTasks { +impl fmt::Debug for HandledNodesTasks { fn fmt(&self, f: &mut fmt::Formatter) -> Result<(), fmt::Error> { f.debug_list() .entries(self.tasks.keys().cloned()) @@ -79,15 +79,19 @@ impl fmt::Debug for HandledNodesTasks /// Event that can happen on the `HandledNodesTasks`. #[derive(Debug)] -pub enum HandledNodesEvent { +pub enum HandledNodesEvent { /// A task has been closed. /// /// This happens once the node handler closes or an error happens. + // TODO: send back undelivered events? TaskClosed { /// Identifier of the task that closed. id: TaskId, /// What happened. result: Result<(), IoError>, + /// If the task closed before reaching the node, this contains the handler that was passed + /// to `add_reach_attempt`. + handler: Option, }, /// A task has succeesfully connected to a node. @@ -111,7 +115,7 @@ pub enum HandledNodesEvent { #[derive(Debug, Copy, Clone, Hash, PartialEq, Eq, PartialOrd, Ord)] pub struct TaskId(usize); -impl HandledNodesTasks { +impl HandledNodesTasks { /// Creates a new empty collection. #[inline] pub fn new() -> Self { @@ -130,12 +134,11 @@ impl HandledNodesTasks { /// /// This method spawns a task dedicated to resolving this future and processing the node's /// events. - pub fn add_reach_attempt(&mut self, future: TFut, handler: THandler) + pub fn add_reach_attempt(&mut self, future: TFut, handler: THandler) -> TaskId where - TFut: Future + Send + 'static, - TAddrFut: Future + Send + 'static, - THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, + TFut: Future + Send + 'static, + THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, TInEvent: Send + 'static, TOutEvent: Send + 'static, THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be required? @@ -193,7 +196,7 @@ impl HandledNodesTasks { } /// Provides an API similar to `Stream`, except that it cannot error. - pub fn poll(&mut self) -> Async>> { + pub fn poll(&mut self) -> Async> { for to_spawn in self.to_spawn.drain() { tokio_executor::spawn(to_spawn); } @@ -210,22 +213,22 @@ impl HandledNodesTasks { match message { InToExtMessage::NodeEvent(event) => { - break Async::Ready(Some(HandledNodesEvent::NodeEvent { + break Async::Ready(HandledNodesEvent::NodeEvent { id: task_id, event, - })); + }); }, InToExtMessage::NodeReached(peer_id) => { - break Async::Ready(Some(HandledNodesEvent::NodeReached { + break Async::Ready(HandledNodesEvent::NodeReached { id: task_id, peer_id, - })); + }); }, - InToExtMessage::TaskClosed(result) => { + InToExtMessage::TaskClosed(result, handler) => { let _ = self.tasks.remove(&task_id); - break Async::Ready(Some(HandledNodesEvent::TaskClosed { - id: task_id, result - })); + break Async::Ready(HandledNodesEvent::TaskClosed { + id: task_id, result, handler + }); }, } } @@ -249,6 +252,7 @@ pub struct Task<'a, TInEvent: 'a> { impl<'a, TInEvent> Task<'a, TInEvent> { /// 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 @@ -279,48 +283,48 @@ impl<'a, TInEvent> fmt::Debug for Task<'a, TInEvent> { } } -impl Stream for HandledNodesTasks { - type Item = HandledNodesEvent; +impl Stream for HandledNodesTasks { + type Item = HandledNodesEvent; type Error = Void; // TODO: use ! once stable #[inline] fn poll(&mut self) -> Poll, Self::Error> { - Ok(self.poll()) + Ok(self.poll().map(Option::Some)) } } /// Message to transmit from a task to the public API. #[derive(Debug)] -enum InToExtMessage { +enum InToExtMessage { /// A connection to a node has succeeded. NodeReached(PeerId), /// The task closed. - TaskClosed(Result<(), IoError>), + TaskClosed(Result<(), IoError>, Option), /// 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 +struct NodeTask where TMuxer: StreamMuxer, - THandler: NodeHandler>, + THandler: NodeHandler>, { /// Sender to transmit events to the outside. - events_tx: mpsc::UnboundedSender<(InToExtMessage, TaskId)>, + events_tx: mpsc::UnboundedSender<(InToExtMessage, TaskId)>, /// Receiving end for events sent from the main `HandledNodesTasks`. in_events_rx: stream::Fuse>, /// Inner state of the `NodeTask`. - inner: NodeTaskInner, + inner: NodeTaskInner, /// Identifier of the attempt. id: TaskId, } -enum NodeTaskInner +enum NodeTaskInner where TMuxer: StreamMuxer, - THandler: NodeHandler>, + THandler: NodeHandler>, { /// Future to resolve to connect to the node. Future { @@ -335,19 +339,18 @@ where }, /// Fully functional node. - Node(HandledNode), + Node(HandledNode), /// A panic happened while polling. Poisoned, } -impl Future for - NodeTask +impl Future for + NodeTask where TMuxer: StreamMuxer, - TFut: Future, - TAddrFut: Future, - THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent>, + TFut: Future, + THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent>, { type Item = (); type Error = (); @@ -369,9 +372,9 @@ where // Check whether dialing succeeded. match future.poll() { - Ok(Async::Ready(((peer_id, muxer), addr_fut))) => { + Ok(Async::Ready((peer_id, muxer))) => { let event = InToExtMessage::NodeReached(peer_id); - let mut node = HandledNode::new(muxer, addr_fut, handler); + let mut node = HandledNode::new(muxer, handler); for event in events_buffer { node.inject_event(event); } @@ -386,7 +389,7 @@ where }, Err(err) => { // End the task - let event = InToExtMessage::TaskClosed(Err(err)); + let event = InToExtMessage::TaskClosed(Err(err), Some(handler)); let _ = self.events_tx.unbounded_send((event, self.id)); return Ok(Async::Ready(())); } @@ -427,12 +430,12 @@ where } } Ok(Async::Ready(None)) => { - let event = InToExtMessage::TaskClosed(Ok(())); + let event = InToExtMessage::TaskClosed(Ok(()), None); let _ = self.events_tx.unbounded_send((event, self.id)); return Ok(Async::Ready(())); // End the task. } Err(err) => { - let event = InToExtMessage::TaskClosed(Err(err)); + let event = InToExtMessage::TaskClosed(Err(err), None); let _ = self.events_tx.unbounded_send((event, self.id)); return Ok(Async::Ready(())); // End the task. } diff --git a/core/src/nodes/listeners.rs b/core/src/nodes/listeners.rs index 80ded1f1d86..c47a05ec6f8 100644 --- a/core/src/nodes/listeners.rs +++ b/core/src/nodes/listeners.rs @@ -60,7 +60,7 @@ use {Multiaddr, Transport}; /// ListenersEvent::Closed { listen_addr, listener, result } => { /// println!("Listener {} has been closed: {:?}", listen_addr, result); /// }, -/// ListenersEvent::Incoming { upgrade, listen_addr } => { +/// ListenersEvent::Incoming { upgrade, listen_addr, .. } => { /// println!("A connection has arrived on {}", listen_addr); /// // We don't do anything with the newly-opened connection, but in a real-life /// // program you probably want to use it! @@ -107,6 +107,8 @@ where upgrade: TTrans::ListenerUpgrade, /// Address of the listener which received the connection. listen_addr: Multiaddr, + /// Address used to send back data to the incoming client. + send_back_addr: Multiaddr, }, /// A listener has closed, either gracefully or with an error. @@ -177,7 +179,7 @@ where } /// Provides an API similar to `Stream`, except that it cannot error. - pub fn poll(&mut self) -> Async>> { + pub fn poll(&mut self) -> Async> { // We remove each element from `listeners` one by one and add them back. for n in (0..self.listeners.len()).rev() { let mut listener = self.listeners.swap_remove(n); @@ -185,27 +187,28 @@ where Ok(Async::NotReady) => { self.listeners.push(listener); } - Ok(Async::Ready(Some(upgrade))) => { + Ok(Async::Ready(Some((upgrade, send_back_addr)))) => { let listen_addr = listener.address.clone(); self.listeners.push(listener); - return Async::Ready(Some(ListenersEvent::Incoming { + return Async::Ready(ListenersEvent::Incoming { upgrade, listen_addr, - })); + send_back_addr, + }); } Ok(Async::Ready(None)) => { - return Async::Ready(Some(ListenersEvent::Closed { + return Async::Ready(ListenersEvent::Closed { listen_addr: listener.address, listener: listener.listener, result: Ok(()), - })); + }); } Err(err) => { - return Async::Ready(Some(ListenersEvent::Closed { + return Async::Ready(ListenersEvent::Closed { listen_addr: listener.address, listener: listener.listener, result: Err(err), - })); + }); } } } @@ -224,7 +227,7 @@ where #[inline] fn poll(&mut self) -> Poll, Self::Error> { - Ok(self.poll()) + Ok(self.poll().map(Option::Some)) } } @@ -294,7 +297,7 @@ mod tests { Async::Ready(Some(n)) => { let addr = l.address.clone(); let stream = stream::iter_ok(n..) - .map(move |stream| future::ok( (stream, future::ok(addr.clone())) )); + .map(move |stream| (future::ok(stream), addr.clone())); Box::new(stream) } Async::Ready(None) => { @@ -320,8 +323,9 @@ mod tests { .map_err(|(err, _)| err) .and_then(|(event, _)| { match event { - Some(ListenersEvent::Incoming { listen_addr, upgrade }) => { + Some(ListenersEvent::Incoming { listen_addr, upgrade, send_back_addr }) => { assert_eq!(listen_addr, "/memory".parse().unwrap()); + assert_eq!(send_back_addr, "/memory".parse().unwrap()); upgrade.map(|_| ()).map_err(|_| panic!()) }, _ => panic!() @@ -384,11 +388,11 @@ mod tests { ls.listen_on(addr1).expect("listen_on failed"); ls.listen_on(addr2).expect("listen_on failed"); - assert_matches!(ls.poll(), Async::Ready(Some(listeners_event)) => { - assert_matches!(listeners_event, ListenersEvent::Incoming{mut upgrade, listen_addr} => { + assert_matches!(ls.poll(), Async::Ready(listeners_event) => { + assert_matches!(listeners_event, ListenersEvent::Incoming{mut upgrade, listen_addr, ..} => { assert_eq!(listen_addr.to_string(), "/ip4/127.0.0.2/tcp/4321"); assert_matches!(upgrade.poll().unwrap(), Async::Ready(tup) => { - assert_matches!(tup, (1, _)) + assert_eq!(tup, 1) }); }) }); @@ -407,11 +411,11 @@ mod tests { // Make the second listener return NotReady so we get the first listener next poll() set_listener_state(&mut ls, 1, ListenerState::Ok(Async::NotReady)); - assert_matches!(ls.poll(), Async::Ready(Some(listeners_event)) => { - assert_matches!(listeners_event, ListenersEvent::Incoming{mut upgrade, listen_addr} => { + assert_matches!(ls.poll(), Async::Ready(listeners_event) => { + assert_matches!(listeners_event, ListenersEvent::Incoming{mut upgrade, listen_addr, ..} => { assert_eq!(listen_addr.to_string(), "/ip4/127.0.0.1/tcp/1234"); assert_matches!(upgrade.poll().unwrap(), Async::Ready(tup) => { - assert_matches!(tup, (1, _)) + assert_eq!(tup, 1) }); }) }); @@ -425,7 +429,7 @@ mod tests { let mut ls = ListenersStream::new(t); ls.listen_on(addr).expect("listen_on failed"); set_listener_state(&mut ls, 0, ListenerState::Ok(Async::Ready(None))); - assert_matches!(ls.poll(), Async::Ready(Some(listeners_event)) => { + assert_matches!(ls.poll(), Async::Ready(listeners_event) => { assert_matches!(listeners_event, ListenersEvent::Closed{..}) }); assert_eq!(ls.listeners.len(), 0); // it's gone @@ -439,7 +443,7 @@ mod tests { let mut ls = ListenersStream::new(t); ls.listen_on(addr).expect("listen_on failed"); set_listener_state(&mut ls, 0, ListenerState::Error); // simulate an error on the socket - assert_matches!(ls.poll(), Async::Ready(Some(listeners_event)) => { + assert_matches!(ls.poll(), Async::Ready(listeners_event) => { assert_matches!(listeners_event, ListenersEvent::Closed{..}) }); assert_eq!(ls.listeners.len(), 0); // it's gone @@ -458,14 +462,14 @@ mod tests { // polling processes listeners in reverse order // Only the last listener ever gets processed for _n in 0..10 { - assert_matches!(ls.poll(), Async::Ready(Some(ListenersEvent::Incoming{listen_addr, ..})) => { + assert_matches!(ls.poll(), Async::Ready(ListenersEvent::Incoming{listen_addr, ..}) => { assert_eq!(listen_addr.to_string(), "/ip4/127.0.0.3/tcp/1233") }) } // Make last listener NotReady so now only the third listener is processed set_listener_state(&mut ls, 3, ListenerState::Ok(Async::NotReady)); for _n in 0..10 { - assert_matches!(ls.poll(), Async::Ready(Some(ListenersEvent::Incoming{listen_addr, ..})) => { + assert_matches!(ls.poll(), Async::Ready(ListenersEvent::Incoming{listen_addr, ..}) => { assert_eq!(listen_addr.to_string(), "/ip4/127.0.0.2/tcp/1232") }) } @@ -483,7 +487,7 @@ mod tests { // If the listeners do not yield items continuously (the normal case) we // process them in the expected, reverse, order. for n in (0..4).rev() { - assert_matches!(ls.poll(), Async::Ready(Some(ListenersEvent::Incoming{listen_addr, ..})) => { + assert_matches!(ls.poll(), Async::Ready(ListenersEvent::Incoming{listen_addr, ..}) => { assert_eq!(listen_addr.to_string(), format!("/ip4/127.0.0.{}/tcp/123{}", n, n)); }); // kick the last listener (current) to NotReady state diff --git a/core/src/nodes/mod.rs b/core/src/nodes/mod.rs index bf206e94b75..1d78ea7b3a3 100644 --- a/core/src/nodes/mod.rs +++ b/core/src/nodes/mod.rs @@ -18,10 +18,13 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -mod handled_node_tasks; - pub mod collection; pub mod handled_node; +pub mod handled_node_tasks; pub mod listeners; pub mod node; -pub mod swarm; +pub mod raw_swarm; + +pub use self::node::Substream; +pub use self::handled_node::{NodeHandlerEvent, NodeHandlerEndpoint}; +pub use self::raw_swarm::{ConnectedPoint, Peer, RawSwarm, RawSwarmEvent}; diff --git a/core/src/nodes/node.rs b/core/src/nodes/node.rs index 10c2098b3d1..e072e57d068 100644 --- a/core/src/nodes/node.rs +++ b/core/src/nodes/node.rs @@ -24,16 +24,12 @@ use smallvec::SmallVec; use std::fmt; use std::io::Error as IoError; use std::sync::Arc; -use Multiaddr; // Implementor notes // ================= // // In order to minimize the risk of bugs in higher-level code, we want to avoid as much as // possible having a racy API. The behaviour of methods should be well-defined and predictable. -// As an example, calling the `multiaddr()` method should return `Some` only after a -// `MultiaddrResolved` event has been emitted and never before, even if we technically already -// know the address. // // In order to respect this coding practice, we should theoretically provide events such as "data // incoming on a substream", or "a substream is ready to be written". This would however make the @@ -53,7 +49,7 @@ use Multiaddr; /// /// The stream will close once both the inbound and outbound channels are closed, and no more /// outbound substream attempt is pending. -pub struct NodeStream +pub struct NodeStream where TMuxer: muxing::StreamMuxer, { @@ -63,23 +59,10 @@ where inbound_state: StreamState, /// Tracks the state of the muxers outbound direction. outbound_state: StreamState, - /// Address of the node ; can be empty if the address hasn't been resolved yet. - address: Addr, /// List of substreams we are currently opening. outbound_substreams: SmallVec<[(TUserData, TMuxer::OutboundSubstream); 8]>, } -/// Address of the node. -#[derive(Debug, Clone)] -enum Addr { - /// Future that will resolve the address. - Future(TAddrFut), - /// The address is now known. - Resolved(Multiaddr), - /// An error happened while resolving the future. - Errored, -} - /// A successfully opened substream. pub type Substream = muxing::SubstreamRef>; @@ -102,12 +85,6 @@ pub enum NodeEvent where TMuxer: muxing::StreamMuxer, { - /// The multiaddress future of the node has been resolved. - /// - /// If this succeeded, after this event has been emitted calling `multiaddr()` will return - /// `Some`. - Multiaddr(Result), - /// A new inbound substream arrived. InboundSubstream { /// The newly-opened substream. @@ -137,35 +114,21 @@ where #[derive(Debug, Copy, Clone, Hash, PartialEq, Eq, PartialOrd, Ord)] pub struct OutboundSubstreamId(usize); -impl NodeStream +impl NodeStream where TMuxer: muxing::StreamMuxer, - TAddrFut: Future, { /// Creates a new node events stream. #[inline] - pub fn new(muxer: TMuxer, multiaddr_future: TAddrFut) -> Self { + pub fn new(muxer: TMuxer) -> Self { NodeStream { muxer: Arc::new(muxer), inbound_state: StreamState::Open, outbound_state: StreamState::Open, - address: Addr::Future(multiaddr_future), outbound_substreams: SmallVec::new(), } } - /// Returns the multiaddress of the node, if already known. - /// - /// This method will always return `None` before a successful `Multiaddr` event has been - /// returned by `poll()`, and will always return `Some` afterwards. - #[inline] - pub fn multiaddr(&self) -> Option<&Multiaddr> { - match self.address { - Addr::Resolved(ref addr) => Some(addr), - Addr::Future(_) | Addr::Errored => None, - } - } - /// Starts the process of opening a new outbound substream. /// /// Returns an error if the outbound side of the muxer is closed. @@ -286,10 +249,9 @@ where } } -impl Stream for NodeStream +impl Stream for NodeStream where TMuxer: muxing::StreamMuxer, - TAddrFut: Future, { type Item = NodeEvent; type Error = IoError; @@ -345,26 +307,6 @@ where } } - // Check whether the multiaddress is resolved. - { - let poll = match self.address { - Addr::Future(ref mut fut) => Some(fut.poll()), - Addr::Resolved(_) | Addr::Errored => None, - }; - - match poll { - Some(Ok(Async::NotReady)) | None => {} - Some(Ok(Async::Ready(addr))) => { - self.address = Addr::Resolved(addr.clone()); - return Ok(Async::Ready(Some(NodeEvent::Multiaddr(Ok(addr))))); - } - Some(Err(err)) => { - self.address = Addr::Errored; - return Ok(Async::Ready(Some(NodeEvent::Multiaddr(Err(err))))); - } - } - } - // Closing the node if there's no way we can do anything more. if self.inbound_state == StreamState::Closed && self.outbound_state == StreamState::Closed @@ -378,14 +320,12 @@ where } } -impl fmt::Debug for NodeStream +impl fmt::Debug for NodeStream where TMuxer: muxing::StreamMuxer, - TAddrFut: Future, { fn fmt(&self, f: &mut fmt::Formatter) -> Result<(), fmt::Error> { f.debug_struct("NodeStream") - .field("address", &self.multiaddr()) .field("inbound_state", &self.inbound_state) .field("outbound_state", &self.outbound_state) .field("outbound_substreams", &self.outbound_substreams.len()) @@ -393,7 +333,7 @@ where } } -impl Drop for NodeStream +impl Drop for NodeStream where TMuxer: muxing::StreamMuxer, { @@ -436,40 +376,22 @@ where TTrans: Transport, #[cfg(test)] mod node_stream { - use multiaddr::Multiaddr; use super::NodeStream; - use futures::{future::self, prelude::*, Future}; + use futures::prelude::*; use tokio_mock_task::MockTask; use super::NodeEvent; use tests::dummy_muxer::{DummyMuxer, DummyConnectionState}; - use std::io::Error as IoError; - - fn build_node_stream() -> NodeStream, Vec> { - let addr = future::ok("/ip4/127.0.0.1/tcp/1234".parse::().expect("bad maddr")); + fn build_node_stream() -> NodeStream> { let muxer = DummyMuxer::new(); - NodeStream::<_, _, Vec>::new(muxer, addr) - } - - #[test] - fn multiaddr_is_available_once_polled() { - let mut node_stream = build_node_stream(); - assert!(node_stream.multiaddr().is_none()); - match node_stream.poll() { - Ok(Async::Ready(Some(NodeEvent::Multiaddr(Ok(addr))))) => { - assert_eq!(addr.to_string(), "/ip4/127.0.0.1/tcp/1234") - } - _ => panic!("unexpected poll return value" ) - } - assert!(node_stream.multiaddr().is_some()); + NodeStream::<_, Vec>::new(muxer) } #[test] fn can_open_outbound_substreams_until_an_outbound_channel_is_closed() { - let addr = future::ok("/ip4/127.0.0.1/tcp/1234".parse::().expect("bad maddr")); let mut muxer = DummyMuxer::new(); muxer.set_outbound_connection_state(DummyConnectionState::Closed); - let mut ns = NodeStream::<_, _, Vec>::new(muxer, addr); + let mut ns = NodeStream::<_, Vec>::new(muxer); // open first substream works assert!(ns.open_substream(vec![1,2,3]).is_ok()); @@ -498,10 +420,9 @@ mod node_stream { #[test] fn query_inbound_state() { - let addr = future::ok("/ip4/127.0.0.1/tcp/1234".parse::().expect("bad maddr")); let mut muxer = DummyMuxer::new(); muxer.set_inbound_connection_state(DummyConnectionState::Closed); - let mut ns = NodeStream::<_, _, Vec>::new(muxer, addr); + let mut ns = NodeStream::<_, Vec>::new(muxer); assert_matches!(ns.poll(), Ok(Async::Ready(Some(node_event))) => { assert_matches!(node_event, NodeEvent::InboundClosed) @@ -512,10 +433,9 @@ mod node_stream { #[test] fn query_outbound_state() { - let addr = future::ok("/ip4/127.0.0.1/tcp/1234".parse::().expect("bad multiaddr")); let mut muxer = DummyMuxer::new(); muxer.set_outbound_connection_state(DummyConnectionState::Closed); - let mut ns = NodeStream::<_, _, Vec>::new(muxer, addr); + let mut ns = NodeStream::<_, Vec>::new(muxer); assert!(ns.is_outbound_open()); @@ -548,13 +468,12 @@ mod node_stream { let mut task = MockTask::new(); task.enter(|| { // ensure the address never resolves - let addr = future::empty(); let mut muxer = DummyMuxer::new(); // ensure muxer.poll_inbound() returns Async::NotReady muxer.set_inbound_connection_state(DummyConnectionState::Pending); // ensure muxer.poll_outbound() returns Async::NotReady muxer.set_outbound_connection_state(DummyConnectionState::Pending); - let mut ns = NodeStream::<_, _, Vec>::new(muxer, addr); + let mut ns = NodeStream::<_, Vec>::new(muxer); assert_matches!(ns.poll(), Ok(Async::NotReady)); }); @@ -562,13 +481,12 @@ mod node_stream { #[test] fn poll_closes_the_node_stream_when_no_more_work_can_be_done() { - let addr = future::ok("/ip4/127.0.0.1/tcp/1234".parse::().expect("bad multiaddr")); let mut muxer = DummyMuxer::new(); // ensure muxer.poll_inbound() returns Async::Ready(None) muxer.set_inbound_connection_state(DummyConnectionState::Closed); // ensure muxer.poll_outbound() returns Async::Ready(None) muxer.set_outbound_connection_state(DummyConnectionState::Closed); - let mut ns = NodeStream::<_, _, Vec>::new(muxer, addr); + let mut ns = NodeStream::<_, Vec>::new(muxer); ns.open_substream(vec![]).unwrap(); ns.poll().unwrap(); // poll_inbound() ns.poll().unwrap(); // poll_outbound() @@ -577,32 +495,14 @@ mod node_stream { assert_matches!(ns.poll(), Ok(Async::Ready(None))); } - #[test] - fn poll_resolves_the_address() { - let addr = future::ok("/ip4/127.0.0.1/tcp/1234".parse::().expect("bad multiaddr")); - let mut muxer = DummyMuxer::new(); - // ensure muxer.poll_inbound() returns Async::Ready(None) - muxer.set_inbound_connection_state(DummyConnectionState::Closed); - // ensure muxer.poll_outbound() returns Async::Ready(None) - muxer.set_outbound_connection_state(DummyConnectionState::Closed); - let mut ns = NodeStream::<_, _, Vec>::new(muxer, addr); - ns.open_substream(vec![]).unwrap(); - ns.poll().unwrap(); // poll_inbound() - ns.poll().unwrap(); // poll_outbound() - assert_matches!(ns.poll(), Ok(Async::Ready(Some(node_event))) => { - assert_matches!(node_event, NodeEvent::Multiaddr(Ok(_))) - }); - } - #[test] fn poll_sets_up_substreams_yielding_them_in_reverse_order() { - let addr = future::ok("/ip4/127.0.0.1/tcp/1234".parse::().expect("bad multiaddr")); let mut muxer = DummyMuxer::new(); // ensure muxer.poll_inbound() returns Async::Ready(None) muxer.set_inbound_connection_state(DummyConnectionState::Closed); // ensure muxer.poll_outbound() returns Async::Ready(Some(substream)) muxer.set_outbound_connection_state(DummyConnectionState::Opened); - let mut ns = NodeStream::<_, _, Vec>::new(muxer, addr); + let mut ns = NodeStream::<_, Vec>::new(muxer); ns.open_substream(vec![1]).unwrap(); ns.open_substream(vec![2]).unwrap(); ns.poll().unwrap(); // poll_inbound() @@ -623,13 +523,12 @@ mod node_stream { #[test] fn poll_keeps_outbound_substreams_when_the_outgoing_connection_is_not_ready() { - let addr = future::ok("/ip4/127.0.0.1/tcp/1234".parse::().expect("bad multiaddr")); let mut muxer = DummyMuxer::new(); // ensure muxer.poll_inbound() returns Async::Ready(None) muxer.set_inbound_connection_state(DummyConnectionState::Closed); // ensure muxer.poll_outbound() returns Async::NotReady muxer.set_outbound_connection_state(DummyConnectionState::Pending); - let mut ns = NodeStream::<_, _, Vec>::new(muxer, addr); + let mut ns = NodeStream::<_, Vec>::new(muxer); ns.open_substream(vec![1]).unwrap(); ns.poll().unwrap(); // poll past inbound ns.poll().unwrap(); // poll outbound @@ -639,11 +538,10 @@ mod node_stream { #[test] fn poll_returns_incoming_substream() { - let addr = future::ok("/ip4/127.0.0.1/tcp/1234".parse::().expect("bad multiaddr")); let mut muxer = DummyMuxer::new(); // ensure muxer.poll_inbound() returns Async::Ready(Some(subs)) muxer.set_inbound_connection_state(DummyConnectionState::Opened); - let mut ns = NodeStream::<_, _, Vec>::new(muxer, addr); + let mut ns = NodeStream::<_, Vec>::new(muxer); assert_matches!(ns.poll(), Ok(Async::Ready(Some(node_event))) => { assert_matches!(node_event, NodeEvent::InboundSubstream{ substream: _ }); }); diff --git a/core/src/nodes/swarm.rs b/core/src/nodes/raw_swarm.rs similarity index 66% rename from core/src/nodes/swarm.rs rename to core/src/nodes/raw_swarm.rs index 97e96250bad..e1b8ed2b301 100644 --- a/core/src/nodes/swarm.rs +++ b/core/src/nodes/raw_swarm.rs @@ -29,11 +29,10 @@ use nodes::listeners::{ListenersEvent, ListenersStream}; use nodes::node::Substream; use std::collections::hash_map::{Entry, OccupiedEntry}; use std::io::{Error as IoError, ErrorKind as IoErrorKind}; -use void::Void; use {Endpoint, Multiaddr, PeerId, Transport}; /// Implementation of `Stream` that handles the nodes. -pub struct Swarm +pub struct RawSwarm where TTrans: Transport, { @@ -41,14 +40,11 @@ where listeners: ListenersStream, /// The nodes currently active. - active_nodes: CollectionStream, + active_nodes: CollectionStream, /// The reach attempts of the swarm. /// This needs to be a separate struct in order to handle multiple mutable borrows issues. reach_attempts: ReachAttempts, - - /// Object that builds new handlers. - handler_build: THandlerBuild, } struct ReachAttempts { @@ -59,8 +55,8 @@ struct ReachAttempts { /// the peer ID. other_reach_attempts: Vec<(ReachAttemptId, ConnectedPoint)>, - /// For each peer ID we're connected to, contains the multiaddress we're connected to. - connected_multiaddresses: FnvHashMap, + /// For each peer ID we're connected to, contains the endpoint we're connected to. + connected_points: FnvHashMap, } /// Attempt to reach a peer. @@ -74,8 +70,8 @@ struct OutReachAttempt { next_attempts: Vec, } -/// Event that can happen on the `Swarm`. -pub enum SwarmEvent +/// Event that can happen on the `RawSwarm`. +pub enum RawSwarmEvent<'a, TTrans: 'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a> where TTrans: Transport, { @@ -90,15 +86,17 @@ where }, /// A new connection arrived on a listener. - IncomingConnection { - /// Address of the listener which received the connection. - listen_addr: Multiaddr, - }, + IncomingConnection(IncomingConnectionEvent<'a, TTrans, TInEvent, TOutEvent, THandler>), - /// An error happened when negotiating a new connection. + /// A new connection was arriving on a listener, but an error happened when negotiating it. + /// + /// This can include, for example, an error during the handshake of the encryption layer, or + /// the connection unexpectedly closed. IncomingConnectionError { /// Address of the listener which received the connection. listen_addr: Multiaddr, + /// Address used to send back data to the remote. + send_back_addr: Multiaddr, /// The error that happened. error: IoError, }, @@ -116,8 +114,8 @@ where Replaced { /// Id of the peer. peer_id: PeerId, - /// Multiaddr we were connected to, or `None` if it was unknown. - closed_multiaddr: Option, + /// Endpoint we were connected to. + closed_endpoint: ConnectedPoint, /// If `Listener`, then we received the connection. If `Dial`, then it's a connection that /// we opened. endpoint: ConnectedPoint, @@ -130,16 +128,16 @@ where NodeClosed { /// Identifier of the node. peer_id: PeerId, - /// Address we were connected to. `None` if not known. - address: Option, + /// Endpoint we were connected to. + endpoint: ConnectedPoint, }, /// The muxer of a node has produced an error. NodeError { /// Identifier of the node. peer_id: PeerId, - /// Address we were connected to. `None` if not known. - address: Option, + /// Endpoint we were connected to. + endpoint: ConnectedPoint, /// The error that happened. error: IoError, }, @@ -165,26 +163,12 @@ where UnknownPeerDialError { /// The multiaddr we failed to reach. multiaddr: Multiaddr, + /// The error that happened. error: IoError, - }, - /// When dialing a peer, we successfully connected to a remote whose peer id doesn't match - /// what we expected. - PublicKeyMismatch { - /// Id of the peer we were expecting. - expected_peer_id: PeerId, - - /// Id of the peer we actually obtained. - actual_peer_id: PeerId, - - /// The multiaddr we failed to reach. - multiaddr: Multiaddr, - - /// Returns the number of multiaddresses that still need to be attempted in order to reach - /// `expected_peer_id`. If this is non-zero, then there's still a chance we can connect to - /// this node. If this is zero, then we have definitely failed. - remain_addrs_attempt: usize, + /// The handler that was passed to `dial()`. + handler: THandler, }, /// A node produced a custom event. @@ -196,7 +180,82 @@ where }, } +/// A new connection arrived on a listener. +pub struct IncomingConnectionEvent<'a, TTrans: 'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a> +where TTrans: Transport +{ + /// The produced upgrade. + upgrade: TTrans::ListenerUpgrade, + /// Address of the listener which received the connection. + listen_addr: Multiaddr, + /// Address used to send back data to the remote. + send_back_addr: Multiaddr, + /// Reference to the `active_nodes` field of the swarm. + active_nodes: &'a mut CollectionStream, + /// Reference to the `other_reach_attempts` field of the swarm. + other_reach_attempts: &'a mut Vec<(ReachAttemptId, ConnectedPoint)>, +} + +impl<'a, TTrans, TInEvent, TOutEvent, TMuxer, THandler> IncomingConnectionEvent<'a, TTrans, TInEvent, TOutEvent, THandler> +where + TTrans: Transport + Clone, + TTrans::Dial: Send + 'static, + TTrans::ListenerUpgrade: Send + 'static, + THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, + THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary + TMuxer: StreamMuxer + Send + Sync + 'static, + TMuxer::OutboundSubstream: Send, + TMuxer::Substream: Send, + TInEvent: Send + 'static, + TOutEvent: Send + 'static, +{ + /// Starts processing the incoming connection and sets the handler to use for it. + #[inline] + pub fn accept(self, handler: THandler) { + self.accept_with_builder(|_| handler) + } + + /// Same as `accept`, but accepts a closure that turns a `ConnectedPoint` into a handler. + pub fn accept_with_builder(self, builder: TBuilder) + where TBuilder: FnOnce(&ConnectedPoint) -> THandler + { + let connected_point = self.to_connected_point(); + let handler = builder(&connected_point); + let id = self.active_nodes.add_reach_attempt(self.upgrade, handler); + self.other_reach_attempts.push(( + id, + connected_point, + )); + } +} + +impl<'a, TTrans, TInEvent, TOutEvent, THandler> IncomingConnectionEvent<'a, TTrans, TInEvent, TOutEvent, THandler> +where TTrans: Transport +{ + /// Address of the listener that received the connection. + #[inline] + pub fn listen_addr(&self) -> &Multiaddr { + &self.listen_addr + } + + /// Address used to send back data to the dialer. + #[inline] + pub fn send_back_addr(&self) -> &Multiaddr { + &self.send_back_addr + } + + /// Builds the `ConnectedPoint` corresponding to the incoming connection. + #[inline] + pub fn to_connected_point(&self) -> ConnectedPoint { + ConnectedPoint::Listener { + listen_addr: self.listen_addr.clone(), + send_back_addr: self.send_back_addr.clone(), + } + } +} + /// How we connected to a node. +// TODO: move definition #[derive(Debug, Clone)] pub enum ConnectedPoint { /// We dialed the node. @@ -208,20 +267,35 @@ pub enum ConnectedPoint { Listener { /// Address of the listener that received the connection. listen_addr: Multiaddr, + /// Stack of protocols used to send back data to the remote. + send_back_addr: Multiaddr, }, } +impl<'a> From<&'a ConnectedPoint> for Endpoint { + #[inline] + fn from(endpoint: &'a ConnectedPoint) -> Endpoint { + endpoint.to_endpoint() + } +} + impl From for Endpoint { #[inline] fn from(endpoint: ConnectedPoint) -> Endpoint { - match endpoint { + endpoint.to_endpoint() + } +} + +impl ConnectedPoint { + /// Turns the `ConnectedPoint` into the corresponding `Endpoint`. + #[inline] + pub fn to_endpoint(&self) -> Endpoint { + match *self { ConnectedPoint::Dialer { .. } => Endpoint::Dialer, ConnectedPoint::Listener { .. } => Endpoint::Listener, } } -} -impl ConnectedPoint { /// Returns true if we are `Dialer`. #[inline] pub fn is_dialer(&self) -> bool { @@ -241,64 +315,26 @@ impl ConnectedPoint { } } -/// Trait for structures that can create new factories. -pub trait HandlerFactory { - /// The generated handler. - type Handler; - - /// Creates a new handler. - fn new_handler(&self) -> Self::Handler; -} - -impl HandlerFactory for T where T: Fn() -> THandler { - type Handler = THandler; - - #[inline] - fn new_handler(&self) -> THandler { - (*self)() - } -} - -impl - Swarm +impl + RawSwarm where TTrans: Transport + Clone, TMuxer: StreamMuxer, - THandlerBuild: HandlerFactory, - THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, + THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary { /// Creates a new node events stream. #[inline] - pub fn new(transport: TTrans) -> Swarm THandler> - where THandler: Default, - { - // TODO: with_capacity? - Swarm { - listeners: ListenersStream::new(transport), - active_nodes: CollectionStream::new(), - reach_attempts: ReachAttempts { - out_reach_attempts: Default::default(), - other_reach_attempts: Vec::new(), - connected_multiaddresses: Default::default(), - }, - handler_build: Default::default, - } - } - - /// Same as `new`, but lets you specify a way to build a node handler. - #[inline] - pub fn with_handler_builder(transport: TTrans, handler_build: THandlerBuild) -> Self { + pub fn new(transport: TTrans) -> Self { // TODO: with_capacity? - Swarm { + RawSwarm { listeners: ListenersStream::new(transport), active_nodes: CollectionStream::new(), reach_attempts: ReachAttempts { out_reach_attempts: Default::default(), other_reach_attempts: Vec::new(), - connected_multiaddresses: Default::default(), + connected_points: Default::default(), }, - handler_build, } } @@ -340,11 +376,12 @@ where } /// Dials a multiaddress without knowing the peer ID we're going to obtain. - pub fn dial(&mut self, addr: Multiaddr) -> Result<(), Multiaddr> + /// + /// The second parameter is the handler to use if we manage to reach a node. + pub fn dial(&mut self, addr: Multiaddr, handler: THandler) -> Result<(), Multiaddr> where TTrans: Transport + Clone, TTrans::Dial: Send + 'static, - TTrans::MultiaddrFuture: Send + 'static, TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer::OutboundSubstream: Send, TMuxer::Substream: Send, @@ -356,9 +393,9 @@ where Err((_, addr)) => return Err(addr), }; - let reach_id = self.active_nodes.add_reach_attempt(future, self.handler_build.new_handler()); - self.reach_attempts.other_reach_attempts - .push((reach_id, ConnectedPoint::Dialer { address: addr })); + let connected_point = ConnectedPoint::Dialer { address: addr }; + let reach_id = self.active_nodes.add_reach_attempt(future, handler); + self.reach_attempts.other_reach_attempts.push((reach_id, connected_point)); Ok(()) } @@ -387,7 +424,7 @@ where /// Grants access to a struct that represents a peer. #[inline] - pub fn peer(&mut self, peer_id: PeerId) -> Peer { + pub fn peer(&mut self, peer_id: PeerId) -> Peer { // TODO: we do `peer_mut(...).is_some()` followed with `peer_mut(...).unwrap()`, otherwise // the borrow checker yells at us. @@ -399,12 +436,12 @@ where .peer_mut(&peer_id) .expect("we checked for Some just above"), peer_id, - connected_multiaddresses: &mut self.reach_attempts.connected_multiaddresses, + connected_points: &mut self.reach_attempts.connected_points, }); } if self.reach_attempts.out_reach_attempts.get_mut(&peer_id).is_some() { - debug_assert!(!self.reach_attempts.connected_multiaddresses.contains_key(&peer_id)); + debug_assert!(!self.reach_attempts.connected_points.contains_key(&peer_id)); return Peer::PendingConnect(PeerPendingConnect { attempt: match self.reach_attempts.out_reach_attempts.entry(peer_id.clone()) { Entry::Occupied(e) => e, @@ -414,7 +451,7 @@ where }); } - debug_assert!(!self.reach_attempts.connected_multiaddresses.contains_key(&peer_id)); + debug_assert!(!self.reach_attempts.connected_points.contains_key(&peer_id)); Peer::NotConnected(PeerNotConnected { nodes: self, peer_id, @@ -426,11 +463,11 @@ where /// /// It is a logic error to call this method if we already have an outgoing attempt to the /// given peer. - fn start_dial_out(&mut self, peer_id: PeerId, first: Multiaddr, rest: Vec) + fn start_dial_out(&mut self, peer_id: PeerId, handler: THandler, first: Multiaddr, + rest: Vec) where TTrans: Transport + Clone, TTrans::Dial: Send + 'static, - TTrans::MultiaddrFuture: Send + 'static, TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer::OutboundSubstream: Send, TMuxer::Substream: Send, @@ -438,11 +475,23 @@ where TOutEvent: Send + 'static, { let reach_id = match self.transport().clone().dial(first.clone()) { - Ok(fut) => self.active_nodes.add_reach_attempt(fut, self.handler_build.new_handler()), + Ok(fut) => { + let expected_peer_id = peer_id.clone(); + let fut = fut.and_then(move |(actual_peer_id, muxer)| { + if actual_peer_id == expected_peer_id { + Ok((actual_peer_id, muxer)) + } else { + let msg = format!("public key mismatch ; expected = {:?} ; obtained = {:?}", + expected_peer_id, actual_peer_id); + Err(IoError::new(IoErrorKind::Other, msg)) + } + }); + self.active_nodes.add_reach_attempt(fut, handler) + }, Err((_, addr)) => { let msg = format!("unsupported multiaddr {}", addr); let fut = future::err(IoError::new(IoErrorKind::Other, msg)); - self.active_nodes.add_reach_attempt::<_, _, future::FutureResult, _>(fut, self.handler_build.new_handler()) + self.active_nodes.add_reach_attempt(fut, handler) }, }; @@ -459,51 +508,48 @@ where } /// Provides an API similar to `Stream`, except that it cannot error. - pub fn poll(&mut self) -> Async>> + pub fn poll(&mut self) -> Async> where TTrans: Transport + Clone, TTrans::Dial: Send + 'static, - TTrans::MultiaddrFuture: Future + Send + 'static, TTrans::ListenerUpgrade: Send + 'static, TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer::OutboundSubstream: Send, TMuxer::Substream: Send, TInEvent: Send + 'static, TOutEvent: Send + 'static, - THandlerBuild: HandlerFactory, - THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, + THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary { // Start by polling the listeners for events. match self.listeners.poll() { Async::NotReady => (), - Async::Ready(Some(ListenersEvent::Incoming { + Async::Ready(ListenersEvent::Incoming { upgrade, listen_addr, - })) => { - let id = self.active_nodes.add_reach_attempt(upgrade, self.handler_build.new_handler()); - self.reach_attempts.other_reach_attempts.push(( - id, - ConnectedPoint::Listener { - listen_addr: listen_addr.clone(), - }, - )); - return Async::Ready(Some(SwarmEvent::IncomingConnection { + send_back_addr, + }) => { + let event = IncomingConnectionEvent { + upgrade, listen_addr, - })); + send_back_addr, + active_nodes: &mut self.active_nodes, + other_reach_attempts: &mut self.reach_attempts.other_reach_attempts, + }; + + return Async::Ready(RawSwarmEvent::IncomingConnection(event)); } - Async::Ready(Some(ListenersEvent::Closed { + Async::Ready(ListenersEvent::Closed { listen_addr, listener, result, - })) => { - return Async::Ready(Some(SwarmEvent::ListenerClosed { + }) => { + return Async::Ready(RawSwarmEvent::ListenerClosed { listen_addr, listener, result, - })); + }); } - Async::Ready(None) => unreachable!("The listeners stream never finishes"), } // Poll the existing nodes. @@ -511,44 +557,53 @@ where let (action, out_event); match self.active_nodes.poll() { Async::NotReady => break, - Async::Ready(Some(CollectionEvent::NodeReached(reach_event))) => { + Async::Ready(CollectionEvent::NodeReached(reach_event)) => { let (a, e) = handle_node_reached(&mut self.reach_attempts, reach_event); action = a; out_event = e; } - Async::Ready(Some(CollectionEvent::ReachError { id, error })) => { - let (a, e) = handle_reach_error(&mut self.reach_attempts, id, error); + Async::Ready(CollectionEvent::ReachError { id, error, handler }) => { + let (a, e) = handle_reach_error(&mut self.reach_attempts, id, error, handler); action = a; out_event = e; } - Async::Ready(Some(CollectionEvent::NodeError { + Async::Ready(CollectionEvent::NodeError { peer_id, error, - })) => { - let address = self.reach_attempts.connected_multiaddresses.remove(&peer_id); + }) => { + let endpoint = self.reach_attempts.connected_points.remove(&peer_id) + .expect("We insert into connected_points whenever a connection is \ + opened and remove only when a connection is closed ; the \ + underlying API is guaranteed to always deliver a connection \ + closed message after it has been opened, and no two closed \ + messages ; qed"); debug_assert!(!self.reach_attempts.out_reach_attempts.contains_key(&peer_id)); action = Default::default(); - out_event = SwarmEvent::NodeError { + out_event = RawSwarmEvent::NodeError { peer_id, - address, + endpoint, error, }; } - Async::Ready(Some(CollectionEvent::NodeClosed { peer_id })) => { - let address = self.reach_attempts.connected_multiaddresses.remove(&peer_id); + Async::Ready(CollectionEvent::NodeClosed { peer_id }) => { + let endpoint = self.reach_attempts.connected_points.remove(&peer_id) + .expect("We insert into connected_points whenever a connection is \ + opened and remove only when a connection is closed ; the \ + underlying API is guaranteed to always deliver a connection \ + closed message after it has been opened, and no two closed \ + messages ; qed"); debug_assert!(!self.reach_attempts.out_reach_attempts.contains_key(&peer_id)); action = Default::default(); - out_event = SwarmEvent::NodeClosed { peer_id, address }; + out_event = RawSwarmEvent::NodeClosed { peer_id, endpoint }; } - Async::Ready(Some(CollectionEvent::NodeEvent { peer_id, event })) => { + Async::Ready(CollectionEvent::NodeEvent { peer_id, event }) => { action = Default::default(); - out_event = SwarmEvent::NodeEvent { peer_id, event }; + out_event = RawSwarmEvent::NodeEvent { peer_id, event }; } - Async::Ready(None) => unreachable!("CollectionStream never ends"), }; - if let Some((peer_id, first, rest)) = action.start_dial_out { - self.start_dial_out(peer_id, first, rest); + if let Some((peer_id, handler, first, rest)) = action.start_dial_out { + self.start_dial_out(peer_id, handler, first, rest); } if let Some(interrupt) = action.interrupt { @@ -563,7 +618,7 @@ where attempts ; qed"); } - return Async::Ready(Some(out_event)); + return Async::Ready(out_event); } Async::NotReady @@ -571,27 +626,35 @@ where } /// Internal struct indicating an action to perform of the swarm. -#[derive(Debug, Default)] +#[derive(Debug)] #[must_use] -struct ActionItem { - start_dial_out: Option<(PeerId, Multiaddr, Vec)>, +struct ActionItem { + start_dial_out: Option<(PeerId, THandler, Multiaddr, Vec)>, interrupt: Option, } +impl Default for ActionItem { + fn default() -> Self { + ActionItem { + start_dial_out: None, + interrupt: None, + } + } +} + /// Handles a node reached event from the collection. /// /// Returns an event to return from the stream. /// /// > **Note**: The event **must** have been produced by the collection of nodes, otherwise /// > panics will likely happen. -fn handle_node_reached( +fn handle_node_reached<'a, TTrans, TMuxer, TInEvent, TOutEvent, THandler>( reach_attempts: &mut ReachAttempts, - event: CollectionReachEvent -) -> (ActionItem, SwarmEvent) + event: CollectionReachEvent +) -> (ActionItem, RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler>) where TTrans: Transport + Clone, TTrans::Dial: Send + 'static, - TTrans::MultiaddrFuture: Send + 'static, TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer::OutboundSubstream: Send, TMuxer::Substream: Send, @@ -605,10 +668,11 @@ where .iter() .position(|i| i.0 == event.reach_attempt_id()) { - let (_, endpoint) = reach_attempts.other_reach_attempts.swap_remove(in_pos); + let (_, opened_endpoint) = reach_attempts.other_reach_attempts.swap_remove(in_pos); + + // Set the endpoint for this peer. + let closed_endpoint = reach_attempts.connected_points.insert(event.peer_id().clone(), opened_endpoint.clone()); - // Clear the known multiaddress for this peer. - let closed_multiaddr = reach_attempts.connected_multiaddresses.remove(&event.peer_id()); // Cancel any outgoing attempt to this peer. let action = if let Some(attempt) = reach_attempts.out_reach_attempts.remove(&event.peer_id()) { debug_assert_ne!(attempt.id, event.reach_attempt_id()); @@ -622,13 +686,18 @@ where let (outcome, peer_id) = event.accept(); if outcome == CollectionNodeAccept::ReplacedExisting { - return (action, SwarmEvent::Replaced { + let closed_endpoint = closed_endpoint + .expect("We insert into connected_points whenever a connection is opened and \ + remove only when a connection is closed ; the underlying API is \ + guaranteed to always deliver a connection closed message after it has \ + been opened, and no two closed messages ; qed"); + return (action, RawSwarmEvent::Replaced { peer_id, - endpoint, - closed_multiaddr, + endpoint: opened_endpoint, + closed_endpoint, }); } else { - return (action, SwarmEvent::Connected { peer_id, endpoint }); + return (action, RawSwarmEvent::Connected { peer_id, endpoint: opened_endpoint }); } } @@ -646,60 +715,29 @@ where .expect("is_outgoing_and_ok is true only if reach_attempts.out_reach_attempts.get(event.peer_id()) \ returned Some"); - let closed_multiaddr = reach_attempts.connected_multiaddresses - .insert(event.peer_id().clone(), attempt.cur_attempted.clone()); - let endpoint = ConnectedPoint::Dialer { + let opened_endpoint = ConnectedPoint::Dialer { address: attempt.cur_attempted, }; + let closed_endpoint = reach_attempts.connected_points + .insert(event.peer_id().clone(), opened_endpoint.clone()) + .expect("We insert into connected_points whenever a connection is opened and remove \ + only when a connection is closed ; the underlying API is guaranteed to always \ + deliver a connection closed message after it has been opened, and no two \ + closed messages ; qed"); + let (outcome, peer_id) = event.accept(); if outcome == CollectionNodeAccept::ReplacedExisting { - return (Default::default(), SwarmEvent::Replaced { + return (Default::default(), RawSwarmEvent::Replaced { peer_id, - endpoint, - closed_multiaddr, + endpoint: opened_endpoint, + closed_endpoint, }); } else { - return (Default::default(), SwarmEvent::Connected { peer_id, endpoint }); + return (Default::default(), RawSwarmEvent::Connected { peer_id, endpoint: opened_endpoint }); } } - // If in neither, check outgoing reach attempts again as we may have a public - // key mismatch. - let expected_peer_id = reach_attempts - .out_reach_attempts - .iter() - .find(|(_, a)| a.id == event.reach_attempt_id()) - .map(|(p, _)| p.clone()); - if let Some(expected_peer_id) = expected_peer_id { - debug_assert_ne!(&expected_peer_id, event.peer_id()); - let attempt = reach_attempts.out_reach_attempts.remove(&expected_peer_id) - .expect("expected_peer_id is a key that is grabbed from out_reach_attempts"); - - let num_remain = attempt.next_attempts.len(); - let failed_addr = attempt.cur_attempted.clone(); - - let peer_id = event.deny(); - - let action = if !attempt.next_attempts.is_empty() { - let mut attempt = attempt; - let next = attempt.next_attempts.remove(0); - ActionItem { - start_dial_out: Some((expected_peer_id.clone(), next, attempt.next_attempts)), - .. Default::default() - } - } else { - Default::default() - }; - - return (action, SwarmEvent::PublicKeyMismatch { - remain_addrs_attempt: num_remain, - expected_peer_id, - actual_peer_id: peer_id, - multiaddr: failed_addr, - }); - } - // We didn't find any entry in neither the outgoing connections not ingoing connections. // TODO: improve proof or remove ; this is too complicated right now panic!("The API of collection guarantees that the id sent back in NodeReached (which is where \ @@ -715,11 +753,12 @@ where /// /// > **Note**: The event **must** have been produced by the collection of nodes, otherwise /// > panics will likely happen. -fn handle_reach_error( +fn handle_reach_error<'a, TTrans, TInEvent, TOutEvent, THandler>( reach_attempts: &mut ReachAttempts, reach_id: ReachAttemptId, error: IoError, -) -> (ActionItem, SwarmEvent) + handler: THandler, +) -> (ActionItem, RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler>) where TTrans: Transport { // Search for the attempt in `out_reach_attempts`. @@ -740,14 +779,14 @@ where TTrans: Transport let mut attempt = attempt; let next_attempt = attempt.next_attempts.remove(0); ActionItem { - start_dial_out: Some((peer_id.clone(), next_attempt, attempt.next_attempts)), + start_dial_out: Some((peer_id.clone(), handler, next_attempt, attempt.next_attempts)), .. Default::default() } } else { Default::default() }; - return (action, SwarmEvent::DialError { + return (action, RawSwarmEvent::DialError { remain_addrs_attempt: num_remain, peer_id, multiaddr: failed_addr, @@ -764,13 +803,14 @@ where TTrans: Transport let (_, endpoint) = reach_attempts.other_reach_attempts.swap_remove(in_pos); match endpoint { ConnectedPoint::Dialer { address } => { - return (Default::default(), SwarmEvent::UnknownPeerDialError { + return (Default::default(), RawSwarmEvent::UnknownPeerDialError { multiaddr: address, error, + handler, }); } - ConnectedPoint::Listener { listen_addr } => { - return (Default::default(), SwarmEvent::IncomingConnectionError { listen_addr, error }); + ConnectedPoint::Listener { listen_addr, send_back_addr } => { + return (Default::default(), RawSwarmEvent::IncomingConnectionError { listen_addr, send_back_addr, error }); } } } @@ -786,7 +826,7 @@ where TTrans: Transport } /// State of a peer in the system. -pub enum Peer<'a, TTrans: 'a, TInEvent: 'a, TOutEvent: 'a, THandlerBuild: 'a> +pub enum Peer<'a, TTrans: 'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a> where TTrans: Transport, { @@ -794,23 +834,22 @@ where Connected(PeerConnected<'a, TInEvent>), /// We are currently attempting to connect to this peer. - PendingConnect(PeerPendingConnect<'a, TInEvent, TOutEvent>), + PendingConnect(PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>), /// We are not connected to this peer at all. /// /// > **Note**: It is however possible that a pending incoming connection is being negotiated /// > and will connect to this peer, but we don't know it yet. - NotConnected(PeerNotConnected<'a, TTrans, TInEvent, TOutEvent, THandlerBuild>), + NotConnected(PeerNotConnected<'a, TTrans, TInEvent, TOutEvent, THandler>), } // TODO: add other similar methods that wrap to the ones of `PeerNotConnected` -impl<'a, TTrans, TMuxer, TInEvent, TOutEvent, THandler, THandlerBuild> - Peer<'a, TTrans, TInEvent, TOutEvent, THandlerBuild> +impl<'a, TTrans, TMuxer, TInEvent, TOutEvent, THandler> + Peer<'a, TTrans, TInEvent, TOutEvent, THandler> where TTrans: Transport, TMuxer: StreamMuxer, - THandlerBuild: HandlerFactory, - THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, + THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary { /// If we are connected, returns the `PeerConnected`. @@ -824,7 +863,7 @@ where /// If a connection is pending, returns the `PeerPendingConnect`. #[inline] - pub fn as_pending_connect(self) -> Option> { + pub fn as_pending_connect(self) -> Option> { match self { Peer::PendingConnect(peer) => Some(peer), _ => None, @@ -833,7 +872,7 @@ where /// If we are not connected, returns the `PeerNotConnected`. #[inline] - pub fn as_not_connected(self) -> Option> { + pub fn as_not_connected(self) -> Option> { match self { Peer::NotConnected(peer) => Some(peer), _ => None, @@ -841,36 +880,50 @@ where } /// If we're not connected, opens a new connection to this peer using the given multiaddr. + /// + /// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then + /// the whole connection is immediately closed. + /// + /// > **Note**: It is possible that the attempt reaches a node that doesn't have the peer id + /// > that we are expecting, in which case the handler will be used for this "wrong" + /// > node. #[inline] pub fn or_connect( self, addr: Multiaddr, - ) -> Result, Self> + handler: THandler, + ) -> Result, Self> where TTrans: Transport + Clone, TTrans::Dial: Send + 'static, - TTrans::MultiaddrFuture: Send + 'static, TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer::OutboundSubstream: Send, TMuxer::Substream: Send, TInEvent: Send + 'static, TOutEvent: Send + 'static, { - self.or_connect_with(move |_| addr) + self.or_connect_with(move |_| addr, handler) } /// If we're not connected, calls the function passed as parameter and opens a new connection /// using the returned address. + /// + /// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then + /// the whole connection is immediately closed. + /// + /// > **Note**: It is possible that the attempt reaches a node that doesn't have the peer id + /// > that we are expecting, in which case the handler will be used for this "wrong" + /// > node. #[inline] pub fn or_connect_with( self, addr: TFn, - ) -> Result, Self> + handler: THandler, + ) -> Result, Self> where TFn: FnOnce(&PeerId) -> Multiaddr, TTrans: Transport + Clone, TTrans::Dial: Send + 'static, - TTrans::MultiaddrFuture: Send + 'static, TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer::OutboundSubstream: Send, TMuxer::Substream: Send, @@ -882,7 +935,7 @@ where Peer::PendingConnect(peer) => Ok(PeerPotentialConnect::PendingConnect(peer)), Peer::NotConnected(peer) => { let addr = addr(&peer.peer_id); - match peer.connect(addr) { + match peer.connect(addr, handler) { Ok(peer) => Ok(PeerPotentialConnect::PendingConnect(peer)), Err(peer) => Err(Peer::NotConnected(peer)), } @@ -892,15 +945,15 @@ where } /// Peer we are potentially going to connect to. -pub enum PeerPotentialConnect<'a, TInEvent: 'a, TOutEvent: 'a> { +pub enum PeerPotentialConnect<'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a> { /// We are connected to this peer. Connected(PeerConnected<'a, TInEvent>), /// We are currently attempting to connect to this peer. - PendingConnect(PeerPendingConnect<'a, TInEvent, TOutEvent>), + PendingConnect(PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>), } -impl<'a, TInEvent, TOutEvent> PeerPotentialConnect<'a, TInEvent, TOutEvent> { +impl<'a, TInEvent, TOutEvent, THandler> PeerPotentialConnect<'a, TInEvent, TOutEvent, THandler> { /// Closes the connection or the connection attempt. /// /// If the connection was active, returns the list of outbound substream openings that were @@ -925,7 +978,7 @@ impl<'a, TInEvent, TOutEvent> PeerPotentialConnect<'a, TInEvent, TOutEvent> { /// If a connection is pending, returns the `PeerPendingConnect`. #[inline] - pub fn as_pending_connect(self) -> Option> { + pub fn as_pending_connect(self) -> Option> { match self { PeerPotentialConnect::PendingConnect(peer) => Some(peer), _ => None, @@ -936,8 +989,8 @@ impl<'a, TInEvent, TOutEvent> PeerPotentialConnect<'a, TInEvent, TOutEvent> { /// Access to a peer we are connected to. pub struct PeerConnected<'a, TInEvent: 'a> { peer: CollecPeerMut<'a, TInEvent>, - /// Reference to the `connected_multiaddresses` field of the parent. - connected_multiaddresses: &'a mut FnvHashMap, + /// Reference to the `connected_points` field of the parent. + connected_points: &'a mut FnvHashMap, peer_id: PeerId, } @@ -948,14 +1001,18 @@ impl<'a, TInEvent> PeerConnected<'a, TInEvent> { // TODO: consider returning a `PeerNotConnected` ; however this makes all the borrows things // much more annoying to deal with pub fn close(self) { - self.connected_multiaddresses.remove(&self.peer_id); + self.connected_points.remove(&self.peer_id); self.peer.close() } - /// Returns the outcome of the future that resolves the multiaddress of the peer. + /// Returns the endpoint we're connected to. #[inline] - pub fn multiaddr(&self) -> Option<&Multiaddr> { - self.connected_multiaddresses.get(&self.peer_id) + pub fn endpoint(&self) -> &ConnectedPoint { + self.connected_points.get(&self.peer_id) + .expect("We insert into connected_points whenever a connection is opened and remove \ + only when a connection is closed ; the underlying API is guaranteed to always \ + deliver a connection closed message after it has been opened, and no two \ + closed messages ; qed") } /// Sends an event to the node. @@ -966,12 +1023,12 @@ impl<'a, TInEvent> PeerConnected<'a, TInEvent> { } /// Access to a peer we are attempting to connect to. -pub struct PeerPendingConnect<'a, TInEvent: 'a, TOutEvent: 'a> { +pub struct PeerPendingConnect<'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a> { attempt: OccupiedEntry<'a, PeerId, OutReachAttempt>, - active_nodes: &'a mut CollectionStream, + active_nodes: &'a mut CollectionStream, } -impl<'a, TInEvent, TOutEvent> PeerPendingConnect<'a, TInEvent, TOutEvent> { +impl<'a, TInEvent, TOutEvent, THandler> PeerPendingConnect<'a, TInEvent, TOutEvent, THandler> { /// Interrupt this connection attempt. // TODO: consider returning a PeerNotConnected ; however that is really pain in terms of // borrows @@ -1013,54 +1070,58 @@ impl<'a, TInEvent, TOutEvent> PeerPendingConnect<'a, TInEvent, TOutEvent> { } /// Access to a peer we're not connected to. -pub struct PeerNotConnected<'a, TTrans: 'a, TInEvent: 'a, TOutEvent: 'a, THandlerBuild: 'a> +pub struct PeerNotConnected<'a, TTrans: 'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a> where TTrans: Transport, { peer_id: PeerId, - nodes: &'a mut Swarm, + nodes: &'a mut RawSwarm, } -impl<'a, TTrans, TInEvent, TOutEvent, TMuxer, THandler, THandlerBuild> - PeerNotConnected<'a, TTrans, TInEvent, TOutEvent, THandlerBuild> +impl<'a, TTrans, TInEvent, TOutEvent, TMuxer, THandler> + PeerNotConnected<'a, TTrans, TInEvent, TOutEvent, THandler> where TTrans: Transport, TMuxer: StreamMuxer, - THandlerBuild: HandlerFactory, - THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, + THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary { /// Attempts a new connection to this node using the given multiaddress. + /// + /// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then + /// the whole connection is immediately closed. #[inline] - pub fn connect(self, addr: Multiaddr) -> Result, Self> + pub fn connect(self, addr: Multiaddr, handler: THandler) -> Result, Self> where TTrans: Transport + Clone, TTrans::Dial: Send + 'static, - TTrans::MultiaddrFuture: Send + 'static, TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer::OutboundSubstream: Send, TMuxer::Substream: Send, TInEvent: Send + 'static, TOutEvent: Send + 'static, { - self.connect_inner(addr, Vec::new()) + self.connect_inner(handler, addr, Vec::new()) } /// Attempts a new connection to this node using the given multiaddresses. /// - /// The multiaddresses passes as parameter will be tried one by one. + /// The multiaddresses passed as parameter will be tried one by one. /// /// If the iterator is empty, TODO: what to do? at the moment we unwrap + /// + /// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then + /// the whole connection is immediately closed. #[inline] pub fn connect_iter( self, addrs: TIter, - ) -> Result, Self> + handler: THandler, + ) -> Result, Self> where TIter: IntoIterator, TTrans: Transport + Clone, TTrans::Dial: Send + 'static, - TTrans::MultiaddrFuture: Send + 'static, TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer::OutboundSubstream: Send, TMuxer::Substream: Send, @@ -1070,26 +1131,26 @@ where let mut addrs = addrs.into_iter(); let first = addrs.next().unwrap(); // TODO: bad let rest = addrs.collect(); - self.connect_inner(first, rest) + self.connect_inner(handler, first, rest) } /// Inner implementation of `connect`. fn connect_inner( self, + handler: THandler, first: Multiaddr, rest: Vec, - ) -> Result, Self> + ) -> Result, Self> where TTrans: Transport + Clone, TTrans::Dial: Send + 'static, - TTrans::MultiaddrFuture: Send + 'static, TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer::OutboundSubstream: Send, TMuxer::Substream: Send, TInEvent: Send + 'static, TOutEvent: Send + 'static, { - self.nodes.start_dial_out(self.peer_id.clone(), first, rest); + self.nodes.start_dial_out(self.peer_id.clone(), handler, first, rest); Ok(PeerPendingConnect { attempt: match self.nodes.reach_attempts.out_reach_attempts.entry(self.peer_id) { @@ -1102,28 +1163,3 @@ where }) } } - -impl Stream for - Swarm -where - TTrans: Transport + Clone, - TTrans::Dial: Send + 'static, - TTrans::MultiaddrFuture: Future + Send + 'static, - TTrans::ListenerUpgrade: Send + 'static, - TMuxer: StreamMuxer + Send + Sync + 'static, - TMuxer::OutboundSubstream: Send, - TMuxer::Substream: Send, - TInEvent: Send + 'static, - TOutEvent: Send + 'static, - THandlerBuild: HandlerFactory, - THandler: NodeHandler, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, - THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary -{ - type Item = SwarmEvent; - type Error = Void; // TODO: use `!` once stable - - #[inline] - fn poll(&mut self) -> Poll, Self::Error> { - Ok(self.poll()) - } -} diff --git a/core/src/tests/dummy_transport.rs b/core/src/tests/dummy_transport.rs index 6241a4c1260..37d1ad6ce58 100644 --- a/core/src/tests/dummy_transport.rs +++ b/core/src/tests/dummy_transport.rs @@ -47,10 +47,9 @@ impl DummyTransport { } impl Transport for DummyTransport { type Output = usize; - type Listener = Box + Send>; - type ListenerUpgrade = FutureResult<(Self::Output, Self::MultiaddrFuture), io::Error>; - type MultiaddrFuture = FutureResult; - type Dial = Box + Send>; + type Listener = Box + Send>; + type ListenerUpgrade = FutureResult; + type Dial = Box + Send>; fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> where @@ -59,7 +58,7 @@ impl Transport for DummyTransport { let addr2 = addr.clone(); match self.listener_state { ListenerState::Ok(async) => { - let tupelize = move |stream| future::ok( (stream, future::ok(addr.clone())) ); + let tupelize = move |stream| (future::ok(stream), addr.clone()); Ok(match async { Async::NotReady => { let stream = stream::poll_fn(|| Ok(Async::NotReady)).map(tupelize); diff --git a/core/src/transport/and_then.rs b/core/src/transport/and_then.rs index dcfb6664e16..461448a5d09 100644 --- a/core/src/transport/and_then.rs +++ b/core/src/transport/and_then.rs @@ -20,9 +20,9 @@ use futures::prelude::*; use multiaddr::Multiaddr; +use nodes::raw_swarm::ConnectedPoint; use std::io::Error as IoError; -use transport::{MuxedTransport, Transport}; -use upgrade::Endpoint; +use transport::Transport; /// See the `Transport::and_then` method. #[inline] @@ -37,21 +37,19 @@ pub struct AndThen { upgrade: C, } -impl Transport for AndThen +impl Transport for AndThen where T: Transport + 'static, T::Dial: Send, T::Listener: Send, T::ListenerUpgrade: Send, - C: FnOnce(T::Output, Endpoint, T::MultiaddrFuture) -> F + Clone + Send + 'static, - F: Future + Send + 'static, - Maf: Future + 'static, + C: FnOnce(T::Output, ConnectedPoint) -> F + Clone + Send + 'static, + F: Future + Send + 'static, { type Output = O; - type MultiaddrFuture = Maf; - type Listener = Box + Send>; - type ListenerUpgrade = Box + Send>; - type Dial = Box + Send>; + type Listener = Box + Send>; + type ListenerUpgrade = Box + Send>; + type Dial = Box + Send>; #[inline] fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { @@ -69,17 +67,24 @@ where } }; + let listen_addr = new_addr.clone(); + // Try to negotiate the protocol. // Note that failing to negotiate a protocol will never produce a future with an error. // Instead the `stream` will produce `Ok(Err(...))`. // `stream` can only produce an `Err` if `listening_stream` produces an `Err`. - let stream = listening_stream.map(move |connection| { + let stream = listening_stream.map(move |(connection, client_addr)| { let upgrade = upgrade.clone(); - let future = connection.and_then(move |(stream, client_addr)| { - upgrade(stream, Endpoint::Listener, client_addr) + let connected_point = ConnectedPoint::Listener { + listen_addr: listen_addr.clone(), + send_back_addr: client_addr.clone(), + }; + + let future = connection.and_then(move |stream| { + upgrade(stream, connected_point) }); - Box::new(future) as Box<_> + (Box::new(future) as Box<_>, client_addr) }); Ok((Box::new(stream), new_addr)) @@ -101,10 +106,14 @@ where } }; + let connected_point = ConnectedPoint::Dialer { + address: addr, + }; + let future = dialed_fut // Try to negotiate the protocol. - .and_then(move |(connection, client_addr)| { - upgrade(connection, Endpoint::Dialer, client_addr) + .and_then(move |connection| { + upgrade(connection, connected_point) }); Ok(Box::new(future)) @@ -115,36 +124,3 @@ where self.transport.nat_traversal(server, observed) } } - -impl MuxedTransport for AndThen -where - T: MuxedTransport + 'static, - T::Dial: Send, - T::Listener: Send, - T::ListenerUpgrade: Send, - T::Incoming: Send, - T::IncomingUpgrade: Send, - C: FnOnce(T::Output, Endpoint, T::MultiaddrFuture) -> F + Clone + Send + 'static, - F: Future + Send + 'static, - Maf: Future + 'static, -{ - type Incoming = Box + Send>; - type IncomingUpgrade = Box + Send>; - - #[inline] - fn next_incoming(self) -> Self::Incoming { - let upgrade = self.upgrade; - - let future = self.transport.next_incoming().map(|future| { - // Try to negotiate the protocol. - let future = future.and_then(move |(connection, client_addr)| { - let upgrade = upgrade.clone(); - upgrade(connection, Endpoint::Listener, client_addr) - }); - - Box::new(future) as Box + Send> - }); - - Box::new(future) as Box<_> - } -} diff --git a/core/src/transport/boxed.rs b/core/src/transport/boxed.rs index dff8298f91a..7b47d150844 100644 --- a/core/src/transport/boxed.rs +++ b/core/src/transport/boxed.rs @@ -23,7 +23,7 @@ use multiaddr::Multiaddr; use std::fmt; use std::io::Error as IoError; use std::sync::Arc; -use transport::{MuxedTransport, Transport}; +use transport::Transport; /// See the `Transport::boxed` method. #[inline] @@ -33,35 +33,17 @@ where T::Dial: Send + 'static, T::Listener: Send + 'static, T::ListenerUpgrade: Send + 'static, - T::MultiaddrFuture: Send + 'static, { Boxed { inner: Arc::new(transport) as Arc<_>, } } -/// See the `Transport::boxed_muxed` method. -#[inline] -pub fn boxed_muxed(transport: T) -> BoxedMuxed -where - T: MuxedTransport + Clone + Send + Sync + 'static, - T::Dial: Send + 'static, - T::Listener: Send + 'static, - T::ListenerUpgrade: Send + 'static, - T::MultiaddrFuture: Send + 'static, - T::Incoming: Send + 'static, - T::IncomingUpgrade: Send + 'static, -{ - BoxedMuxed { - inner: Arc::new(transport) as Arc<_>, - } -} -pub type MultiaddrFuture = Box + Send>; -pub type Dial = Box + Send>; -pub type Listener = Box, Error = IoError> + Send>; -pub type ListenerUpgrade = Box + Send>; -pub type Incoming = Box, Error = IoError> + Send>; -pub type IncomingUpgrade = Box + Send>; +pub type Dial = Box + Send>; +pub type Listener = Box, Multiaddr), Error = IoError> + Send>; +pub type ListenerUpgrade = Box + Send>; +pub type Incoming = Box, Multiaddr), Error = IoError> + Send>; +pub type IncomingUpgrade = Box + Send>; trait Abstract { fn listen_on(&self, addr: Multiaddr) -> Result<(Listener, Multiaddr), Multiaddr>; @@ -75,22 +57,19 @@ where T::Dial: Send + 'static, T::Listener: Send + 'static, T::ListenerUpgrade: Send + 'static, - T::MultiaddrFuture: Send + 'static, { fn listen_on(&self, addr: Multiaddr) -> Result<(Listener, Multiaddr), Multiaddr> { let (listener, new_addr) = Transport::listen_on(self.clone(), addr).map_err(|(_, addr)| addr)?; - let fut = listener.map(|upgrade| { - let fut = upgrade.map(|(out, addr)| (out, Box::new(addr) as MultiaddrFuture)); - Box::new(fut) as ListenerUpgrade + let fut = listener.map(|(upgrade, addr)| { + (Box::new(upgrade) as ListenerUpgrade, addr) }); Ok((Box::new(fut) as Box<_>, new_addr)) } fn dial(&self, addr: Multiaddr) -> Result, Multiaddr> { let fut = Transport::dial(self.clone(), addr) - .map_err(|(_, addr)| addr)? - .map(|(out, addr)| (out, Box::new(addr) as MultiaddrFuture)); + .map_err(|(_, addr)| addr)?; Ok(Box::new(fut) as Box<_>) } @@ -100,29 +79,6 @@ where } } -trait AbstractMuxed: Abstract { - fn next_incoming(&self) -> Incoming; -} - -impl AbstractMuxed for T -where - T: MuxedTransport + Clone + 'static, - T::Dial: Send + 'static, - T::Listener: Send + 'static, - T::ListenerUpgrade: Send + 'static, - T::MultiaddrFuture: Send + 'static, - T::Incoming: Send + 'static, - T::IncomingUpgrade: Send + 'static, -{ - fn next_incoming(&self) -> Incoming { - let fut = MuxedTransport::next_incoming(self.clone()).map(|upgrade| { - let fut = upgrade.map(|(out, addr)| (out, Box::new(addr) as MultiaddrFuture)); - Box::new(fut) as IncomingUpgrade - }); - Box::new(fut) as Box<_> - } -} - /// See the `Transport::boxed` method. pub struct Boxed { inner: Arc + Send + Sync>, @@ -145,7 +101,6 @@ impl Clone for Boxed { impl Transport for Boxed { type Output = O; - type MultiaddrFuture = MultiaddrFuture; type Listener = Listener; type ListenerUpgrade = ListenerUpgrade; type Dial = Dial; @@ -171,62 +126,3 @@ impl Transport for Boxed { self.inner.nat_traversal(server, observed) } } - -/// See the `Transport::boxed_muxed` method. -pub struct BoxedMuxed { - inner: Arc + Send + Sync>, -} - -impl fmt::Debug for BoxedMuxed { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "BoxedMuxedTransport") - } -} - -impl Clone for BoxedMuxed { - #[inline] - fn clone(&self) -> Self { - BoxedMuxed { - inner: self.inner.clone(), - } - } -} - -impl Transport for BoxedMuxed { - type Output = O; - type MultiaddrFuture = MultiaddrFuture; - type Listener = Listener; - type ListenerUpgrade = ListenerUpgrade; - type Dial = Dial; - - #[inline] - fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { - match self.inner.listen_on(addr) { - Ok(listen) => Ok(listen), - Err(addr) => Err((self, addr)), - } - } - - #[inline] - fn dial(self, addr: Multiaddr) -> Result { - match self.inner.dial(addr) { - Ok(dial) => Ok(dial), - Err(addr) => Err((self, addr)), - } - } - - #[inline] - fn nat_traversal(&self, server: &Multiaddr, observed: &Multiaddr) -> Option { - self.inner.nat_traversal(server, observed) - } -} - -impl MuxedTransport for BoxedMuxed { - type Incoming = Incoming; - type IncomingUpgrade = IncomingUpgrade; - - #[inline] - fn next_incoming(self) -> Self::Incoming { - self.inner.next_incoming() - } -} diff --git a/core/src/transport/choice.rs b/core/src/transport/choice.rs index 78288cf98b5..3f5b684d89d 100644 --- a/core/src/transport/choice.rs +++ b/core/src/transport/choice.rs @@ -18,11 +18,9 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use either::{EitherListenStream, EitherListenUpgrade, EitherOutput}; -use futures::{prelude::*, future}; +use either::{EitherListenStream, EitherOutput, EitherFuture}; use multiaddr::Multiaddr; -use std::io::Error as IoError; -use transport::{MuxedTransport, Transport}; +use transport::Transport; /// Struct returned by `or_transport()`. #[derive(Debug, Copy, Clone)] @@ -41,10 +39,8 @@ where { type Output = EitherOutput; type Listener = EitherListenStream; - type ListenerUpgrade = EitherListenUpgrade; - type MultiaddrFuture = future::Either; - type Dial = - EitherListenUpgrade<::Future, ::Future>; + type ListenerUpgrade = EitherFuture; + type Dial = EitherFuture; fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { let (first, addr) = match self.0.listen_on(addr) { @@ -60,12 +56,12 @@ where fn dial(self, addr: Multiaddr) -> Result { let (first, addr) = match self.0.dial(addr) { - Ok(connec) => return Ok(EitherListenUpgrade::First(connec)), + Ok(connec) => return Ok(EitherFuture::First(connec)), Err(err) => err, }; match self.1.dial(addr) { - Ok(connec) => Ok(EitherListenUpgrade::Second(connec)), + Ok(connec) => Ok(EitherFuture::Second(connec)), Err((second, addr)) => Err((OrTransport(first, second), addr)), } } @@ -80,33 +76,3 @@ where self.1.nat_traversal(server, observed) } } - -impl MuxedTransport for OrTransport -where - A: MuxedTransport, - B: MuxedTransport, - A::Incoming: Send + 'static, // TODO: meh :-/ - B::Incoming: Send + 'static, // TODO: meh :-/ - A::IncomingUpgrade: Send + 'static, // TODO: meh :-/ - B::IncomingUpgrade: Send + 'static, // TODO: meh :-/ - A::Output: 'static, // TODO: meh :-/ - B::Output: 'static, // TODO: meh :-/ -{ - type Incoming = Box + Send>; - type IncomingUpgrade = - Box, Self::MultiaddrFuture), Error = IoError> + Send>; - - #[inline] - fn next_incoming(self) -> Self::Incoming { - let first = self.0.next_incoming().map(|out| { - let fut = out.map(move |(v, addr)| (EitherOutput::First(v), future::Either::A(addr))); - Box::new(fut) as Box + Send> - }); - let second = self.1.next_incoming().map(|out| { - let fut = out.map(move |(v, addr)| (EitherOutput::Second(v), future::Either::B(addr))); - Box::new(fut) as Box + Send> - }); - let future = first.select(second).map(|(i, _)| i).map_err(|(e, _)| e); - Box::new(future) as Box<_> - } -} diff --git a/core/src/transport/denied.rs b/core/src/transport/denied.rs index 2f77c79ae22..87a0beb41e5 100644 --- a/core/src/transport/denied.rs +++ b/core/src/transport/denied.rs @@ -18,11 +18,9 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use futures::future; use futures::prelude::*; use multiaddr::Multiaddr; use std::io::{self, Cursor}; -use transport::MuxedTransport; use transport::Transport; /// Dummy implementation of `Transport` that just denies every single attempt. @@ -32,10 +30,9 @@ pub struct DeniedTransport; impl Transport for DeniedTransport { // TODO: could use `!` for associated types once stable type Output = Cursor>; - type MultiaddrFuture = Box + Send + Sync>; - type Listener = Box + Send + Sync>; - type ListenerUpgrade = Box + Send + Sync>; - type Dial = Box + Send + Sync>; + type Listener = Box + Send + Sync>; + type ListenerUpgrade = Box + Send + Sync>; + type Dial = Box + Send + Sync>; #[inline] fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { @@ -52,13 +49,3 @@ impl Transport for DeniedTransport { None } } - -impl MuxedTransport for DeniedTransport { - type Incoming = future::Empty; - type IncomingUpgrade = future::Empty<(Self::Output, Self::MultiaddrFuture), io::Error>; - - #[inline] - fn next_incoming(self) -> Self::Incoming { - future::empty() - } -} diff --git a/core/src/transport/interruptible.rs b/core/src/transport/interruptible.rs index 3638b8fef51..db24e99518f 100644 --- a/core/src/transport/interruptible.rs +++ b/core/src/transport/interruptible.rs @@ -20,7 +20,7 @@ use futures::{future, prelude::*, sync::oneshot}; use std::io::{Error as IoError, ErrorKind as IoErrorKind}; -use transport::{MuxedTransport, Transport}; +use transport::Transport; use Multiaddr; /// See `Transport::interruptible`. @@ -46,7 +46,6 @@ where T: Transport, { type Output = T::Output; - type MultiaddrFuture = T::MultiaddrFuture; type Listener = T::Listener; type ListenerUpgrade = T::ListenerUpgrade; type Dial = InterruptibleDial; @@ -78,19 +77,6 @@ where } } -impl MuxedTransport for Interruptible -where - T: MuxedTransport, -{ - type Incoming = T::Incoming; - type IncomingUpgrade = T::IncomingUpgrade; - - #[inline] - fn next_incoming(self) -> Self::Incoming { - self.transport.next_incoming() - } -} - /// Dropping this object interrupts the dialing of the corresponding `Interruptible`. pub struct Interrupt { _tx: oneshot::Sender<()>, diff --git a/core/src/transport/map.rs b/core/src/transport/map.rs index d4e09068fa0..f7dad92294a 100644 --- a/core/src/transport/map.rs +++ b/core/src/transport/map.rs @@ -21,7 +21,7 @@ use futures::prelude::*; use multiaddr::Multiaddr; use std::io::Error as IoError; -use transport::{MuxedTransport, Transport}; +use transport::Transport; use Endpoint; /// See `Transport::map`. @@ -48,22 +48,21 @@ where F: FnOnce(T::Output, Endpoint) -> D + Clone + Send + 'static, // TODO: 'static :-/ { type Output = D; - type MultiaddrFuture = T::MultiaddrFuture; - type Listener = Box + Send>; - type ListenerUpgrade = Box + Send>; - type Dial = Box + Send>; + type Listener = Box + Send>; + type ListenerUpgrade = Box + Send>; + type Dial = Box + Send>; fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { let map = self.map; match self.transport.listen_on(addr) { Ok((stream, listen_addr)) => { - let stream = stream.map(move |future| { + let stream = stream.map(move |(future, addr)| { let map = map.clone(); let future = future .into_future() - .map(move |(output, addr)| (map(output, Endpoint::Listener), addr)); - Box::new(future) as Box<_> + .map(move |output| map(output, Endpoint::Listener)); + (Box::new(future) as Box<_>, addr) }); Ok((Box::new(stream), listen_addr)) } @@ -78,7 +77,7 @@ where Ok(future) => { let future = future .into_future() - .map(move |(output, addr)| (map(output, Endpoint::Dialer), addr)); + .map(move |output| map(output, Endpoint::Dialer)); Ok(Box::new(future)) } Err((transport, addr)) => Err((Map { transport, map }, addr)), @@ -90,28 +89,3 @@ where self.transport.nat_traversal(server, observed) } } - -impl MuxedTransport for Map -where - T: MuxedTransport + 'static, // TODO: 'static :-/ - T::Dial: Send, - T::Listener: Send, - T::ListenerUpgrade: Send, - T::Incoming: Send, - T::IncomingUpgrade: Send, - F: FnOnce(T::Output, Endpoint) -> D + Clone + Send + 'static, // TODO: 'static :-/ -{ - type Incoming = Box + Send>; - type IncomingUpgrade = Box + Send>; - - fn next_incoming(self) -> Self::Incoming { - let map = self.map; - let future = self.transport.next_incoming().map(move |upgrade| { - let future = upgrade.map(move |(output, addr)| { - (map(output, Endpoint::Listener), addr) - }); - Box::new(future) as Box<_> - }); - Box::new(future) - } -} diff --git a/core/src/transport/map_err.rs b/core/src/transport/map_err.rs index 3008f0fd7cf..f8b92839572 100644 --- a/core/src/transport/map_err.rs +++ b/core/src/transport/map_err.rs @@ -21,7 +21,7 @@ use futures::prelude::*; use multiaddr::Multiaddr; use std::io::Error as IoError; -use transport::{MuxedTransport, Transport}; +use transport::Transport; /// See `Transport::map_err`. #[derive(Debug, Copy, Clone)] @@ -44,7 +44,6 @@ where F: FnOnce(IoError) -> IoError + Clone, { type Output = T::Output; - type MultiaddrFuture = T::MultiaddrFuture; type Listener = MapErrListener; type ListenerUpgrade = MapErrListenerUpgrade; type Dial = MapErrDial; @@ -76,23 +75,6 @@ where } } -impl MuxedTransport for MapErr -where - T: MuxedTransport, - F: FnOnce(IoError) -> IoError + Clone, -{ - type Incoming = MapErrIncoming; - type IncomingUpgrade = MapErrIncomingUpgrade; - - #[inline] - fn next_incoming(self) -> Self::Incoming { - MapErrIncoming { - inner: self.transport.next_incoming(), - map: Some(self.map), - } - } -} - /// Listening stream for `MapErr`. pub struct MapErrListener where T: Transport { @@ -104,14 +86,14 @@ impl Stream for MapErrListener where T: Transport, F: FnOnce(IoError) -> IoError + Clone, { - type Item = MapErrListenerUpgrade; + type Item = (MapErrListenerUpgrade, Multiaddr); type Error = IoError; #[inline] fn poll(&mut self) -> Poll, Self::Error> { match try_ready!(self.inner.poll()) { - Some(value) => Ok(Async::Ready( - Some(MapErrListenerUpgrade { inner: value, map: Some(self.map.clone()) }))), + Some((value, addr)) => Ok(Async::Ready( + Some((MapErrListenerUpgrade { inner: value, map: Some(self.map.clone()) }, addr)))), None => Ok(Async::Ready(None)) } } @@ -128,7 +110,7 @@ impl Future for MapErrListenerUpgrade where T: Transport, F: FnOnce(IoError) -> IoError, { - type Item = (T::Output, T::MultiaddrFuture); + type Item = T::Output; type Error = IoError; #[inline] @@ -159,72 +141,7 @@ impl Future for MapErrDial where T: Transport, F: FnOnce(IoError) -> IoError, { - type Item = (T::Output, T::MultiaddrFuture); - type Error = IoError; - - #[inline] - fn poll(&mut self) -> Poll { - match self.inner.poll() { - Ok(Async::Ready(value)) => { - Ok(Async::Ready(value)) - }, - Ok(Async::NotReady) => Ok(Async::NotReady), - Err(err) => { - let map = self.map.take().expect("poll() called again after error"); - Err(map(err)) - } - } - } -} - -/// Incoming future for `MapErr`. -pub struct MapErrIncoming -where T: MuxedTransport -{ - inner: T::Incoming, - map: Option, -} - -impl Future for MapErrIncoming -where T: MuxedTransport, - F: FnOnce(IoError) -> IoError, -{ - type Item = MapErrIncomingUpgrade; - type Error = IoError; - - #[inline] - fn poll(&mut self) -> Poll { - match self.inner.poll() { - Ok(Async::Ready(value)) => { - let map = self.map.take().expect("poll() called again after error"); - let value = MapErrIncomingUpgrade { - inner: value, - map: Some(map), - }; - Ok(Async::Ready(value)) - }, - Ok(Async::NotReady) => Ok(Async::NotReady), - Err(err) => { - let map = self.map.take().expect("poll() called again after error"); - Err(map(err)) - } - } - } -} - -/// Incoming upgrade future for `MapErr`. -pub struct MapErrIncomingUpgrade -where T: MuxedTransport -{ - inner: T::IncomingUpgrade, - map: Option, -} - -impl Future for MapErrIncomingUpgrade -where T: MuxedTransport, - F: FnOnce(IoError) -> IoError, -{ - type Item = (T::Output, T::MultiaddrFuture); + type Item = T::Output; type Error = IoError; #[inline] diff --git a/core/src/transport/map_err_dial.rs b/core/src/transport/map_err_dial.rs index f99ff8e9966..8019df65242 100644 --- a/core/src/transport/map_err_dial.rs +++ b/core/src/transport/map_err_dial.rs @@ -21,7 +21,7 @@ use futures::prelude::*; use multiaddr::Multiaddr; use std::io::Error as IoError; -use transport::{MuxedTransport, Transport}; +use transport::Transport; /// See `Transport::map_err_dial`. #[derive(Debug, Copy, Clone)] @@ -45,10 +45,9 @@ where F: FnOnce(IoError, Multiaddr) -> IoError + Clone + Send + 'static, // TODO: 'static :-/ { type Output = T::Output; - type MultiaddrFuture = T::MultiaddrFuture; type Listener = T::Listener; type ListenerUpgrade = T::ListenerUpgrade; - type Dial = Box + Send>; + type Dial = Box + Send>; fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { match self.transport.listen_on(addr) { @@ -74,18 +73,3 @@ where self.transport.nat_traversal(server, observed) } } - -impl MuxedTransport for MapErrDial -where - T: MuxedTransport + 'static, // TODO: 'static :-/ - T::Dial: Send, - F: FnOnce(IoError, Multiaddr) -> IoError + Clone + Send + 'static, // TODO: 'static :-/ -{ - type Incoming = T::Incoming; - type IncomingUpgrade = T::IncomingUpgrade; - - #[inline] - fn next_incoming(self) -> Self::Incoming { - self.transport.next_incoming() - } -} diff --git a/core/src/transport/memory.rs b/core/src/transport/memory.rs index 14d53111396..f77e9b95251 100644 --- a/core/src/transport/memory.rs +++ b/core/src/transport/memory.rs @@ -52,10 +52,9 @@ impl Clone for Dialer { impl Transport for Dialer { type Output = Channel; - type Listener = Box + Send>; - type ListenerUpgrade = FutureResult<(Self::Output, Self::MultiaddrFuture), io::Error>; - type MultiaddrFuture = FutureResult; - type Dial = Box + Send>; + type Listener = Box + Send>; + type ListenerUpgrade = FutureResult; + type Dial = Box + Send>; fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { Err((self, addr)) @@ -70,7 +69,7 @@ impl Transport for Dialer { let a = Chan { incoming: a_rx, outgoing: b_tx }; let b = Chan { incoming: b_rx, outgoing: a_tx }; let future = self.0.send(b) - .map(move |_| (a.into(), future::ok(addr))) + .map(move |_| a.into()) .map_err(|_| io::ErrorKind::ConnectionRefused.into()); Ok(Box::new(future)) } @@ -95,10 +94,9 @@ impl Clone for Listener { impl Transport for Listener { type Output = Channel; - type Listener = Box + Send>; - type ListenerUpgrade = FutureResult<(Self::Output, Self::MultiaddrFuture), io::Error>; - type MultiaddrFuture = FutureResult; - type Dial = Box + Send>; + type Listener = Box + Send>; + type ListenerUpgrade = FutureResult; + type Dial = Box + Send>; fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { if !is_memory_addr(&addr) { @@ -108,7 +106,7 @@ impl Transport for Listener { let receiver = self.0.clone(); let stream = stream::poll_fn(move || receiver.lock().poll()) .map(move |channel| { - future::ok((channel.into(), future::ok(addr.clone()))) + (future::ok(channel.into()), addr.clone()) }) .map_err(|()| unreachable!()); Ok((Box::new(stream), addr2)) diff --git a/core/src/transport/mod.rs b/core/src/transport/mod.rs index 7679e3f09c3..7a353f1e524 100644 --- a/core/src/transport/mod.rs +++ b/core/src/transport/mod.rs @@ -31,6 +31,7 @@ use futures::prelude::*; use multiaddr::Multiaddr; +use nodes::raw_swarm::ConnectedPoint; use std::io::Error as IoError; use tokio_io::{AsyncRead, AsyncWrite}; use upgrade::{ConnectionUpgrade, Endpoint}; @@ -39,21 +40,16 @@ pub mod and_then; pub mod boxed; pub mod choice; pub mod denied; -pub mod dummy; pub mod interruptible; pub mod map; pub mod map_err; pub mod map_err_dial; pub mod memory; -pub mod muxed; pub mod upgrade; -pub use self::boxed::BoxedMuxed; pub use self::choice::OrTransport; pub use self::denied::DeniedTransport; -pub use self::dummy::DummyMuxing; pub use self::memory::connector; -pub use self::muxed::MuxedTransport; pub use self::upgrade::UpgradedNode; /// A transport is an object that can be used to produce connections by listening or dialing a @@ -75,19 +71,15 @@ pub trait Transport { /// An item should be produced whenever a connection is received at the lowest level of the /// transport stack. The item is a `Future` that is signalled once some pre-processing has /// taken place, and that connection has been upgraded to the wanted protocols. - type Listener: Stream; - - /// Future that produces the multiaddress of the remote. - type MultiaddrFuture: Future; + type Listener: Stream; /// After a connection has been received, we may need to do some asynchronous pre-processing /// on it (eg. an intermediary protocol negotiation). While this pre-processing takes place, we /// want to be able to continue polling on the listener. - // TODO: we could move the `MultiaddrFuture` to the `Listener` trait - type ListenerUpgrade: Future; + type ListenerUpgrade: Future; /// A future which indicates that we are currently dialing to a peer. - type Dial: Future; + type Dial: Future; /// Listen on the given multiaddr. Returns a stream of incoming connections, plus a modified /// version of the `Multiaddr`. This new `Multiaddr` is the one that that should be advertised @@ -132,27 +124,10 @@ pub trait Transport { Self::Dial: Send + 'static, Self::Listener: Send + 'static, Self::ListenerUpgrade: Send + 'static, - Self::MultiaddrFuture: Send + 'static, { boxed::boxed(self) } - /// Turns this `Transport` into an abstract boxed transport. - /// - /// This is the version if the transport supports muxing. - #[inline] - fn boxed_muxed(self) -> boxed::BoxedMuxed - where Self: Sized + MuxedTransport + Clone + Send + Sync + 'static, - Self::Dial: Send + 'static, - Self::Listener: Send + 'static, - Self::ListenerUpgrade: Send + 'static, - Self::MultiaddrFuture: Send + 'static, - Self::Incoming: Send + 'static, - Self::IncomingUpgrade: Send + 'static, - { - boxed::boxed_muxed(self) - } - /// Applies a function on the output of the `Transport`. #[inline] fn map(self, map: F) -> map::Map @@ -207,7 +182,7 @@ pub trait Transport { where Self: Sized, Self::Output: AsyncRead + AsyncWrite, - U: ConnectionUpgrade, + U: ConnectionUpgrade, { UpgradedNode::new(self, upgrade) } @@ -218,29 +193,15 @@ pub trait Transport { /// > **Note**: The concept of an *upgrade* for example includes middlewares such *secio* /// > (communication encryption), *multiplex*, but also a protocol handler. #[inline] - fn and_then(self, upgrade: C) -> and_then::AndThen + fn and_then(self, upgrade: C) -> and_then::AndThen where Self: Sized, - C: FnOnce(Self::Output, Endpoint, Self::MultiaddrFuture) -> F + Clone + 'static, - F: Future + 'static, - Maf: Future + 'static, + C: FnOnce(Self::Output, ConnectedPoint) -> F + Clone + 'static, + F: Future + 'static, { and_then::and_then(self, upgrade) } - /// Builds a dummy implementation of `MuxedTransport` that uses this transport. - /// - /// The resulting object will not actually use muxing. This means that dialing the same node - /// twice will result in two different connections instead of two substreams on the same - /// connection. - #[inline] - fn with_dummy_muxing(self) -> DummyMuxing - where - Self: Sized, - { - DummyMuxing::new(self) - } - /// Wraps around the `Transport` and makes it interruptible. #[inline] fn interruptible(self) -> (interruptible::Interruptible, interruptible::Interrupt) diff --git a/core/src/transport/upgrade.rs b/core/src/transport/upgrade.rs index 8d9fc6218a5..e661b535de7 100644 --- a/core/src/transport/upgrade.rs +++ b/core/src/transport/upgrade.rs @@ -22,7 +22,7 @@ use futures::prelude::*; use multiaddr::Multiaddr; use std::io::Error as IoError; use tokio_io::{AsyncRead, AsyncWrite}; -use transport::{MuxedTransport, Transport}; +use transport::Transport; use upgrade::{apply, ConnectionUpgrade, Endpoint}; /// Implements the `Transport` trait. Dials or listens, then upgrades any dialed or received @@ -50,9 +50,8 @@ where T::Dial: Send, T::Listener: Send, T::ListenerUpgrade: Send, - T::MultiaddrFuture: Send, T::Output: Send + AsyncRead + AsyncWrite, - C: ConnectionUpgrade + Send + 'a, + C: ConnectionUpgrade + Send + 'a, C::NamesIter: Send, C::Future: Send, C::UpgradeIdentifier: Send, @@ -72,7 +71,7 @@ where pub fn dial( self, addr: Multiaddr, - ) -> Result + Send + 'a>, (Self, Multiaddr)> + ) -> Result + Send + 'a>, (Self, Multiaddr)> where C::NamesIter: Clone, // TODO: not elegant { @@ -92,48 +91,13 @@ where let future = dialed_fut // Try to negotiate the protocol. - .and_then(move |(connection, client_addr)| { - apply(connection, upgrade, Endpoint::Dialer, client_addr) + .and_then(move |connection| { + apply(connection, upgrade, Endpoint::Dialer) }); Ok(Box::new(future)) } - /// If the underlying transport is a `MuxedTransport`, then after calling `dial` we may receive - /// substreams opened by the dialed nodes. - /// - /// This function returns the next incoming substream. You are strongly encouraged to call it - /// if you have a muxed transport. - pub fn next_incoming( - self, - ) -> Box< - Future< - Item = Box + Send + 'a>, - Error = IoError, - > - + Send + 'a, - > - where - T: MuxedTransport, - T::Incoming: Send, - T::IncomingUpgrade: Send, - C::NamesIter: Clone, // TODO: not elegant - C: Clone, - { - let upgrade = self.upgrade; - - let future = self.transports.next_incoming().map(|future| { - // Try to negotiate the protocol. - let future = future.and_then(move |(connection, client_addr)| { - apply(connection, upgrade, Endpoint::Listener, client_addr) - }); - - Box::new(future) as Box + Send> - }); - - Box::new(future) as Box<_> - } - /// Start listening on the multiaddr using the transport that was passed to `new`. /// Then whenever a connection is opened, it is upgraded. /// @@ -147,7 +111,7 @@ where ( Box< Stream< - Item = Box + Send + 'a>, + Item = (Box + Send + 'a>, Multiaddr), Error = IoError, > + Send @@ -179,15 +143,15 @@ where // Note that failing to negotiate a protocol will never produce a future with an error. // Instead the `stream` will produce `Ok(Err(...))`. // `stream` can only produce an `Err` if `listening_stream` produces an `Err`. - let stream = listening_stream.map(move |connection| { + let stream = listening_stream.map(move |(connection, client_addr)| { let upgrade = upgrade.clone(); let connection = connection // Try to negotiate the protocol. - .and_then(move |(connection, client_addr)| { - apply(connection, upgrade, Endpoint::Listener, client_addr) + .and_then(move |connection| { + apply(connection, upgrade, Endpoint::Listener) }); - Box::new(connection) as Box<_> + (Box::new(connection) as Box<_>, client_addr) }); Ok((Box::new(stream), new_addr)) @@ -200,19 +164,16 @@ where T::Dial: Send, T::Listener: Send, T::ListenerUpgrade: Send, - T::MultiaddrFuture: Send, T::Output: Send + AsyncRead + AsyncWrite, - C: ConnectionUpgrade + Clone + Send + 'static, - C::MultiaddrFuture: Future, + C: ConnectionUpgrade + Clone + Send + 'static, C::NamesIter: Clone + Send, C::Future: Send, C::UpgradeIdentifier: Send, { type Output = C::Output; - type MultiaddrFuture = C::MultiaddrFuture; - type Listener = Box + Send>; - type ListenerUpgrade = Box + Send>; - type Dial = Box + Send>; + type Listener = Box + Send>; + type ListenerUpgrade = Box + Send>; + type Dial = Box + Send>; #[inline] fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { @@ -229,28 +190,3 @@ where self.transports.nat_traversal(server, observed) } } - -impl MuxedTransport for UpgradedNode -where - T: MuxedTransport + 'static, - T::Dial: Send, - T::Listener: Send, - T::ListenerUpgrade: Send, - T::MultiaddrFuture: Send, - T::Output: Send + AsyncRead + AsyncWrite, - T::Incoming: Send, - T::IncomingUpgrade: Send, - C: ConnectionUpgrade + Clone + Send + 'static, - C::MultiaddrFuture: Future, - C::NamesIter: Clone + Send, - C::Future: Send, - C::UpgradeIdentifier: Send, -{ - type Incoming = Box + Send>; - type IncomingUpgrade = Box + Send>; - - #[inline] - fn next_incoming(self) -> Self::Incoming { - self.next_incoming() - } -} diff --git a/core/src/upgrade/apply.rs b/core/src/upgrade/apply.rs index 7389f70e8fb..2e82d981cd4 100644 --- a/core/src/upgrade/apply.rs +++ b/core/src/upgrade/apply.rs @@ -29,9 +29,9 @@ use upgrade::{ConnectionUpgrade, Endpoint}; /// /// Returns a `Future` that returns the outcome of the connection upgrade. #[inline] -pub fn apply(conn: C, upgrade: U, e: Endpoint, remote: Maf) -> UpgradeApplyFuture +pub fn apply(conn: C, upgrade: U, e: Endpoint) -> UpgradeApplyFuture where - U: ConnectionUpgrade, + U: ConnectionUpgrade, U::NamesIter: Clone, // TODO: not elegant C: AsyncRead + AsyncWrite, { @@ -40,31 +40,28 @@ where future: negotiate(conn, &upgrade, e), upgrade, endpoint: e, - remote } } } /// Future, returned from `apply` which performs a connection upgrade. -pub struct UpgradeApplyFuture +pub struct UpgradeApplyFuture where - U: ConnectionUpgrade, + U: ConnectionUpgrade, C: AsyncRead + AsyncWrite { - inner: UpgradeApplyState + inner: UpgradeApplyState } - -enum UpgradeApplyState +enum UpgradeApplyState where - U: ConnectionUpgrade, + U: ConnectionUpgrade, C: AsyncRead + AsyncWrite { Init { future: NegotiationFuture, U::UpgradeIdentifier>, upgrade: U, - endpoint: Endpoint, - remote: Maf + endpoint: Endpoint }, Upgrade { future: U::Future @@ -72,28 +69,28 @@ where Undefined } -impl Future for UpgradeApplyFuture +impl Future for UpgradeApplyFuture where - U: ConnectionUpgrade, + U: ConnectionUpgrade, U::NamesIter: Clone, C: AsyncRead + AsyncWrite { - type Item = (U::Output, U::MultiaddrFuture); + type Item = U::Output; type Error = IoError; fn poll(&mut self) -> Poll { loop { match mem::replace(&mut self.inner, UpgradeApplyState::Undefined) { - UpgradeApplyState::Init { mut future, upgrade, endpoint, remote } => { + UpgradeApplyState::Init { mut future, upgrade, endpoint } => { let (upgrade_id, connection) = match future.poll()? { Async::Ready(x) => x, Async::NotReady => { - self.inner = UpgradeApplyState::Init { future, upgrade, endpoint, remote }; + self.inner = UpgradeApplyState::Init { future, upgrade, endpoint }; return Ok(Async::NotReady) } }; self.inner = UpgradeApplyState::Upgrade { - future: upgrade.upgrade(connection, upgrade_id, endpoint, remote) + future: upgrade.upgrade(connection, upgrade_id, endpoint) }; } UpgradeApplyState::Upgrade { mut future } => { @@ -124,13 +121,13 @@ where /// /// Returns a `Future` that returns the negotiated protocol and the stream. #[inline] -pub fn negotiate( +pub fn negotiate( connection: C, upgrade: &U, endpoint: Endpoint, ) -> NegotiationFuture, U::UpgradeIdentifier> where - U: ConnectionUpgrade, + U: ConnectionUpgrade, U::NamesIter: Clone, // TODO: not elegant C: AsyncRead + AsyncWrite, { @@ -144,7 +141,6 @@ where } } - /// Future, returned by `negotiate`, which negotiates a protocol and stream. pub struct NegotiationFuture { inner: Either, DialerSelectFuture> @@ -175,7 +171,6 @@ where } } - /// Iterator adapter which adds equality matching predicates to items. /// Used in `NegotiationFuture`. #[derive(Clone)] diff --git a/core/src/upgrade/choice.rs b/core/src/upgrade/choice.rs index f7484af1583..7dfa815132e 100644 --- a/core/src/upgrade/choice.rs +++ b/core/src/upgrade/choice.rs @@ -19,8 +19,7 @@ // DEALINGS IN THE SOFTWARE. use bytes::Bytes; -use futures::{future, prelude::*}; -use std::io::Error as IoError; +use futures::future; use tokio_io::{AsyncRead, AsyncWrite}; use upgrade::{ConnectionUpgrade, Endpoint}; @@ -37,11 +36,11 @@ pub fn or(me: A, other: B) -> OrUpgrade { #[derive(Debug, Copy, Clone)] pub struct OrUpgrade(A, B); -impl ConnectionUpgrade for OrUpgrade +impl ConnectionUpgrade for OrUpgrade where C: AsyncRead + AsyncWrite, - A: ConnectionUpgrade, - B: ConnectionUpgrade, + A: ConnectionUpgrade, + B: ConnectionUpgrade, { type NamesIter = NamesIterChain; type UpgradeIdentifier = EitherUpgradeIdentifier; @@ -55,8 +54,7 @@ where } type Output = O; - type MultiaddrFuture = future::Either; - type Future = EitherConnUpgrFuture; + type Future = future::Either; #[inline] fn upgrade( @@ -64,14 +62,13 @@ where socket: C, id: Self::UpgradeIdentifier, ty: Endpoint, - remote_addr: Maf, ) -> Self::Future { match id { EitherUpgradeIdentifier::First(id) => { - EitherConnUpgrFuture::First(self.0.upgrade(socket, id, ty, remote_addr)) + future::Either::A(self.0.upgrade(socket, id, ty)) } EitherUpgradeIdentifier::Second(id) => { - EitherConnUpgrFuture::Second(self.1.upgrade(socket, id, ty, remote_addr)) + future::Either::B(self.1.upgrade(socket, id, ty)) } } } @@ -84,43 +81,6 @@ pub enum EitherUpgradeIdentifier { Second(B), } -/// Implements `Future` and redirects calls to either `First` or `Second`. -/// -/// Additionally, the output will be wrapped inside a `EitherOutput`. -/// -// TODO: This type is needed because of the lack of `impl Trait` in stable Rust. -// If Rust had impl Trait we could use the Either enum from the futures crate and add some -// modifiers to it. This custom enum is a combination of Either and these modifiers. -#[derive(Debug, Copy, Clone)] -#[must_use = "futures do nothing unless polled"] -pub enum EitherConnUpgrFuture { - First(A), - Second(B), -} - -impl Future for EitherConnUpgrFuture -where - A: Future, - B: Future, -{ - type Item = (O, future::Either); - type Error = IoError; - - #[inline] - fn poll(&mut self) -> Poll { - match self { - &mut EitherConnUpgrFuture::First(ref mut a) => { - let (item, fut) = try_ready!(a.poll()); - Ok(Async::Ready((item, future::Either::A(fut)))) - } - &mut EitherConnUpgrFuture::Second(ref mut b) => { - let (item, fut) = try_ready!(b.poll()); - Ok(Async::Ready((item, future::Either::B(fut)))) - } - } - } -} - /// Internal type used by the `OrUpgrade` struct. /// /// > **Note**: This type is needed because of the lack of `-> impl Trait` in Rust. It can be diff --git a/core/src/upgrade/denied.rs b/core/src/upgrade/denied.rs index 3cb9b836ccf..1ead49f93c0 100644 --- a/core/src/upgrade/denied.rs +++ b/core/src/upgrade/denied.rs @@ -20,7 +20,6 @@ use bytes::Bytes; use futures::prelude::*; -use multiaddr::Multiaddr; use std::{io, iter}; use tokio_io::{AsyncRead, AsyncWrite}; use upgrade::{ConnectionUpgrade, Endpoint}; @@ -29,15 +28,14 @@ use upgrade::{ConnectionUpgrade, Endpoint}; #[derive(Debug, Copy, Clone)] pub struct DeniedConnectionUpgrade; -impl ConnectionUpgrade for DeniedConnectionUpgrade +impl ConnectionUpgrade for DeniedConnectionUpgrade where C: AsyncRead + AsyncWrite, { type NamesIter = iter::Empty<(Bytes, ())>; type UpgradeIdentifier = (); // TODO: could use `!` type Output = (); // TODO: could use `!` - type MultiaddrFuture = Box + Send + Sync>; // TODO: could use `!` - type Future = Box + Send + Sync>; // TODO: could use `!` + type Future = Box + Send + Sync>; // TODO: could use `!` #[inline] fn protocol_names(&self) -> Self::NamesIter { @@ -45,7 +43,7 @@ where } #[inline] - fn upgrade(self, _: C, _: Self::UpgradeIdentifier, _: Endpoint, _: Maf) -> Self::Future { + fn upgrade(self, _: C, _: Self::UpgradeIdentifier, _: Endpoint) -> Self::Future { unreachable!("the denied connection upgrade always fails to negotiate") } } diff --git a/core/src/upgrade/loop_upg.rs b/core/src/upgrade/loop_upg.rs index faf761c0010..f597c9eccd2 100644 --- a/core/src/upgrade/loop_upg.rs +++ b/core/src/upgrade/loop_upg.rs @@ -61,23 +61,20 @@ pub struct LoopUpg { } // TODO: 'static :-/ -impl ConnectionUpgrade<(State, Socket), AddrFut> +impl ConnectionUpgrade<(State, Socket)> for LoopUpg where State: Send + 'static, Socket: AsyncRead + AsyncWrite + Send + 'static, Inner: ConnectionUpgrade< (State, Socket), - AddrFut, Output = Loop, - MultiaddrFuture = AddrFut, > + Clone + Send + 'static, Inner::NamesIter: Clone + Send + 'static, Inner::UpgradeIdentifier: Send, Inner::Future: Send, - AddrFut: Send + 'static, Out: Send + 'static, { type NamesIter = Inner::NamesIter; @@ -88,29 +85,27 @@ where } type Output = Out; - type MultiaddrFuture = AddrFut; - type Future = Box + Send>; + type Future = Box + Send>; fn upgrade( self, (state, socket): (State, Socket), id: Self::UpgradeIdentifier, endpoint: Endpoint, - remote_addr: AddrFut, ) -> Self::Future { let inner = self.inner; let fut = future::loop_fn( - (state, socket, id, remote_addr, MAX_LOOPS), - move |(state, socket, id, remote_addr, loops_remaining)| { + (state, socket, id, MAX_LOOPS), + move |(state, socket, id, loops_remaining)| { // When we enter a recursion of the `loop_fn`, a protocol has already been // negotiated. So what we have to do is upgrade then negotiate the next protocol // (if necessary), and then only continue iteration in the `future::loop_fn`. let inner = inner.clone(); inner .clone() - .upgrade((state, socket), id, endpoint, remote_addr) - .and_then(move |(loop_out, remote_addr)| match loop_out { + .upgrade((state, socket), id, endpoint) + .and_then(move |loop_out| match loop_out { Loop::Continue(state, socket) => { // Produce an error if we reached the recursion limit. if loops_remaining == 0 { @@ -126,14 +121,13 @@ where state, socket, id, - remote_addr, loops_remaining - 1, )) }); future::Either::A(fut) } Loop::Break(fin) => { - future::Either::B(future::ok(FutLoop::Break((fin, remote_addr)))) + future::Either::B(future::ok(FutLoop::Break(fin))) } }) }, diff --git a/core/src/upgrade/map.rs b/core/src/upgrade/map.rs index 92f365ec6f1..4ead163d5de 100644 --- a/core/src/upgrade/map.rs +++ b/core/src/upgrade/map.rs @@ -36,9 +36,9 @@ pub struct Map { map: F, } -impl ConnectionUpgrade for Map +impl ConnectionUpgrade for Map where - U: ConnectionUpgrade, + U: ConnectionUpgrade, U::Future: Send + 'static, // TODO: 'static :( C: AsyncRead + AsyncWrite, F: FnOnce(U::Output) -> O + Send + 'static, // TODO: 'static :( @@ -51,20 +51,18 @@ where } type Output = O; - type MultiaddrFuture = U::MultiaddrFuture; - type Future = Box + Send>; + type Future = Box + Send>; fn upgrade( self, socket: C, id: Self::UpgradeIdentifier, ty: Endpoint, - remote_addr: Maf, ) -> Self::Future { let map = self.map; let fut = self.upgrade - .upgrade(socket, id, ty, remote_addr) - .map(move |(out, maf)| (map(out), maf)); + .upgrade(socket, id, ty) + .map(map); Box::new(fut) as Box<_> } } diff --git a/core/src/upgrade/mod.rs b/core/src/upgrade/mod.rs index d01a60fec52..279d5a9b44e 100644 --- a/core/src/upgrade/mod.rs +++ b/core/src/upgrade/mod.rs @@ -23,7 +23,6 @@ pub mod choice; pub mod denied; pub mod loop_upg; pub mod map; -pub mod map_addr; pub mod plaintext; pub mod toggleable; pub mod traits; @@ -33,7 +32,6 @@ pub use self::choice::{or, OrUpgrade}; pub use self::denied::DeniedConnectionUpgrade; pub use self::loop_upg::{loop_upg, Loop}; pub use self::map::map; -pub use self::map_addr::map_with_addr; pub use self::plaintext::PlainTextConfig; pub use self::toggleable::toggleable; pub use self::traits::{ConnectionUpgrade, Endpoint}; diff --git a/core/src/upgrade/plaintext.rs b/core/src/upgrade/plaintext.rs index ceaf179b1a2..ac969853960 100644 --- a/core/src/upgrade/plaintext.rs +++ b/core/src/upgrade/plaintext.rs @@ -32,19 +32,18 @@ use upgrade::{ConnectionUpgrade, Endpoint}; #[derive(Debug, Copy, Clone)] pub struct PlainTextConfig; -impl ConnectionUpgrade for PlainTextConfig +impl ConnectionUpgrade for PlainTextConfig where C: AsyncRead + AsyncWrite, { type Output = C; - type Future = FutureResult<(C, F), IoError>; + type Future = FutureResult; type UpgradeIdentifier = (); - type MultiaddrFuture = F; type NamesIter = iter::Once<(Bytes, ())>; #[inline] - fn upgrade(self, i: C, _: (), _: Endpoint, remote_addr: F) -> Self::Future { - future::ok((i, remote_addr)) + fn upgrade(self, i: C, _: (), _: Endpoint) -> Self::Future { + future::ok(i) } #[inline] diff --git a/core/src/upgrade/toggleable.rs b/core/src/upgrade/toggleable.rs index 33ea8f329e0..5383ca463f9 100644 --- a/core/src/upgrade/toggleable.rs +++ b/core/src/upgrade/toggleable.rs @@ -65,10 +65,10 @@ impl Toggleable { } } -impl ConnectionUpgrade for Toggleable +impl ConnectionUpgrade for Toggleable where C: AsyncRead + AsyncWrite, - U: ConnectionUpgrade, + U: ConnectionUpgrade, { type NamesIter = ToggleableIter; type UpgradeIdentifier = U::UpgradeIdentifier; @@ -82,8 +82,7 @@ where } type Output = U::Output; - type MultiaddrFuture = U::MultiaddrFuture; - type Future = future::Either, U::Future>; + type Future = future::Either, U::Future>; #[inline] fn upgrade( @@ -91,10 +90,9 @@ where socket: C, id: Self::UpgradeIdentifier, ty: Endpoint, - remote_addr: Maf, ) -> Self::Future { if self.enabled { - future::Either::B(self.inner.upgrade(socket, id, ty, remote_addr)) + future::Either::B(self.inner.upgrade(socket, id, ty)) } else { future::Either::A(future::empty()) } diff --git a/core/src/upgrade/traits.rs b/core/src/upgrade/traits.rs index 519992460f2..82efde57b7f 100644 --- a/core/src/upgrade/traits.rs +++ b/core/src/upgrade/traits.rs @@ -50,7 +50,7 @@ impl Not for Endpoint { /// > **Note**: The `upgrade` method of this trait uses `self` and not `&self` or `&mut self`. /// > This has been designed so that you would implement this trait on `&Foo` or /// > `&mut Foo` instead of directly on `Foo`. -pub trait ConnectionUpgrade { +pub trait ConnectionUpgrade { /// Iterator returned by `protocol_names`. type NamesIter: Iterator; /// Type that serves as an identifier for the protocol. This type only exists to be returned @@ -68,10 +68,8 @@ pub trait ConnectionUpgrade { /// > **Note**: For upgrades that add an intermediary layer (such as `secio` or `multiplex`), /// > this associated type must implement `AsyncRead + AsyncWrite`. type Output; - /// Type of the future that will resolve to the remote's multiaddr. - type MultiaddrFuture; /// Type of the future that will resolve to `Self::Output`. - type Future: Future; + type Future: Future; /// This method is called after protocol negotiation has been performed. /// @@ -82,6 +80,5 @@ pub trait ConnectionUpgrade { socket: C, id: Self::UpgradeIdentifier, ty: Endpoint, - remote_addr: TAddrFut, ) -> Self::Future; } diff --git a/muxers/mplex/src/lib.rs b/muxers/mplex/src/lib.rs index 670e11c5bca..2de8c8b9a62 100644 --- a/muxers/mplex/src/lib.rs +++ b/muxers/mplex/src/lib.rs @@ -117,18 +117,17 @@ pub enum MaxBufferBehaviour { Block, } -impl ConnectionUpgrade for MplexConfig +impl ConnectionUpgrade for MplexConfig where C: AsyncRead + AsyncWrite, { type Output = Multiplex; - type MultiaddrFuture = Maf; - type Future = future::FutureResult<(Self::Output, Self::MultiaddrFuture), IoError>; + type Future = future::FutureResult; type UpgradeIdentifier = (); type NamesIter = iter::Once<(Bytes, ())>; #[inline] - fn upgrade(self, i: C, _: (), endpoint: Endpoint, remote_addr: Maf) -> Self::Future { + fn upgrade(self, i: C, _: (), endpoint: Endpoint) -> Self::Future { let max_buffer_len = self.max_buffer_len; let out = Multiplex { @@ -149,7 +148,7 @@ where }) }; - future::ok((out, remote_addr)) + future::ok(out) } #[inline] diff --git a/muxers/mplex/tests/two_peers.rs b/muxers/mplex/tests/two_peers.rs index 7c030fa95a6..26fd1507db4 100644 --- a/muxers/mplex/tests/two_peers.rs +++ b/muxers/mplex/tests/two_peers.rs @@ -52,8 +52,8 @@ fn client_to_server_outbound() { let future = listener .into_future() .map_err(|(err, _)| err) - .and_then(|(client, _)| client.unwrap().map(|v| Arc::new(v.0))) - .and_then(|client| muxing::outbound_from_ref_and_wrap(client)) + .and_then(|(client, _)| client.unwrap().0) + .and_then(|client| muxing::outbound_from_ref_and_wrap(Arc::new(client))) .map(|client| Framed::<_, bytes::BytesMut>::new(client.unwrap())) .and_then(|client| { client @@ -75,7 +75,7 @@ fn client_to_server_outbound() { let future = transport .dial(rx.recv().unwrap()) .unwrap() - .and_then(|client| muxing::inbound_from_ref_and_wrap(Arc::new(client.0))) + .and_then(|client| muxing::inbound_from_ref_and_wrap(Arc::new(client))) .map(|server| Framed::<_, bytes::BytesMut>::new(server.unwrap())) .and_then(|server| server.send("hello world".into())) .map(|_| ()); @@ -102,7 +102,7 @@ fn client_to_server_inbound() { let future = listener .into_future() .map_err(|(err, _)| err) - .and_then(|(client, _)| client.unwrap().map(|v| v.0)) + .and_then(|(client, _)| client.unwrap().0) .and_then(|client| muxing::inbound_from_ref_and_wrap(Arc::new(client))) .map(|client| Framed::<_, bytes::BytesMut>::new(client.unwrap())) .and_then(|client| { @@ -125,7 +125,7 @@ fn client_to_server_inbound() { let future = transport .dial(rx.recv().unwrap()) .unwrap() - .and_then(|(client, _)| muxing::outbound_from_ref_and_wrap(Arc::new(client))) + .and_then(|client| muxing::outbound_from_ref_and_wrap(Arc::new(client))) .map(|server| Framed::<_, bytes::BytesMut>::new(server.unwrap())) .and_then(|server| server.send("hello world".into())) .map(|_| ()); diff --git a/muxers/yamux/src/lib.rs b/muxers/yamux/src/lib.rs index 2b8610c9f75..0ec13430f29 100644 --- a/muxers/yamux/src/lib.rs +++ b/muxers/yamux/src/lib.rs @@ -134,10 +134,9 @@ impl Default for Config { } } -impl core::ConnectionUpgrade for Config +impl core::ConnectionUpgrade for Config where C: AsyncRead + AsyncWrite + 'static, - M: 'static { type UpgradeIdentifier = (); type NamesIter = iter::Once<(Bytes, ())>; @@ -147,15 +146,15 @@ where } type Output = Yamux; - type MultiaddrFuture = M; - type Future = FutureResult<(Yamux, M), io::Error>; + type Future = FutureResult, io::Error>; - fn upgrade(self, i: C, _: (), end: Endpoint, remote: M) -> Self::Future { + fn upgrade(self, i: C, _: (), end: Endpoint) -> Self::Future { let mode = match end { Endpoint::Dialer => yamux::Mode::Client, Endpoint::Listener => yamux::Mode::Server }; - future::ok((Yamux::new(i, self.0, mode), remote)) + + future::ok(Yamux::new(i, self.0, mode)) } } diff --git a/protocols/floodsub/src/lib.rs b/protocols/floodsub/src/lib.rs index 65cb2392d71..c84150bbce1 100644 --- a/protocols/floodsub/src/lib.rs +++ b/protocols/floodsub/src/lib.rs @@ -89,10 +89,9 @@ impl FloodSubUpgrade { } } -impl ConnectionUpgrade for FloodSubUpgrade +impl ConnectionUpgrade for FloodSubUpgrade where C: AsyncRead + AsyncWrite + Send + 'static, - Maf: Future + Send + 'static, { type NamesIter = iter::Once<(Bytes, Self::UpgradeIdentifier)>; type UpgradeIdentifier = (); @@ -103,8 +102,7 @@ where } type Output = FloodSubFuture; - type MultiaddrFuture = future::FutureResult; - type Future = Box + Send>; + type Future = Box + Send>; #[inline] fn upgrade( @@ -112,11 +110,13 @@ where socket: C, _: Self::UpgradeIdentifier, _: Endpoint, - remote_addr: Maf, ) -> Self::Future { debug!("Upgrading connection as floodsub"); - let future = remote_addr.and_then(move |remote_addr| { + let future = { + // FIXME: WRONG + let remote_addr: Multiaddr = "/ip4/127.0.0.1/tcp/5000".parse().unwrap(); + // Whenever a new node connects, we send to it a message containing the topics we are // already subscribed to. let init_msg: Vec = { @@ -168,7 +168,6 @@ where } let inner = self.inner.clone(); - let remote_addr_ret = future::ok(remote_addr.clone()); let future = future::loop_fn( (floodsub_sink, messages), move |(floodsub_sink, messages)| { @@ -215,10 +214,10 @@ where }, ); - future::ok((FloodSubFuture { + future::ok(FloodSubFuture { inner: Box::new(future) as Box<_>, - }, remote_addr_ret)) - }); + }) + }; Box::new(future) as Box<_> } diff --git a/protocols/identify/Cargo.toml b/protocols/identify/Cargo.toml index 365bf0995b6..e055733ea33 100644 --- a/protocols/identify/Cargo.toml +++ b/protocols/identify/Cargo.toml @@ -16,7 +16,9 @@ parking_lot = "0.6" protobuf = "2.0.2" tokio-codec = "0.1" tokio-io = "0.1.0" +tokio-timer = "0.2.6" unsigned-varint = { version = "0.2.1", features = ["codec"] } +void = "1.0" [dev-dependencies] libp2p-tcp-transport = { path = "../../transports/tcp" } diff --git a/protocols/identify/src/lib.rs b/protocols/identify/src/lib.rs index b8fd5ff772b..503bb55cade 100644 --- a/protocols/identify/src/lib.rs +++ b/protocols/identify/src/lib.rs @@ -77,14 +77,12 @@ extern crate parking_lot; extern crate protobuf; extern crate tokio_codec; extern crate tokio_io; +extern crate tokio_timer; extern crate unsigned_varint; +extern crate void; -pub use self::identify_transport::IdentifyTransportOutcome; -pub use self::peer_id_transport::{PeerIdTransport, PeerIdTransportOutput}; pub use self::protocol::{IdentifyInfo, IdentifyOutput}; pub use self::protocol::{IdentifyProtocolConfig, IdentifySender}; -mod identify_transport; -mod peer_id_transport; mod protocol; mod structs_proto; diff --git a/protocols/identify/src/protocol.rs b/protocols/identify/src/protocol.rs index 31506a02ed6..671dbc6bdd1 100644 --- a/protocols/identify/src/protocol.rs +++ b/protocols/identify/src/protocol.rs @@ -20,8 +20,7 @@ use bytes::{Bytes, BytesMut}; use futures::{future, Future, Sink, Stream}; -use libp2p_core::{ConnectionUpgrade, Endpoint, PublicKey}; -use multiaddr::Multiaddr; +use libp2p_core::{ConnectionUpgrade, Endpoint, Multiaddr, PublicKey}; use protobuf::Message as ProtobufMessage; use protobuf::parse_from_bytes as protobuf_parse_from_bytes; use protobuf::RepeatedField; @@ -111,23 +110,21 @@ pub struct IdentifyInfo { pub protocols: Vec, } -impl ConnectionUpgrade for IdentifyProtocolConfig +impl ConnectionUpgrade for IdentifyProtocolConfig where C: AsyncRead + AsyncWrite + Send + 'static, - Maf: Future + Send + 'static, { type NamesIter = iter::Once<(Bytes, Self::UpgradeIdentifier)>; type UpgradeIdentifier = (); type Output = IdentifyOutput; - type MultiaddrFuture = future::Either, Maf>; - type Future = Box + Send>; + type Future = Box + Send>; #[inline] fn protocol_names(&self) -> Self::NamesIter { iter::once((Bytes::from("/ipfs/id/1.0.0"), ())) } - fn upgrade(self, socket: C, _: (), ty: Endpoint, remote_addr: Maf) -> Self::Future { + fn upgrade(self, socket: C, _: (), ty: Endpoint) -> Self::Future { trace!("Upgrading connection as {:?}", ty); let socket = Framed::new(socket, codec::UviBytes::default()); @@ -153,12 +150,10 @@ where trace!("Remote observes us as {:?}", observed_addr); trace!("Information received: {:?}", info); - let out = IdentifyOutput::RemoteInfo { + Ok(IdentifyOutput::RemoteInfo { info, observed_addr: observed_addr.clone(), - }; - - Ok((out, future::Either::A(future::ok(observed_addr)))) + }) } else { debug!("Identify protocol stream closed before receiving info"); Err(IoErrorKind::InvalidData.into()) @@ -170,15 +165,7 @@ where Endpoint::Listener => { let sender = IdentifySender { inner: socket }; - - let future = future::ok({ - let io = IdentifyOutput::Sender { - sender, - }; - - (io, future::Either::B(remote_addr)) - }); - + let future = future::ok(IdentifyOutput::Sender { sender }); Box::new(future) as Box<_> } } @@ -251,7 +238,7 @@ mod tests { let future = listener .into_future() .map_err(|(err, _)| err) - .and_then(|(client, _)| client.unwrap().map(|v| v.0)) + .and_then(|(client, _)| client.unwrap().0) .and_then(|identify| match identify { IdentifyOutput::Sender { sender, .. } => sender.send( IdentifyInfo { @@ -277,7 +264,7 @@ mod tests { let future = transport .dial(rx.recv().unwrap()) .unwrap_or_else(|_| panic!()) - .and_then(|(identify, _)| match identify { + .and_then(|identify| match identify { IdentifyOutput::RemoteInfo { info, observed_addr, diff --git a/protocols/kad/src/kad_server.rs b/protocols/kad/src/kad_server.rs index 2a9c3e4340f..45a7e7c6396 100644 --- a/protocols/kad/src/kad_server.rs +++ b/protocols/kad/src/kad_server.rs @@ -64,7 +64,7 @@ impl KadConnecConfig { } } -impl ConnectionUpgrade for KadConnecConfig +impl ConnectionUpgrade for KadConnecConfig where C: AsyncRead + AsyncWrite + Send + 'static, // TODO: 'static :-/ { @@ -72,22 +72,21 @@ where KadConnecController, Box + Send>, ); - type MultiaddrFuture = Maf; - type Future = future::Map<>::Future, fn((>::Output, Maf)) -> (Self::Output, Maf)>; + type Future = future::Map<>::Future, fn(>::Output) -> Self::Output>; type NamesIter = iter::Once<(Bytes, ())>; type UpgradeIdentifier = (); #[inline] fn protocol_names(&self) -> Self::NamesIter { - ConnectionUpgrade::::protocol_names(&self.raw_proto) + ConnectionUpgrade::::protocol_names(&self.raw_proto) } #[inline] - fn upgrade(self, incoming: C, id: (), endpoint: Endpoint, addr: Maf) -> Self::Future { + fn upgrade(self, incoming: C, id: (), endpoint: Endpoint) -> Self::Future { self.raw_proto - .upgrade(incoming, id, endpoint, addr) - .map:: _, _>(move |(connec, addr)| { - (build_from_sink_stream(connec), addr) + .upgrade(incoming, id, endpoint) + .map:: _, _>(move |connec| { + build_from_sink_stream(connec) }) } } diff --git a/protocols/kad/src/protocol.rs b/protocols/kad/src/protocol.rs index 64120182b34..3ef816486e2 100644 --- a/protocols/kad/src/protocol.rs +++ b/protocols/kad/src/protocol.rs @@ -128,13 +128,12 @@ impl Into for KadPeer { #[derive(Debug, Default, Copy, Clone)] pub struct KademliaProtocolConfig; -impl ConnectionUpgrade for KademliaProtocolConfig +impl ConnectionUpgrade for KademliaProtocolConfig where C: AsyncRead + AsyncWrite + 'static, // TODO: 'static :-/ { type Output = KadStreamSink; - type MultiaddrFuture = Maf; - type Future = future::FutureResult<(Self::Output, Self::MultiaddrFuture), IoError>; + type Future = future::FutureResult; type NamesIter = iter::Once<(Bytes, ())>; type UpgradeIdentifier = (); @@ -144,8 +143,8 @@ where } #[inline] - fn upgrade(self, incoming: C, _: (), _: Endpoint, addr: Maf) -> Self::Future { - future::ok((kademlia_protocol(incoming), addr)) + fn upgrade(self, incoming: C, _: (), _: Endpoint) -> Self::Future { + future::ok(kademlia_protocol(incoming)) } } @@ -489,7 +488,7 @@ mod tests { let future = listener .into_future() .map_err(|(err, _)| err) - .and_then(|(client, _)| client.unwrap().map(|v| v.0)) + .and_then(|(client, _)| client.unwrap().0) .and_then(|proto| proto.into_future().map_err(|(err, _)| err).map(|(v, _)| v)) .map(|recv_msg| { assert_eq!(recv_msg.unwrap(), msg_server); @@ -504,7 +503,7 @@ mod tests { let future = transport .dial(rx.recv().unwrap()) .unwrap_or_else(|_| panic!()) - .and_then(|proto| proto.0.send(msg_client)) + .and_then(|proto| proto.send(msg_client)) .map(|_| ()); let _ = tokio_current_thread::block_on_all(future).unwrap(); diff --git a/protocols/ping/src/lib.rs b/protocols/ping/src/lib.rs index fcb8663daa4..af4092c16e0 100644 --- a/protocols/ping/src/lib.rs +++ b/protocols/ping/src/lib.rs @@ -63,7 +63,7 @@ //! let ping_finished_future = libp2p_tcp_transport::TcpConfig::new() //! .with_upgrade(Ping::default()) //! .dial("127.0.0.1:12345".parse::().unwrap()).unwrap_or_else(|_| panic!()) -//! .and_then(|(out, _)| { +//! .and_then(|out| { //! match out { //! PingOutput::Ponger(processing) => Box::new(processing) as Box + Send>, //! PingOutput::Pinger(mut pinger) => { @@ -123,7 +123,7 @@ pub enum PingOutput { Ponger(PingListener), } -impl ConnectionUpgrade for Ping +impl ConnectionUpgrade for Ping where TSocket: AsyncRead + AsyncWrite, { @@ -136,8 +136,7 @@ where } type Output = PingOutput; - type MultiaddrFuture = Maf; - type Future = FutureResult<(Self::Output, Self::MultiaddrFuture), IoError>; + type Future = FutureResult; #[inline] fn upgrade( @@ -145,14 +144,13 @@ where socket: TSocket, _: Self::UpgradeIdentifier, endpoint: Endpoint, - remote_addr: Maf, ) -> Self::Future { let out = match endpoint { Endpoint::Dialer => upgrade_as_dialer(socket), Endpoint::Listener => upgrade_as_listener(socket), }; - Ok((out, remote_addr)).into_future() + Ok(out).into_future() } } @@ -399,9 +397,8 @@ mod tests { use self::tokio_tcp::TcpListener; use self::tokio_tcp::TcpStream; use super::{Ping, PingOutput}; - use futures::{future, Future, Stream}; - use libp2p_core::{ConnectionUpgrade, Endpoint, Multiaddr}; - use std::io::Error as IoError; + use futures::{Future, Stream}; + use libp2p_core::{ConnectionUpgrade, Endpoint}; // TODO: rewrite tests with the MemoryTransport @@ -419,10 +416,9 @@ mod tests { c.unwrap(), (), Endpoint::Listener, - future::ok::("/ip4/127.0.0.1/tcp/10000".parse().unwrap()), ) }) - .and_then(|(out, _)| match out { + .and_then(|out| match out { PingOutput::Ponger(service) => service, _ => unreachable!(), }); @@ -434,10 +430,9 @@ mod tests { c, (), Endpoint::Dialer, - future::ok::("/ip4/127.0.0.1/tcp/10000".parse().unwrap()), ) }) - .and_then(|(out, _)| match out { + .and_then(|out| match out { PingOutput::Pinger(mut pinger) => { pinger.ping(()); pinger.into_future().map(|_| ()).map_err(|_| panic!()) @@ -464,10 +459,9 @@ mod tests { c.unwrap(), (), Endpoint::Listener, - future::ok::("/ip4/127.0.0.1/tcp/10000".parse().unwrap()), ) }) - .and_then(|(out, _)| match out { + .and_then(|out| match out { PingOutput::Ponger(service) => service, _ => unreachable!(), }); @@ -479,10 +473,9 @@ mod tests { c, (), Endpoint::Dialer, - future::ok::("/ip4/127.0.0.1/tcp/10000".parse().unwrap()), ) }) - .and_then(|(out, _)| match out { + .and_then(|out| match out { PingOutput::Pinger(mut pinger) => { for n in 0..20 { pinger.ping(n); diff --git a/protocols/secio/src/lib.rs b/protocols/secio/src/lib.rs index a7a61756872..8ec9991ac78 100644 --- a/protocols/secio/src/lib.rs +++ b/protocols/secio/src/lib.rs @@ -58,7 +58,7 @@ //! //! let future = transport.dial("/ip4/127.0.0.1/tcp/12345".parse::().unwrap()) //! .unwrap_or_else(|_| panic!("Unable to dial node")) -//! .and_then(|(connection, _)| { +//! .and_then(|connection| { //! // Sends "hello world" on the connection, will be encrypted. //! write_all(connection, "hello world") //! }); @@ -349,14 +349,12 @@ where pub ephemeral_public_key: Vec, } -impl libp2p_core::ConnectionUpgrade for SecioConfig +impl libp2p_core::ConnectionUpgrade for SecioConfig where S: AsyncRead + AsyncWrite + Send + 'static, // TODO: 'static :( - Maf: Send + 'static, // TODO: 'static :( { type Output = SecioOutput; - type MultiaddrFuture = Maf; - type Future = Box + Send>; + type Future = Box + Send>; type NamesIter = iter::Once<(Bytes, ())>; type UpgradeIdentifier = (); @@ -371,7 +369,6 @@ where incoming: S, _: (), _: libp2p_core::Endpoint, - remote_addr: Maf, ) -> Self::Future { debug!("Starting secio upgrade"); @@ -384,7 +381,7 @@ where ephemeral_public_key: ephemeral, } }).map_err(map_err); - Box::new(wrapped.map(move |out| (out, remote_addr))) + Box::new(wrapped) } } diff --git a/src/lib.rs b/src/lib.rs index 2d81fb74bda..b43f807192f 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -212,7 +212,6 @@ impl CommonTransport { impl Transport for CommonTransport { type Output = ::Output; - type MultiaddrFuture = ::MultiaddrFuture; type Listener = ::Listener; type ListenerUpgrade = ::ListenerUpgrade; type Dial = ::Dial; diff --git a/src/simple.rs b/src/simple.rs index f640af86f1a..42dbd789eaa 100644 --- a/src/simple.rs +++ b/src/simple.rs @@ -57,13 +57,12 @@ impl Clone for SimpleProtocol { } } -impl ConnectionUpgrade for SimpleProtocol +impl ConnectionUpgrade for SimpleProtocol where C: AsyncRead + AsyncWrite, F: Fn(C) -> O, O: IntoFuture, O::Future: Send + 'static, - Maf: Send + 'static, { type NamesIter = iter::Once<(Bytes, ())>; type UpgradeIdentifier = (); @@ -74,13 +73,12 @@ where } type Output = O::Item; - type MultiaddrFuture = Maf; - type Future = Box + Send>; + type Future = Box + Send>; #[inline] - fn upgrade(self, socket: C, _: (), _: Endpoint, client_addr: Maf) -> Self::Future { + fn upgrade(self, socket: C, _: (), _: Endpoint) -> Self::Future { let upgrade = &self.upgrade; - let fut = upgrade(socket).into_future().from_err().map(move |out| (out, client_addr)); + let fut = upgrade(socket).into_future().from_err(); Box::new(fut) as Box<_> } } diff --git a/transports/dns/src/lib.rs b/transports/dns/src/lib.rs index d0e63796f98..9b7948ac2ee 100644 --- a/transports/dns/src/lib.rs +++ b/transports/dns/src/lib.rs @@ -98,10 +98,9 @@ where T::Dial: Send, { type Output = T::Output; - type MultiaddrFuture = T::MultiaddrFuture; type Listener = T::Listener; type ListenerUpgrade = T::ListenerUpgrade; - type Dial = Box + Send>; + type Dial = Box + Send>; #[inline] fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { @@ -227,9 +226,9 @@ mod tests { extern crate libp2p_tcp_transport; use self::libp2p_tcp_transport::TcpConfig; use futures::future; + use swarm::Transport; use multiaddr::{Protocol, Multiaddr}; use std::io::Error as IoError; - use swarm::Transport; use DnsConfig; #[test] @@ -238,10 +237,9 @@ mod tests { struct CustomTransport; impl Transport for CustomTransport { type Output = ::Output; - type MultiaddrFuture = ::MultiaddrFuture; type Listener = ::Listener; type ListenerUpgrade = ::ListenerUpgrade; - type Dial = future::Empty<(Self::Output, Self::MultiaddrFuture), IoError>; + type Dial = future::Empty; #[inline] fn listen_on( diff --git a/transports/ratelimit/src/lib.rs b/transports/ratelimit/src/lib.rs index 9fc86a240f4..215806a5503 100644 --- a/transports/ratelimit/src/lib.rs +++ b/transports/ratelimit/src/lib.rs @@ -119,16 +119,16 @@ impl AsyncWrite for Connection { pub struct Listener(RateLimited); impl Stream for Listener { - type Item = ListenerUpgrade; + type Item = (ListenerUpgrade, Multiaddr); type Error = io::Error; fn poll(&mut self) -> Poll, Self::Error> { match try_ready!(self.0.value.poll()) { - Some(upgrade) => { + Some((upgrade, addr)) => { let r = self.0.rlimiter.clone(); let w = self.0.wlimiter.clone(); let u = ListenerUpgrade(RateLimited::from_parts(upgrade, r, w)); - Ok(Async::Ready(Some(u))) + Ok(Async::Ready(Some((u, addr)))) } None => Ok(Async::Ready(None)), } @@ -143,14 +143,14 @@ where T: Transport + 'static, T::Output: AsyncRead + AsyncWrite, { - type Item = (Connection, T::MultiaddrFuture); + type Item = Connection; type Error = io::Error; fn poll(&mut self) -> Poll { - let (conn, addr) = try_ready!(self.0.value.poll()); + let conn = try_ready!(self.0.value.poll()); let r = self.0.rlimiter.clone(); let w = self.0.wlimiter.clone(); - Ok(Async::Ready((Connection::new(conn, r, w)?, addr))) + Ok(Async::Ready(Connection::new(conn, r, w)?)) } } @@ -158,14 +158,12 @@ impl Transport for RateLimited where T: Transport + 'static, T::Dial: Send, - T::MultiaddrFuture: Send, T::Output: AsyncRead + AsyncWrite + Send, { type Output = Connection; - type MultiaddrFuture = T::MultiaddrFuture; type Listener = Listener; type ListenerUpgrade = ListenerUpgrade; - type Dial = Box, Self::MultiaddrFuture), Error = io::Error> + Send>; + type Dial = Box, Error = io::Error> + Send>; fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> where @@ -197,7 +195,7 @@ where .dial(addr) .map(move |dial| { let future = dial - .and_then(move |(conn, addr)| Ok((Connection::new(conn, r, w)?, addr))); + .and_then(move |conn| Ok(Connection::new(conn, r, w)?)); Box::new(future) as Box<_> }) .map_err(|(transport, a)| (RateLimited::from_parts(transport, r2, w2), a)) diff --git a/transports/relay/src/protocol.rs b/transports/relay/src/protocol.rs index b08ba98115e..984ce4d764d 100644 --- a/transports/relay/src/protocol.rs +++ b/transports/relay/src/protocol.rs @@ -49,18 +49,16 @@ pub enum Output { Sealed(Box + Send>) } -impl ConnectionUpgrade for RelayConfig +impl ConnectionUpgrade for RelayConfig where C: AsyncRead + AsyncWrite + Send + 'static, T: Transport + Clone + Send + 'static, T::Dial: Send, T::Listener: Send, T::ListenerUpgrade: Send, - T::MultiaddrFuture: Send, T::Output: AsyncRead + AsyncWrite + Send, P: Deref + Clone + Send + 'static, S: 'static, - Maf: Send + 'static, for<'a> &'a S: Peerstore { type NamesIter = iter::Once<(Bytes, Self::UpgradeIdentifier)>; @@ -71,10 +69,9 @@ where } type Output = Output; - type MultiaddrFuture = Maf; - type Future = Box + Send>; + type Future = Box + Send>; - fn upgrade(self, conn: C, _: (), _: Endpoint, remote_addr: Maf) -> Self::Future { + fn upgrade(self, conn: C, _: (), _: Endpoint) -> Self::Future { let future = Io::new(conn).recv().and_then(move |(message, io)| { let msg = if let Some(m) = message { m @@ -95,7 +92,7 @@ where } } }); - Box::new(future.map(move |out| (out, remote_addr))) + Box::new(future) } } @@ -105,7 +102,6 @@ where T::Dial: Send, // TODO: remove T::Listener: Send, // TODO: remove T::ListenerUpgrade: Send, // TODO: remove - T::MultiaddrFuture: Send, // TODO: remove T::Output: Send + AsyncRead + AsyncWrite, P: Deref + Clone + 'static, for<'a> &'a S: Peerstore, @@ -158,7 +154,7 @@ where .into_future() .map_err(|(err, _stream)| err) .and_then(move |(ok, _stream)| { - if let Some((c, a)) = ok { + if let Some(c) = ok { // send STOP message to destination and expect back a SUCCESS message let future = Io::new(c).send(stop) .and_then(Io::recv) @@ -168,7 +164,7 @@ where None => return Err(io_err("no message from destination")) }; if is_success(&rsp) { - Ok((io.into(), a)) + Ok(io.into()) } else { Err(io_err("no success response from relay")) } @@ -181,7 +177,7 @@ where // signal success or failure to source .then(move |result| { match result { - Ok((c, _)) => { + Ok(c) => { let msg = status(CircuitRelay_Status::SUCCESS); A(io.send(msg).map(|io| (io.into(), c))) } @@ -251,7 +247,7 @@ fn stop_message(from: &Peer, dest: &Peer) -> CircuitRelay { #[derive(Debug, Clone)] struct TrivialUpgrade; -impl ConnectionUpgrade for TrivialUpgrade +impl ConnectionUpgrade for TrivialUpgrade where C: AsyncRead + AsyncWrite + 'static { @@ -263,21 +259,19 @@ where } type Output = C; - type MultiaddrFuture = Maf; - type Future = FutureResult<(Self::Output, Maf), io::Error>; + type Future = FutureResult; - fn upgrade(self, conn: C, _: (), _: Endpoint, remote_addr: Maf) -> Self::Future { - future::ok((conn, remote_addr)) + fn upgrade(self, conn: C, _: (), _: Endpoint) -> Self::Future { + future::ok(conn) } } #[derive(Debug, Clone)] pub(crate) struct Source(pub(crate) CircuitRelay); -impl ConnectionUpgrade for Source +impl ConnectionUpgrade for Source where C: AsyncRead + AsyncWrite + Send + 'static, - Maf: Send + 'static, { type NamesIter = iter::Once<(Bytes, Self::UpgradeIdentifier)>; type UpgradeIdentifier = (); @@ -287,10 +281,9 @@ where } type Output = C; - type MultiaddrFuture = Maf; - type Future = Box + Send>; + type Future = Box + Send>; - fn upgrade(self, conn: C, _: (), _: Endpoint, remote_addr: Maf) -> Self::Future { + fn upgrade(self, conn: C, _: (), _: Endpoint) -> Self::Future { let future = Io::new(conn) .send(self.0) .and_then(Io::recv) @@ -305,7 +298,7 @@ where Err(io_err("no success response from relay")) } }); - Box::new(future.map(move |out| (out, remote_addr))) + Box::new(future) } } diff --git a/transports/relay/src/transport.rs b/transports/relay/src/transport.rs index fe2c8b95b84..0e216b8506d 100644 --- a/transports/relay/src/transport.rs +++ b/transports/relay/src/transport.rs @@ -43,17 +43,15 @@ where T::Dial: Send, T::Listener: Send, T::ListenerUpgrade: Send, - T::MultiaddrFuture: Send, T::Output: AsyncRead + AsyncWrite + Send, P: Deref + Clone + 'static, S: 'static, for<'a> &'a S: Peerstore { type Output = T::Output; - type MultiaddrFuture = T::MultiaddrFuture; - type Listener = Box + Send>; - type ListenerUpgrade = Box + Send>; - type Dial = Box + Send>; + type Listener = Box + Send>; + type ListenerUpgrade = Box + Send>; + type Dial = Box + Send>; fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { Err((self, addr)) @@ -92,7 +90,6 @@ where T::Dial: Send, T::Listener: Send, T::ListenerUpgrade: Send, - T::MultiaddrFuture: Send, T::Output: AsyncRead + AsyncWrite + Send, P: Deref + Clone + 'static, for<'a> &'a S: Peerstore @@ -114,7 +111,7 @@ where } // Relay to destination over any available relay node. - fn relay_to(self, destination: &Peer) -> Result, Self> { + fn relay_to(self, destination: &Peer) -> Result, Self> { trace!("relay_to {:?}", destination.id); let mut dials = Vec::new(); for relay in &*self.relays { @@ -152,7 +149,7 @@ where } // Relay to destination via the given peer. - fn relay_via(self, relay: &Peer, destination: &Peer) -> Result, Self> { + fn relay_via(self, relay: &Peer, destination: &Peer) -> Result, Self> { trace!("relay_via {:?} to {:?}", relay.id, destination.id); let mut addresses = Vec::new(); @@ -183,9 +180,9 @@ where .into_future() .map_err(|(err, _stream)| err) .and_then(move |(ok, _stream)| match ok { - Some((out, addr)) => { + Some(out) => { debug!("connected"); - Ok((out, addr)) + Ok(out) } None => { info!("failed to dial to {:?}", relay.id); diff --git a/transports/tcp/src/lib.rs b/transports/tcp/src/lib.rs index 01f8498fa35..2064003eb39 100644 --- a/transports/tcp/src/lib.rs +++ b/transports/tcp/src/lib.rs @@ -134,8 +134,7 @@ impl TcpConfig { impl Transport for TcpConfig { type Output = TcpTransStream; type Listener = TcpListenStream; - type ListenerUpgrade = FutureResult<(Self::Output, Self::MultiaddrFuture), IoError>; - type MultiaddrFuture = FutureResult; + type ListenerUpgrade = FutureResult; type Dial = TcpDialFut; fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { @@ -181,7 +180,6 @@ impl Transport for TcpConfig { Ok(TcpDialFut { inner: TcpStream::connect(&socket_addr), config: self, - addr: Some(addr), }) } else { debug!("Instantly refusing dialing {}, as it is invalid", addr); @@ -260,32 +258,21 @@ pub struct TcpDialFut { inner: ConnectFuture, /// Original configuration. config: TcpConfig, - /// Address we're dialing. Extracted when the `Future` finishes. - addr: Option, } impl Future for TcpDialFut { - type Item = (TcpTransStream, FutureResult); + type Item = TcpTransStream; type Error = IoError; - fn poll(&mut self) -> Poll<(TcpTransStream, FutureResult), IoError> { + fn poll(&mut self) -> Poll { match self.inner.poll() { Ok(Async::Ready(stream)) => { apply_config(&self.config, &stream)?; - let addr = self - .addr - .take() - .expect("TcpDialFut polled again after finished"); - let out = TcpTransStream { inner: stream }; - Ok(Async::Ready((out, future::ok(addr)))) + Ok(Async::Ready(TcpTransStream { inner: stream })) } Ok(Async::NotReady) => Ok(Async::NotReady), Err(err) => { - let addr = self - .addr - .as_ref() - .expect("TcpDialFut polled again after finished"); - debug!("Error while dialing {:?} => {:?}", addr, err); + debug!("Error while dialing => {:?}", err); Err(err) } } @@ -300,13 +287,13 @@ pub struct TcpListenStream { } impl Stream for TcpListenStream { - type Item = FutureResult<(TcpTransStream, FutureResult), IoError>; + type Item = (FutureResult, Multiaddr); type Error = IoError; fn poll( &mut self, ) -> Poll< - Option), IoError>>, + Option<(FutureResult, Multiaddr)>, IoError, > { let inner = match self.inner { @@ -316,28 +303,36 @@ impl Stream for TcpListenStream { } }; - match inner.poll() { - Ok(Async::Ready(Some(sock))) => { - match apply_config(&self.config, &sock) { - Ok(()) => (), - Err(err) => return Ok(Async::Ready(Some(future::err(err)))), - }; - - let addr = match sock.peer_addr() { - // TODO: remove this expect() - Ok(addr) => addr - .to_multiaddr() - .expect("generating a multiaddr from a socket addr never fails"), - Err(err) => return Ok(Async::Ready(Some(future::err(err)))), - }; - - debug!("Incoming connection from {}", addr); - let ret = future::ok((TcpTransStream { inner: sock }, future::ok(addr))); - Ok(Async::Ready(Some(ret))) + loop { + match inner.poll() { + Ok(Async::Ready(Some(sock))) => { + let addr = match sock.peer_addr() { + // TODO: remove this expect() + Ok(addr) => addr + .to_multiaddr() + .expect("generating a multiaddr from a socket addr never fails"), + Err(err) => { + // If we can't get the address of the newly-opened socket, there's + // nothing we can except ignore this connection attempt. + error!("Ignored incoming because could't determine its \ + address: {:?}", err); + continue + }, + }; + + match apply_config(&self.config, &sock) { + Ok(()) => (), + Err(err) => return Ok(Async::Ready(Some((future::err(err), addr)))), + }; + + debug!("Incoming connection from {}", addr); + let ret = future::ok(TcpTransStream { inner: sock }); + break Ok(Async::Ready(Some((ret, addr)))) + } + Ok(Async::Ready(None)) => break Ok(Async::Ready(None)), + Ok(Async::NotReady) => break Ok(Async::NotReady), + Err(()) => unreachable!("sleep_on_error never produces an error"), } - Ok(Async::Ready(None)) => Ok(Async::Ready(None)), - Ok(Async::NotReady) => Ok(Async::NotReady), - Err(()) => unreachable!("sleep_on_error never produces an error"), } } } @@ -465,8 +460,8 @@ mod tests { std::thread::spawn(move || { let addr = "/ip4/127.0.0.1/tcp/12345".parse::().unwrap(); let tcp = TcpConfig::new(); - let listener = tcp.listen_on(addr).unwrap().0.for_each(|sock| { - sock.and_then(|(sock, _)| { + let listener = tcp.listen_on(addr).unwrap().0.for_each(|(sock, _)| { + sock.and_then(|sock| { // Define what to do with the socket that just connected to us // Which in this case is read 3 bytes let handle_conn = tokio_io::io::read_exact(sock, [0; 3]) @@ -489,7 +484,7 @@ mod tests { let socket = tcp.dial(addr.clone()).unwrap(); // Define what to do with the socket once it's obtained let action = socket.then(|sock| -> Result<(), ()> { - sock.unwrap().0.write(&[0x1, 0x2, 0x3]).unwrap(); + sock.unwrap().write(&[0x1, 0x2, 0x3]).unwrap(); Ok(()) }); // Execute the future in our event loop diff --git a/transports/timeout/src/lib.rs b/transports/timeout/src/lib.rs index 1d2f7a8160f..fa0b39c5141 100644 --- a/transports/timeout/src/lib.rs +++ b/transports/timeout/src/lib.rs @@ -31,7 +31,7 @@ extern crate log; extern crate tokio_timer; use futures::{Async, Future, Poll, Stream}; -use libp2p_core::{Multiaddr, MuxedTransport, Transport}; +use libp2p_core::{Multiaddr, Transport}; use std::io::{Error as IoError, ErrorKind as IoErrorKind}; use std::time::Duration; use tokio_timer::Timeout; @@ -85,7 +85,6 @@ where InnerTrans: Transport, { type Output = InnerTrans::Output; - type MultiaddrFuture = InnerTrans::MultiaddrFuture; type Listener = TimeoutListener; type ListenerUpgrade = TokioTimerMapErr>; type Dial = TokioTimerMapErr>; @@ -135,22 +134,6 @@ where } } -impl MuxedTransport for TransportTimeout -where - InnerTrans: MuxedTransport, -{ - type Incoming = TimeoutIncoming; - type IncomingUpgrade = TokioTimerMapErr>; - - #[inline] - fn next_incoming(self) -> Self::Incoming { - TimeoutIncoming { - inner: self.inner.next_incoming(), - timeout: self.incoming_timeout, - } - } -} - // TODO: can be removed and replaced with an `impl Stream` once impl Trait is fully stable // in Rust (https://github.com/rust-lang/rust/issues/34511) pub struct TimeoutListener { @@ -158,50 +141,26 @@ pub struct TimeoutListener { timeout: Duration, } -impl Stream for TimeoutListener +impl Stream for TimeoutListener where - InnerStream: Stream, + InnerStream: Stream, { - type Item = TokioTimerMapErr>; + type Item = (TokioTimerMapErr>, Multiaddr); type Error = InnerStream::Error; fn poll(&mut self) -> Poll, Self::Error> { - let inner_fut = try_ready!(self.inner.poll()); - if let Some(inner_fut) = inner_fut { + let poll_out = try_ready!(self.inner.poll()); + if let Some((inner_fut, addr)) = poll_out { let fut = TokioTimerMapErr { inner: Timeout::new(inner_fut, self.timeout), }; - Ok(Async::Ready(Some(fut))) + Ok(Async::Ready(Some((fut, addr)))) } else { Ok(Async::Ready(None)) } } } -// TODO: can be removed and replaced with an `impl Future` once impl Trait is fully stable -// in Rust (https://github.com/rust-lang/rust/issues/34511) -#[must_use = "futures do nothing unless polled"] -pub struct TimeoutIncoming { - inner: InnerFut, - timeout: Duration, -} - -impl Future for TimeoutIncoming -where - InnerFut: Future, -{ - type Item = TokioTimerMapErr>; - type Error = InnerFut::Error; - - fn poll(&mut self) -> Poll { - let inner_fut = try_ready!(self.inner.poll()); - let fut = TokioTimerMapErr { - inner: Timeout::new(inner_fut, self.timeout), - }; - Ok(Async::Ready(fut)) - } -} - /// Wraps around a `Future`. Turns the error type from `TimeoutError` to `IoError`. // TODO: can be replaced with `impl Future` once `impl Trait` are fully stable in Rust // (https://github.com/rust-lang/rust/issues/34511) diff --git a/transports/uds/src/lib.rs b/transports/uds/src/lib.rs index c2b76e11b62..c326fa9f7d3 100644 --- a/transports/uds/src/lib.rs +++ b/transports/uds/src/lib.rs @@ -86,10 +86,9 @@ impl UdsConfig { impl Transport for UdsConfig { type Output = UnixStream; - type Listener = Box + Send + Sync>; - type ListenerUpgrade = FutureResult<(Self::Output, Self::MultiaddrFuture), IoError>; - type MultiaddrFuture = FutureResult; - type Dial = Box + Send + Sync>; + type Listener = Box + Send + Sync>; + type ListenerUpgrade = FutureResult; + type Dial = Box + Send + Sync>; // TODO: name this type fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { if let Ok(path) = multiaddr_to_path(&addr) { @@ -109,7 +108,7 @@ impl Transport for UdsConfig { // Pull out a stream of sockets for incoming connections listener.incoming().map(move |sock| { debug!("Incoming connection on {}", addr); - future::ok((sock, future::ok(addr.clone()))) + (future::ok(sock), addr.clone()) }) }) .flatten_stream(); @@ -122,7 +121,7 @@ impl Transport for UdsConfig { fn dial(self, addr: Multiaddr) -> Result { if let Ok(path) = multiaddr_to_path(&addr) { debug!("Dialing {}", addr); - let fut = UnixStream::connect(&path).map(|t| (t, future::ok(addr))); + let fut = UnixStream::connect(&path); Ok(Box::new(fut) as Box<_>) } else { Err((self, addr)) @@ -203,8 +202,8 @@ mod tests { std::thread::spawn(move || { let tcp = UdsConfig::new(); - let listener = tcp.listen_on(addr2).unwrap().0.for_each(|sock| { - sock.and_then(|(sock, _)| { + let listener = tcp.listen_on(addr2).unwrap().0.for_each(|(sock, _)| { + sock.and_then(|sock| { // Define what to do with the socket that just connected to us // Which in this case is read 3 bytes let handle_conn = tokio_io::io::read_exact(sock, [0; 3]) @@ -226,7 +225,7 @@ mod tests { let socket = tcp.dial(addr.clone()).unwrap(); // Define what to do with the socket once it's obtained let action = socket.then(|sock| -> Result<(), ()> { - sock.unwrap().0.write(&[0x1, 0x2, 0x3]).unwrap(); + sock.unwrap().write(&[0x1, 0x2, 0x3]).unwrap(); Ok(()) }); // Execute the future in our event loop diff --git a/transports/websocket/src/browser.rs b/transports/websocket/src/browser.rs index e62a69c18f6..d646c8a2e6a 100644 --- a/transports/websocket/src/browser.rs +++ b/transports/websocket/src/browser.rs @@ -20,7 +20,7 @@ use futures::stream::Then as StreamThen; use futures::sync::{mpsc, oneshot}; -use futures::{future, future::FutureResult, Async, Future, Poll, Stream}; +use futures::{Async, Future, Poll, Stream}; use multiaddr::{Protocol, Multiaddr}; use rw_stream_sink::RwStreamSink; use std::io::{Error as IoError, ErrorKind as IoErrorKind}; @@ -53,11 +53,9 @@ impl BrowserWsConfig { impl Transport for BrowserWsConfig { type Output = BrowserWsConn; - type MultiaddrFuture = FutureResult; - type Listener = Box + Send>; // TODO: use `!` - type ListenerUpgrade = - Box + Send>; // TODO: use `!` - type Dial = Box + Send>; + type Listener = Box + Send>; // TODO: use `!` + type ListenerUpgrade = Box + Send>; // TODO: use `!` + type Dial = Box + Send>; #[inline] fn listen_on(self, a: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { @@ -196,7 +194,7 @@ impl Transport for BrowserWsConfig { Ok(Box::new(open_rx.then(|result| { match result { - Ok(Ok(r)) => Ok((r, future::ok(original_addr))), + Ok(Ok(r)) => Ok(r), Ok(Err(e)) => Err(e), // `Err` would happen here if `open_tx` is destroyed. `open_tx` is captured by // the `WebSocket`, and the `WebSocket` is captured by `open_cb`, which is itself diff --git a/transports/websocket/src/desktop.rs b/transports/websocket/src/desktop.rs index a86d46361ba..14275074d60 100644 --- a/transports/websocket/src/desktop.rs +++ b/transports/websocket/src/desktop.rs @@ -60,19 +60,16 @@ where // TODO: this 'static is pretty arbitrary and is necessary because of the websocket library T: Transport + 'static, T::Dial: Send, - T::MultiaddrFuture: Send, T::Listener: Send, T::ListenerUpgrade: Send, // TODO: this Send is pretty arbitrary and is necessary because of the websocket library T::Output: AsyncRead + AsyncWrite + Send, { type Output = Box; - type MultiaddrFuture = Box + Send>; type Listener = - stream::Map::ListenerUpgrade) -> Self::ListenerUpgrade>; - type ListenerUpgrade = - Box + Send>; - type Dial = Box + Send>; + stream::Map::ListenerUpgrade, Multiaddr)) -> (Self::ListenerUpgrade, Multiaddr)>; + type ListenerUpgrade = Box + Send>; + type Dial = Box + Send>; fn listen_on( self, @@ -102,14 +99,12 @@ where debug!("Listening on {}", new_addr); - let listen = inner_listen.map::<_, fn(_) -> _>(|stream| { + let listen = inner_listen.map::<_, fn(_) -> _>(|(stream, mut client_addr)| { + // Need to suffix `/ws` to each client address. + client_addr.append(Protocol::Ws); + // Upgrade the listener to websockets like the websockets library requires us to do. - let upgraded = stream.and_then(|(stream, client_addr)| { - // Need to suffix `/ws` to each client address. - let client_addr = client_addr.map(|mut addr| { - addr.append(Protocol::Ws); - addr - }); + let upgraded = stream.and_then(move |stream| { debug!("Incoming connection"); stream @@ -149,10 +144,9 @@ where .map(|s| Box::new(Ok(s).into_future()) as Box + Send>) .into_future() .flatten() - .map(move |v| (v, Box::new(client_addr) as Box + Send>)) }); - Box::new(upgraded) as Box + Send> + (Box::new(upgraded) as Box + Send>, client_addr) }); Ok((listen, new_addr)) @@ -194,16 +188,7 @@ where let dial = inner_dial .into_future() - .and_then(move |(connec, client_addr)| { - let client_addr = Box::new(client_addr.map(move |mut addr| { - if is_wss { - addr.append(Protocol::Wss); - } else { - addr.append(Protocol::Ws); - }; - addr - })) as Box + Send>; - + .and_then(move |connec| { ClientBuilder::new(&ws_addr) .expect("generated ws address is always valid") .async_connect_on(connec) @@ -229,7 +214,6 @@ where let read_write = RwStreamSink::new(framed_data); Box::new(read_write) as Box }) - .map(move |c| (c, client_addr)) }); Ok(Box::new(dial) as Box<_>) @@ -294,8 +278,8 @@ mod tests { let listener = listener .into_future() .map_err(|(e, _)| e) - .and_then(|(c, _)| c.unwrap().map(|v| v.0)); - let dialer = ws_config.clone().dial(addr).unwrap().map(|v| v.0); + .and_then(|(c, _)| c.unwrap().0); + let dialer = ws_config.clone().dial(addr).unwrap(); let future = listener .select(dialer) @@ -317,8 +301,8 @@ mod tests { let listener = listener .into_future() .map_err(|(e, _)| e) - .and_then(|(c, _)| c.unwrap().map(|v| v.0)); - let dialer = ws_config.clone().dial(addr).unwrap().map(|v| v.0); + .and_then(|(c, _)| c.unwrap().0); + let dialer = ws_config.clone().dial(addr).unwrap(); let future = listener .select(dialer) From b76f6cce6b1b72a078acb4eff8f116b0a8bed581 Mon Sep 17 00:00:00 2001 From: David Palm Date: Wed, 17 Oct 2018 14:04:24 +0200 Subject: [PATCH 2/2] Fix tests after recent changes on master --- core/src/nodes/handled_node.rs | 46 ++++++++++----------------------- core/src/nodes/node.rs | 3 +-- core/src/tests/dummy_handler.rs | 14 +++++----- 3 files changed, 20 insertions(+), 43 deletions(-) diff --git a/core/src/nodes/handled_node.rs b/core/src/nodes/handled_node.rs index 39bb02bcf64..9fbf19fd2d2 100644 --- a/core/src/nodes/handled_node.rs +++ b/core/src/nodes/handled_node.rs @@ -291,18 +291,15 @@ where #[cfg(test)] mod tests { use super::*; - use futures::future; - use futures::future::FutureResult; - // use muxing::StreamMuxer; use tokio::runtime::current_thread; use tests::dummy_muxer::{DummyMuxer, DummyConnectionState}; use tests::dummy_handler::{Handler, HandlerState, Event}; + use std::marker::PhantomData; // Concrete `HandledNode` - type TestHandledNode = HandledNode, Handler>; + type TestHandledNode = HandledNode; struct TestBuilder { - addr: Multiaddr, muxer: DummyMuxer, handler: Handler, want_open_substream: bool, @@ -312,7 +309,6 @@ mod tests { impl TestBuilder { fn new() -> Self { TestBuilder { - addr: "/ip4/127.0.0.1/tcp/1234".parse::().expect("bad multiaddr"), muxer: DummyMuxer::new(), handler: Handler::default(), want_open_substream: false, @@ -341,9 +337,8 @@ mod tests { self } - // TODO: Is there a way to consume `self` here and get rid of the clones? fn handled_node(&mut self) -> TestHandledNode { - let mut h = HandledNode::new(self.muxer.clone(), future::ok(self.addr.clone()), self.handler.clone()); + let mut h = HandledNode::new(self.muxer.clone(), self.handler.clone()); if self.want_open_substream { h.node.get_mut().open_substream(self.substream_user_data).expect("open substream should work"); } @@ -362,18 +357,19 @@ mod tests { #[test] fn proper_shutdown() { - struct ShutdownHandler { + struct ShutdownHandler { did_substream_attempt: bool, inbound_closed: bool, substream_attempt_cancelled: bool, shutdown_called: bool, + marker: PhantomData } - impl NodeHandler for ShutdownHandler { + impl NodeHandler for ShutdownHandler { type InEvent = (); type OutEvent = (); type Substream = T; type OutboundOpenInfo = (); - fn inject_substream(&mut self, _: T, _: NodeHandlerEndpoint<()>) { panic!() } + fn inject_substream(&mut self, _: Self::Substream, _: NodeHandlerEndpoint) { panic!() } fn inject_inbound_closed(&mut self) { assert!(!self.inbound_closed); self.inbound_closed = true; @@ -400,7 +396,7 @@ mod tests { } } - impl Drop for ShutdownHandler { + impl Drop for ShutdownHandler { fn drop(&mut self) { if self.did_substream_attempt { assert!(self.shutdown_called); @@ -412,11 +408,12 @@ mod tests { let mut muxer = DummyMuxer::new(); muxer.set_inbound_connection_state(DummyConnectionState::Closed); muxer.set_outbound_connection_state(DummyConnectionState::Closed); - let handled = HandledNode::new(muxer, future::empty(), ShutdownHandler { + let handled = HandledNode::new(muxer, ShutdownHandler { did_substream_attempt: false, inbound_closed: false, substream_attempt_cancelled: false, shutdown_called: false, + marker: PhantomData, }); current_thread::Runtime::new().unwrap().block_on(handled.for_each(|_| Ok(()))).unwrap(); @@ -503,7 +500,6 @@ mod tests { .handled_node(); assert_matches!(handled.poll(), Ok(Async::Ready(None))); - assert_eq!(handled.handler.events, vec![Event::Multiaddr]); } #[test] @@ -519,7 +515,6 @@ mod tests { assert_matches!(handled.poll(), Ok(Async::Ready(Some(event))) => { assert_matches!(event, Event::Custom("pineapple")) }); - assert_eq!(handled.handler.events, vec![Event::Multiaddr]); } #[test] @@ -535,8 +530,8 @@ mod tests { &mut handled, HandlerState::Ready(Some(NodeHandlerEvent::Custom(Event::Custom("pear")))) ); - handled.poll(); - assert_eq!(handled.handler.events, vec![Event::Multiaddr, Event::OutboundClosed]); + handled.poll().expect("poll works"); + assert_eq!(handled.handler.events, vec![Event::OutboundClosed]); } #[test] @@ -565,9 +560,6 @@ mod tests { // more outbound substreams). // - Next we poll the handler again which again does nothing because // HandlerState is NotReady (and the node is still there) - // - Polls the node again; now we will hit the address resolution - // - Address resolves and yields a `Multiaddr` event and we resume the - // loop // - HandledNode polls the node again: we skip inbound and there are no // more outbound substreams so we skip that too; the addr is now // Resolved so that part is skipped too @@ -577,7 +569,7 @@ mod tests { // – which in turn makes the HandledNode to yield Async::Ready(None) as well assert_matches!(handled.poll(), Ok(Async::Ready(None))); assert_eq!(handled.handler.events, vec![ - Event::InboundClosed, Event::OutboundClosed, Event::Multiaddr + Event::InboundClosed, Event::OutboundClosed ]); } @@ -607,18 +599,6 @@ mod tests { assert_eq!(h.handler.events, vec![Event::OutboundClosed]); } - #[test] - fn poll_yields_multiaddr_event() { - let mut h = TestBuilder::new() - .with_muxer_inbound_state(DummyConnectionState::Pending) - .with_handler_state(HandlerState::Err) // stop the loop - .handled_node(); - - assert_eq!(h.handler.events, vec![]); - let _ = h.poll(); - assert_eq!(h.handler.events, vec![Event::Multiaddr]); - } - #[test] fn poll_yields_outbound_substream() { let mut h = TestBuilder::new() diff --git a/core/src/nodes/node.rs b/core/src/nodes/node.rs index 2b0ed15b850..db6a7f2e21e 100644 --- a/core/src/nodes/node.rs +++ b/core/src/nodes/node.rs @@ -381,11 +381,10 @@ where TTrans: Transport, #[cfg(test)] mod node_stream { use super::NodeStream; - use futures::{future::self, prelude::*, Future}; + use futures::prelude::*; use tokio_mock_task::MockTask; use super::NodeEvent; use tests::dummy_muxer::{DummyMuxer, DummyConnectionState}; - use std::io::Error as IoError; fn build_node_stream() -> NodeStream> { let muxer = DummyMuxer::new(); diff --git a/core/src/tests/dummy_handler.rs b/core/src/tests/dummy_handler.rs index e784e641380..fd05d60c805 100644 --- a/core/src/tests/dummy_handler.rs +++ b/core/src/tests/dummy_handler.rs @@ -23,8 +23,10 @@ use std::io::{self, Error as IoError}; use futures::prelude::*; -use Multiaddr; use nodes::handled_node::{NodeHandler, NodeHandlerEndpoint, NodeHandlerEvent}; +use super::dummy_muxer::DummyMuxer; +use muxing::SubstreamRef; +use std::sync::Arc; #[derive(Debug, PartialEq, Clone)] pub(crate) struct Handler { @@ -56,15 +58,14 @@ pub(crate) enum Event { Substream(Option), OutboundClosed, InboundClosed, - Multiaddr, } - -impl NodeHandler for Handler { +impl NodeHandler for Handler { type InEvent = Event; type OutEvent = Event; type OutboundOpenInfo = usize; - fn inject_substream(&mut self, _: T, endpoint: NodeHandlerEndpoint) { + type Substream = SubstreamRef>; + fn inject_substream(&mut self, _: Self::Substream, endpoint: NodeHandlerEndpoint) { let user_data = match endpoint { NodeHandlerEndpoint::Dialer(user_data) => Some(user_data), NodeHandlerEndpoint::Listener => None @@ -80,9 +81,6 @@ impl NodeHandler for Handler { self.state = Some(state.clone()); } } - fn inject_multiaddr(&mut self, _: Result) { - self.events.push(Event::Multiaddr); - } fn inject_event(&mut self, inevent: Self::InEvent) { self.events.push(inevent) }