Split ConnectionUpgrade. (#642)

Introduce `InboundUpgrade` and `OutboundUpgrade`.
This commit is contained in:
Toralf Wittner
2018-11-15 17:41:11 +01:00
committed by Pierre Krieger
parent 466385a58a
commit 2e549884ef
52 changed files with 2010 additions and 1658 deletions

View File

@ -58,21 +58,22 @@ members = [
"misc/multistream-select", "misc/multistream-select",
"misc/peer-id-generator", "misc/peer-id-generator",
"misc/rw-stream-sink", "misc/rw-stream-sink",
"transports/dns", "muxers/mplex",
"muxers/yamux",
"protocols/floodsub", "protocols/floodsub",
"protocols/identify", "protocols/identify",
"protocols/kad", "protocols/kad",
"protocols/ping",
"protocols/observed", "protocols/observed",
"transports/relay", "protocols/ping",
"protocols/plaintext",
"protocols/secio", "protocols/secio",
"muxers/mplex",
"muxers/yamux",
"stores/peerstore",
"stores/datastore", "stores/datastore",
"transports/tcp", "stores/peerstore",
"transports/uds", "transports/dns",
"transports/websocket",
"transports/timeout",
"transports/ratelimit", "transports/ratelimit",
"transports/relay",
"transports/tcp",
"transports/timeout",
"transports/uds",
"transports/websocket"
] ]

View File

@ -18,11 +18,42 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::{muxing::{Shutdown, StreamMuxer}, Multiaddr};
use futures::prelude::*; use futures::prelude::*;
use muxing::{Shutdown, StreamMuxer}; use std::{fmt, io::{Error as IoError, Read, Write}};
use std::io::{Error as IoError, Read, Write};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use Multiaddr;
#[derive(Debug, Copy, Clone)]
pub enum EitherError<A, B> {
A(A),
B(B)
}
impl<A, B> fmt::Display for EitherError<A, B>
where
A: fmt::Display,
B: fmt::Display
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
EitherError::A(a) => a.fmt(f),
EitherError::B(b) => b.fmt(f)
}
}
}
impl<A, B> std::error::Error for EitherError<A, B>
where
A: fmt::Debug + std::error::Error,
B: fmt::Debug + std::error::Error
{
fn cause(&self) -> Option<&dyn std::error::Error> {
match self {
EitherError::A(a) => a.cause(),
EitherError::B(b) => b.cause()
}
}
}
/// Implements `AsyncRead` and `AsyncWrite` and dispatches all method calls to /// Implements `AsyncRead` and `AsyncWrite` and dispatches all method calls to
/// either `First` or `Second`. /// either `First` or `Second`.

View File

@ -89,7 +89,7 @@
//! //!
//! # fn main() { //! # fn main() {
//! let tcp_transport = libp2p_tcp_transport::TcpConfig::new(); //! let tcp_transport = libp2p_tcp_transport::TcpConfig::new();
//! let upgraded = tcp_transport.with_upgrade(libp2p_core::upgrade::PlainTextConfig); //! let upgraded = tcp_transport.with_upgrade(libp2p_core::upgrade::DeniedUpgrade);
//! //!
//! // upgraded.dial(...) // automatically applies the plain text protocol on the socket //! // upgraded.dial(...) // automatically applies the plain text protocol on the socket
//! # } //! # }
@ -132,31 +132,28 @@
//! extern crate tokio; //! extern crate tokio;
//! //!
//! use futures::{Future, Stream}; //! use futures::{Future, Stream};
//! use libp2p_ping::protocol::{Ping, PingOutput}; //! use libp2p_ping::protocol::Ping;
//! use libp2p_core::Transport; //! use libp2p_core::{Transport, upgrade::apply_outbound};
//! use tokio::runtime::current_thread::Runtime; //! use tokio::runtime::current_thread::Runtime;
//! //!
//! # fn main() { //! # fn main() {
//! let ping_finished_future = libp2p_tcp_transport::TcpConfig::new() //! let ping_dialer = libp2p_tcp_transport::TcpConfig::new()
//! // We have a `TcpConfig` struct that implements `Transport`, and apply a `Ping` upgrade on it. //! // We have a `TcpConfig` struct that implements `Dialer`, and apply a `Ping` upgrade on it.
//! .with_upgrade(Ping::default()) //! .and_then(|socket, _| {
//! apply_outbound(socket, Ping::default()).map_err(|e| e.into_io_error())
//! })
//! // TODO: right now the only available protocol is ping, but we want to replace it with //! // TODO: right now the only available protocol is ping, but we want to replace it with
//! // something that is more simple to use //! // something that is more simple to use
//! .dial("127.0.0.1:12345".parse::<libp2p_core::Multiaddr>().unwrap()).unwrap_or_else(|_| panic!()) //! .dial("/ip4/127.0.0.1/tcp/12345".parse::<libp2p_core::Multiaddr>().unwrap()).unwrap_or_else(|_| panic!())
//! .and_then(|out| { //! .and_then(|mut pinger| {
//! match out {
//! PingOutput::Ponger(processing) => Box::new(processing) as Box<Future<Item = _, Error = _>>,
//! PingOutput::Pinger(mut pinger) => {
//! pinger.ping(()); //! pinger.ping(());
//! let f = pinger.into_future().map(|_| ()).map_err(|(err, _)| err); //! let f = pinger.into_future().map(|_| ()).map_err(|(e, _)| e);
//! Box::new(f) as Box<Future<Item = _, Error = _>> //! Box::new(f) as Box<Future<Item = _, Error = _>>
//! },
//! }
//! }); //! });
//! //!
//! // Runs until the ping arrives. //! // Runs until the ping arrives.
//! let mut rt = Runtime::new().unwrap(); //! let mut rt = Runtime::new().unwrap();
//! let _ = rt.block_on(ping_finished_future).unwrap(); //! let _ = rt.block_on(ping_dialer).unwrap();
//! # } //! # }
//! ``` //! ```
//! //!
@ -221,4 +218,24 @@ pub use self::muxing::StreamMuxer;
pub use self::peer_id::PeerId; pub use self::peer_id::PeerId;
pub use self::public_key::PublicKey; pub use self::public_key::PublicKey;
pub use self::transport::Transport; pub use self::transport::Transport;
pub use self::upgrade::{ConnectionUpgrade, Endpoint}; pub use self::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, UpgradeError};
#[derive(Debug, Copy, Clone, PartialEq, Eq, Hash)]
pub enum Endpoint {
/// The socket comes from a dialer.
Dialer,
/// The socket comes from a listener.
Listener,
}
impl std::ops::Not for Endpoint {
type Output = Endpoint;
fn not(self) -> Self::Output {
match self {
Endpoint::Dialer => Endpoint::Listener,
Endpoint::Listener => Endpoint::Dialer
}
}
}

View File

@ -18,15 +18,19 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::{
PeerId,
muxing::StreamMuxer,
nodes::{
node::Substream,
handled_node_tasks::{HandledNodesEvent, HandledNodesTasks},
handled_node_tasks::{Task as HandledNodesTask, TaskId},
handled_node::NodeHandler
}
};
use fnv::FnvHashMap; use fnv::FnvHashMap;
use futures::prelude::*; use futures::prelude::*;
use muxing::StreamMuxer;
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, io, mem}; use std::{collections::hash_map::Entry, fmt, io, mem};
use PeerId;
// TODO: make generic over PeerId // TODO: make generic over PeerId
@ -276,7 +280,8 @@ impl<TInEvent, TOutEvent, THandler> CollectionStream<TInEvent, TOutEvent, THandl
pub fn add_reach_attempt<TFut, TMuxer>(&mut self, future: TFut, handler: THandler) pub fn add_reach_attempt<TFut, TMuxer>(&mut self, future: TFut, handler: THandler)
-> ReachAttemptId -> ReachAttemptId
where where
TFut: Future<Item = (PeerId, TMuxer), Error = io::Error> + Send + 'static, TFut: Future<Item = (PeerId, TMuxer)> + Send + 'static,
TFut::Error: std::error::Error + Send + Sync + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static,
TInEvent: Send + 'static, TInEvent: Send + 'static,
TOutEvent: Send + 'static, TOutEvent: Send + 'static,

View File

@ -18,18 +18,25 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::{
PeerId,
muxing::StreamMuxer,
nodes::{
handled_node::{HandledNode, NodeHandler},
node::Substream
}
};
use fnv::FnvHashMap; use fnv::FnvHashMap;
use futures::{prelude::*, stream, sync::mpsc}; use futures::{prelude::*, stream, sync::mpsc};
use muxing::StreamMuxer;
use nodes::node::Substream;
use nodes::handled_node::{HandledNode, NodeHandler};
use smallvec::SmallVec; use smallvec::SmallVec;
use std::collections::hash_map::{Entry, OccupiedEntry}; use std::{
use std::io::Error as IoError; collections::hash_map::{Entry, OccupiedEntry},
use std::{fmt, mem}; fmt,
io::{self, Error as IoError},
mem
};
use tokio_executor; use tokio_executor;
use void::Void; use void::Void;
use PeerId;
// TODO: make generic over PeerId // TODO: make generic over PeerId
@ -135,10 +142,10 @@ impl<TInEvent, TOutEvent, THandler> HandledNodesTasks<TInEvent, TOutEvent, THand
/// ///
/// This method spawns a task dedicated to resolving this future and processing the node's /// This method spawns a task dedicated to resolving this future and processing the node's
/// events. /// events.
pub fn add_reach_attempt<TFut, TMuxer>(&mut self, future: TFut, handler: THandler) pub fn add_reach_attempt<TFut, TMuxer>(&mut self, future: TFut, handler: THandler) -> TaskId
-> TaskId
where where
TFut: Future<Item = (PeerId, TMuxer), Error = IoError> + Send + 'static, TFut: Future<Item = (PeerId, TMuxer)> + Send + 'static,
TFut::Error: std::error::Error + Send + Sync + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static,
TInEvent: Send + 'static, TInEvent: Send + 'static,
TOutEvent: Send + 'static, TOutEvent: Send + 'static,
@ -349,7 +356,8 @@ impl<TFut, TMuxer, THandler, TInEvent, TOutEvent> Future for
NodeTask<TFut, TMuxer, THandler, TInEvent, TOutEvent> NodeTask<TFut, TMuxer, THandler, TInEvent, TOutEvent>
where where
TMuxer: StreamMuxer, TMuxer: StreamMuxer,
TFut: Future<Item = (PeerId, TMuxer), Error = IoError>, TFut: Future<Item = (PeerId, TMuxer)>,
TFut::Error: std::error::Error + Send + Sync + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent>, THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent>,
{ {
type Item = (); type Item = ();
@ -388,7 +396,8 @@ where
}, },
Err(err) => { Err(err) => {
// End the task // End the task
let event = InToExtMessage::TaskClosed(Err(err), Some(handler)); let ioerr = IoError::new(io::ErrorKind::Other, err);
let event = InToExtMessage::TaskClosed(Err(ioerr), Some(handler));
let _ = self.events_tx.unbounded_send((event, self.id)); let _ = self.events_tx.unbounded_send((event, self.id));
return Ok(Async::Ready(())); return Ok(Async::Ready(()));
} }
@ -464,6 +473,7 @@ mod tests {
use tests::dummy_muxer::{DummyMuxer, DummyConnectionState}; use tests::dummy_muxer::{DummyMuxer, DummyConnectionState};
use tokio::runtime::Builder; use tokio::runtime::Builder;
use tokio::runtime::current_thread::Runtime; use tokio::runtime::current_thread::Runtime;
use void::Void;
use {PeerId, PublicKey}; use {PeerId, PublicKey};
type TestNodeTask = NodeTask< type TestNodeTask = NodeTask<
@ -570,7 +580,7 @@ mod tests {
let peer_id = PublicKey::Rsa((0 .. 2048).map(|_| -> u8 { random() }).collect()).into_peer_id(); let peer_id = PublicKey::Rsa((0 .. 2048).map(|_| -> u8 { random() }).collect()).into_peer_id();
let mut task_ids = Vec::new(); let mut task_ids = Vec::new();
for _i in 0..self.task_count { for _i in 0..self.task_count {
let fut = future::ok((peer_id.clone(), self.muxer.clone())); let fut = future::ok::<_, Void>((peer_id.clone(), self.muxer.clone()));
task_ids.push( task_ids.push(
handled_nodes.add_reach_attempt(fut, self.handler.clone()) handled_nodes.add_reach_attempt(fut, self.handler.clone())
); );
@ -719,7 +729,7 @@ mod tests {
assert_eq!(handled_nodes.tasks().count(), 0); assert_eq!(handled_nodes.tasks().count(), 0);
assert_eq!(handled_nodes.to_spawn.len(), 0); assert_eq!(handled_nodes.to_spawn.len(), 0);
handled_nodes.add_reach_attempt( future::empty(), Handler::default() ); handled_nodes.add_reach_attempt( future::empty::<_, Void>(), Handler::default() );
assert_eq!(handled_nodes.tasks().count(), 1); assert_eq!(handled_nodes.tasks().count(), 1);
assert_eq!(handled_nodes.to_spawn.len(), 1); assert_eq!(handled_nodes.to_spawn.len(), 1);

View File

@ -18,15 +18,26 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use either::EitherOutput; use crate::{
either::{EitherError, EitherOutput},
nodes::handled_node::{NodeHandler, NodeHandlerEndpoint, NodeHandlerEvent},
upgrade::{
self,
InboundUpgrade,
InboundUpgradeExt,
OutboundUpgrade,
OutboundUpgradeExt,
UpgradeInfo,
InboundUpgradeApply,
OutboundUpgradeApply,
DeniedUpgrade
}
};
use futures::prelude::*; use futures::prelude::*;
use nodes::handled_node::{NodeHandler, NodeHandlerEndpoint, NodeHandlerEvent};
use std::{io, marker::PhantomData, time::Duration}; use std::{io, marker::PhantomData, time::Duration};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use tokio_timer::Timeout; use tokio_timer::Timeout;
use upgrade::{self, apply::UpgradeApplyFuture, DeniedConnectionUpgrade};
use void::Void; use void::Void;
use {ConnectionUpgrade, Endpoint};
/// Handler for a set of protocols for a specific connection with a remote. /// Handler for a set of protocols for a specific connection with a remote.
/// ///
@ -81,7 +92,9 @@ pub trait ProtocolsHandler {
/// The type of the substream that contains the raw data. /// The type of the substream that contains the raw data.
type Substream: AsyncRead + AsyncWrite; type Substream: AsyncRead + AsyncWrite;
/// The upgrade for the protocol or protocols handled by this handler. /// The upgrade for the protocol or protocols handled by this handler.
type Protocol: ConnectionUpgrade<Self::Substream>; type InboundProtocol: InboundUpgrade<Self::Substream>;
/// The upgrade for the protocol or protocols handled by this handler.
type OutboundProtocol: OutboundUpgrade<Self::Substream>;
/// Information about a substream. Can be sent to the handler through a `NodeHandlerEndpoint`, /// 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`. /// and will be passed back in `inject_substream` or `inject_outbound_closed`.
type OutboundOpenInfo; type OutboundOpenInfo;
@ -92,15 +105,22 @@ pub trait ProtocolsHandler {
/// > context you wouldn't accept one in particular (eg. only allow one substream at /// > context you wouldn't accept one in particular (eg. only allow one substream at
/// > a time for a given protocol). The reason is that remotes are allowed to put the /// > a time for a given protocol). The reason is that remotes are allowed to put the
/// > list of supported protocols in a cache in order to avoid spurious queries. /// > list of supported protocols in a cache in order to avoid spurious queries.
fn listen_protocol(&self) -> Self::Protocol; fn listen_protocol(&self) -> Self::InboundProtocol;
fn dialer_protocol(&self) -> Self::OutboundProtocol;
/// Injects a fully-negotiated substream in the handler. /// Injects a fully-negotiated substream in the handler.
/// ///
/// This method is called when a substream has been successfully opened and negotiated. /// This method is called when a substream has been successfully opened and negotiated.
fn inject_fully_negotiated( fn inject_fully_negotiated_inbound(
&mut self, &mut self,
protocol: <Self::Protocol as ConnectionUpgrade<Self::Substream>>::Output, protocol: <Self::InboundProtocol as InboundUpgrade<Self::Substream>>::Output
endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>, );
fn inject_fully_negotiated_outbound(
&mut self,
protocol: <Self::OutboundProtocol as OutboundUpgrade<Self::Substream>>::Output,
info: Self::OutboundOpenInfo
); );
/// Injects an event coming from the outside in the handler. /// Injects an event coming from the outside in the handler.
@ -126,12 +146,7 @@ pub trait ProtocolsHandler {
/// > **Note**: If this handler is combined with other handlers, as soon as `poll()` returns /// > **Note**: If this handler is combined with other handlers, as soon as `poll()` returns
/// > `Ok(Async::Ready(None))`, all the other handlers will receive a call to /// > `Ok(Async::Ready(None))`, all the other handlers will receive a call to
/// > `shutdown()` and will eventually be closed and destroyed. /// > `shutdown()` and will eventually be closed and destroyed.
fn poll( fn poll(&mut self) -> Poll<Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>, io::Error>;
&mut self,
) -> Poll<
Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>,
io::Error,
>;
/// Adds a closure that turns the input event into something else. /// Adds a closure that turns the input event into something else.
#[inline] #[inline]
@ -296,19 +311,32 @@ where
type InEvent = Void; type InEvent = Void;
type OutEvent = Void; type OutEvent = Void;
type Substream = TSubstream; type Substream = TSubstream;
type Protocol = DeniedConnectionUpgrade; type InboundProtocol = DeniedUpgrade;
type OutboundProtocol = DeniedUpgrade;
type OutboundOpenInfo = Void; type OutboundOpenInfo = Void;
#[inline] #[inline]
fn listen_protocol(&self) -> Self::Protocol { fn listen_protocol(&self) -> Self::InboundProtocol {
DeniedConnectionUpgrade DeniedUpgrade
} }
#[inline] #[inline]
fn inject_fully_negotiated( fn dialer_protocol(&self) -> Self::OutboundProtocol {
DeniedUpgrade
}
#[inline]
fn inject_fully_negotiated_inbound(
&mut self, &mut self,
_: <Self::Protocol as ConnectionUpgrade<TSubstream>>::Output, _: <Self::InboundProtocol as InboundUpgrade<TSubstream>>::Output
_: NodeHandlerEndpoint<Self::OutboundOpenInfo>, ) {
}
#[inline]
fn inject_fully_negotiated_outbound(
&mut self,
_: <Self::OutboundProtocol as OutboundUpgrade<TSubstream>>::Output,
_: Self::OutboundOpenInfo
) { ) {
} }
@ -330,7 +358,7 @@ where
fn poll( fn poll(
&mut self, &mut self,
) -> Poll< ) -> Poll<
Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>, Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>,
io::Error, io::Error,
> { > {
if self.shutting_down { if self.shutting_down {
@ -356,21 +384,35 @@ where
type InEvent = TNewIn; type InEvent = TNewIn;
type OutEvent = TProtoHandler::OutEvent; type OutEvent = TProtoHandler::OutEvent;
type Substream = TProtoHandler::Substream; type Substream = TProtoHandler::Substream;
type Protocol = TProtoHandler::Protocol; type InboundProtocol = TProtoHandler::InboundProtocol;
type OutboundProtocol = TProtoHandler::OutboundProtocol;
type OutboundOpenInfo = TProtoHandler::OutboundOpenInfo; type OutboundOpenInfo = TProtoHandler::OutboundOpenInfo;
#[inline] #[inline]
fn listen_protocol(&self) -> Self::Protocol { fn listen_protocol(&self) -> Self::InboundProtocol {
self.inner.listen_protocol() self.inner.listen_protocol()
} }
#[inline] #[inline]
fn inject_fully_negotiated( fn dialer_protocol(&self) -> Self::OutboundProtocol {
self.inner.dialer_protocol()
}
#[inline]
fn inject_fully_negotiated_inbound(
&mut self, &mut self,
protocol: <Self::Protocol as ConnectionUpgrade<Self::Substream>>::Output, protocol: <Self::InboundProtocol as InboundUpgrade<Self::Substream>>::Output
endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>,
) { ) {
self.inner.inject_fully_negotiated(protocol, endpoint) self.inner.inject_fully_negotiated_inbound(protocol)
}
#[inline]
fn inject_fully_negotiated_outbound(
&mut self,
protocol: <Self::OutboundProtocol as OutboundUpgrade<Self::Substream>>::Output,
info: Self::OutboundOpenInfo
) {
self.inner.inject_fully_negotiated_outbound(protocol, info)
} }
#[inline] #[inline]
@ -399,7 +441,7 @@ where
fn poll( fn poll(
&mut self, &mut self,
) -> Poll< ) -> Poll<
Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>, Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>,
io::Error, io::Error,
> { > {
self.inner.poll() self.inner.poll()
@ -420,21 +462,35 @@ where
type InEvent = TProtoHandler::InEvent; type InEvent = TProtoHandler::InEvent;
type OutEvent = TNewOut; type OutEvent = TNewOut;
type Substream = TProtoHandler::Substream; type Substream = TProtoHandler::Substream;
type Protocol = TProtoHandler::Protocol; type InboundProtocol = TProtoHandler::InboundProtocol;
type OutboundProtocol = TProtoHandler::OutboundProtocol;
type OutboundOpenInfo = TProtoHandler::OutboundOpenInfo; type OutboundOpenInfo = TProtoHandler::OutboundOpenInfo;
#[inline] #[inline]
fn listen_protocol(&self) -> Self::Protocol { fn listen_protocol(&self) -> Self::InboundProtocol {
self.inner.listen_protocol() self.inner.listen_protocol()
} }
#[inline] #[inline]
fn inject_fully_negotiated( fn dialer_protocol(&self) -> Self::OutboundProtocol {
self.inner.dialer_protocol()
}
#[inline]
fn inject_fully_negotiated_inbound(
&mut self, &mut self,
protocol: <Self::Protocol as ConnectionUpgrade<Self::Substream>>::Output, protocol: <Self::InboundProtocol as InboundUpgrade<Self::Substream>>::Output
endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>,
) { ) {
self.inner.inject_fully_negotiated(protocol, endpoint) self.inner.inject_fully_negotiated_inbound(protocol)
}
#[inline]
fn inject_fully_negotiated_outbound(
&mut self,
protocol: <Self::OutboundProtocol as OutboundUpgrade<Self::Substream>>::Output,
info: Self::OutboundOpenInfo
) {
self.inner.inject_fully_negotiated_outbound(protocol, info)
} }
#[inline] #[inline]
@ -461,7 +517,7 @@ where
fn poll( fn poll(
&mut self, &mut self,
) -> Poll< ) -> Poll<
Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>, Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>,
io::Error, io::Error,
> { > {
Ok(self.inner.poll()?.map(|ev| { Ok(self.inner.poll()?.map(|ev| {
@ -531,12 +587,12 @@ where
handler: TProtoHandler, handler: TProtoHandler,
/// Futures that upgrade incoming substreams. /// Futures that upgrade incoming substreams.
negotiating_in: negotiating_in:
Vec<Timeout<UpgradeApplyFuture<TProtoHandler::Substream, TProtoHandler::Protocol>>>, Vec<Timeout<InboundUpgradeApply<TProtoHandler::Substream, TProtoHandler::InboundProtocol>>>,
/// Futures that upgrade outgoing substreams. The first element of the tuple is the userdata /// Futures that upgrade outgoing substreams. The first element of the tuple is the userdata
/// to pass back once successfully opened. /// to pass back once successfully opened.
negotiating_out: Vec<( negotiating_out: Vec<(
TProtoHandler::OutboundOpenInfo, TProtoHandler::OutboundOpenInfo,
Timeout<UpgradeApplyFuture<TProtoHandler::Substream, TProtoHandler::Protocol>>, Timeout<OutboundUpgradeApply<TProtoHandler::Substream, TProtoHandler::OutboundProtocol>>,
)>, )>,
/// Timeout for incoming substreams negotiation. /// Timeout for incoming substreams negotiation.
in_timeout: Duration, in_timeout: Duration,
@ -544,7 +600,7 @@ where
out_timeout: Duration, out_timeout: Duration,
/// For each outbound substream request, how to upgrade it. The first element of the tuple /// For each outbound substream request, how to upgrade it. The first element of the tuple
/// is the unique identifier (see `unique_dial_upgrade_id`). /// is the unique identifier (see `unique_dial_upgrade_id`).
queued_dial_upgrades: Vec<(u64, TProtoHandler::Protocol)>, queued_dial_upgrades: Vec<(u64, TProtoHandler::OutboundProtocol)>,
/// Unique identifier assigned to each queued dial upgrade. /// Unique identifier assigned to each queued dial upgrade.
unique_dial_upgrade_id: u64, unique_dial_upgrade_id: u64,
} }
@ -552,7 +608,8 @@ where
impl<TProtoHandler> NodeHandler for NodeHandlerWrapper<TProtoHandler> impl<TProtoHandler> NodeHandler for NodeHandlerWrapper<TProtoHandler>
where where
TProtoHandler: ProtocolsHandler, TProtoHandler: ProtocolsHandler,
<TProtoHandler::Protocol as ConnectionUpgrade<TProtoHandler::Substream>>::NamesIter: Clone, <TProtoHandler::InboundProtocol as UpgradeInfo>::NamesIter: Clone,
<TProtoHandler::OutboundProtocol as OutboundUpgrade<<TProtoHandler as ProtocolsHandler>::Substream>>::Error: std::fmt::Debug
{ {
type InEvent = TProtoHandler::InEvent; type InEvent = TProtoHandler::InEvent;
type OutEvent = TProtoHandler::OutEvent; type OutEvent = TProtoHandler::OutEvent;
@ -569,7 +626,7 @@ where
match endpoint { match endpoint {
NodeHandlerEndpoint::Listener => { NodeHandlerEndpoint::Listener => {
let protocol = self.handler.listen_protocol(); let protocol = self.handler.listen_protocol();
let upgrade = upgrade::apply(substream, protocol, Endpoint::Listener); let upgrade = upgrade::apply_inbound(substream, protocol);
let with_timeout = Timeout::new(upgrade, self.in_timeout); let with_timeout = Timeout::new(upgrade, self.in_timeout);
self.negotiating_in.push(with_timeout); self.negotiating_in.push(with_timeout);
} }
@ -587,7 +644,7 @@ where
}; };
let (_, proto_upgrade) = self.queued_dial_upgrades.remove(pos); let (_, proto_upgrade) = self.queued_dial_upgrades.remove(pos);
let upgrade = upgrade::apply(substream, proto_upgrade, Endpoint::Dialer); let upgrade = upgrade::apply_outbound(substream, proto_upgrade);
let with_timeout = Timeout::new(upgrade, self.out_timeout); let with_timeout = Timeout::new(upgrade, self.out_timeout);
self.negotiating_out.push((user_data, with_timeout)); self.negotiating_out.push((user_data, with_timeout));
} }
@ -630,21 +687,15 @@ where
self.handler.shutdown(); self.handler.shutdown();
} }
fn poll( fn poll(&mut self) -> Poll<Option<NodeHandlerEvent<Self::OutboundOpenInfo, Self::OutEvent>>, io::Error> {
&mut self,
) -> Poll<Option<NodeHandlerEvent<Self::OutboundOpenInfo, Self::OutEvent>>, io::Error> {
// Continue negotiation of newly-opened substreams on the listening side. // Continue negotiation of newly-opened substreams on the listening side.
// We remove each element from `negotiating_in` one by one and add them back if not ready. // We remove each element from `negotiating_in` one by one and add them back if not ready.
for n in (0..self.negotiating_in.len()).rev() { for n in (0..self.negotiating_in.len()).rev() {
let mut in_progress = self.negotiating_in.swap_remove(n); let mut in_progress = self.negotiating_in.swap_remove(n);
match in_progress.poll() { match in_progress.poll() {
Ok(Async::Ready(upgrade)) => { Ok(Async::Ready(upgrade)) =>
self.handler self.handler.inject_fully_negotiated_inbound(upgrade),
.inject_fully_negotiated(upgrade, NodeHandlerEndpoint::Listener); Ok(Async::NotReady) => self.negotiating_in.push(in_progress),
}
Ok(Async::NotReady) => {
self.negotiating_in.push(in_progress);
}
// TODO: return a diagnostic event? // TODO: return a diagnostic event?
Err(_err) => {} Err(_err) => {}
} }
@ -656,8 +707,7 @@ where
let (upgr_info, mut in_progress) = self.negotiating_out.swap_remove(n); let (upgr_info, mut in_progress) = self.negotiating_out.swap_remove(n);
match in_progress.poll() { match in_progress.poll() {
Ok(Async::Ready(upgrade)) => { Ok(Async::Ready(upgrade)) => {
let endpoint = NodeHandlerEndpoint::Dialer(upgr_info); self.handler.inject_fully_negotiated_outbound(upgrade, upgr_info);
self.handler.inject_fully_negotiated(upgrade, endpoint);
} }
Ok(Async::NotReady) => { Ok(Async::NotReady) => {
self.negotiating_out.push((upgr_info, in_progress)); self.negotiating_out.push((upgr_info, in_progress));
@ -704,45 +754,138 @@ pub struct ProtocolsHandlerSelect<TProto1, TProto2> {
impl<TSubstream, TProto1, TProto2, TProto1Out, TProto2Out> impl<TSubstream, TProto1, TProto2, TProto1Out, TProto2Out>
ProtocolsHandler for ProtocolsHandlerSelect<TProto1, TProto2> ProtocolsHandler for ProtocolsHandlerSelect<TProto1, TProto2>
where TProto1: ProtocolsHandler<Substream = TSubstream>, where
TProto1: ProtocolsHandler<Substream = TSubstream>,
TProto2: ProtocolsHandler<Substream = TSubstream>, TProto2: ProtocolsHandler<Substream = TSubstream>,
TSubstream: AsyncRead + AsyncWrite, TSubstream: AsyncRead + AsyncWrite,
TProto1::Protocol: ConnectionUpgrade<TSubstream, Output = TProto1Out>, TProto1::InboundProtocol: InboundUpgrade<TSubstream, Output = TProto1Out>,
TProto2::Protocol: ConnectionUpgrade<TSubstream, Output = TProto2Out>, TProto2::InboundProtocol: InboundUpgrade<TSubstream, Output = TProto2Out>,
TProto1::OutboundProtocol: OutboundUpgrade<TSubstream, Output = TProto1Out>,
TProto2::OutboundProtocol: OutboundUpgrade<TSubstream, Output = TProto2Out>
{ {
type InEvent = EitherOutput<TProto1::InEvent, TProto2::InEvent>; type InEvent = EitherOutput<TProto1::InEvent, TProto2::InEvent>;
type OutEvent = EitherOutput<TProto1::OutEvent, TProto2::OutEvent>; type OutEvent = EitherOutput<TProto1::OutEvent, TProto2::OutEvent>;
type Substream = TSubstream; type Substream = TSubstream;
type Protocol = upgrade::OrUpgrade<upgrade::toggleable::Toggleable<upgrade::map::Map<TProto1::Protocol, fn(TProto1Out) -> EitherOutput<TProto1Out, TProto2Out>>>, upgrade::toggleable::Toggleable<upgrade::map::Map<TProto2::Protocol, fn(TProto2Out) -> EitherOutput<TProto1Out, TProto2Out>>>>;
type InboundProtocol =
upgrade::OrUpgrade<
upgrade::Toggleable<
upgrade::MapUpgradeErr<
upgrade::MapUpgrade<
TProto1::InboundProtocol,
fn(TProto1Out) -> EitherOutput<TProto1Out, TProto2Out>
>,
fn(<TProto1::InboundProtocol as InboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::InboundProtocol as InboundUpgrade<TSubstream>>::Error,
<TProto2::InboundProtocol as InboundUpgrade<TSubstream>>::Error
>
>
>,
upgrade::Toggleable<
upgrade::MapUpgradeErr<
upgrade::MapUpgrade<
TProto2::InboundProtocol,
fn(TProto2Out) -> EitherOutput<TProto1Out, TProto2Out>
>,
fn(<TProto2::InboundProtocol as InboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::InboundProtocol as InboundUpgrade<TSubstream>>::Error,
<TProto2::InboundProtocol as InboundUpgrade<TSubstream>>::Error
>
>
>
>;
type OutboundProtocol =
upgrade::OrUpgrade<
upgrade::Toggleable<
upgrade::MapUpgradeErr<
upgrade::MapUpgrade<
TProto1::OutboundProtocol,
fn(TProto1Out) -> EitherOutput<TProto1Out, TProto2Out>
>,
fn(<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>
>
>,
upgrade::Toggleable<
upgrade::MapUpgradeErr<
upgrade::MapUpgrade<
TProto2::OutboundProtocol,
fn(TProto2Out) -> EitherOutput<TProto1Out, TProto2Out>
>,
fn(<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>
>
>
>;
type OutboundOpenInfo = EitherOutput<TProto1::OutboundOpenInfo, TProto2::OutboundOpenInfo>; type OutboundOpenInfo = EitherOutput<TProto1::OutboundOpenInfo, TProto2::OutboundOpenInfo>;
#[inline] #[inline]
fn listen_protocol(&self) -> Self::Protocol { fn listen_protocol(&self) -> Self::InboundProtocol {
let proto1 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(self.proto1.listen_protocol(), EitherOutput::First)); let proto1 = self.proto1.listen_protocol()
let proto2 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(self.proto2.listen_protocol(), EitherOutput::Second)); .map_inbound(EitherOutput::First as fn(TProto1Out) -> EitherOutput<TProto1Out, TProto2Out>)
upgrade::or(proto1, proto2) .map_inbound_err(EitherError::A as fn(<TProto1::InboundProtocol as InboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::InboundProtocol as InboundUpgrade<TSubstream>>::Error,
<TProto2::InboundProtocol as InboundUpgrade<TSubstream>>::Error
>);
let proto2 = self.proto2.listen_protocol()
.map_inbound(EitherOutput::Second as fn(TProto2Out) -> EitherOutput<TProto1Out, TProto2Out>)
.map_inbound_err(EitherError::B as fn(<TProto2::InboundProtocol as InboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::InboundProtocol as InboundUpgrade<TSubstream>>::Error,
<TProto2::InboundProtocol as InboundUpgrade<TSubstream>>::Error
>);
upgrade::toggleable(proto1).or_inbound(upgrade::toggleable(proto2))
} }
fn inject_fully_negotiated(&mut self, protocol: <Self::Protocol as ConnectionUpgrade<TSubstream>>::Output, endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>) { #[inline]
fn dialer_protocol(&self) -> Self::OutboundProtocol {
let proto1 = self.proto1.dialer_protocol()
.map_outbound(EitherOutput::First as fn(TProto1Out) -> EitherOutput<TProto1Out, TProto2Out>)
.map_outbound_err(EitherError::A as fn(<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>);
let proto2 = self.proto2.dialer_protocol()
.map_outbound(EitherOutput::Second as fn(TProto2Out) -> EitherOutput<TProto1Out, TProto2Out>)
.map_outbound_err(EitherError::B as fn(<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>);
upgrade::toggleable(proto1).or_outbound(upgrade::toggleable(proto2))
}
fn inject_fully_negotiated_outbound(&mut self, protocol: <Self::OutboundProtocol as OutboundUpgrade<TSubstream>>::Output, endpoint: Self::OutboundOpenInfo) {
match (protocol, endpoint) { match (protocol, endpoint) {
(EitherOutput::First(protocol), NodeHandlerEndpoint::Dialer(EitherOutput::First(info))) => { (EitherOutput::First(protocol), EitherOutput::First(info)) =>
self.proto1.inject_fully_negotiated(protocol, NodeHandlerEndpoint::Dialer(info)); self.proto1.inject_fully_negotiated_outbound(protocol, info),
}, (EitherOutput::Second(protocol), EitherOutput::Second(info)) =>
(EitherOutput::Second(protocol), NodeHandlerEndpoint::Dialer(EitherOutput::Second(info))) => { self.proto2.inject_fully_negotiated_outbound(protocol, info),
self.proto2.inject_fully_negotiated(protocol, NodeHandlerEndpoint::Dialer(info)); (EitherOutput::First(_), EitherOutput::Second(_)) =>
}, panic!("wrong API usage: the protocol doesn't match the upgrade info"),
(EitherOutput::First(_), NodeHandlerEndpoint::Dialer(EitherOutput::Second(_))) => { (EitherOutput::Second(_), EitherOutput::First(_)) =>
panic!("wrong API usage: the protocol doesn't match the upgrade info") panic!("wrong API usage: the protocol doesn't match the upgrade info")
}, }
(EitherOutput::Second(_), NodeHandlerEndpoint::Dialer(EitherOutput::First(_))) => { }
panic!("wrong API usage: the protocol doesn't match the upgrade info")
}, fn inject_fully_negotiated_inbound(&mut self, protocol: <Self::InboundProtocol as InboundUpgrade<TSubstream>>::Output) {
(EitherOutput::First(protocol), NodeHandlerEndpoint::Listener) => { match protocol {
self.proto1.inject_fully_negotiated(protocol, NodeHandlerEndpoint::Listener); EitherOutput::First(protocol) =>
}, self.proto1.inject_fully_negotiated_inbound(protocol),
(EitherOutput::Second(protocol), NodeHandlerEndpoint::Listener) => { EitherOutput::Second(protocol) =>
self.proto2.inject_fully_negotiated(protocol, NodeHandlerEndpoint::Listener); self.proto2.inject_fully_negotiated_inbound(protocol)
},
} }
} }
@ -774,19 +917,33 @@ where TProto1: ProtocolsHandler<Substream = TSubstream>,
self.proto2.shutdown(); self.proto2.shutdown();
} }
fn poll(&mut self) -> Poll<Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>, io::Error> { fn poll(&mut self) -> Poll<Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>, io::Error> {
match self.proto1.poll()? { match self.proto1.poll()? {
Async::Ready(Some(ProtocolsHandlerEvent::Custom(event))) => { Async::Ready(Some(ProtocolsHandlerEvent::Custom(event))) => {
return Ok(Async::Ready(Some(ProtocolsHandlerEvent::Custom(EitherOutput::First(event))))); return Ok(Async::Ready(Some(ProtocolsHandlerEvent::Custom(EitherOutput::First(event)))));
}, },
Async::Ready(Some(ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info})) => { Async::Ready(Some(ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info})) => {
let upgrade = { let upgrade = {
let proto1 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(upgrade, EitherOutput::First)); let proto1 = upgrade
let mut proto2 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(self.proto2.listen_protocol(), EitherOutput::Second)); .map_outbound(EitherOutput::First as fn(TProto1Out) -> EitherOutput<TProto1Out, TProto2Out>)
proto2.disable(); .map_outbound_err(EitherError::A as fn(<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
upgrade::or(proto1, proto2) EitherError<
}; <TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>);
let proto2 = self.proto2.dialer_protocol()
.map_outbound(EitherOutput::Second as fn(TProto2Out) -> EitherOutput<TProto1Out, TProto2Out>)
.map_outbound_err(EitherError::B as fn(<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>);
let proto1 = upgrade::toggleable(proto1);
let mut proto2 = upgrade::toggleable(proto2);
proto2.disable();
proto1.or_outbound(proto2)
};
return Ok(Async::Ready(Some(ProtocolsHandlerEvent::OutboundSubstreamRequest { return Ok(Async::Ready(Some(ProtocolsHandlerEvent::OutboundSubstreamRequest {
upgrade, upgrade,
info: EitherOutput::First(info), info: EitherOutput::First(info),
@ -802,12 +959,26 @@ where TProto1: ProtocolsHandler<Substream = TSubstream>,
}, },
Async::Ready(Some(ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info })) => { Async::Ready(Some(ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info })) => {
let upgrade = { let upgrade = {
let mut proto1 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(self.proto1.listen_protocol(), EitherOutput::First)); let proto1 = self.proto1.dialer_protocol()
proto1.disable(); .map_outbound(EitherOutput::First as fn(TProto1Out) -> EitherOutput<TProto1Out, TProto2Out>)
let proto2 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(upgrade, EitherOutput::Second)); .map_outbound_err(EitherError::A as fn(<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
upgrade::or(proto1, proto2) EitherError<
}; <TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>);
let proto2 = upgrade
.map_outbound(EitherOutput::Second as fn(TProto2Out) -> EitherOutput<TProto1Out, TProto2Out>)
.map_outbound_err(EitherError::B as fn(<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error) ->
EitherError<
<TProto1::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error,
<TProto2::OutboundProtocol as OutboundUpgrade<TSubstream>>::Error
>);
let mut proto1 = upgrade::toggleable(proto1);
proto1.disable();
let proto2 = upgrade::toggleable(proto2);
proto1.or_outbound(proto2)
};
return Ok(Async::Ready(Some(ProtocolsHandlerEvent::OutboundSubstreamRequest { return Ok(Async::Ready(Some(ProtocolsHandlerEvent::OutboundSubstreamRequest {
upgrade, upgrade,
info: EitherOutput::Second(info), info: EitherOutput::Second(info),

View File

@ -18,18 +18,30 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::muxing::StreamMuxer;
use crate::{
Endpoint, Multiaddr, PeerId,
nodes::{
collection::{
CollectionEvent,
CollectionNodeAccept,
CollectionReachEvent,
CollectionStream,
PeerMut as CollecPeerMut,
ReachAttemptId
},
handled_node::NodeHandler,
node::Substream
},
nodes::listeners::{ListenersEvent, ListenersStream},
transport::Transport
};
use fnv::FnvHashMap; use fnv::FnvHashMap;
use futures::{prelude::*, future}; use futures::{prelude::*, future};
use muxing::StreamMuxer; use std::{
use nodes::collection::{ collections::hash_map::{Entry, OccupiedEntry},
CollectionEvent, CollectionNodeAccept, CollectionReachEvent, CollectionStream, PeerMut as CollecPeerMut, ReachAttemptId, io::{Error as IoError, ErrorKind as IoErrorKind}
}; };
use nodes::handled_node::NodeHandler;
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 {Endpoint, Multiaddr, PeerId, Transport};
/// Implementation of `Stream` that handles the nodes. /// Implementation of `Stream` that handles the nodes.
pub struct RawSwarm<TTrans, TInEvent, TOutEvent, THandler> pub struct RawSwarm<TTrans, TInEvent, TOutEvent, THandler>
@ -198,8 +210,7 @@ where TTrans: Transport
impl<'a, TTrans, TInEvent, TOutEvent, TMuxer, THandler> IncomingConnectionEvent<'a, TTrans, TInEvent, TOutEvent, THandler> impl<'a, TTrans, TInEvent, TOutEvent, TMuxer, THandler> IncomingConnectionEvent<'a, TTrans, TInEvent, TOutEvent, THandler>
where where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone, TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::Dial: Send + 'static,
TTrans::ListenerUpgrade: Send + 'static, TTrans::ListenerUpgrade: Send + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
@ -318,7 +329,7 @@ impl ConnectedPoint {
impl<TTrans, TInEvent, TOutEvent, TMuxer, THandler> impl<TTrans, TInEvent, TOutEvent, TMuxer, THandler>
RawSwarm<TTrans, TInEvent, TOutEvent, THandler> RawSwarm<TTrans, TInEvent, TOutEvent, THandler>
where where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone, TTrans: Transport + Clone,
TMuxer: StreamMuxer, TMuxer: StreamMuxer,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
@ -380,7 +391,7 @@ where
/// The second parameter is the handler to use if we manage to reach a node. /// 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> pub fn dial(&mut self, addr: Multiaddr, handler: THandler) -> Result<(), Multiaddr>
where where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone, TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::Dial: Send + 'static, TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send, TMuxer::OutboundSubstream: Send,
@ -463,10 +474,9 @@ where
/// ///
/// It is a logic error to call this method if we already have an outgoing attempt to the /// It is a logic error to call this method if we already have an outgoing attempt to the
/// given peer. /// given peer.
fn start_dial_out(&mut self, peer_id: PeerId, handler: THandler, first: Multiaddr, fn start_dial_out(&mut self, peer_id: PeerId, handler: THandler, first: Multiaddr, rest: Vec<Multiaddr>)
rest: Vec<Multiaddr>)
where where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone, TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::Dial: Send + 'static, TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send, TMuxer::OutboundSubstream: Send,
@ -510,7 +520,7 @@ where
/// Provides an API similar to `Stream`, except that it cannot error. /// Provides an API similar to `Stream`, except that it cannot error.
pub fn poll(&mut self) -> Async<RawSwarmEvent<TTrans, TInEvent, TOutEvent, THandler>> pub fn poll(&mut self) -> Async<RawSwarmEvent<TTrans, TInEvent, TOutEvent, THandler>>
where where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone, TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::Dial: Send + 'static, TTrans::Dial: Send + 'static,
TTrans::ListenerUpgrade: Send + 'static, TTrans::ListenerUpgrade: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer: StreamMuxer + Send + Sync + 'static,
@ -524,11 +534,7 @@ where
// Start by polling the listeners for events. // Start by polling the listeners for events.
match self.listeners.poll() { match self.listeners.poll() {
Async::NotReady => (), Async::NotReady => (),
Async::Ready(ListenersEvent::Incoming { Async::Ready(ListenersEvent::Incoming { upgrade, listen_addr, send_back_addr }) => {
upgrade,
listen_addr,
send_back_addr,
}) => {
let event = IncomingConnectionEvent { let event = IncomingConnectionEvent {
upgrade, upgrade,
listen_addr, listen_addr,
@ -536,14 +542,9 @@ where
active_nodes: &mut self.active_nodes, active_nodes: &mut self.active_nodes,
other_reach_attempts: &mut self.reach_attempts.other_reach_attempts, other_reach_attempts: &mut self.reach_attempts.other_reach_attempts,
}; };
return Async::Ready(RawSwarmEvent::IncomingConnection(event)); return Async::Ready(RawSwarmEvent::IncomingConnection(event));
} }
Async::Ready(ListenersEvent::Closed { Async::Ready(ListenersEvent::Closed { listen_addr, listener, result }) => {
listen_addr,
listener,
result,
}) => {
return Async::Ready(RawSwarmEvent::ListenerClosed { return Async::Ready(RawSwarmEvent::ListenerClosed {
listen_addr, listen_addr,
listener, listener,
@ -654,7 +655,6 @@ fn handle_node_reached<'a, TTrans, TMuxer, TInEvent, TOutEvent, THandler>(
) -> (ActionItem<THandler>, RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler>) ) -> (ActionItem<THandler>, RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler>)
where where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone, TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send, TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send, TMuxer::Substream: Send,
@ -848,10 +848,15 @@ where
impl<'a, TTrans, TMuxer, TInEvent, TOutEvent, THandler> impl<'a, TTrans, TMuxer, TInEvent, TOutEvent, THandler>
Peer<'a, TTrans, TInEvent, TOutEvent, THandler> Peer<'a, TTrans, TInEvent, TOutEvent, THandler>
where where
TTrans: Transport<Output = (PeerId, TMuxer)>, TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TMuxer: StreamMuxer, TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary THandler::OutboundOpenInfo: Send + 'static,
{ {
/// If we are connected, returns the `PeerConnected`. /// If we are connected, returns the `PeerConnected`.
#[inline] #[inline]
@ -889,19 +894,8 @@ where
/// > that we are expecting, in which case the handler will be used for this "wrong" /// > that we are expecting, in which case the handler will be used for this "wrong"
/// > node. /// > node.
#[inline] #[inline]
pub fn or_connect( pub fn or_connect(self, addr: Multiaddr, handler: THandler)
self, -> Result<PeerPotentialConnect<'a, TInEvent, TOutEvent, THandler>, Self>
addr: Multiaddr,
handler: THandler,
) -> Result<PeerPotentialConnect<'a, TInEvent, TOutEvent, THandler>, Self>
where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Dial: 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, handler) self.or_connect_with(move |_| addr, handler)
} }
@ -916,20 +910,10 @@ where
/// > that we are expecting, in which case the handler will be used for this "wrong" /// > that we are expecting, in which case the handler will be used for this "wrong"
/// > node. /// > node.
#[inline] #[inline]
pub fn or_connect_with<TFn>( pub fn or_connect_with<TFn>(self, addr: TFn, handler: THandler)
self, -> Result<PeerPotentialConnect<'a, TInEvent, TOutEvent, THandler>, Self>
addr: TFn,
handler: THandler,
) -> Result<PeerPotentialConnect<'a, TInEvent, TOutEvent, THandler>, Self>
where where
TFn: FnOnce(&PeerId) -> Multiaddr, TFn: FnOnce(&PeerId) -> Multiaddr,
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
{ {
match self { match self {
Peer::Connected(peer) => Ok(PeerPotentialConnect::Connected(peer)), Peer::Connected(peer) => Ok(PeerPotentialConnect::Connected(peer)),
@ -1082,26 +1066,22 @@ where
impl<'a, TTrans, TInEvent, TOutEvent, TMuxer, THandler> impl<'a, TTrans, TInEvent, TOutEvent, TMuxer, THandler>
PeerNotConnected<'a, TTrans, TInEvent, TOutEvent, THandler> PeerNotConnected<'a, TTrans, TInEvent, TOutEvent, THandler>
where where
TTrans: Transport<Output = (PeerId, TMuxer)>, TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TMuxer: StreamMuxer, TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static, THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
{ {
/// Attempts a new connection to this node using the given multiaddress. /// 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 /// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then
/// the whole connection is immediately closed. /// the whole connection is immediately closed.
#[inline] #[inline]
pub fn connect(self, addr: Multiaddr, handler: THandler) -> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self> pub fn connect(self, addr: Multiaddr, handler: THandler) -> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self> {
where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
{
self.connect_inner(handler, addr, Vec::new()) self.connect_inner(handler, addr, Vec::new())
} }
@ -1114,20 +1094,10 @@ where
/// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then /// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then
/// the whole connection is immediately closed. /// the whole connection is immediately closed.
#[inline] #[inline]
pub fn connect_iter<TIter>( pub fn connect_iter<TIter>(self, addrs: TIter, handler: THandler)
self, -> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self>
addrs: TIter,
handler: THandler,
) -> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self>
where where
TIter: IntoIterator<Item = Multiaddr>, TIter: IntoIterator<Item = Multiaddr>,
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send,
TInEvent: Send + 'static,
TOutEvent: Send + 'static,
{ {
let mut addrs = addrs.into_iter(); let mut addrs = addrs.into_iter();
let first = match addrs.next() { let first = match addrs.next() {
@ -1139,23 +1109,10 @@ where
} }
/// Inner implementation of `connect`. /// Inner implementation of `connect`.
fn connect_inner( fn connect_inner(self, handler: THandler, first: Multiaddr, rest: Vec<Multiaddr>)
self, -> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self>
handler: THandler,
first: Multiaddr,
rest: Vec<Multiaddr>,
) -> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self>
where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Dial: 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(), handler, first, rest); self.nodes.start_dial_out(self.peer_id.clone(), handler, first, rest);
Ok(PeerPendingConnect { Ok(PeerPendingConnect {
attempt: match self.nodes.reach_attempts.out_reach_attempts.entry(self.peer_id) { attempt: match self.nodes.reach_attempts.out_reach_attempts.entry(self.peer_id) {
Entry::Occupied(e) => e, Entry::Occupied(e) => e,

View File

@ -18,15 +18,19 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::{
Transport, Multiaddr, PeerId, InboundUpgrade, OutboundUpgrade, UpgradeInfo,
muxing::StreamMuxer,
nodes::{
handled_node::NodeHandler,
node::Substream,
protocols_handler::{NodeHandlerWrapper, ProtocolsHandler},
raw_swarm::{RawSwarm, RawSwarmEvent, ConnectedPoint}
},
topology::Topology
};
use futures::prelude::*; use futures::prelude::*;
use muxing::StreamMuxer; use std::{fmt, io, ops::{Deref, DerefMut}};
use nodes::handled_node::NodeHandler;
use nodes::node::Substream;
use nodes::protocols_handler::{NodeHandlerWrapper, ProtocolsHandler};
use nodes::raw_swarm::{RawSwarm, RawSwarmEvent, ConnectedPoint};
use std::{io, ops::{Deref, DerefMut}};
use topology::Topology;
use {ConnectionUpgrade, Multiaddr, PeerId, Transport};
/// Contains the state of the network, plus the way it should behave. /// Contains the state of the network, plus the way it should behave.
pub struct Swarm<TTransport, TBehaviour, TTopology> pub struct Swarm<TTransport, TBehaviour, TTopology>
@ -77,17 +81,23 @@ where TBehaviour: NetworkBehavior,
<TMuxer as StreamMuxer>::OutboundSubstream: Send + 'static, <TMuxer as StreamMuxer>::OutboundSubstream: Send + 'static,
<TMuxer as StreamMuxer>::Substream: Send + 'static, <TMuxer as StreamMuxer>::Substream: Send + 'static,
TTransport: Transport<Output = (PeerId, TMuxer)> + Clone, TTransport: Transport<Output = (PeerId, TMuxer)> + Clone,
TTransport::Dial: Send + 'static,
TTransport::Listener: Send + 'static, TTransport::Listener: Send + 'static,
TTransport::ListenerUpgrade: Send + 'static, TTransport::ListenerUpgrade: Send + 'static,
TTransport::Dial: Send + 'static,
TBehaviour::ProtocolsHandler: ProtocolsHandler<Substream = Substream<TMuxer>> + Send + 'static, TBehaviour::ProtocolsHandler: ProtocolsHandler<Substream = Substream<TMuxer>> + Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InEvent: Send + 'static, <TBehaviour::ProtocolsHandler as ProtocolsHandler>::InEvent: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutEvent: Send + 'static, <TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutEvent: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol: ConnectionUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol as ConnectionUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol as ConnectionUpgrade<Substream<TMuxer>>>::NamesIter: Clone + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol as ConnectionUpgrade<Substream<TMuxer>>>::UpgradeIdentifier: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary <TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol: InboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::NamesIter: Clone + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::UpgradeId: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::NamesIter: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::UpgradeId: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static,
<NodeHandlerWrapper<TBehaviour::ProtocolsHandler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary <NodeHandlerWrapper<TBehaviour::ProtocolsHandler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
TTopology: Topology, TTopology: Topology,
{ {
@ -164,17 +174,23 @@ where TBehaviour: NetworkBehavior,
<TMuxer as StreamMuxer>::OutboundSubstream: Send + 'static, <TMuxer as StreamMuxer>::OutboundSubstream: Send + 'static,
<TMuxer as StreamMuxer>::Substream: Send + 'static, <TMuxer as StreamMuxer>::Substream: Send + 'static,
TTransport: Transport<Output = (PeerId, TMuxer)> + Clone, TTransport: Transport<Output = (PeerId, TMuxer)> + Clone,
TTransport::Dial: Send + 'static,
TTransport::Listener: Send + 'static, TTransport::Listener: Send + 'static,
TTransport::ListenerUpgrade: Send + 'static, TTransport::ListenerUpgrade: Send + 'static,
TTransport::Dial: Send + 'static,
TBehaviour::ProtocolsHandler: ProtocolsHandler<Substream = Substream<TMuxer>> + Send + 'static, TBehaviour::ProtocolsHandler: ProtocolsHandler<Substream = Substream<TMuxer>> + Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InEvent: Send + 'static, <TBehaviour::ProtocolsHandler as ProtocolsHandler>::InEvent: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutEvent: Send + 'static, <TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutEvent: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol: ConnectionUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol as ConnectionUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol as ConnectionUpgrade<Substream<TMuxer>>>::NamesIter: Clone + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::Protocol as ConnectionUpgrade<Substream<TMuxer>>>::UpgradeIdentifier: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary <TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol: InboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as InboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::NamesIter: Clone + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InboundProtocol as UpgradeInfo>::UpgradeId: Send + 'static,
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol: OutboundUpgrade<Substream<TMuxer>> + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Future: Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as OutboundUpgrade<Substream<TMuxer>>>::Error: fmt::Debug + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::NamesIter: Clone + Send + 'static,
<<TBehaviour::ProtocolsHandler as ProtocolsHandler>::OutboundProtocol as UpgradeInfo>::UpgradeId: Send + 'static,
<NodeHandlerWrapper<TBehaviour::ProtocolsHandler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary <NodeHandlerWrapper<TBehaviour::ProtocolsHandler> as NodeHandler>::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
TTopology: Topology, TTopology: Topology,
{ {

View File

@ -1,4 +1,4 @@
// Copyright 2017 Parity Technologies (UK) Ltd. // Copyright 2017-2018 Parity Technologies (UK) Ltd.
// //
// Permission is hereby granted, free of charge, to any person obtaining a // Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"), // copy of this software and associated documentation files (the "Software"),
@ -29,12 +29,12 @@
//! `UpgradedNode::or_upgrade` methods, you can combine multiple transports and/or upgrades //! `UpgradedNode::or_upgrade` methods, you can combine multiple transports and/or upgrades
//! together in a complex chain of protocols negotiation. //! together in a complex chain of protocols negotiation.
use crate::{InboundUpgrade, OutboundUpgrade, Endpoint};
use futures::prelude::*; use futures::prelude::*;
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use nodes::raw_swarm::ConnectedPoint; use nodes::raw_swarm::ConnectedPoint;
use std::io::Error as IoError; use std::io::Error as IoError;
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use upgrade::{ConnectionUpgrade, Endpoint};
pub mod and_then; pub mod and_then;
pub mod boxed; pub mod boxed;
@ -50,7 +50,7 @@ pub mod upgrade;
pub use self::choice::OrTransport; pub use self::choice::OrTransport;
pub use self::denied::DeniedTransport; pub use self::denied::DeniedTransport;
pub use self::memory::connector; pub use self::memory::connector;
pub use self::upgrade::UpgradedNode; pub use self::upgrade::Upgrade;
/// A transport is an object that can be used to produce connections by listening or dialing a /// A transport is an object that can be used to produce connections by listening or dialing a
/// peer. /// peer.
@ -178,13 +178,14 @@ pub trait Transport {
/// > **Note**: The concept of an *upgrade* for example includes middlewares such *secio* /// > **Note**: The concept of an *upgrade* for example includes middlewares such *secio*
/// > (communication encryption), *multiplex*, but also a protocol handler. /// > (communication encryption), *multiplex*, but also a protocol handler.
#[inline] #[inline]
fn with_upgrade<U>(self, upgrade: U) -> UpgradedNode<Self, U> fn with_upgrade<U, O, E>(self, upgrade: U) -> Upgrade<Self, U>
where where
Self: Sized, Self: Sized,
Self::Output: AsyncRead + AsyncWrite, Self::Output: AsyncRead + AsyncWrite,
U: ConnectionUpgrade<Self::Output>, U: InboundUpgrade<Self::Output, Output = O, Error = E>,
U: OutboundUpgrade<Self::Output, Output = O, Error = E>
{ {
UpgradedNode::new(self, upgrade) Upgrade::new(self, upgrade)
} }
/// Wraps this transport inside an upgrade. Whenever a connection that uses this transport /// Wraps this transport inside an upgrade. Whenever a connection that uses this transport

View File

@ -1,4 +1,4 @@
// Copyright 2017 Parity Technologies (UK) Ltd. // Copyright 2017-2018 Parity Technologies (UK) Ltd.
// //
// Permission is hereby granted, free of charge, to any person obtaining a // Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"), // copy of this software and associated documentation files (the "Software"),
@ -20,173 +20,75 @@
use futures::prelude::*; use futures::prelude::*;
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use std::io::Error as IoError; use crate::{
transport::Transport,
upgrade::{OutboundUpgrade, InboundUpgrade, UpgradeInfo, apply_inbound, apply_outbound}
};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use transport::Transport;
use upgrade::{apply, ConnectionUpgrade, Endpoint};
/// Implements the `Transport` trait. Dials or listens, then upgrades any dialed or received #[derive(Debug, Copy, Clone)]
/// connection. pub struct Upgrade<T, U> { inner: T, upgrade: U }
///
/// See the `Transport::with_upgrade` method.
#[derive(Debug, Clone)]
pub struct UpgradedNode<T, C> {
transports: T,
upgrade: C,
}
impl<T, C> UpgradedNode<T, C> { impl<T, U> Upgrade<T, U> {
pub fn new(transports: T, upgrade: C) -> UpgradedNode<T, C> { pub fn new(inner: T, upgrade: U) -> Self {
UpgradedNode { Upgrade { inner, upgrade }
transports,
upgrade,
}
} }
} }
impl<'a, T, C> UpgradedNode<T, C> impl<D, U, O, E> Transport for Upgrade<D, U>
where where
T: Transport + 'a, D: Transport,
T::Dial: Send, D::Dial: Send + 'static,
T::Listener: Send, D::Listener: Send + 'static,
T::ListenerUpgrade: Send, D::ListenerUpgrade: Send + 'static,
T::Output: Send + AsyncRead + AsyncWrite, D::Output: AsyncRead + AsyncWrite + Send + 'static,
C: ConnectionUpgrade<T::Output> + Send + 'a, U: InboundUpgrade<D::Output, Output = O, Error = E>,
C::NamesIter: Send, U: OutboundUpgrade<D::Output, Output = O, Error = E> + Send + Clone + 'static,
C::Future: Send, <U as UpgradeInfo>::NamesIter: Clone + Send,
C::UpgradeIdentifier: Send, <U as UpgradeInfo>::UpgradeId: Send,
<U as InboundUpgrade<D::Output>>::Future: Send,
<U as OutboundUpgrade<D::Output>>::Future: Send,
E: std::error::Error + Send + Sync + 'static
{ {
/// Returns a reference to the inner `Transport`. type Output = O;
#[inline] type Listener = Box<Stream<Item = (Self::ListenerUpgrade, Multiaddr), Error = std::io::Error> + Send>;
pub fn transport(&self) -> &T { type ListenerUpgrade = Box<Future<Item = Self::Output, Error = std::io::Error> + Send>;
&self.transports type Dial = Box<Future<Item = Self::Output, Error = std::io::Error> + Send>;
}
/// Tries to dial on the `Multiaddr` using the transport that was passed to `new`, then upgrade fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> {
/// the connection.
///
/// Note that this does the same as `Transport::dial`, but with less restrictions on the trait
/// requirements.
#[inline]
pub fn dial(
self,
addr: Multiaddr,
) -> Result<Box<Future<Item = C::Output, Error = IoError> + Send + 'a>, (Self, Multiaddr)>
where
C::NamesIter: Clone, // TODO: not elegant
{
let upgrade = self.upgrade; let upgrade = self.upgrade;
match self.inner.dial(addr.clone()) {
let dialed_fut = match self.transports.dial(addr.clone()) { Ok(outbound) => {
Ok(f) => f, let future = outbound
Err((trans, addr)) => { .and_then(move |x| apply_outbound(x, upgrade).map_err(|e| {
let builder = UpgradedNode { std::io::Error::new(std::io::ErrorKind::Other, e)
transports: trans, }));
upgrade: upgrade,
};
return Err((builder, addr));
}
};
let future = dialed_fut
// Try to negotiate the protocol.
.and_then(move |connection| {
apply(connection, upgrade, Endpoint::Dialer)
});
Ok(Box::new(future)) Ok(Box::new(future))
} }
Err((dialer, addr)) => Err((Upgrade::new(dialer, upgrade), addr))
/// Start listening on the multiaddr using the transport that was passed to `new`.
/// Then whenever a connection is opened, it is upgraded.
///
/// Note that this does the same as `Transport::listen_on`, but with less restrictions on the
/// trait requirements.
#[inline]
pub fn listen_on(
self,
addr: Multiaddr,
) -> Result<
(
Box<
Stream<
Item = (Box<Future<Item = C::Output, Error = IoError> + Send + 'a>, Multiaddr),
Error = IoError,
>
+ Send
+ 'a,
>,
Multiaddr,
),
(Self, Multiaddr),
>
where
C::NamesIter: Clone, // TODO: not elegant
C: Clone,
{
let upgrade = self.upgrade;
let (listening_stream, new_addr) = match self.transports.listen_on(addr) {
Ok((l, new_addr)) => (l, new_addr),
Err((trans, addr)) => {
let builder = UpgradedNode {
transports: trans,
upgrade: upgrade,
};
return Err((builder, addr));
} }
};
// 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, client_addr)| {
let upgrade = upgrade.clone();
let connection = connection
// Try to negotiate the protocol.
.and_then(move |connection| {
apply(connection, upgrade, Endpoint::Listener)
});
(Box::new(connection) as Box<_>, client_addr)
});
Ok((Box::new(stream), new_addr))
} }
}
impl<T, C> Transport for UpgradedNode<T, C>
where
T: Transport + 'static,
T::Dial: Send,
T::Listener: Send,
T::ListenerUpgrade: Send,
T::Output: Send + AsyncRead + AsyncWrite,
C: ConnectionUpgrade<T::Output> + Clone + Send + 'static,
C::NamesIter: Clone + Send,
C::Future: Send,
C::UpgradeIdentifier: Send,
{
type Output = C::Output;
type Listener = Box<Stream<Item = (Self::ListenerUpgrade, Multiaddr), Error = IoError> + Send>;
type ListenerUpgrade = Box<Future<Item = C::Output, Error = IoError> + Send>;
type Dial = Box<Future<Item = C::Output, Error = IoError> + Send>;
#[inline]
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> {
self.listen_on(addr) let upgrade = self.upgrade;
match self.inner.listen_on(addr) {
Ok((inbound, addr)) => {
let stream = inbound
.map(move |(future, addr)| {
let upgrade = upgrade.clone();
let future = future
.and_then(move |x| apply_inbound(x, upgrade).map_err(|e| {
std::io::Error::new(std::io::ErrorKind::Other, e)
}));
(Box::new(future) as Box<_>, addr)
});
Ok((Box::new(stream), addr))
}
Err((listener, addr)) => Err((Upgrade::new(listener, upgrade), addr)),
}
} }
#[inline]
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> {
self.dial(addr)
}
#[inline]
fn nat_traversal(&self, server: &Multiaddr, observed: &Multiaddr) -> Option<Multiaddr> { fn nat_traversal(&self, server: &Multiaddr, observed: &Multiaddr) -> Option<Multiaddr> {
self.transports.nat_traversal(server, observed) self.inner.nat_traversal(server, observed)
} }
} }

View File

@ -19,49 +19,53 @@
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use bytes::Bytes; use bytes::Bytes;
use futures::{prelude::*, future::Either}; use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeError};
use futures::prelude::*;
use multistream_select::{self, DialerSelectFuture, ListenerSelectFuture}; use multistream_select::{self, DialerSelectFuture, ListenerSelectFuture};
use std::{io::{Error as IoError, ErrorKind as IoErrorKind}, mem}; use std::mem;
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use upgrade::{ConnectionUpgrade, Endpoint};
/// Applies a connection upgrade on a socket. pub fn apply_inbound<C, U>(conn: C, up: U) -> InboundUpgradeApply<C, U>
///
/// Returns a `Future` that returns the outcome of the connection upgrade.
#[inline]
pub fn apply<C, U>(conn: C, upgrade: U, e: Endpoint) -> UpgradeApplyFuture<C, U>
where where
U: ConnectionUpgrade<C>,
U::NamesIter: Clone, // TODO: not elegant
C: AsyncRead + AsyncWrite, C: AsyncRead + AsyncWrite,
U: InboundUpgrade<C>,
U::NamesIter: Clone
{ {
UpgradeApplyFuture { let iter = ProtocolNames(up.protocol_names());
inner: UpgradeApplyState::Init { let future = multistream_select::listener_select_proto(conn, iter);
future: negotiate(conn, &upgrade, e), InboundUpgradeApply {
upgrade, inner: InboundUpgradeApplyState::Init { future, upgrade: up }
endpoint: e,
}
} }
} }
/// Future, returned from `apply` which performs a connection upgrade. pub fn apply_outbound<C, U>(conn: C, up: U) -> OutboundUpgradeApply<C, U>
pub struct UpgradeApplyFuture<C, U>
where where
U: ConnectionUpgrade<C>, C: AsyncRead + AsyncWrite,
C: AsyncRead + AsyncWrite U: OutboundUpgrade<C>
{ {
inner: UpgradeApplyState<C, U> let iter = ProtocolNames(up.protocol_names());
let future = multistream_select::dialer_select_proto(conn, iter);
OutboundUpgradeApply {
inner: OutboundUpgradeApplyState::Init { future, upgrade: up }
}
} }
enum UpgradeApplyState<C, U> pub struct InboundUpgradeApply<C, U>
where where
U: ConnectionUpgrade<C>, C: AsyncRead + AsyncWrite,
C: AsyncRead + AsyncWrite U: InboundUpgrade<C>
{
inner: InboundUpgradeApplyState<C, U>
}
enum InboundUpgradeApplyState<C, U>
where
C: AsyncRead + AsyncWrite,
U: InboundUpgrade<C>
{ {
Init { Init {
future: NegotiationFuture<C, ProtocolNames<U::NamesIter>, U::UpgradeIdentifier>, future: ListenerSelectFuture<C, ProtocolNames<U::NamesIter>, U::UpgradeId>,
upgrade: U, upgrade: U
endpoint: Endpoint
}, },
Upgrade { Upgrade {
future: U::Future future: U::Future
@ -69,34 +73,34 @@ where
Undefined Undefined
} }
impl<C, U> Future for UpgradeApplyFuture<C, U> impl<C, U> Future for InboundUpgradeApply<C, U>
where where
U: ConnectionUpgrade<C>, C: AsyncRead + AsyncWrite,
U::NamesIter: Clone, U: InboundUpgrade<C>,
C: AsyncRead + AsyncWrite U::NamesIter: Clone
{ {
type Item = U::Output; type Item = U::Output;
type Error = IoError; type Error = UpgradeError<U::Error>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
loop { loop {
match mem::replace(&mut self.inner, UpgradeApplyState::Undefined) { match mem::replace(&mut self.inner, InboundUpgradeApplyState::Undefined) {
UpgradeApplyState::Init { mut future, upgrade, endpoint } => { InboundUpgradeApplyState::Init { mut future, upgrade } => {
let (upgrade_id, connection) = match future.poll()? { let (upgrade_id, connection) = match future.poll()? {
Async::Ready(x) => x, Async::Ready(x) => x,
Async::NotReady => { Async::NotReady => {
self.inner = UpgradeApplyState::Init { future, upgrade, endpoint }; self.inner = InboundUpgradeApplyState::Init { future, upgrade };
return Ok(Async::NotReady) return Ok(Async::NotReady)
} }
}; };
self.inner = UpgradeApplyState::Upgrade { self.inner = InboundUpgradeApplyState::Upgrade {
future: upgrade.upgrade(connection, upgrade_id, endpoint) future: upgrade.upgrade_inbound(connection, upgrade_id)
}; };
} }
UpgradeApplyState::Upgrade { mut future } => { InboundUpgradeApplyState::Upgrade { mut future } => {
match future.poll() { match future.poll() {
Ok(Async::NotReady) => { Ok(Async::NotReady) => {
self.inner = UpgradeApplyState::Upgrade { future }; self.inner = InboundUpgradeApplyState::Upgrade { future };
return Ok(Async::NotReady) return Ok(Async::NotReady)
} }
Ok(Async::Ready(x)) => { Ok(Async::Ready(x)) => {
@ -104,68 +108,82 @@ where
return Ok(Async::Ready(x)) return Ok(Async::Ready(x))
} }
Err(e) => { Err(e) => {
debug!("Failed to apply negotiated protocol: {:?}", e); debug!("Failed to apply negotiated protocol");
return Err(e) return Err(UpgradeError::Apply(e))
} }
} }
} }
UpgradeApplyState::Undefined => InboundUpgradeApplyState::Undefined =>
panic!("UpgradeApplyState::poll called after completion") panic!("InboundUpgradeApplyState::poll called after completion")
} }
} }
} }
} }
pub struct OutboundUpgradeApply<C, U>
/// Negotiates a protocol on a stream.
///
/// Returns a `Future` that returns the negotiated protocol and the stream.
#[inline]
pub fn negotiate<C, I, U>(
connection: C,
upgrade: &U,
endpoint: Endpoint,
) -> NegotiationFuture<C, ProtocolNames<U::NamesIter>, U::UpgradeIdentifier>
where where
U: ConnectionUpgrade<I>,
U::NamesIter: Clone, // TODO: not elegant
C: AsyncRead + AsyncWrite, C: AsyncRead + AsyncWrite,
U: OutboundUpgrade<C>
{ {
debug!("Starting protocol negotiation"); inner: OutboundUpgradeApplyState<C, U>
let iter = ProtocolNames(upgrade.protocol_names());
NegotiationFuture {
inner: match endpoint {
Endpoint::Listener => Either::A(multistream_select::listener_select_proto(connection, iter)),
Endpoint::Dialer => Either::B(multistream_select::dialer_select_proto(connection, iter)),
}
}
} }
/// Future, returned by `negotiate`, which negotiates a protocol and stream. enum OutboundUpgradeApplyState<C, U>
pub struct NegotiationFuture<R: AsyncRead + AsyncWrite, I, P> {
inner: Either<ListenerSelectFuture<R, I, P>, DialerSelectFuture<R, I, P>>
}
impl<R, I, M, P> Future for NegotiationFuture<R, I, P>
where where
R: AsyncRead + AsyncWrite, C: AsyncRead + AsyncWrite,
I: Iterator<Item=(Bytes, M, P)> + Clone, U: OutboundUpgrade<C>
M: FnMut(&Bytes, &Bytes) -> bool,
{ {
type Item = (P, R); Init {
type Error = IoError; future: DialerSelectFuture<C, ProtocolNames<U::NamesIter>, U::UpgradeId>,
upgrade: U
},
Upgrade {
future: U::Future
},
Undefined
}
impl<C, U> Future for OutboundUpgradeApply<C, U>
where
C: AsyncRead + AsyncWrite,
U: OutboundUpgrade<C>
{
type Item = U::Output;
type Error = UpgradeError<U::Error>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
match self.inner.poll() { loop {
Ok(Async::NotReady) => Ok(Async::NotReady), match mem::replace(&mut self.inner, OutboundUpgradeApplyState::Undefined) {
OutboundUpgradeApplyState::Init { mut future, upgrade } => {
let (upgrade_id, connection) = match future.poll()? {
Async::Ready(x) => x,
Async::NotReady => {
self.inner = OutboundUpgradeApplyState::Init { future, upgrade };
return Ok(Async::NotReady)
}
};
self.inner = OutboundUpgradeApplyState::Upgrade {
future: upgrade.upgrade_outbound(connection, upgrade_id)
};
}
OutboundUpgradeApplyState::Upgrade { mut future } => {
match future.poll() {
Ok(Async::NotReady) => {
self.inner = OutboundUpgradeApplyState::Upgrade { future };
return Ok(Async::NotReady)
}
Ok(Async::Ready(x)) => { Ok(Async::Ready(x)) => {
debug!("Successfully negotiated protocol upgrade"); debug!("Successfully applied negotiated protocol");
Ok(Async::Ready(x)) return Ok(Async::Ready(x))
} }
Err(e) => { Err(e) => {
let err = IoError::new(IoErrorKind::Other, e); debug!("Failed to apply negotiated protocol");
debug!("Error while negotiated protocol upgrade: {:?}", err); return Err(UpgradeError::Apply(e))
Err(err) }
}
}
OutboundUpgradeApplyState::Undefined =>
panic!("OutboundUpgradeApplyState::poll called after completion")
} }
} }
} }
@ -191,5 +209,3 @@ where
self.0.size_hint() self.0.size_hint()
} }
} }

View File

@ -1,122 +0,0 @@
// Copyright 2017 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use bytes::Bytes;
use futures::future;
use tokio_io::{AsyncRead, AsyncWrite};
use upgrade::{ConnectionUpgrade, Endpoint};
/// Builds a new `ConnectionUpgrade` that chooses between `A` and `B`.
///
/// If both `A` and `B` are supported by the remote, then `A` will be chosen.
// TODO: write a test for this ^
#[inline]
pub fn or<A, B>(me: A, other: B) -> OrUpgrade<A, B> {
OrUpgrade(me, other)
}
/// See `upgrade::or`.
#[derive(Debug, Copy, Clone)]
pub struct OrUpgrade<A, B>(A, B);
impl<C, A, B, O> ConnectionUpgrade<C> for OrUpgrade<A, B>
where
C: AsyncRead + AsyncWrite,
A: ConnectionUpgrade<C, Output = O>,
B: ConnectionUpgrade<C, Output = O>,
{
type NamesIter = NamesIterChain<A::NamesIter, B::NamesIter>;
type UpgradeIdentifier = EitherUpgradeIdentifier<A::UpgradeIdentifier, B::UpgradeIdentifier>;
#[inline]
fn protocol_names(&self) -> Self::NamesIter {
NamesIterChain {
first: self.0.protocol_names(),
second: self.1.protocol_names(),
}
}
type Output = O;
type Future = future::Either<A::Future, B::Future>;
#[inline]
fn upgrade(
self,
socket: C,
id: Self::UpgradeIdentifier,
ty: Endpoint,
) -> Self::Future {
match id {
EitherUpgradeIdentifier::First(id) => {
future::Either::A(self.0.upgrade(socket, id, ty))
}
EitherUpgradeIdentifier::Second(id) => {
future::Either::B(self.1.upgrade(socket, id, ty))
}
}
}
}
/// Internal struct used by the `OrUpgrade` trait.
#[derive(Debug, Copy, Clone)]
pub enum EitherUpgradeIdentifier<A, B> {
First(A),
Second(B),
}
/// Internal type used by the `OrUpgrade` struct.
///
/// > **Note**: This type is needed because of the lack of `-> impl Trait` in Rust. It can be
/// > removed eventually.
#[derive(Debug, Copy, Clone)]
pub struct NamesIterChain<A, B> {
first: A,
second: B,
}
impl<A, B, AId, BId> Iterator for NamesIterChain<A, B>
where
A: Iterator<Item = (Bytes, AId)>,
B: Iterator<Item = (Bytes, BId)>,
{
type Item = (Bytes, EitherUpgradeIdentifier<AId, BId>);
#[inline]
fn next(&mut self) -> Option<Self::Item> {
if let Some((name, id)) = self.first.next() {
return Some((name, EitherUpgradeIdentifier::First(id)));
}
if let Some((name, id)) = self.second.next() {
return Some((name, EitherUpgradeIdentifier::Second(id)));
}
None
}
#[inline]
fn size_hint(&self) -> (usize, Option<usize>) {
let (min1, max1) = self.first.size_hint();
let (min2, max2) = self.second.size_hint();
let max = match (max1, max2) {
(Some(max1), Some(max2)) => max1.checked_add(max2),
_ => None,
};
(min1.saturating_add(min2), max)
}
}

View File

@ -1,4 +1,4 @@
// Copyright 2017 Parity Technologies (UK) Ltd. // Copyright 2017-2018 Parity Technologies (UK) Ltd.
// //
// Permission is hereby granted, free of charge, to any person obtaining a // Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"), // copy of this software and associated documentation files (the "Software"),
@ -19,31 +19,40 @@
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use bytes::Bytes; use bytes::Bytes;
use futures::prelude::*; use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
use std::{io, iter}; use futures::future::FutureResult;
use tokio_io::{AsyncRead, AsyncWrite}; use std::iter;
use upgrade::{ConnectionUpgrade, Endpoint}; use void::{unreachable, Void};
/// Implementation of `ConnectionUpgrade` that always fails to negotiate.
#[derive(Debug, Copy, Clone)] #[derive(Debug, Copy, Clone)]
pub struct DeniedConnectionUpgrade; pub struct DeniedUpgrade;
impl<C> ConnectionUpgrade<C> for DeniedConnectionUpgrade impl UpgradeInfo for DeniedUpgrade {
where type UpgradeId = Void;
C: AsyncRead + AsyncWrite, type NamesIter = iter::Empty<(Bytes, Self::UpgradeId)>;
{
type NamesIter = iter::Empty<(Bytes, ())>;
type UpgradeIdentifier = (); // TODO: could use `!`
type Output = (); // TODO: could use `!`
type Future = Box<Future<Item = (), Error = io::Error> + Send + Sync>; // TODO: could use `!`
#[inline]
fn protocol_names(&self) -> Self::NamesIter { fn protocol_names(&self) -> Self::NamesIter {
iter::empty() iter::empty()
} }
}
#[inline] impl<C> InboundUpgrade<C> for DeniedUpgrade {
fn upgrade(self, _: C, _: Self::UpgradeIdentifier, _: Endpoint) -> Self::Future { type Output = Void;
unreachable!("the denied connection upgrade always fails to negotiate") type Error = Void;
type Future = FutureResult<Self::Output, Self::Error>;
fn upgrade_inbound(self, _: C, id: Self::UpgradeId) -> Self::Future {
unreachable(id)
} }
} }
impl<C> OutboundUpgrade<C> for DeniedUpgrade {
type Output = Void;
type Error = Void;
type Future = FutureResult<Self::Output, Self::Error>;
fn upgrade_outbound(self, _: C, id: Self::UpgradeId) -> Self::Future {
unreachable(id)
}
}

92
core/src/upgrade/error.rs Normal file
View File

@ -0,0 +1,92 @@
// Copyright 2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use multistream_select::ProtocolChoiceError;
use std::fmt;
#[derive(Debug)]
pub enum UpgradeError<E> {
Select(ProtocolChoiceError),
Apply(E),
#[doc(hidden)]
__Nonexhaustive
}
impl<E> UpgradeError<E>
where
E: std::error::Error + Send + Sync + 'static
{
pub fn into_io_error(self) -> std::io::Error {
std::io::Error::new(std::io::ErrorKind::Other, self)
}
}
impl<E> UpgradeError<E> {
pub fn map_err<F, T>(self, f: F) -> UpgradeError<T>
where
F: FnOnce(E) -> T
{
match self {
UpgradeError::Select(e) => UpgradeError::Select(e),
UpgradeError::Apply(e) => UpgradeError::Apply(f(e)),
UpgradeError::__Nonexhaustive => UpgradeError::__Nonexhaustive
}
}
pub fn from_err<T>(self) -> UpgradeError<T>
where
T: From<E>
{
self.map_err(Into::into)
}
}
impl<E> fmt::Display for UpgradeError<E>
where
E: fmt::Display
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
UpgradeError::Select(e) => write!(f, "select error: {}", e),
UpgradeError::Apply(e) => write!(f, "upgrade apply error: {}", e),
UpgradeError::__Nonexhaustive => f.write_str("__Nonexhaustive")
}
}
}
impl<E> std::error::Error for UpgradeError<E>
where
E: std::error::Error
{
fn cause(&self) -> Option<&dyn std::error::Error> {
match self {
UpgradeError::Select(e) => Some(e),
UpgradeError::Apply(e) => Some(e),
UpgradeError::__Nonexhaustive => None
}
}
}
impl<E> From<ProtocolChoiceError> for UpgradeError<E> {
fn from(e: ProtocolChoiceError) -> Self {
UpgradeError::Select(e)
}
}

View File

@ -1,138 +0,0 @@
// Copyright 2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use futures::{future, future::Loop as FutLoop, prelude::*};
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use tokio_io::{AsyncRead, AsyncWrite};
use upgrade::{negotiate, ConnectionUpgrade, Endpoint};
/// Looping connection upgrade.
///
/// Applies a modifier around a `ConnectionUpgrade`.
/// The `ConnectionUpgrade` is expected to produce a `Loop`. If upgrading produces
/// `Loop::Continue`, then the protocol will be negotiated again on the returned stream.
/// If upgrading produces `Loop::Break`, then the loop will stop.
///
/// This is useful for upgrades that produce a stream over which you want to negotiate a protocol.
///
/// Note that there is a maximum number of looping after which a runtime error is produced, in
/// order to avoid DoS attacks if your code happens to be wrong.
#[inline]
pub fn loop_upg<U>(inner: U) -> LoopUpg<U> {
LoopUpg { inner }
}
/// Maximum number of loops after which a runtime error is produced.
pub const MAX_LOOPS: u32 = 64;
/// See the documentation of `loop_upg`.
pub enum Loop<State, Socket, Final> {
/// Looping should continue. `Socket` must implement `AsyncRead` and `AsyncWrite`, and will
/// be used to continue negotiating a protocol. `State` is passed around and can contain
/// anything.
Continue(State, Socket),
/// Stop looping. `Final` is the output of the `loop_upg`.
Break(Final),
}
/// Looping connection upgrade.
///
/// See the documentation of `loop_upg`.
#[derive(Debug, Copy, Clone)]
pub struct LoopUpg<Inner> {
inner: Inner,
}
// TODO: 'static :-/
impl<State, Socket, Inner, Out> ConnectionUpgrade<(State, Socket)>
for LoopUpg<Inner>
where
State: Send + 'static,
Socket: AsyncRead + AsyncWrite + Send + 'static,
Inner: ConnectionUpgrade<
(State, Socket),
Output = Loop<State, Socket, Out>,
> + Clone
+ Send
+ 'static,
Inner::NamesIter: Clone + Send + 'static,
Inner::UpgradeIdentifier: Send,
Inner::Future: Send,
Out: Send + 'static,
{
type NamesIter = Inner::NamesIter;
type UpgradeIdentifier = Inner::UpgradeIdentifier;
fn protocol_names(&self) -> Self::NamesIter {
self.inner.protocol_names()
}
type Output = Out;
type Future = Box<Future<Item = Out, Error = IoError> + Send>;
fn upgrade(
self,
(state, socket): (State, Socket),
id: Self::UpgradeIdentifier,
endpoint: Endpoint,
) -> Self::Future {
let inner = self.inner;
let fut = future::loop_fn(
(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)
.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 {
return future::Either::B(future::err(IoError::new(
IoErrorKind::Other,
"protocol negotiation maximum recursion limit reached",
)));
}
let nego = negotiate(socket, &inner, endpoint);
let fut = nego.map(move |(id, socket)| {
FutLoop::Continue((
state,
socket,
id,
loops_remaining - 1,
))
});
future::Either::A(fut)
}
Loop::Break(fin) => {
future::Either::B(future::ok(FutLoop::Break(fin)))
}
})
},
);
Box::new(fut) as Box<_>
}
}

View File

@ -1,4 +1,4 @@
// Copyright 2017 Parity Technologies (UK) Ltd. // Copyright 2018 Parity Technologies (UK) Ltd.
// //
// Permission is hereby granted, free of charge, to any person obtaining a // Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"), // copy of this software and associated documentation files (the "Software"),
@ -19,47 +19,114 @@
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use futures::prelude::*; use futures::prelude::*;
use tokio_io::{AsyncRead, AsyncWrite}; use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
use upgrade::{ConnectionUpgrade, Endpoint};
/// Applies a closure on the output of a connection upgrade. #[derive(Debug, Clone)]
#[inline] pub struct MapUpgrade<U, F> { upgrade: U, fun: F }
pub fn map<U, F>(upgrade: U, map: F) -> Map<U, F> {
Map { upgrade, map } impl<U, F> MapUpgrade<U, F> {
pub fn new(upgrade: U, fun: F) -> Self {
MapUpgrade { upgrade, fun }
}
} }
/// Application of a closure on the output of a connection upgrade. impl<U, F> UpgradeInfo for MapUpgrade<U, F>
#[derive(Debug, Copy, Clone)]
pub struct Map<U, F> {
upgrade: U,
map: F,
}
impl<C, U, F, O> ConnectionUpgrade<C> for Map<U, F>
where where
U: ConnectionUpgrade<C>, U: UpgradeInfo
C: AsyncRead + AsyncWrite,
F: FnOnce(U::Output) -> O,
{ {
type UpgradeId = U::UpgradeId;
type NamesIter = U::NamesIter; type NamesIter = U::NamesIter;
type UpgradeIdentifier = U::UpgradeIdentifier;
fn protocol_names(&self) -> Self::NamesIter { fn protocol_names(&self) -> Self::NamesIter {
self.upgrade.protocol_names() self.upgrade.protocol_names()
} }
}
type Output = O; impl<C, U, F, T> InboundUpgrade<C> for MapUpgrade<U, F>
where
U: InboundUpgrade<C>,
F: FnOnce(U::Output) -> T
{
type Output = T;
type Error = U::Error;
type Future = MapFuture<U::Future, F>; type Future = MapFuture<U::Future, F>;
fn upgrade( fn upgrade_inbound(self, sock: C, id: Self::UpgradeId) -> Self::Future {
self,
socket: C,
id: Self::UpgradeIdentifier,
ty: Endpoint,
) -> Self::Future {
MapFuture { MapFuture {
inner: self.upgrade.upgrade(socket, id, ty), inner: self.upgrade.upgrade_inbound(sock, id),
map: Some(self.map), map: Some(self.fun)
}
}
}
impl<C, U, F, T> OutboundUpgrade<C> for MapUpgrade<U, F>
where
U: OutboundUpgrade<C>,
F: FnOnce(U::Output) -> T
{
type Output = T;
type Error = U::Error;
type Future = MapFuture<U::Future, F>;
fn upgrade_outbound(self, sock: C, id: Self::UpgradeId) -> Self::Future {
MapFuture {
inner: self.upgrade.upgrade_outbound(sock, id),
map: Some(self.fun)
}
}
}
#[derive(Debug, Clone)]
pub struct MapUpgradeErr<U, F> { upgrade: U, fun: F }
impl<U, F> MapUpgradeErr<U, F> {
pub fn new(upgrade: U, fun: F) -> Self {
MapUpgradeErr { upgrade, fun }
}
}
impl<U, F> UpgradeInfo for MapUpgradeErr<U, F>
where
U: UpgradeInfo
{
type UpgradeId = U::UpgradeId;
type NamesIter = U::NamesIter;
fn protocol_names(&self) -> Self::NamesIter {
self.upgrade.protocol_names()
}
}
impl<C, U, F, T> InboundUpgrade<C> for MapUpgradeErr<U, F>
where
U: InboundUpgrade<C>,
F: FnOnce(U::Error) -> T
{
type Output = U::Output;
type Error = T;
type Future = MapErrFuture<U::Future, F>;
fn upgrade_inbound(self, sock: C, id: Self::UpgradeId) -> Self::Future {
MapErrFuture {
fut: self.upgrade.upgrade_inbound(sock, id),
fun: Some(self.fun)
}
}
}
impl<C, U, F, T> OutboundUpgrade<C> for MapUpgradeErr<U, F>
where
U: OutboundUpgrade<C>,
F: FnOnce(U::Error) -> T,
{
type Output = U::Output;
type Error = T;
type Future = MapErrFuture<U::Future, F>;
fn upgrade_outbound(self, sock: C, id: Self::UpgradeId) -> Self::Future {
MapErrFuture {
fut: self.upgrade.upgrade_outbound(sock, id),
fun: Some(self.fun)
} }
} }
} }
@ -70,7 +137,8 @@ pub struct MapFuture<TInnerFut, TMap> {
} }
impl<TInnerFut, TIn, TMap, TOut> Future for MapFuture<TInnerFut, TMap> impl<TInnerFut, TIn, TMap, TOut> Future for MapFuture<TInnerFut, TMap>
where TInnerFut: Future<Item = TIn>, where
TInnerFut: Future<Item = TIn>,
TMap: FnOnce(TIn) -> TOut, TMap: FnOnce(TIn) -> TOut,
{ {
type Item = TOut; type Item = TOut;
@ -82,3 +150,29 @@ where TInnerFut: Future<Item = TIn>,
Ok(Async::Ready(map(item))) Ok(Async::Ready(map(item)))
} }
} }
pub struct MapErrFuture<T, F> {
fut: T,
fun: Option<F>,
}
impl<T, E, F, A> Future for MapErrFuture<T, F>
where
T: Future<Error = E>,
F: FnOnce(E) -> A,
{
type Item = T::Item;
type Error = A;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
match self.fut.poll() {
Ok(Async::NotReady) => Ok(Async::NotReady),
Ok(Async::Ready(x)) => Ok(Async::Ready(x)),
Err(e) => {
let f = self.fun.take().expect("Future has not resolved yet");
Err(f(e))
}
}
}
}

View File

@ -1,4 +1,4 @@
// Copyright 2017 Parity Technologies (UK) Ltd. // Copyright 2018 Parity Technologies (UK) Ltd.
// //
// Permission is hereby granted, free of charge, to any person obtaining a // Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"), // copy of this software and associated documentation files (the "Software"),
@ -18,20 +18,101 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
pub mod apply; mod apply;
pub mod choice; mod denied;
pub mod denied; mod error;
pub mod loop_upg; mod map;
pub mod map; mod or;
pub mod plaintext; mod toggleable;
pub mod toggleable;
pub mod traits; use bytes::Bytes;
use futures::future::Future;
pub use self::{
apply::{apply_inbound, apply_outbound, InboundUpgradeApply, OutboundUpgradeApply},
denied::DeniedUpgrade,
error::UpgradeError,
map::{MapUpgrade, MapUpgradeErr},
or::OrUpgrade,
toggleable::{toggleable, Toggleable}
};
pub trait UpgradeInfo {
type UpgradeId;
type NamesIter: Iterator<Item = (Bytes, Self::UpgradeId)>;
fn protocol_names(&self) -> Self::NamesIter;
}
pub trait InboundUpgrade<C>: UpgradeInfo {
type Output;
type Error;
type Future: Future<Item = Self::Output, Error = Self::Error>;
fn upgrade_inbound(self, socket: C, id: Self::UpgradeId) -> Self::Future;
}
pub trait InboundUpgradeExt<C>: InboundUpgrade<C> {
fn map_inbound<F, T>(self, f: F) -> MapUpgrade<Self, F>
where
Self: Sized,
F: FnOnce(Self::Output) -> T
{
MapUpgrade::new(self, f)
}
fn map_inbound_err<F, T>(self, f: F) -> MapUpgradeErr<Self, F>
where
Self: Sized,
F: FnOnce(Self::Error) -> T
{
MapUpgradeErr::new(self, f)
}
fn or_inbound<U>(self, upgrade: U) -> OrUpgrade<Self, U>
where
Self: Sized,
U: InboundUpgrade<C, Output = Self::Output, Error = Self::Error>
{
OrUpgrade::new(self, upgrade)
}
}
impl<C, U: InboundUpgrade<C>> InboundUpgradeExt<C> for U {}
pub trait OutboundUpgrade<C>: UpgradeInfo {
type Output;
type Error;
type Future: Future<Item = Self::Output, Error = Self::Error>;
fn upgrade_outbound(self, socket: C, id: Self::UpgradeId) -> Self::Future;
}
pub trait OutboundUpgradeExt<C>: OutboundUpgrade<C> {
fn map_outbound<F, T>(self, f: F) -> MapUpgrade<Self, F>
where
Self: Sized,
F: FnOnce(Self::Output) -> T
{
MapUpgrade::new(self, f)
}
fn map_outbound_err<F, T>(self, f: F) -> MapUpgradeErr<Self, F>
where
Self: Sized,
F: FnOnce(Self::Error) -> T
{
MapUpgradeErr::new(self, f)
}
fn or_outbound<U>(self, upgrade: U) -> OrUpgrade<Self, U>
where
Self: Sized,
U: OutboundUpgrade<C, Output = Self::Output, Error = Self::Error>
{
OrUpgrade::new(self, upgrade)
}
}
impl<C, U: OutboundUpgrade<C>> OutboundUpgradeExt<C> for U {}
pub use self::apply::{apply, negotiate};
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::plaintext::PlainTextConfig;
pub use self::toggleable::toggleable;
pub use self::traits::{ConnectionUpgrade, Endpoint};

108
core/src/upgrade/or.rs Normal file
View File

@ -0,0 +1,108 @@
// Copyright 2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use bytes::Bytes;
use futures::future::Either;
use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
#[derive(Debug, Clone)]
pub struct OrUpgrade<A, B>(A, B);
impl<A, B> OrUpgrade<A, B> {
pub fn new(a: A, b: B) -> Self {
OrUpgrade(a, b)
}
}
impl<A, B> UpgradeInfo for OrUpgrade<A, B>
where
A: UpgradeInfo,
B: UpgradeInfo
{
type UpgradeId = Either<A::UpgradeId, B::UpgradeId>;
type NamesIter = NamesIterChain<A::NamesIter, B::NamesIter>;
fn protocol_names(&self) -> Self::NamesIter {
NamesIterChain(self.0.protocol_names(), self.1.protocol_names())
}
}
impl<C, A, B, T, E> InboundUpgrade<C> for OrUpgrade<A, B>
where
A: InboundUpgrade<C, Output = T, Error = E>,
B: InboundUpgrade<C, Output = T, Error = E>,
{
type Output = T; // TODO: different output types
type Error = E; // TODO: different error types
type Future = Either<A::Future, B::Future>;
fn upgrade_inbound(self, sock: C, id: Self::UpgradeId) -> Self::Future {
match id {
Either::A(id) => Either::A(self.0.upgrade_inbound(sock, id)),
Either::B(id) => Either::B(self.1.upgrade_inbound(sock, id))
}
}
}
impl<C, A, B, T, E> OutboundUpgrade<C> for OrUpgrade<A, B>
where
A: OutboundUpgrade<C, Output = T, Error = E>,
B: OutboundUpgrade<C, Output = T, Error = E>,
{
type Output = T; // TODO: different output types
type Error = E; // TODO: different error types
type Future = Either<A::Future, B::Future>;
fn upgrade_outbound(self, sock: C, id: Self::UpgradeId) -> Self::Future {
match id {
Either::A(id) => Either::A(self.0.upgrade_outbound(sock, id)),
Either::B(id) => Either::B(self.1.upgrade_outbound(sock, id))
}
}
}
#[derive(Debug, Clone)]
pub struct NamesIterChain<A, B>(A, B);
impl<A, B, AId, BId> Iterator for NamesIterChain<A, B>
where
A: Iterator<Item = (Bytes, AId)>,
B: Iterator<Item = (Bytes, BId)>,
{
type Item = (Bytes, Either<AId, BId>);
fn next(&mut self) -> Option<Self::Item> {
if let Some((name, id)) = self.0.next() {
return Some((name, Either::A(id)))
}
if let Some((name, id)) = self.1.next() {
return Some((name, Either::B(id)))
}
None
}
fn size_hint(&self) -> (usize, Option<usize>) {
let (min1, max1) = self.0.size_hint();
let (min2, max2) = self.1.size_hint();
let max = max1.and_then(move |m1| max2.and_then(move |m2| m1.checked_add(m2)));
(min1.saturating_add(min2), max)
}
}

View File

@ -18,12 +18,11 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
use futures::future; use futures::future;
use std::io::Error as IoError;
use tokio_io::{AsyncRead, AsyncWrite};
use upgrade::{ConnectionUpgrade, Endpoint};
/// Wraps around a `ConnectionUpgrade` and makes it possible to enable or disable an upgrade. /// Wraps around a `InboundUpgrade` or `OutboundUpgrade` and makes it possible
/// to enable or disable the upgrade.
#[inline] #[inline]
pub fn toggleable<U>(upgrade: U) -> Toggleable<U> { pub fn toggleable<U>(upgrade: U) -> Toggleable<U> {
Toggleable { Toggleable {
@ -32,7 +31,7 @@ pub fn toggleable<U>(upgrade: U) -> Toggleable<U> {
} }
} }
/// See `upgrade::toggleable`. /// See `toggleable`.
#[derive(Debug, Copy, Clone)] #[derive(Debug, Copy, Clone)]
pub struct Toggleable<U> { pub struct Toggleable<U> {
inner: U, inner: U,
@ -65,13 +64,12 @@ impl<U> Toggleable<U> {
} }
} }
impl<C, U> ConnectionUpgrade<C> for Toggleable<U> impl<U> UpgradeInfo for Toggleable<U>
where where
C: AsyncRead + AsyncWrite, U: UpgradeInfo
U: ConnectionUpgrade<C>,
{ {
type UpgradeId = U::UpgradeId;
type NamesIter = ToggleableIter<U::NamesIter>; type NamesIter = ToggleableIter<U::NamesIter>;
type UpgradeIdentifier = U::UpgradeIdentifier;
#[inline] #[inline]
fn protocol_names(&self) -> Self::NamesIter { fn protocol_names(&self) -> Self::NamesIter {
@ -80,19 +78,38 @@ where
enabled: self.enabled, enabled: self.enabled,
} }
} }
}
impl<C, U> InboundUpgrade<C> for Toggleable<U>
where
U: InboundUpgrade<C>
{
type Output = U::Output; type Output = U::Output;
type Future = future::Either<future::Empty<U::Output, IoError>, U::Future>; type Error = U::Error;
type Future = future::Either<future::Empty<Self::Output, Self::Error>, U::Future>;
#[inline] #[inline]
fn upgrade( fn upgrade_inbound(self, socket: C, id: Self::UpgradeId) -> Self::Future {
self,
socket: C,
id: Self::UpgradeIdentifier,
ty: Endpoint,
) -> Self::Future {
if self.enabled { if self.enabled {
future::Either::B(self.inner.upgrade(socket, id, ty)) future::Either::B(self.inner.upgrade_inbound(socket, id))
} else {
future::Either::A(future::empty())
}
}
}
impl<C, U> OutboundUpgrade<C> for Toggleable<U>
where
U: OutboundUpgrade<C>
{
type Output = U::Output;
type Error = U::Error;
type Future = future::Either<future::Empty<Self::Output, Self::Error>, U::Future>;
#[inline]
fn upgrade_outbound(self, socket: C, id: Self::UpgradeId) -> Self::Future {
if self.enabled {
future::Either::B(self.inner.upgrade_outbound(socket, id))
} else { } else {
future::Either::A(future::empty()) future::Either::A(future::empty())
} }
@ -130,4 +147,7 @@ where I: Iterator
} }
impl<I> ExactSizeIterator for ToggleableIter<I> impl<I> ExactSizeIterator for ToggleableIter<I>
where I: ExactSizeIterator {} where
I: ExactSizeIterator
{}

View File

@ -1,79 +0,0 @@
// Copyright 2017 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use bytes::Bytes;
use futures::future::Future;
use std::{io::Error as IoError, ops::Not};
/// Type of connection for the upgrade.
#[derive(Debug, Copy, Clone, PartialEq, Eq, Hash)]
pub enum Endpoint {
/// The socket comes from a dialer.
Dialer,
/// The socket comes from a listener.
Listener,
}
impl Not for Endpoint {
type Output = Endpoint;
fn not(self) -> Self::Output {
match self {
Endpoint::Dialer => Endpoint::Listener,
Endpoint::Listener => Endpoint::Dialer
}
}
}
/// Implemented on structs that describe a possible upgrade to a connection between two peers.
///
/// The generic `C` is the type of the incoming connection before it is upgraded.
///
/// > **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<C> {
/// Iterator returned by `protocol_names`.
type NamesIter: Iterator<Item = (Bytes, Self::UpgradeIdentifier)>;
/// Type that serves as an identifier for the protocol. This type only exists to be returned
/// by the `NamesIter` and then be passed to `upgrade`.
///
/// This is only useful on implementations that dispatch between multiple possible upgrades.
/// Any basic implementation will probably just use the `()` type.
type UpgradeIdentifier;
/// Returns the name of the protocols to advertise to the remote.
fn protocol_names(&self) -> Self::NamesIter;
/// Type of the stream that has been upgraded. Generally wraps around `C` and `Self`.
///
/// > **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 `Self::Output`.
type Future: Future<Item = Self::Output, Error = IoError>;
/// This method is called after protocol negotiation has been performed.
///
/// Because performing the upgrade may not be instantaneous (e.g. it may require a handshake),
/// this function returns a future instead of the direct output.
fn upgrade(self, socket: C, id: Self::UpgradeIdentifier, ty: Endpoint) -> Self::Future;
}

View File

@ -68,29 +68,44 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream {
quote!{#n} quote!{#n}
}; };
let output_types = {
let mut start = 1;
// Avoid collisions.
while ast.generics.type_params().any(|tp| tp.ident.to_string() == format!("TOut{}", start)) {
start += 1;
}
data_struct.fields.iter()
.filter(|x| !is_ignored(x))
.enumerate()
.map(move |(i, _)| Ident::new(&format!("TOut{}", start + i), name.span()))
.collect::<Vec<_>>()
};
// Build the generics. // Build the generics.
let impl_generics = { let impl_generics = {
let tp = ast.generics.type_params(); let tp = ast.generics.type_params();
let lf = ast.generics.lifetimes(); let lf = ast.generics.lifetimes();
let cst = ast.generics.const_params(); let cst = ast.generics.const_params();
quote!{<#(#lf,)* #(#tp,)* #(#cst,)* #substream_generic>} let out = output_types.clone();
quote!{<#(#lf,)* #(#tp,)* #(#cst,)* #substream_generic, #(#out),*>}
}; };
// Build the `where ...` clause of the trait implementation. // Build the `where ...` clause of the trait implementation.
let where_clause = { let where_clause = {
let mut additional = data_struct.fields.iter().flat_map(|field| { let mut additional = data_struct.fields.iter()
if is_ignored(&field) { .filter(|x| !is_ignored(x))
return vec![]; .zip(output_types)
} .flat_map(|(field, out)| {
let ty = &field.ty; let ty = &field.ty;
vec![ vec![
quote!{#ty: #trait_to_impl}, quote!{#ty: #trait_to_impl},
quote!{<#ty as #trait_to_impl>::ProtocolsHandler: #protocols_handler<Substream = #substream_generic>}, quote!{<#ty as #trait_to_impl>::ProtocolsHandler: #protocols_handler<Substream = #substream_generic>},
// Note: this bound is required because of https://github.com/rust-lang/rust/issues/55697 // Note: this bound is required because of https://github.com/rust-lang/rust/issues/55697
quote!{<<#ty as #trait_to_impl>::ProtocolsHandler as #protocols_handler>::Protocol: ::libp2p::core::ConnectionUpgrade<#substream_generic>}, quote!{<<#ty as #trait_to_impl>::ProtocolsHandler as #protocols_handler>::InboundProtocol: ::libp2p::core::InboundUpgrade<#substream_generic, Output = #out>},
quote!{<<#ty as #trait_to_impl>::ProtocolsHandler as #protocols_handler>::OutboundProtocol: ::libp2p::core::OutboundUpgrade<#substream_generic, Output = #out>},
] ]
}).collect::<Vec<_>>(); })
.collect::<Vec<_>>();
additional.push(quote!{#substream_generic: ::libp2p::tokio_io::AsyncRead}); additional.push(quote!{#substream_generic: ::libp2p::tokio_io::AsyncRead});
additional.push(quote!{#substream_generic: ::libp2p::tokio_io::AsyncWrite}); additional.push(quote!{#substream_generic: ::libp2p::tokio_io::AsyncWrite});

View File

@ -36,7 +36,12 @@ use std::{cmp, iter, mem};
use std::io::{Error as IoError, ErrorKind as IoErrorKind}; use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use std::sync::{atomic::AtomicUsize, atomic::Ordering, Arc}; use std::sync::{atomic::AtomicUsize, atomic::Ordering, Arc};
use bytes::Bytes; use bytes::Bytes;
use core::{ConnectionUpgrade, Endpoint, StreamMuxer, muxing::Shutdown}; use core::{
Endpoint,
StreamMuxer,
muxing::Shutdown,
upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}
};
use parking_lot::Mutex; use parking_lot::Mutex;
use fnv::{FnvHashMap, FnvHashSet}; use fnv::{FnvHashMap, FnvHashSet};
use futures::prelude::*; use futures::prelude::*;
@ -91,6 +96,31 @@ impl MplexConfig {
self.max_buffer_behaviour = behaviour; self.max_buffer_behaviour = behaviour;
self self
} }
#[inline]
fn upgrade<C>(self, i: C, endpoint: Endpoint) -> Multiplex<C>
where
C: AsyncRead + AsyncWrite
{
let max_buffer_len = self.max_buffer_len;
Multiplex {
inner: Mutex::new(MultiplexInner {
error: Ok(()),
inner: executor::spawn(Framed::new(i, codec::Codec::new()).fuse()),
config: self,
buffer: Vec::with_capacity(cmp::min(max_buffer_len, 512)),
opened_substreams: Default::default(),
next_outbound_stream_id: if endpoint == Endpoint::Dialer { 0 } else { 1 },
notifier_read: Arc::new(Notifier {
to_notify: Mutex::new(Default::default()),
}),
notifier_write: Arc::new(Notifier {
to_notify: Mutex::new(Default::default()),
}),
is_shutdown: false
})
}
}
} }
impl Default for MplexConfig { impl Default for MplexConfig {
@ -117,39 +147,9 @@ pub enum MaxBufferBehaviour {
Block, Block,
} }
impl<C> ConnectionUpgrade<C> for MplexConfig impl UpgradeInfo for MplexConfig {
where type UpgradeId = ();
C: AsyncRead + AsyncWrite, type NamesIter = iter::Once<(Bytes, Self::UpgradeId)>;
{
type Output = Multiplex<C>;
type Future = future::FutureResult<Self::Output, IoError>;
type UpgradeIdentifier = ();
type NamesIter = iter::Once<(Bytes, ())>;
#[inline]
fn upgrade(self, i: C, _: (), endpoint: Endpoint) -> Self::Future {
let max_buffer_len = self.max_buffer_len;
let out = Multiplex {
inner: Mutex::new(MultiplexInner {
error: Ok(()),
inner: executor::spawn(Framed::new(i, codec::Codec::new()).fuse()),
config: self,
buffer: Vec::with_capacity(cmp::min(max_buffer_len, 512)),
opened_substreams: Default::default(),
next_outbound_stream_id: if endpoint == Endpoint::Dialer { 0 } else { 1 },
notifier_read: Arc::new(Notifier {
to_notify: Mutex::new(Default::default()),
}),
notifier_write: Arc::new(Notifier {
to_notify: Mutex::new(Default::default()),
}),
is_shutdown: false
})
};
future::ok(out)
}
#[inline] #[inline]
fn protocol_names(&self) -> Self::NamesIter { fn protocol_names(&self) -> Self::NamesIter {
@ -157,6 +157,32 @@ where
} }
} }
impl<C> InboundUpgrade<C> for MplexConfig
where
C: AsyncRead + AsyncWrite,
{
type Output = Multiplex<C>;
type Error = IoError;
type Future = future::FutureResult<Self::Output, IoError>;
fn upgrade_inbound(self, socket: C, _: Self::UpgradeId) -> Self::Future {
future::ok(self.upgrade(socket, Endpoint::Listener))
}
}
impl<C> OutboundUpgrade<C> for MplexConfig
where
C: AsyncRead + AsyncWrite,
{
type Output = Multiplex<C>;
type Error = IoError;
type Future = future::FutureResult<Self::Output, IoError>;
fn upgrade_outbound(self, socket: C, _: Self::UpgradeId) -> Self::Future {
future::ok(self.upgrade(socket, Endpoint::Dialer))
}
}
/// Multiplexer. Implements the `StreamMuxer` trait. /// Multiplexer. Implements the `StreamMuxer` trait.
pub struct Multiplex<C> { pub struct Multiplex<C> {
inner: Mutex<MultiplexInner<C>>, inner: Mutex<MultiplexInner<C>>,

View File

@ -22,14 +22,14 @@ extern crate bytes;
extern crate futures; extern crate futures;
#[macro_use] #[macro_use]
extern crate log; extern crate log;
extern crate libp2p_core as core; extern crate libp2p_core;
extern crate parking_lot; extern crate parking_lot;
extern crate tokio_io; extern crate tokio_io;
extern crate yamux; extern crate yamux;
use bytes::Bytes; use bytes::Bytes;
use core::{Endpoint, muxing::Shutdown};
use futures::{future::{self, FutureResult}, prelude::*}; use futures::{future::{self, FutureResult}, prelude::*};
use libp2p_core::{muxing::Shutdown, upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}};
use parking_lot::Mutex; use parking_lot::Mutex;
use std::{io, iter}; use std::{io, iter};
use std::io::{Error as IoError}; use std::io::{Error as IoError};
@ -47,7 +47,7 @@ where
} }
} }
impl<C> core::StreamMuxer for Yamux<C> impl<C> libp2p_core::StreamMuxer for Yamux<C>
where where
C: AsyncRead + AsyncWrite + 'static C: AsyncRead + AsyncWrite + 'static
{ {
@ -134,27 +134,38 @@ impl Default for Config {
} }
} }
impl<C> core::ConnectionUpgrade<C> for Config impl UpgradeInfo for Config {
where type UpgradeId = ();
C: AsyncRead + AsyncWrite + 'static, type NamesIter = iter::Once<(Bytes, Self::UpgradeId)>;
{
type UpgradeIdentifier = ();
type NamesIter = iter::Once<(Bytes, ())>;
fn protocol_names(&self) -> Self::NamesIter { fn protocol_names(&self) -> Self::NamesIter {
iter::once((Bytes::from("/yamux/1.0.0"), ())) iter::once((Bytes::from("/yamux/1.0.0"), ()))
} }
}
impl<C> InboundUpgrade<C> for Config
where
C: AsyncRead + AsyncWrite + 'static,
{
type Output = Yamux<C>; type Output = Yamux<C>;
type Error = io::Error;
type Future = FutureResult<Yamux<C>, io::Error>; type Future = FutureResult<Yamux<C>, io::Error>;
fn upgrade(self, i: C, _: (), end: Endpoint) -> Self::Future { fn upgrade_inbound(self, i: C, _: Self::UpgradeId) -> Self::Future {
let mode = match end { future::ok(Yamux::new(i, self.0, yamux::Mode::Server))
Endpoint::Dialer => yamux::Mode::Client, }
Endpoint::Listener => yamux::Mode::Server }
};
impl<C> OutboundUpgrade<C> for Config
future::ok(Yamux::new(i, self.0, mode)) where
C: AsyncRead + AsyncWrite + 'static,
{
type Output = Yamux<C>;
type Error = io::Error;
type Future = FutureResult<Yamux<C>, io::Error>;
fn upgrade_outbound(self, i: C, _: Self::UpgradeId) -> Self::Future {
future::ok(Yamux::new(i, self.0, yamux::Mode::Client))
} }
} }

View File

@ -18,10 +18,12 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::protocol::{FloodsubCodec, FloodsubConfig, FloodsubRpc};
use futures::prelude::*; use futures::prelude::*;
use libp2p_core::nodes::{NodeHandlerEndpoint, ProtocolsHandler, ProtocolsHandlerEvent}; use libp2p_core::{
use libp2p_core::ConnectionUpgrade; nodes::{ProtocolsHandler, ProtocolsHandlerEvent},
use protocol::{FloodsubCodec, FloodsubConfig, FloodsubRpc}; upgrade::{InboundUpgrade, OutboundUpgrade}
};
use smallvec::SmallVec; use smallvec::SmallVec;
use std::{fmt, io}; use std::{fmt, io};
use tokio_codec::Framed; use tokio_codec::Framed;
@ -103,32 +105,39 @@ where
type InEvent = FloodsubRpc; type InEvent = FloodsubRpc;
type OutEvent = FloodsubRpc; type OutEvent = FloodsubRpc;
type Substream = TSubstream; type Substream = TSubstream;
type Protocol = FloodsubConfig; type InboundProtocol = FloodsubConfig;
type OutboundProtocol = FloodsubConfig;
type OutboundOpenInfo = FloodsubRpc; type OutboundOpenInfo = FloodsubRpc;
#[inline] #[inline]
fn listen_protocol(&self) -> Self::Protocol { fn listen_protocol(&self) -> Self::InboundProtocol {
self.config.clone() self.config.clone()
} }
fn inject_fully_negotiated( #[inline]
&mut self, fn dialer_protocol(&self) -> Self::OutboundProtocol {
protocol: <Self::Protocol as ConnectionUpgrade<TSubstream>>::Output, self.config.clone()
endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>,
) {
if self.shutting_down {
return;
} }
match endpoint { fn inject_fully_negotiated_inbound(
NodeHandlerEndpoint::Dialer(message) => { &mut self,
self.substreams protocol: <Self::InboundProtocol as InboundUpgrade<TSubstream>>::Output
.push(SubstreamState::PendingSend(protocol, message)); ) {
if self.shutting_down {
return ()
} }
NodeHandlerEndpoint::Listener => { self.substreams.push(SubstreamState::WaitingInput(protocol))
self.substreams.push(SubstreamState::WaitingInput(protocol));
} }
fn inject_fully_negotiated_outbound(
&mut self,
protocol: <Self::OutboundProtocol as OutboundUpgrade<TSubstream>>::Output,
message: Self::OutboundOpenInfo
) {
if self.shutting_down {
return ()
} }
self.substreams.push(SubstreamState::PendingSend(protocol, message))
} }
#[inline] #[inline]
@ -154,7 +163,7 @@ where
fn poll( fn poll(
&mut self, &mut self,
) -> Poll< ) -> Poll<
Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>, Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>,
io::Error, io::Error,
> { > {
if !self.send_queue.is_empty() { if !self.send_queue.is_empty() {

View File

@ -19,10 +19,10 @@
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use bytes::{BufMut, Bytes, BytesMut}; use bytes::{BufMut, Bytes, BytesMut};
use crate::rpc_proto;
use futures::future; use futures::future;
use libp2p_core::{ConnectionUpgrade, Endpoint, PeerId}; use libp2p_core::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, PeerId};
use protobuf::Message as ProtobufMessage; use protobuf::Message as ProtobufMessage;
use rpc_proto;
use std::{io, iter}; use std::{io, iter};
use tokio_codec::{Decoder, Encoder, Framed}; use tokio_codec::{Decoder, Encoder, Framed};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
@ -41,29 +41,41 @@ impl FloodsubConfig {
} }
} }
impl<TSocket> ConnectionUpgrade<TSocket> for FloodsubConfig impl UpgradeInfo for FloodsubConfig {
where type UpgradeId = ();
TSocket: AsyncRead + AsyncWrite, type NamesIter = iter::Once<(Bytes, Self::UpgradeId)>;
{
type NamesIter = iter::Once<(Bytes, Self::UpgradeIdentifier)>;
type UpgradeIdentifier = ();
#[inline] #[inline]
fn protocol_names(&self) -> Self::NamesIter { fn protocol_names(&self) -> Self::NamesIter {
iter::once(("/floodsub/1.0.0".into(), ())) iter::once(("/floodsub/1.0.0".into(), ()))
} }
}
impl<TSocket> InboundUpgrade<TSocket> for FloodsubConfig
where
TSocket: AsyncRead + AsyncWrite,
{
type Output = Framed<TSocket, FloodsubCodec>; type Output = Framed<TSocket, FloodsubCodec>;
type Future = future::FutureResult<Self::Output, io::Error>; type Error = io::Error;
type Future = future::FutureResult<Self::Output, Self::Error>;
#[inline] #[inline]
fn upgrade(self, socket: TSocket, _: Self::UpgradeIdentifier, _: Endpoint) -> Self::Future { fn upgrade_inbound(self, socket: TSocket, _: Self::UpgradeId) -> Self::Future {
future::ok(Framed::new( future::ok(Framed::new(socket, FloodsubCodec { length_prefix: Default::default() }))
socket, }
FloodsubCodec { }
length_prefix: Default::default(),
}, impl<TSocket> OutboundUpgrade<TSocket> for FloodsubConfig
)) where
TSocket: AsyncRead + AsyncWrite,
{
type Output = Framed<TSocket, FloodsubCodec>;
type Error = io::Error;
type Future = future::FutureResult<Self::Output, Self::Error>;
#[inline]
fn upgrade_outbound(self, socket: TSocket, _: Self::UpgradeId) -> Self::Future {
future::ok(Framed::new(socket, FloodsubCodec { length_prefix: Default::default() }))
} }
} }

View File

@ -21,8 +21,11 @@
//! Contains the `IdentifyTransport` type. //! Contains the `IdentifyTransport` type.
use futures::prelude::*; use futures::prelude::*;
use libp2p_core::{Endpoint, Multiaddr, PeerId, PublicKey, Transport, muxing, upgrade::apply}; use libp2p_core::{
use protocol::{IdentifyOutput, IdentifyProtocolConfig}; Multiaddr, PeerId, PublicKey, muxing, Transport,
upgrade::{self, OutboundUpgradeApply, UpgradeError}
};
use protocol::{RemoteInfo, IdentifyProtocolConfig};
use std::io::{Error as IoError, ErrorKind as IoErrorKind}; use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use std::mem; use std::mem;
use std::sync::Arc; use std::sync::Arc;
@ -71,26 +74,7 @@ where
#[inline] #[inline]
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> {
let (listener, new_addr) = match self.transport.listen_on(addr) { Err((self, addr))
Ok((l, a)) => (l, a),
Err((inner, addr)) => {
let id = IdentifyTransport {
transport: inner,
};
return Err((id, addr));
}
};
let listener = listener
.map(move |(upgrade, remote_addr)| {
let upgr = upgrade
.and_then(move |muxer| {
IdRetriever::new(muxer, IdentifyProtocolConfig, Endpoint::Listener)
});
(Box::new(upgr) as Box<Future<Item = _, Error = _> + Send>, remote_addr)
});
Ok((Box::new(listener) as Box<_>, new_addr))
} }
#[inline] #[inline]
@ -107,7 +91,7 @@ where
}; };
let dial = dial.and_then(move |muxer| { let dial = dial.and_then(move |muxer| {
IdRetriever::new(muxer, IdentifyProtocolConfig, Endpoint::Dialer) IdRetriever::new(muxer, IdentifyProtocolConfig).map_err(|e| e.into_io_error())
}); });
Ok(Box::new(dial) as Box<_>) Ok(Box::new(dial) as Box<_>)
@ -126,9 +110,7 @@ where TMuxer: muxing::StreamMuxer + Send + Sync + 'static,
TMuxer::Substream: Send, TMuxer::Substream: Send,
{ {
/// Internal state. /// Internal state.
state: IdRetrieverState<TMuxer>, state: IdRetrieverState<TMuxer>
/// Whether we're dialing or listening.
endpoint: Endpoint,
} }
enum IdRetrieverState<TMuxer> enum IdRetrieverState<TMuxer>
@ -138,7 +120,7 @@ where TMuxer: muxing::StreamMuxer + Send + Sync + 'static,
/// We are in the process of opening a substream with the remote. /// We are in the process of opening a substream with the remote.
OpeningSubstream(Arc<TMuxer>, muxing::OutboundSubstreamRefWrapFuture<Arc<TMuxer>>, IdentifyProtocolConfig), OpeningSubstream(Arc<TMuxer>, muxing::OutboundSubstreamRefWrapFuture<Arc<TMuxer>>, IdentifyProtocolConfig),
/// We opened the substream and are currently negotiating the identify protocol. /// We opened the substream and are currently negotiating the identify protocol.
NegotiatingIdentify(Arc<TMuxer>, apply::UpgradeApplyFuture<muxing::SubstreamRef<Arc<TMuxer>>, IdentifyProtocolConfig>), NegotiatingIdentify(Arc<TMuxer>, OutboundUpgradeApply<muxing::SubstreamRef<Arc<TMuxer>>, IdentifyProtocolConfig>),
/// We retreived the remote's public key and are ready to yield it when polled again. /// We retreived the remote's public key and are ready to yield it when polled again.
Finishing(Arc<TMuxer>, PublicKey), Finishing(Arc<TMuxer>, PublicKey),
/// Something bad happend, or the `Future` is finished, and shouldn't be polled again. /// Something bad happend, or the `Future` is finished, and shouldn't be polled again.
@ -150,13 +132,12 @@ where TMuxer: muxing::StreamMuxer + Send + Sync + 'static,
TMuxer::Substream: Send, TMuxer::Substream: Send,
{ {
/// Creates a new `IdRetriever` ready to be polled. /// Creates a new `IdRetriever` ready to be polled.
fn new(muxer: TMuxer, config: IdentifyProtocolConfig, endpoint: Endpoint) -> Self { fn new(muxer: TMuxer, config: IdentifyProtocolConfig) -> Self {
let muxer = Arc::new(muxer); let muxer = Arc::new(muxer);
let opening = muxing::outbound_from_ref_and_wrap(muxer.clone()); let opening = muxing::outbound_from_ref_and_wrap(muxer.clone());
IdRetriever { IdRetriever {
state: IdRetrieverState::OpeningSubstream(muxer, opening, config), state: IdRetrieverState::OpeningSubstream(muxer, opening, config)
endpoint,
} }
} }
} }
@ -166,7 +147,7 @@ where TMuxer: muxing::StreamMuxer + Send + Sync + 'static,
TMuxer::Substream: Send, TMuxer::Substream: Send,
{ {
type Item = (PeerId, TMuxer); type Item = (PeerId, TMuxer);
type Error = IoError; type Error = UpgradeError<IoError>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
// This loop is here so that we can continue polling until we're ready. // This loop is here so that we can continue polling until we're ready.
@ -177,28 +158,24 @@ where TMuxer: muxing::StreamMuxer + Send + Sync + 'static,
IdRetrieverState::OpeningSubstream(muxer, mut opening, config) => { IdRetrieverState::OpeningSubstream(muxer, mut opening, config) => {
match opening.poll() { match opening.poll() {
Ok(Async::Ready(Some(substream))) => { Ok(Async::Ready(Some(substream))) => {
let upgrade = apply::apply(substream, config, self.endpoint); let upgrade = upgrade::apply_outbound(substream, config);
self.state = IdRetrieverState::NegotiatingIdentify(muxer, upgrade); self.state = IdRetrieverState::NegotiatingIdentify(muxer, upgrade)
}, },
Ok(Async::Ready(None)) => { Ok(Async::Ready(None)) => {
return Err(IoError::new(IoErrorKind::Other, "remote refused our identify attempt")); return Err(UpgradeError::Apply(IoError::new(IoErrorKind::Other, "remote refused our identify attempt")))
}, }
Ok(Async::NotReady) => { Ok(Async::NotReady) => {
self.state = IdRetrieverState::OpeningSubstream(muxer, opening, config); self.state = IdRetrieverState::OpeningSubstream(muxer, opening, config);
return Ok(Async::NotReady); return Ok(Async::NotReady);
}, },
Err(err) => return Err(err), Err(err) => return Err(UpgradeError::Apply(err))
} }
}, },
IdRetrieverState::NegotiatingIdentify(muxer, mut nego) => { IdRetrieverState::NegotiatingIdentify(muxer, mut nego) => {
match nego.poll() { match nego.poll() {
Ok(Async::Ready(IdentifyOutput::RemoteInfo { info, .. })) => { Ok(Async::Ready(RemoteInfo { info, .. })) => {
self.state = IdRetrieverState::Finishing(muxer, info.public_key); self.state = IdRetrieverState::Finishing(muxer, info.public_key);
}, },
Ok(Async::Ready(IdentifyOutput::Sender { .. })) => {
unreachable!("IdentifyOutput::Sender can never be the output from \
the dialing side");
},
Ok(Async::NotReady) => { Ok(Async::NotReady) => {
self.state = IdRetrieverState::NegotiatingIdentify(muxer, nego); self.state = IdRetrieverState::NegotiatingIdentify(muxer, nego);
return Ok(Async::NotReady); return Ok(Async::NotReady);

View File

@ -70,7 +70,6 @@ extern crate fnv;
extern crate futures; extern crate futures;
extern crate libp2p_peerstore; extern crate libp2p_peerstore;
extern crate libp2p_core; extern crate libp2p_core;
#[macro_use]
extern crate log; extern crate log;
extern crate multiaddr; extern crate multiaddr;
extern crate parking_lot; extern crate parking_lot;
@ -87,7 +86,7 @@ pub use self::listen_handler::IdentifyListenHandler;
pub use self::listen_layer::IdentifyListen; pub use self::listen_layer::IdentifyListen;
pub use self::periodic_id_handler::{PeriodicIdentification, PeriodicIdentificationEvent}; pub use self::periodic_id_handler::{PeriodicIdentification, PeriodicIdentificationEvent};
pub use self::periodic_id_layer::{PeriodicIdentifyBehaviour, PeriodicIdentifyBehaviourEvent}; pub use self::periodic_id_layer::{PeriodicIdentifyBehaviour, PeriodicIdentifyBehaviourEvent};
pub use self::protocol::{IdentifyInfo, IdentifyOutput}; pub use self::protocol::{IdentifyInfo, RemoteInfo};
pub use self::protocol::{IdentifyProtocolConfig, IdentifySender}; pub use self::protocol::{IdentifyProtocolConfig, IdentifySender};
mod id_transport; mod id_transport;

View File

@ -18,15 +18,16 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::{IdentifySender, IdentifyProtocolConfig};
use futures::prelude::*; use futures::prelude::*;
use libp2p_core::nodes::handled_node::NodeHandlerEndpoint; use libp2p_core::{
use libp2p_core::nodes::protocols_handler::{ProtocolsHandler, ProtocolsHandlerEvent}; nodes::protocols_handler::{ProtocolsHandler, ProtocolsHandlerEvent},
use libp2p_core::ConnectionUpgrade; upgrade::{DeniedUpgrade, InboundUpgrade}
};
use smallvec::SmallVec; use smallvec::SmallVec;
use std::io; use std::io;
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use void::Void; use void::{Void, unreachable};
use {IdentifySender, IdentifyOutput, IdentifyProtocolConfig};
/// Protocol handler that identifies the remote at a regular period. /// Protocol handler that identifies the remote at a regular period.
pub struct IdentifyListenHandler<TSubstream> { pub struct IdentifyListenHandler<TSubstream> {
@ -59,28 +60,29 @@ where
type InEvent = Void; type InEvent = Void;
type OutEvent = IdentifySender<TSubstream>; type OutEvent = IdentifySender<TSubstream>;
type Substream = TSubstream; type Substream = TSubstream;
type Protocol = IdentifyProtocolConfig; type InboundProtocol = IdentifyProtocolConfig;
type OutboundProtocol = DeniedUpgrade;
type OutboundOpenInfo = (); type OutboundOpenInfo = ();
#[inline] #[inline]
fn listen_protocol(&self) -> Self::Protocol { fn listen_protocol(&self) -> Self::InboundProtocol {
self.config.clone() self.config.clone()
} }
fn inject_fully_negotiated( #[inline]
fn dialer_protocol(&self) -> Self::OutboundProtocol {
DeniedUpgrade
}
fn inject_fully_negotiated_inbound(
&mut self, &mut self,
protocol: <Self::Protocol as ConnectionUpgrade<TSubstream>>::Output, protocol: <Self::InboundProtocol as InboundUpgrade<TSubstream>>::Output
endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>,
) { ) {
match protocol { self.pending_result.push(protocol)
IdentifyOutput::Sender { sender } => {
debug_assert!(if let NodeHandlerEndpoint::Listener = endpoint { true } else { false });
self.pending_result.push(sender);
}
IdentifyOutput::RemoteInfo { .. } => unreachable!(
"RemoteInfo can only be produced if we dial the protocol, but we never do that"
),
} }
fn inject_fully_negotiated_outbound(&mut self, protocol: Void, _: Self::OutboundOpenInfo) {
unreachable(protocol)
} }
#[inline] #[inline]
@ -102,7 +104,7 @@ where
) -> Poll< ) -> Poll<
Option< Option<
ProtocolsHandlerEvent< ProtocolsHandlerEvent<
Self::Protocol, Self::OutboundProtocol,
Self::OutboundOpenInfo, Self::OutboundOpenInfo,
Self::OutEvent, Self::OutEvent,
>, >,

View File

@ -18,18 +18,16 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::{RemoteInfo, IdentifyProtocolConfig};
use futures::prelude::*; use futures::prelude::*;
use libp2p_core::nodes::handled_node::NodeHandlerEndpoint; use libp2p_core::{
use libp2p_core::nodes::protocols_handler::{ProtocolsHandler, ProtocolsHandlerEvent}; nodes::protocols_handler::{ProtocolsHandler, ProtocolsHandlerEvent},
use libp2p_core::upgrade::{self, toggleable::Toggleable}; upgrade::{self, DeniedUpgrade, OutboundUpgrade, Toggleable}
use libp2p_core::{ConnectionUpgrade, Multiaddr}; };
use std::io; use std::{io, marker::PhantomData, time::{Duration, Instant}};
use std::marker::PhantomData;
use std::time::{Duration, Instant};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use tokio_timer::Delay; use tokio_timer::Delay;
use void::Void; use void::{Void, unreachable};
use {IdentifyInfo, IdentifyOutput, IdentifyProtocolConfig};
/// Delay between the moment we connect and the first time we identify. /// Delay between the moment we connect and the first time we identify.
const DELAY_TO_FIRST_ID: Duration = Duration::from_millis(500); const DELAY_TO_FIRST_ID: Duration = Duration::from_millis(500);
@ -59,13 +57,7 @@ pub struct PeriodicIdentification<TSubstream> {
#[derive(Debug)] #[derive(Debug)]
pub enum PeriodicIdentificationEvent { pub enum PeriodicIdentificationEvent {
/// We obtained identification information from the remote /// We obtained identification information from the remote
Identified { Identified(RemoteInfo),
/// Information of the remote.
info: IdentifyInfo,
/// Address the remote observes us as.
observed_addr: Multiaddr,
},
/// Failed to identify the remote. /// Failed to identify the remote.
IdentificationError(io::Error), IdentificationError(io::Error),
} }
@ -90,36 +82,30 @@ where
type InEvent = Void; type InEvent = Void;
type OutEvent = PeriodicIdentificationEvent; type OutEvent = PeriodicIdentificationEvent;
type Substream = TSubstream; type Substream = TSubstream;
type Protocol = Toggleable<IdentifyProtocolConfig>; type InboundProtocol = DeniedUpgrade;
type OutboundProtocol = Toggleable<IdentifyProtocolConfig>;
type OutboundOpenInfo = (); type OutboundOpenInfo = ();
#[inline] #[inline]
fn listen_protocol(&self) -> Self::Protocol { fn listen_protocol(&self) -> Self::InboundProtocol {
let mut upgrade = self.config.clone(); DeniedUpgrade
upgrade.disable();
upgrade
} }
fn inject_fully_negotiated( #[inline]
fn dialer_protocol(&self) -> Self::OutboundProtocol {
self.config.clone()
}
fn inject_fully_negotiated_inbound(&mut self, protocol: Void) {
unreachable(protocol)
}
fn inject_fully_negotiated_outbound(
&mut self, &mut self,
protocol: <Self::Protocol as ConnectionUpgrade<TSubstream>>::Output, protocol: <Self::OutboundProtocol as OutboundUpgrade<TSubstream>>::Output,
_endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>, _info: Self::OutboundOpenInfo,
) { ) {
match protocol { self.pending_result = Some(PeriodicIdentificationEvent::Identified(protocol))
IdentifyOutput::RemoteInfo {
info,
observed_addr,
} => {
self.pending_result = Some(PeriodicIdentificationEvent::Identified {
info,
observed_addr,
});
}
IdentifyOutput::Sender { .. } => unreachable!(
"Sender can only be produced if we listen for the identify \
protocol ; however we disable it in listen_protocol"
),
}
} }
#[inline] #[inline]
@ -146,7 +132,7 @@ where
) -> Poll< ) -> Poll<
Option< Option<
ProtocolsHandlerEvent< ProtocolsHandlerEvent<
Self::Protocol, Self::OutboundProtocol,
Self::OutboundOpenInfo, Self::OutboundOpenInfo,
PeriodicIdentificationEvent, PeriodicIdentificationEvent,
>, >,

View File

@ -65,15 +65,12 @@ where
event: <Self::ProtocolsHandler as ProtocolsHandler>::OutEvent, event: <Self::ProtocolsHandler as ProtocolsHandler>::OutEvent,
) { ) {
match event { match event {
PeriodicIdentificationEvent::Identified { PeriodicIdentificationEvent::Identified(remote) => {
info,
observed_addr,
} => {
self.events self.events
.push_back(PeriodicIdentifyBehaviourEvent::Identified { .push_back(PeriodicIdentifyBehaviourEvent::Identified {
peer_id: peer_id, peer_id: peer_id,
info: info, info: remote.info,
observed_addr: observed_addr, observed_addr: remote.observed_addr,
}); });
} }
_ => (), // TODO: exhaustive pattern _ => (), // TODO: exhaustive pattern

View File

@ -19,8 +19,12 @@
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use bytes::{Bytes, BytesMut}; use bytes::{Bytes, BytesMut};
use futures::{future, Future, Sink, Stream}; use futures::{future::{self, FutureResult}, Future, Sink, Stream};
use libp2p_core::{ConnectionUpgrade, Endpoint, Multiaddr, PublicKey}; use libp2p_core::{
Multiaddr, PublicKey,
upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}
};
use log::{debug, trace};
use protobuf::Message as ProtobufMessage; use protobuf::Message as ProtobufMessage;
use protobuf::parse_from_bytes as protobuf_parse_from_bytes; use protobuf::parse_from_bytes as protobuf_parse_from_bytes;
use protobuf::RepeatedField; use protobuf::RepeatedField;
@ -35,22 +39,14 @@ use unsigned_varint::codec;
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub struct IdentifyProtocolConfig; pub struct IdentifyProtocolConfig;
/// Output of the connection upgrade. #[derive(Debug, Clone)]
pub enum IdentifyOutput<T> { pub struct RemoteInfo {
/// We obtained information from the remote. Happens when we are the dialer.
RemoteInfo {
/// Information about the remote. /// Information about the remote.
info: IdentifyInfo, pub info: IdentifyInfo,
/// Address the remote sees for us. /// Address the remote sees for us.
observed_addr: Multiaddr, pub observed_addr: Multiaddr,
},
/// We opened a connection to the remote and need to send it information. Happens when we are _priv: ()
/// the listener.
Sender {
/// Object used to send identify info to the client.
sender: IdentifySender<T>,
},
} }
/// Object used to send back information to the client. /// Object used to send back information to the client.
@ -64,11 +60,9 @@ where
{ {
/// Sends back information to the client. Returns a future that is signalled whenever the /// Sends back information to the client. Returns a future that is signalled whenever the
/// info have been sent. /// info have been sent.
pub fn send( pub fn send(self, info: IdentifyInfo, observed_addr: &Multiaddr)
self, -> Box<Future<Item = (), Error = IoError> + Send + 'a>
info: IdentifyInfo, {
observed_addr: &Multiaddr,
) -> Box<Future<Item = (), Error = IoError> + Send + 'a> {
debug!("Sending identify info to client"); debug!("Sending identify info to client");
trace!("Sending: {:?}", info); trace!("Sending: {:?}", info);
@ -108,36 +102,52 @@ pub struct IdentifyInfo {
pub listen_addrs: Vec<Multiaddr>, pub listen_addrs: Vec<Multiaddr>,
/// Protocols supported by the node, e.g. `/ipfs/ping/1.0.0`. /// Protocols supported by the node, e.g. `/ipfs/ping/1.0.0`.
pub protocols: Vec<String>, pub protocols: Vec<String>,
_priv: ()
} }
impl<C> ConnectionUpgrade<C> for IdentifyProtocolConfig impl UpgradeInfo for IdentifyProtocolConfig {
where type UpgradeId = ();
C: AsyncRead + AsyncWrite + Send + 'static, type NamesIter = iter::Once<(Bytes, Self::UpgradeId)>;
{
type NamesIter = iter::Once<(Bytes, Self::UpgradeIdentifier)>;
type UpgradeIdentifier = ();
type Output = IdentifyOutput<C>;
type Future = Box<Future<Item = Self::Output, Error = IoError> + Send>;
#[inline] #[inline]
fn protocol_names(&self) -> Self::NamesIter { fn protocol_names(&self) -> Self::NamesIter {
iter::once((Bytes::from("/ipfs/id/1.0.0"), ())) iter::once((Bytes::from("/ipfs/id/1.0.0"), ()))
} }
}
fn upgrade(self, socket: C, _: (), ty: Endpoint) -> Self::Future { impl<C> InboundUpgrade<C> for IdentifyProtocolConfig
trace!("Upgrading connection as {:?}", ty); where
C: AsyncRead + AsyncWrite + Send + 'static,
{
type Output = IdentifySender<C>;
type Error = IoError;
type Future = FutureResult<Self::Output, IoError>;
fn upgrade_inbound(self, socket: C, _: ()) -> Self::Future {
trace!("Upgrading inbound connection");
let socket = Framed::new(socket, codec::UviBytes::default()); let socket = Framed::new(socket, codec::UviBytes::default());
let sender = IdentifySender { inner: socket };
future::ok(sender)
}
}
match ty { impl<C> OutboundUpgrade<C> for IdentifyProtocolConfig
Endpoint::Dialer => { where
C: AsyncRead + AsyncWrite + Send + 'static,
{
type Output = RemoteInfo;
type Error = IoError;
type Future = Box<dyn Future<Item=Self::Output, Error=IoError> + Send>;
fn upgrade_outbound(self, socket: C, _: ()) -> Self::Future {
let socket = Framed::new(socket, codec::UviBytes::<BytesMut>::default());
let future = socket let future = socket
.into_future() .into_future()
.map(|(msg, _)| msg) .map(|(msg, _)| msg)
.map_err(|(err, _)| err) .map_err(|(err, _)| err)
.and_then(|msg| { .and_then(|msg| {
debug!("Received identify message"); debug!("Received identify message");
if let Some(msg) = msg { if let Some(msg) = msg {
let (info, observed_addr) = match parse_proto_msg(msg) { let (info, observed_addr) = match parse_proto_msg(msg) {
Ok(v) => v, Ok(v) => v,
@ -146,30 +156,20 @@ where
return Err(err.into()); return Err(err.into());
} }
}; };
trace!("Remote observes us as {:?}", observed_addr); trace!("Remote observes us as {:?}", observed_addr);
trace!("Information received: {:?}", info); trace!("Information received: {:?}", info);
Ok(RemoteInfo {
Ok(IdentifyOutput::RemoteInfo {
info, info,
observed_addr: observed_addr.clone(), observed_addr: observed_addr.clone(),
_priv: ()
}) })
} else { } else {
debug!("Identify protocol stream closed before receiving info"); debug!("Identify protocol stream closed before receiving info");
Err(IoErrorKind::InvalidData.into()) Err(IoErrorKind::InvalidData.into())
} }
}); });
Box::new(future) as Box<_> Box::new(future) as Box<_>
} }
Endpoint::Listener => {
let sender = IdentifySender { inner: socket };
let future = future::ok(IdentifyOutput::Sender { sender });
Box::new(future) as Box<_>
}
}
}
} }
// Turns a protobuf message into an `IdentifyInfo` and an observed address. If something bad // Turns a protobuf message into an `IdentifyInfo` and an observed address. If something bad
@ -199,6 +199,7 @@ fn parse_proto_msg(msg: BytesMut) -> Result<(IdentifyInfo, Multiaddr), IoError>
agent_version: msg.take_agentVersion(), agent_version: msg.take_agentVersion(),
listen_addrs: listen_addrs, listen_addrs: listen_addrs,
protocols: msg.take_protocols().into_vec(), protocols: msg.take_protocols().into_vec(),
_priv: ()
}; };
Ok((info, observed_addr)) Ok((info, observed_addr))
@ -216,10 +217,10 @@ mod tests {
use self::tokio::runtime::current_thread::Runtime; use self::tokio::runtime::current_thread::Runtime;
use self::libp2p_tcp_transport::TcpConfig; use self::libp2p_tcp_transport::TcpConfig;
use futures::{Future, Stream}; use futures::{Future, Stream};
use libp2p_core::{PublicKey, Transport}; use libp2p_core::{PublicKey, Transport, upgrade::{apply_outbound, apply_inbound}};
use std::sync::mpsc; use std::sync::mpsc;
use std::thread; use std::thread;
use {IdentifyInfo, IdentifyOutput, IdentifyProtocolConfig}; use {IdentifyInfo, RemoteInfo, IdentifyProtocolConfig};
#[test] #[test]
fn correct_transfer() { fn correct_transfer() {
@ -229,19 +230,23 @@ mod tests {
let (tx, rx) = mpsc::channel(); let (tx, rx) = mpsc::channel();
let bg_thread = thread::spawn(move || { let bg_thread = thread::spawn(move || {
let transport = TcpConfig::new().with_upgrade(IdentifyProtocolConfig); let transport = TcpConfig::new();
let (listener, addr) = transport let (listener, addr) = transport
.listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()) .listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap())
.unwrap(); .unwrap();
tx.send(addr).unwrap(); tx.send(addr).unwrap();
let future = listener let future = listener
.into_future() .into_future()
.map_err(|(err, _)| err) .map_err(|(err, _)| err)
.and_then(|(client, _)| client.unwrap().0) .and_then(|(client, _)| client.unwrap().0)
.and_then(|identify| match identify { .and_then(|socket| {
IdentifyOutput::Sender { sender, .. } => sender.send( apply_inbound(socket, IdentifyProtocolConfig).map_err(|e| e.into_io_error())
})
.and_then(|sender| {
sender.send(
IdentifyInfo { IdentifyInfo {
public_key: PublicKey::Ed25519(vec![1, 2, 3, 4, 5, 7]), public_key: PublicKey::Ed25519(vec![1, 2, 3, 4, 5, 7]),
protocol_version: "proto_version".to_owned(), protocol_version: "proto_version".to_owned(),
@ -251,47 +256,34 @@ mod tests {
"/ip6/::1/udp/1000".parse().unwrap(), "/ip6/::1/udp/1000".parse().unwrap(),
], ],
protocols: vec!["proto1".to_string(), "proto2".to_string()], protocols: vec!["proto1".to_string(), "proto2".to_string()],
_priv: ()
}, },
&"/ip4/100.101.102.103/tcp/5000".parse().unwrap(), &"/ip4/100.101.102.103/tcp/5000".parse().unwrap(),
), )
_ => panic!(),
}); });
let mut rt = Runtime::new().unwrap(); let mut rt = Runtime::new().unwrap();
let _ = rt.block_on(future).unwrap(); let _ = rt.block_on(future).unwrap();
}); });
let transport = TcpConfig::new().with_upgrade(IdentifyProtocolConfig); let transport = TcpConfig::new();
let future = transport let future = transport.dial(rx.recv().unwrap())
.dial(rx.recv().unwrap())
.unwrap_or_else(|_| panic!()) .unwrap_or_else(|_| panic!())
.and_then(|identify| match identify { .and_then(|socket| {
IdentifyOutput::RemoteInfo { apply_outbound(socket, IdentifyProtocolConfig).map_err(|e| e.into_io_error())
info, })
observed_addr, .and_then(|RemoteInfo { info, observed_addr, .. }| {
} => { assert_eq!(observed_addr, "/ip4/100.101.102.103/tcp/5000".parse().unwrap());
assert_eq!(
observed_addr,
"/ip4/100.101.102.103/tcp/5000".parse().unwrap()
);
assert_eq!(info.public_key, PublicKey::Ed25519(vec![1, 2, 3, 4, 5, 7])); assert_eq!(info.public_key, PublicKey::Ed25519(vec![1, 2, 3, 4, 5, 7]));
assert_eq!(info.protocol_version, "proto_version"); assert_eq!(info.protocol_version, "proto_version");
assert_eq!(info.agent_version, "agent_version"); assert_eq!(info.agent_version, "agent_version");
assert_eq!( assert_eq!(info.listen_addrs,
info.listen_addrs, &["/ip4/80.81.82.83/tcp/500".parse().unwrap(),
&[ "/ip6/::1/udp/1000".parse().unwrap()]);
"/ip4/80.81.82.83/tcp/500".parse().unwrap(), assert_eq!(info.protocols, &["proto1".to_string(), "proto2".to_string()]);
"/ip6/::1/udp/1000".parse().unwrap()
]
);
assert_eq!(
info.protocols,
&["proto1".to_string(), "proto2".to_string()]
);
Ok(()) Ok(())
}
_ => panic!(),
}); });
let mut rt = Runtime::new().unwrap(); let mut rt = Runtime::new().unwrap();
let _ = rt.block_on(future).unwrap(); let _ = rt.block_on(future).unwrap();
bg_thread.join().unwrap(); bg_thread.join().unwrap();

View File

@ -23,6 +23,7 @@ use futures::{future, Future, IntoFuture, stream, Stream};
use kad_server::KadConnecController; use kad_server::KadConnecController;
use kbucket::{KBucketsTable, KBucketsPeerId}; use kbucket::{KBucketsTable, KBucketsPeerId};
use libp2p_core::PeerId; use libp2p_core::PeerId;
use log::{debug, trace};
use protocol; use protocol;
use rand; use rand;
use smallvec::SmallVec; use smallvec::SmallVec;

View File

@ -36,7 +36,8 @@
use bytes::Bytes; use bytes::Bytes;
use futures::sync::{mpsc, oneshot}; use futures::sync::{mpsc, oneshot};
use futures::{future, Future, Sink, stream, Stream}; use futures::{future, Future, Sink, stream, Stream};
use libp2p_core::{ConnectionUpgrade, Endpoint, PeerId}; use libp2p_core::{PeerId, upgrade::{InboundUpgrade, UpgradeInfo}};
use log::{debug, warn};
use multihash::Multihash; use multihash::Multihash;
use protocol::{self, KadMsg, KademliaProtocolConfig, KadPeer}; use protocol::{self, KadMsg, KademliaProtocolConfig, KadPeer};
use std::collections::VecDeque; use std::collections::VecDeque;
@ -64,7 +65,17 @@ impl KadConnecConfig {
} }
} }
impl<C> ConnectionUpgrade<C> for KadConnecConfig impl UpgradeInfo for KadConnecConfig {
type NamesIter = iter::Once<(Bytes, Self::UpgradeId)>;
type UpgradeId = ();
#[inline]
fn protocol_names(&self) -> Self::NamesIter {
self.raw_proto.protocol_names()
}
}
impl<C> InboundUpgrade<C> for KadConnecConfig
where where
C: AsyncRead + AsyncWrite + Send + 'static, // TODO: 'static :-/ C: AsyncRead + AsyncWrite + Send + 'static, // TODO: 'static :-/
{ {
@ -72,22 +83,14 @@ where
KadConnecController, KadConnecController,
Box<Stream<Item = KadIncomingRequest, Error = IoError> + Send>, Box<Stream<Item = KadIncomingRequest, Error = IoError> + Send>,
); );
type Future = future::Map<<KademliaProtocolConfig as ConnectionUpgrade<C>>::Future, fn(<KademliaProtocolConfig as ConnectionUpgrade<C>>::Output) -> Self::Output>; type Error = IoError;
type NamesIter = iter::Once<(Bytes, ())>; type Future = future::Map<<KademliaProtocolConfig as InboundUpgrade<C>>::Future, fn(<KademliaProtocolConfig as InboundUpgrade<C>>::Output) -> Self::Output>;
type UpgradeIdentifier = ();
#[inline] #[inline]
fn protocol_names(&self) -> Self::NamesIter { fn upgrade_inbound(self, incoming: C, id: Self::UpgradeId) -> Self::Future {
ConnectionUpgrade::<C>::protocol_names(&self.raw_proto)
}
#[inline]
fn upgrade(self, incoming: C, id: (), endpoint: Endpoint) -> Self::Future {
self.raw_proto self.raw_proto
.upgrade(incoming, id, endpoint) .upgrade_inbound(incoming, id)
.map::<fn(_) -> _, _>(move |connec| { .map(build_from_sink_stream)
build_from_sink_stream(connec)
})
} }
} }

View File

@ -66,7 +66,6 @@ extern crate futures;
extern crate libp2p_identify; extern crate libp2p_identify;
extern crate libp2p_ping; extern crate libp2p_ping;
extern crate libp2p_core; extern crate libp2p_core;
#[macro_use]
extern crate log; extern crate log;
extern crate multiaddr; extern crate multiaddr;
extern crate multihash; extern crate multihash;

View File

@ -27,7 +27,10 @@
use bytes::{Bytes, BytesMut}; use bytes::{Bytes, BytesMut};
use futures::{future, sink, Sink, stream, Stream}; use futures::{future, sink, Sink, stream, Stream};
use libp2p_core::{ConnectionUpgrade, Endpoint, Multiaddr, PeerId}; use libp2p_core::{
Multiaddr, PeerId,
upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}
};
use multihash::Multihash; use multihash::Multihash;
use protobuf::{self, Message}; use protobuf::{self, Message};
use protobuf_structs; use protobuf_structs;
@ -128,22 +131,40 @@ impl Into<protobuf_structs::dht::Message_Peer> for KadPeer {
#[derive(Debug, Default, Copy, Clone)] #[derive(Debug, Default, Copy, Clone)]
pub struct KademliaProtocolConfig; pub struct KademliaProtocolConfig;
impl<C> ConnectionUpgrade<C> for KademliaProtocolConfig impl UpgradeInfo for KademliaProtocolConfig {
where type UpgradeId = ();
C: AsyncRead + AsyncWrite + 'static, // TODO: 'static :-/ type NamesIter = iter::Once<(Bytes, Self::UpgradeId)>;
{
type Output = KadStreamSink<C>;
type Future = future::FutureResult<Self::Output, IoError>;
type NamesIter = iter::Once<(Bytes, ())>;
type UpgradeIdentifier = ();
#[inline] #[inline]
fn protocol_names(&self) -> Self::NamesIter { fn protocol_names(&self) -> Self::NamesIter {
iter::once(("/ipfs/kad/1.0.0".into(), ())) iter::once(("/ipfs/kad/1.0.0".into(), ()))
} }
}
impl<C> InboundUpgrade<C> for KademliaProtocolConfig
where
C: AsyncRead + AsyncWrite + 'static, // TODO: 'static :-/
{
type Output = KadStreamSink<C>;
type Error = IoError;
type Future = future::FutureResult<Self::Output, Self::Error>;
#[inline] #[inline]
fn upgrade(self, incoming: C, _: (), _: Endpoint) -> Self::Future { fn upgrade_inbound(self, incoming: C, _: Self::UpgradeId) -> Self::Future {
future::ok(kademlia_protocol(incoming))
}
}
impl<C> OutboundUpgrade<C> for KademliaProtocolConfig
where
C: AsyncRead + AsyncWrite + 'static, // TODO: 'static :-/
{
type Output = KadStreamSink<C>;
type Error = IoError;
type Future = future::FutureResult<Self::Output, Self::Error>;
#[inline]
fn upgrade_outbound(self, incoming: C, _: Self::UpgradeId) -> Self::Future {
future::ok(kademlia_protocol(incoming)) future::ok(kademlia_protocol(incoming))
} }
} }
@ -151,9 +172,7 @@ where
type KadStreamSink<S> = stream::AndThen<sink::With<stream::FromErr<Framed<S, codec::UviBytes<Vec<u8>>>, IoError>, KadMsg, fn(KadMsg) -> Result<Vec<u8>, IoError>, Result<Vec<u8>, IoError>>, fn(BytesMut) -> Result<KadMsg, IoError>, Result<KadMsg, IoError>>; type KadStreamSink<S> = stream::AndThen<sink::With<stream::FromErr<Framed<S, codec::UviBytes<Vec<u8>>>, IoError>, KadMsg, fn(KadMsg) -> Result<Vec<u8>, IoError>, Result<Vec<u8>, IoError>>, fn(BytesMut) -> Result<KadMsg, IoError>, Result<KadMsg, IoError>>;
// Upgrades a socket to use the Kademlia protocol. // Upgrades a socket to use the Kademlia protocol.
fn kademlia_protocol<S>( fn kademlia_protocol<S>(socket: S) -> KadStreamSink<S>
socket: S,
) -> KadStreamSink<S>
where where
S: AsyncRead + AsyncWrite, S: AsyncRead + AsyncWrite,
{ {
@ -485,6 +504,7 @@ mod tests {
let (listener, addr) = transport let (listener, addr) = transport
.listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()) .listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap())
.unwrap(); .unwrap();
tx.send(addr).unwrap(); tx.send(addr).unwrap();
let future = listener let future = listener

View File

@ -30,19 +30,67 @@ extern crate unsigned_varint;
use bytes::Bytes; use bytes::Bytes;
use futures::{future, prelude::*}; use futures::{future, prelude::*};
use libp2p_core::{ConnectionUpgrade, Endpoint, Multiaddr}; use libp2p_core::{Multiaddr, upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}};
use std::{io, iter}; use std::{io, iter};
use tokio_codec::{FramedRead, FramedWrite}; use tokio_codec::{FramedRead, FramedWrite};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use unsigned_varint::codec::UviBytes; use unsigned_varint::codec::UviBytes;
/// The output, this connection upgrade produces. /// The connection upgrade type to retrieve or report externally visible addresses.
pub enum Output<C> { pub struct Observed {}
/// As `Dialer`, we get our own externally observed address.
Address(Multiaddr), impl Observed {
/// As `Listener`, we return a sender which allows reporting the observed pub fn new() -> Self {
/// address the client. Observed {}
Sender(Sender<C>) }
}
impl UpgradeInfo for Observed {
type UpgradeId = ();
type NamesIter = iter::Once<(Bytes, Self::UpgradeId)>;
fn protocol_names(&self) -> Self::NamesIter {
iter::once((Bytes::from("/paritytech/observed-address/0.1.0"), ()))
}
}
impl<C> InboundUpgrade<C> for Observed
where
C: AsyncRead + AsyncWrite + Send + 'static
{
type Output = Sender<C>;
type Error = io::Error;
type Future = Box<dyn Future<Item=Self::Output, Error=Self::Error> + Send>;
fn upgrade_inbound(self, conn: C, _: ()) -> Self::Future {
let io = FramedWrite::new(conn, UviBytes::default());
Box::new(future::ok(Sender { io }))
}
}
impl<C> OutboundUpgrade<C> for Observed
where
C: AsyncRead + AsyncWrite + Send + 'static
{
type Output = Multiaddr;
type Error = io::Error;
type Future = Box<dyn Future<Item=Self::Output, Error=Self::Error> + Send>;
fn upgrade_outbound(self, conn: C, _: ()) -> Self::Future {
let io = FramedRead::new(conn, UviBytes::default());
let future = io.into_future()
.map_err(|(e, _): (io::Error, FramedRead<C, UviBytes>)| e)
.and_then(move |(bytes, _)| {
if let Some(b) = bytes {
let ma = Multiaddr::from_bytes(b.to_vec())
.map_err(|e| io::Error::new(io::ErrorKind::InvalidData, e))?;
Ok(ma)
} else {
Err(io::ErrorKind::InvalidData.into())
}
});
Box::new(future)
}
} }
/// `Sender` allows reporting back the observed address to the remote endpoint. /// `Sender` allows reporting back the observed address to the remote endpoint.
@ -57,58 +105,11 @@ impl<C: AsyncWrite> Sender<C> {
} }
} }
/// The connection upgrade type to retrieve or report externally visible addresses.
pub struct Observed {}
impl Observed {
pub fn new() -> Self {
Observed {}
}
}
impl<C> ConnectionUpgrade<C> for Observed
where
C: AsyncRead + AsyncWrite + Send + 'static
{
type NamesIter = iter::Once<(Bytes, Self::UpgradeIdentifier)>;
type UpgradeIdentifier = ();
type Output = Output<C>;
type Future = Box<dyn Future<Item=Self::Output, Error=io::Error> + Send>;
fn protocol_names(&self) -> Self::NamesIter {
iter::once((Bytes::from("/paritytech/observed-address/0.1.0"), ()))
}
fn upgrade(self, conn: C, _: (), role: Endpoint) -> Self::Future {
match role {
Endpoint::Dialer => {
let io = FramedRead::new(conn, UviBytes::default());
let future = io.into_future()
.map_err(|(e, _): (io::Error, FramedRead<C, UviBytes>)| e)
.and_then(move |(bytes, _)| {
if let Some(b) = bytes {
let ma = Multiaddr::from_bytes(b.to_vec())
.map_err(|e| io::Error::new(io::ErrorKind::InvalidData, e))?;
Ok(Output::Address(ma))
} else {
Err(io::ErrorKind::InvalidData.into())
}
});
Box::new(future)
}
Endpoint::Listener => {
let io = FramedWrite::new(conn, UviBytes::default());
Box::new(future::ok(Output::Sender(Sender { io })))
}
}
}
}
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
extern crate tokio; extern crate tokio;
use libp2p_core::{ConnectionUpgrade, Endpoint, Multiaddr}; use libp2p_core::{Multiaddr, upgrade::{InboundUpgrade, OutboundUpgrade}};
use self::tokio::runtime::current_thread; use self::tokio::runtime::current_thread;
use self::tokio::net::{TcpListener, TcpStream}; use self::tokio::net::{TcpListener, TcpStream};
use super::*; use super::*;
@ -125,28 +126,18 @@ mod tests {
.into_future() .into_future()
.map_err(|(e, _)| e.into()) .map_err(|(e, _)| e.into())
.and_then(move |(conn, _)| { .and_then(move |(conn, _)| {
Observed::new().upgrade(conn.unwrap(), (), Endpoint::Listener) Observed::new().upgrade_inbound(conn.unwrap(), ())
}) })
.and_then(move |output| { .and_then(move |sender| sender.send_address(observed_addr1));
match output {
Output::Sender(s) => s.send_address(observed_addr1),
Output::Address(_) => unreachable!()
}
});
let client = TcpStream::connect(&server_addr) let client = TcpStream::connect(&server_addr)
.map_err(|e| e.into()) .map_err(|e| e.into())
.and_then(|conn| { .and_then(|conn| {
Observed::new().upgrade(conn, (), Endpoint::Dialer) Observed::new().upgrade_outbound(conn, ())
}) })
.map(move |output| { .map(move |addr| {
match output {
Output::Address(addr) => {
eprintln!("{} {}", addr, observed_addr2); eprintln!("{} {}", addr, observed_addr2);
assert_eq!(addr, observed_addr2) assert_eq!(addr, observed_addr2)
}
_ => unreachable!()
}
}); });
current_thread::block_on_all(future::lazy(move || { current_thread::block_on_all(future::lazy(move || {

View File

@ -20,25 +20,26 @@
use futures::prelude::*; use futures::prelude::*;
use libp2p_core::{ use libp2p_core::{
nodes::{NodeHandlerEndpoint, ProtocolsHandler, ProtocolsHandlerEvent}, OutboundUpgrade,
upgrade::toggleable, nodes::{ProtocolsHandler, ProtocolsHandlerEvent},
ConnectionUpgrade, upgrade::{self, DeniedUpgrade}
}; };
use protocol::{Ping, PingDialer, PingOutput}; use log::warn;
use protocol::{Ping, PingDialer};
use std::{ use std::{
io, mem, io, mem,
time::{Duration, Instant}, time::{Duration, Instant},
}; };
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use tokio_timer::Delay; use tokio_timer::Delay;
use void::Void; use void::{Void, unreachable};
/// Protocol handler that handles pinging the remote at a regular period. /// Protocol handler that handles pinging the remote at a regular period.
/// ///
/// If the remote doesn't respond, produces `Unresponsive` and closes the connection. /// If the remote doesn't respond, produces `Unresponsive` and closes the connection.
pub struct PeriodicPingHandler<TSubstream> { pub struct PeriodicPingHandler<TSubstream> {
/// Configuration for the ping protocol. /// Configuration for the ping protocol.
ping_config: toggleable::Toggleable<Ping<Instant>>, ping_config: upgrade::Toggleable<Ping<Instant>>,
/// State of the outgoing ping. /// State of the outgoing ping.
out_state: OutState<TSubstream>, out_state: OutState<TSubstream>,
@ -126,7 +127,7 @@ impl<TSubstream> PeriodicPingHandler<TSubstream> {
let ping_timeout = Duration::from_secs(30); let ping_timeout = Duration::from_secs(30);
PeriodicPingHandler { PeriodicPingHandler {
ping_config: toggleable::toggleable(Default::default()), ping_config: upgrade::toggleable(Default::default()),
out_state: OutState::NeedToOpen { out_state: OutState::NeedToOpen {
expires: Delay::new(Instant::now() + ping_timeout), expires: Delay::new(Instant::now() + ping_timeout),
}, },
@ -151,24 +152,29 @@ where
type InEvent = Void; type InEvent = Void;
type OutEvent = OutEvent; type OutEvent = OutEvent;
type Substream = TSubstream; type Substream = TSubstream;
type Protocol = toggleable::Toggleable<Ping<Instant>>; type InboundProtocol = DeniedUpgrade;
type OutboundProtocol = upgrade::Toggleable<Ping<Instant>>;
type OutboundOpenInfo = (); type OutboundOpenInfo = ();
#[inline] #[inline]
fn listen_protocol(&self) -> Self::Protocol { fn listen_protocol(&self) -> Self::InboundProtocol {
let mut config = self.ping_config; DeniedUpgrade
config.disable();
config
} }
fn inject_fully_negotiated( #[inline]
fn dialer_protocol(&self) -> Self::OutboundProtocol {
self.ping_config
}
fn inject_fully_negotiated_inbound(&mut self, protocol: Void) {
unreachable(protocol)
}
fn inject_fully_negotiated_outbound(
&mut self, &mut self,
protocol: <Self::Protocol as ConnectionUpgrade<TSubstream>>::Output, mut substream: <Self::OutboundProtocol as OutboundUpgrade<TSubstream>>::Output,
_endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>, _info: Self::OutboundOpenInfo
) { ) {
match protocol {
PingOutput::Pinger(mut substream) => {
debug_assert!(_endpoint.is_dialer());
match mem::replace(&mut self.out_state, OutState::Poisoned) { match mem::replace(&mut self.out_state, OutState::Poisoned) {
OutState::Upgrading { expires } => { OutState::Upgrading { expires } => {
// We always upgrade with the intent of immediately pinging. // We always upgrade with the intent of immediately pinging.
@ -178,11 +184,6 @@ where
_ => (), _ => (),
} }
} }
PingOutput::Ponger(_) => {
debug_assert!(false, "Received an unexpected incoming ping substream");
}
}
}
fn inject_event(&mut self, _: Self::InEvent) {} fn inject_event(&mut self, _: Self::InEvent) {}
@ -218,7 +219,7 @@ where
fn poll( fn poll(
&mut self, &mut self,
) -> Poll< ) -> Poll<
Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>, Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>,
io::Error, io::Error,
> { > {
// Shortcut for polling a `tokio_timer::Delay` // Shortcut for polling a `tokio_timer::Delay`

View File

@ -56,38 +56,35 @@
//! extern crate tokio; //! extern crate tokio;
//! //!
//! use futures::{Future, Stream}; //! use futures::{Future, Stream};
//! use libp2p_ping::protocol::{Ping, PingOutput}; //! use libp2p_core::{transport::Transport, upgrade::apply_outbound};
//! use libp2p_core::Transport; //! use libp2p_ping::protocol::Ping;
//! use tokio::runtime::current_thread::Runtime; //! use tokio::runtime::current_thread::Runtime;
//! //!
//! # fn main() { //! # fn main() {
//! let ping_finished_future = libp2p_tcp_transport::TcpConfig::new() //! let ping_dialer = libp2p_tcp_transport::TcpConfig::new()
//! .with_upgrade(Ping::default()) //! .and_then(|socket, _| {
//! .dial("127.0.0.1:12345".parse::<libp2p_core::Multiaddr>().unwrap()).unwrap_or_else(|_| panic!()) //! apply_outbound(socket, Ping::default()).map_err(|e| e.into_io_error())
//! .and_then(|out| { //! })
//! match out { //! .dial("/ip4/127.0.0.1/tcp/12345".parse::<libp2p_core::Multiaddr>().unwrap()).unwrap_or_else(|_| panic!())
//! PingOutput::Ponger(processing) => Box::new(processing) as Box<Future<Item = _, Error = _> + Send>, //! .and_then(|mut pinger| {
//! PingOutput::Pinger(mut pinger) => {
//! pinger.ping(()); //! pinger.ping(());
//! let f = pinger.into_future().map(|_| ()).map_err(|(err, _)| err); //! let f = pinger.into_future()
//! .map(|_| println!("received pong"))
//! .map_err(|(e, _)| e);
//! Box::new(f) as Box<Future<Item = _, Error = _> + Send> //! Box::new(f) as Box<Future<Item = _, Error = _> + Send>
//! },
//! }
//! }); //! });
//! //!
//! // Runs until the ping arrives. //! // Runs until the ping arrives.
//! let mut rt = Runtime::new().unwrap(); //! let mut rt = Runtime::new().unwrap();
//! let _ = rt.block_on(ping_finished_future).unwrap(); //! let _ = rt.block_on(ping_dialer).unwrap();
//! # } //! # }
//! ``` //! ```
//! //!
extern crate arrayvec; extern crate arrayvec;
extern crate bytes; extern crate bytes;
#[macro_use]
extern crate futures; extern crate futures;
extern crate libp2p_core; extern crate libp2p_core;
#[macro_use]
extern crate log; extern crate log;
extern crate multistream_select; extern crate multistream_select;
extern crate parking_lot; extern crate parking_lot;

View File

@ -21,13 +21,15 @@
use arrayvec::ArrayVec; use arrayvec::ArrayVec;
use futures::prelude::*; use futures::prelude::*;
use libp2p_core::{ use libp2p_core::{
nodes::{NodeHandlerEndpoint, ProtocolsHandler, ProtocolsHandlerEvent}, InboundUpgrade,
ConnectionUpgrade, nodes::{ProtocolsHandler, ProtocolsHandlerEvent},
upgrade::DeniedUpgrade
}; };
use protocol::{Ping, PingListener, PingOutput}; use log::warn;
use protocol::{Ping, PingListener};
use std::io; use std::io;
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use void::Void; use void::{Void, unreachable};
/// Handler for handling pings received from a remote. /// Handler for handling pings received from a remote.
pub struct PingListenHandler<TSubstream> { pub struct PingListenHandler<TSubstream> {
@ -68,33 +70,32 @@ where
type InEvent = Void; type InEvent = Void;
type OutEvent = Void; type OutEvent = Void;
type Substream = TSubstream; type Substream = TSubstream;
type Protocol = Ping<()>; type InboundProtocol = Ping<()>;
type OutboundProtocol = DeniedUpgrade;
type OutboundOpenInfo = (); type OutboundOpenInfo = ();
#[inline] #[inline]
fn listen_protocol(&self) -> Self::Protocol { fn listen_protocol(&self) -> Self::InboundProtocol {
self.ping_config self.ping_config
} }
fn inject_fully_negotiated( #[inline]
fn dialer_protocol(&self) -> Self::OutboundProtocol {
DeniedUpgrade
}
fn inject_fully_negotiated_inbound(
&mut self, &mut self,
protocol: <Self::Protocol as ConnectionUpgrade<TSubstream>>::Output, protocol: <Self::InboundProtocol as InboundUpgrade<TSubstream>>::Output
_endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>,
) { ) {
if self.shutdown { if self.shutdown {
return; return;
} }
let _ = self.ping_in_substreams.try_push(protocol);
}
match protocol { fn inject_fully_negotiated_outbound(&mut self, protocol: Void, _info: Self::OutboundOpenInfo) {
PingOutput::Pinger(_) => { unreachable(protocol)
debug_assert!(false, "Received an unexpected outgoing ping substream");
}
PingOutput::Ponger(listener) => {
debug_assert!(_endpoint.is_listener());
// Try insert the element, but don't care if the list is full.
let _ = self.ping_in_substreams.try_push(listener);
}
}
} }
#[inline] #[inline]
@ -118,7 +119,7 @@ where
fn poll( fn poll(
&mut self, &mut self,
) -> Poll< ) -> Poll<
Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>, Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>,
io::Error, io::Error,
> { > {
// Removes each substream one by one, and pushes them back if they're not ready (which // Removes each substream one by one, and pushes them back if they're not ready (which

View File

@ -19,8 +19,9 @@
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use bytes::{BufMut, Bytes, BytesMut}; use bytes::{BufMut, Bytes, BytesMut};
use futures::{prelude::*, future::FutureResult, future::IntoFuture}; use futures::{prelude::*, future::{self, FutureResult}, try_ready};
use libp2p_core::{ConnectionUpgrade, Endpoint}; use libp2p_core::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
use log::debug;
use rand::{distributions::Standard, prelude::*, rngs::EntropyRng}; use rand::{distributions::Standard, prelude::*, rngs::EntropyRng};
use std::collections::VecDeque; use std::collections::VecDeque;
use std::io::Error as IoError; use std::io::Error as IoError;
@ -42,49 +43,43 @@ impl<TUserData> Default for Ping<TUserData> {
} }
} }
/// Output of a `Ping` upgrade. impl<TUserData> UpgradeInfo for Ping<TUserData> {
pub enum PingOutput<TSocket, TUserData> { type UpgradeId = ();
/// We are on the dialing side. type NamesIter = iter::Once<(Bytes, Self::UpgradeId)>;
Pinger(PingDialer<TSocket, TUserData>),
/// We are on the listening side.
Ponger(PingListener<TSocket>),
}
impl<TSocket, TUserData> ConnectionUpgrade<TSocket> for Ping<TUserData>
where
TSocket: AsyncRead + AsyncWrite,
{
type NamesIter = iter::Once<(Bytes, Self::UpgradeIdentifier)>;
type UpgradeIdentifier = ();
#[inline]
fn protocol_names(&self) -> Self::NamesIter { fn protocol_names(&self) -> Self::NamesIter {
iter::once(("/ipfs/ping/1.0.0".into(), ())) iter::once(("/ipfs/ping/1.0.0".into(), ()))
} }
}
type Output = PingOutput<TSocket, TUserData>; impl<TSocket, TUserData> InboundUpgrade<TSocket> for Ping<TUserData>
type Future = FutureResult<Self::Output, IoError>; where
TSocket: AsyncRead + AsyncWrite,
{
type Output = PingListener<TSocket>;
type Error = IoError;
type Future = FutureResult<Self::Output, Self::Error>;
#[inline] #[inline]
fn upgrade( fn upgrade_inbound(self, socket: TSocket, _: Self::UpgradeId) -> Self::Future {
self, let listener = PingListener {
socket: TSocket, inner: Framed::new(socket, Codec),
_: Self::UpgradeIdentifier, state: PingListenerState::Listening,
endpoint: Endpoint,
) -> Self::Future {
let out = match endpoint {
Endpoint::Dialer => upgrade_as_dialer(socket),
Endpoint::Listener => upgrade_as_listener(socket),
}; };
future::ok(listener)
Ok(out).into_future()
} }
} }
/// Upgrades a connection from the dialer side. impl<TSocket, TUserData> OutboundUpgrade<TSocket> for Ping<TUserData>
fn upgrade_as_dialer<TSocket, TUserData>(socket: TSocket) -> PingOutput<TSocket, TUserData> where
where TSocket: AsyncRead + AsyncWrite, TSocket: AsyncRead + AsyncWrite,
{ {
type Output = PingDialer<TSocket, TUserData>;
type Error = IoError;
type Future = FutureResult<Self::Output, Self::Error>;
#[inline]
fn upgrade_outbound(self, socket: TSocket, _: Self::UpgradeId) -> Self::Future {
let dialer = PingDialer { let dialer = PingDialer {
inner: Framed::new(socket, Codec), inner: Framed::new(socket, Codec),
need_writer_flush: false, need_writer_flush: false,
@ -93,20 +88,8 @@ where TSocket: AsyncRead + AsyncWrite,
rng: EntropyRng::default(), rng: EntropyRng::default(),
pings_to_send: VecDeque::with_capacity(4), pings_to_send: VecDeque::with_capacity(4),
}; };
future::ok(dialer)
PingOutput::Pinger(dialer) }
}
/// Upgrades a connection from the listener side.
fn upgrade_as_listener<TSocket, TUserData>(socket: TSocket) -> PingOutput<TSocket, TUserData>
where TSocket: AsyncRead + AsyncWrite,
{
let listener = PingListener {
inner: Framed::new(socket, Codec),
state: PingListenerState::Listening,
};
PingOutput::Ponger(listener)
} }
/// Sends pings and receives the pongs. /// Sends pings and receives the pongs.
@ -342,9 +325,9 @@ mod tests {
use self::tokio_tcp::TcpListener; use self::tokio_tcp::TcpListener;
use self::tokio_tcp::TcpStream; use self::tokio_tcp::TcpStream;
use super::{Ping, PingOutput}; use super::Ping;
use futures::{Future, Stream}; use futures::{Future, Stream};
use libp2p_core::{ConnectionUpgrade, Endpoint}; use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade};
// TODO: rewrite tests with the MemoryTransport // TODO: rewrite tests with the MemoryTransport
@ -358,32 +341,18 @@ mod tests {
.into_future() .into_future()
.map_err(|(e, _)| e.into()) .map_err(|(e, _)| e.into())
.and_then(|(c, _)| { .and_then(|(c, _)| {
Ping::<()>::default().upgrade( Ping::<()>::default().upgrade_inbound(c.unwrap(), ())
c.unwrap(),
(),
Endpoint::Listener,
)
}) })
.and_then(|out| match out { .flatten();
PingOutput::Ponger(service) => service,
_ => unreachable!(),
});
let client = TcpStream::connect(&listener_addr) let client = TcpStream::connect(&listener_addr)
.map_err(|e| e.into()) .map_err(|e| e.into())
.and_then(|c| { .and_then(|c| {
Ping::<()>::default().upgrade( Ping::<()>::default().upgrade_outbound(c, ())
c,
(),
Endpoint::Dialer,
)
}) })
.and_then(|out| match out { .and_then(|mut pinger| {
PingOutput::Pinger(mut pinger) => {
pinger.ping(()); pinger.ping(());
pinger.into_future().map(|_| ()).map_err(|_| panic!()) pinger.into_future().map(|_| ()).map_err(|_| panic!())
},
_ => unreachable!(),
}) })
.map(|_| ()); .map(|_| ());
@ -402,39 +371,23 @@ mod tests {
.into_future() .into_future()
.map_err(|(e, _)| e.into()) .map_err(|(e, _)| e.into())
.and_then(|(c, _)| { .and_then(|(c, _)| {
Ping::<u32>::default().upgrade( Ping::<u32>::default().upgrade_inbound(c.unwrap(), ())
c.unwrap(),
(),
Endpoint::Listener,
)
}) })
.and_then(|out| match out { .flatten();
PingOutput::Ponger(service) => service,
_ => unreachable!(),
});
let client = TcpStream::connect(&listener_addr) let client = TcpStream::connect(&listener_addr)
.map_err(|e| e.into()) .map_err(|e| e.into())
.and_then(|c| { .and_then(|c| {
Ping::<u32>::default().upgrade( Ping::<u32>::default().upgrade_outbound(c, ())
c,
(),
Endpoint::Dialer,
)
}) })
.and_then(|out| match out { .and_then(|mut pinger| {
PingOutput::Pinger(mut pinger) => {
for n in 0..20 { for n in 0..20 {
pinger.ping(n); pinger.ping(n);
} }
pinger.take(20)
pinger
.take(20)
.collect() .collect()
.map(|val| { assert_eq!(val, (0..20).collect::<Vec<_>>()); }) .map(|val| { assert_eq!(val, (0..20).collect::<Vec<_>>()); })
.map_err(|_| panic!()) .map_err(|_| panic!())
},
_ => unreachable!(),
}); });
let mut runtime = tokio::runtime::Runtime::new().unwrap(); let mut runtime = tokio::runtime::Runtime::new().unwrap();

View File

@ -0,0 +1,12 @@
[package]
name = "libp2p-plaintext"
version = "0.1.0"
authors = ["Parity Technologies <admin@parity.io>"]
license = "MIT"
[dependencies]
bytes = "0.4"
futures = "0.1"
libp2p-core = { path = "../../core" }
void = "1"

View File

@ -1,4 +1,4 @@
// Copyright 2017 Parity Technologies (UK) Ltd. // Copyright 2018 Parity Technologies (UK) Ltd.
// //
// Permission is hereby granted, free of charge, to any person obtaining a // Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"), // copy of this software and associated documentation files (the "Software"),
@ -18,36 +18,46 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
extern crate bytes;
extern crate futures;
extern crate libp2p_core;
extern crate void;
use bytes::Bytes; use bytes::Bytes;
use futures::future::{self, FutureResult}; use futures::future::{self, FutureResult};
use std::{iter, io::Error as IoError}; use libp2p_core::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
use tokio_io::{AsyncRead, AsyncWrite}; use std::iter;
use upgrade::{ConnectionUpgrade, Endpoint}; use void::Void;
/// Implementation of the `ConnectionUpgrade` that negotiates the `/plaintext/1.0.0` protocol and
/// simply passes communications through without doing anything more.
///
/// > **Note**: Generally used as an alternative to `secio` if a security layer is not desirable.
// TODO: move to a separate crate?
#[derive(Debug, Copy, Clone)] #[derive(Debug, Copy, Clone)]
pub struct PlainTextConfig; pub struct PlainTextConfig;
impl<C> ConnectionUpgrade<C> for PlainTextConfig impl UpgradeInfo for PlainTextConfig {
where type UpgradeId = ();
C: AsyncRead + AsyncWrite, type NamesIter = iter::Once<(Bytes, Self::UpgradeId)>;
{
type Output = C;
type Future = FutureResult<C, IoError>;
type UpgradeIdentifier = ();
type NamesIter = iter::Once<(Bytes, ())>;
#[inline]
fn upgrade(self, i: C, _: (), _: Endpoint) -> Self::Future {
future::ok(i)
}
#[inline]
fn protocol_names(&self) -> Self::NamesIter { fn protocol_names(&self) -> Self::NamesIter {
iter::once((Bytes::from("/plaintext/1.0.0"), ())) iter::once((Bytes::from("/plaintext/1.0.0"), ()))
} }
} }
impl<C> InboundUpgrade<C> for PlainTextConfig {
type Output = C;
type Error = Void;
type Future = FutureResult<C, Self::Error>;
fn upgrade_inbound(self, i: C, _: Self::UpgradeId) -> Self::Future {
future::ok(i)
}
}
impl<C> OutboundUpgrade<C> for PlainTextConfig {
type Output = C;
type Error = Void;
type Future = FutureResult<C, Self::Error>;
fn upgrade_outbound(self, i: C, _: Self::UpgradeId) -> Self::Future {
future::ok(i)
}
}

View File

@ -39,12 +39,13 @@
//! # fn main() { //! # fn main() {
//! use futures::Future; //! use futures::Future;
//! use libp2p_secio::{SecioConfig, SecioKeyPair, SecioOutput}; //! use libp2p_secio::{SecioConfig, SecioKeyPair, SecioOutput};
//! use libp2p_core::{Multiaddr, Transport, upgrade}; //! use libp2p_core::{Multiaddr, upgrade::apply_inbound};
//! use libp2p_core::transport::Transport;
//! use libp2p_tcp_transport::TcpConfig; //! use libp2p_tcp_transport::TcpConfig;
//! use tokio_io::io::write_all; //! use tokio_io::io::write_all;
//! use tokio::runtime::current_thread::Runtime; //! use tokio::runtime::current_thread::Runtime;
//! //!
//! let transport = TcpConfig::new() //! let dialer = TcpConfig::new()
//! .with_upgrade({ //! .with_upgrade({
//! # let private_key = b""; //! # let private_key = b"";
//! //let private_key = include_bytes!("test-rsa-private-key.pk8"); //! //let private_key = include_bytes!("test-rsa-private-key.pk8");
@ -52,17 +53,17 @@
//! //let public_key = include_bytes!("test-rsa-public-key.der").to_vec(); //! //let public_key = include_bytes!("test-rsa-public-key.der").to_vec();
//! // See the documentation of `SecioKeyPair`. //! // See the documentation of `SecioKeyPair`.
//! let keypair = SecioKeyPair::rsa_from_pkcs8(private_key, public_key).unwrap(); //! let keypair = SecioKeyPair::rsa_from_pkcs8(private_key, public_key).unwrap();
//! let upgrade = SecioConfig::new(keypair); //! SecioConfig::new(keypair)
//! })
//! .map(|out: SecioOutput<_>, _| out.stream);
//! //!
//! upgrade::map(upgrade, |out: SecioOutput<_>| out.stream) //! let future = dialer.dial("/ip4/127.0.0.1/tcp/12345".parse::<Multiaddr>().unwrap())
//! });
//!
//! let future = transport.dial("/ip4/127.0.0.1/tcp/12345".parse::<Multiaddr>().unwrap())
//! .unwrap_or_else(|_| panic!("Unable to dial node")) //! .unwrap_or_else(|_| panic!("Unable to dial node"))
//! .and_then(|connection| { //! .and_then(|connection| {
//! // Sends "hello world" on the connection, will be encrypted. //! // Sends "hello world" on the connection, will be encrypted.
//! write_all(connection, "hello world") //! write_all(connection, "hello world")
//! }); //! })
//! .map_err(|e| panic!("error: {:?}", e));
//! //!
//! let mut rt = Runtime::new().unwrap(); //! let mut rt = Runtime::new().unwrap();
//! let _ = rt.block_on(future).unwrap(); //! let _ = rt.block_on(future).unwrap();
@ -119,7 +120,7 @@ use bytes::{Bytes, BytesMut};
use ed25519_dalek::Keypair as Ed25519KeyPair; use ed25519_dalek::Keypair as Ed25519KeyPair;
use futures::stream::MapErr as StreamMapErr; use futures::stream::MapErr as StreamMapErr;
use futures::{Future, Poll, Sink, StartSend, Stream}; use futures::{Future, Poll, Sink, StartSend, Stream};
use libp2p_core::{PeerId, PublicKey}; use libp2p_core::{PeerId, PublicKey, upgrade::{UpgradeInfo, InboundUpgrade, OutboundUpgrade}};
#[cfg(all(feature = "rsa", not(target_os = "emscripten")))] #[cfg(all(feature = "rsa", not(target_os = "emscripten")))]
use ring::signature::RSAKeyPair; use ring::signature::RSAKeyPair;
use rw_stream_sink::RwStreamSink; use rw_stream_sink::RwStreamSink;
@ -191,6 +192,22 @@ impl SecioConfig {
self.digests_prop = Some(algo_support::digests_proposition(xs)); self.digests_prop = Some(algo_support::digests_proposition(xs));
self self
} }
fn handshake<T>(self, socket: T, _: ()) -> impl Future<Item=SecioOutput<T>, Error=SecioError>
where
T: AsyncRead + AsyncWrite + Send + 'static
{
debug!("Starting secio upgrade");
SecioMiddleware::handshake(socket, self)
.map(|(stream_sink, pubkey, ephemeral)| {
let mapped = stream_sink.map_err(map_err as fn(_) -> _);
SecioOutput {
stream: RwStreamSink::new(mapped),
remote_key: pubkey,
ephemeral_public_key: ephemeral
}
})
}
} }
/// Private and public keys of the local node. /// Private and public keys of the local node.
@ -353,39 +370,38 @@ where
pub ephemeral_public_key: Vec<u8>, pub ephemeral_public_key: Vec<u8>,
} }
impl<S> libp2p_core::ConnectionUpgrade<S> for SecioConfig impl UpgradeInfo for SecioConfig {
where type UpgradeId = ();
S: AsyncRead + AsyncWrite + Send + 'static, // TODO: 'static :( type NamesIter = iter::Once<(Bytes, Self::UpgradeId)>;
{
type Output = SecioOutput<S>;
type Future = Box<Future<Item = Self::Output, Error = IoError> + Send>;
type NamesIter = iter::Once<(Bytes, ())>;
type UpgradeIdentifier = ();
#[inline]
fn protocol_names(&self) -> Self::NamesIter { fn protocol_names(&self) -> Self::NamesIter {
iter::once(("/secio/1.0.0".into(), ())) iter::once(("/secio/1.0.0".into(), ()))
} }
}
#[inline] impl<T> InboundUpgrade<T> for SecioConfig
fn upgrade( where
self, T: AsyncRead + AsyncWrite + Send + 'static
incoming: S, {
_: (), type Output = SecioOutput<T>;
_: libp2p_core::Endpoint, type Error = SecioError;
) -> Self::Future { type Future = Box<dyn Future<Item = Self::Output, Error = Self::Error> + Send>;
debug!("Starting secio upgrade");
let fut = SecioMiddleware::handshake(incoming, self); fn upgrade_inbound(self, socket: T, id: Self::UpgradeId) -> Self::Future {
let wrapped = fut.map(|(stream_sink, pubkey, ephemeral)| { Box::new(self.handshake(socket, id))
let mapped = stream_sink.map_err(map_err as fn(_) -> _);
SecioOutput {
stream: RwStreamSink::new(mapped),
remote_key: pubkey,
ephemeral_public_key: ephemeral,
} }
}).map_err(map_err); }
Box::new(wrapped)
impl<T> OutboundUpgrade<T> for SecioConfig
where
T: AsyncRead + AsyncWrite + Send + 'static
{
type Output = SecioOutput<T>;
type Error = SecioError;
type Future = Box<dyn Future<Item = Self::Output, Error = Self::Error> + Send>;
fn upgrade_outbound(self, socket: T, id: Self::UpgradeId) -> Self::Future {
Box::new(self.handshake(socket, id))
} }
} }

View File

@ -164,8 +164,11 @@ mod transport_ext;
pub mod simple; pub mod simple;
pub use self::core::{
Transport, PeerId,
upgrade::{InboundUpgrade, InboundUpgradeExt, OutboundUpgrade, OutboundUpgradeExt}
};
pub use libp2p_core_derive::NetworkBehaviour; pub use libp2p_core_derive::NetworkBehaviour;
pub use self::core::{Transport, ConnectionUpgrade, PeerId};
pub use self::multiaddr::Multiaddr; pub use self::multiaddr::Multiaddr;
pub use self::simple::SimpleProtocol; pub use self::simple::SimpleProtocol;
pub use self::transport_ext::TransportExt; pub use self::transport_ext::TransportExt;

View File

@ -19,7 +19,7 @@
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use bytes::Bytes; use bytes::Bytes;
use core::upgrade::{ConnectionUpgrade, Endpoint}; use core::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
use futures::prelude::*; use futures::prelude::*;
use std::{iter, io::Error as IoError, sync::Arc}; use std::{iter, io::Error as IoError, sync::Arc};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
@ -57,26 +57,48 @@ impl<F> Clone for SimpleProtocol<F> {
} }
} }
impl<C, F, O> ConnectionUpgrade<C> for SimpleProtocol<F> impl<F> UpgradeInfo for SimpleProtocol<F> {
type UpgradeId = ();
type NamesIter = iter::Once<(Bytes, Self::UpgradeId)>;
#[inline]
fn protocol_names(&self) -> Self::NamesIter {
iter::once((self.name.clone(), ()))
}
}
impl<C, F, O> InboundUpgrade<C> for SimpleProtocol<F>
where where
C: AsyncRead + AsyncWrite, C: AsyncRead + AsyncWrite,
F: Fn(C) -> O, F: Fn(C) -> O,
O: IntoFuture<Error = IoError>, O: IntoFuture<Error = IoError>,
O::Future: Send + 'static, O::Future: Send + 'static,
{ {
type NamesIter = iter::Once<(Bytes, ())>;
type UpgradeIdentifier = ();
#[inline]
fn protocol_names(&self) -> Self::NamesIter {
iter::once((self.name.clone(), ()))
}
type Output = O::Item; type Output = O::Item;
type Future = Box<Future<Item = O::Item, Error = IoError> + Send>; type Error = IoError;
type Future = Box<Future<Item = O::Item, Error = Self::Error> + Send>;
#[inline] #[inline]
fn upgrade(self, socket: C, _: (), _: Endpoint) -> Self::Future { fn upgrade_inbound(self, socket: C, _: Self::UpgradeId) -> Self::Future {
let upgrade = &self.upgrade;
let fut = upgrade(socket).into_future().from_err();
Box::new(fut) as Box<_>
}
}
impl<C, F, O> OutboundUpgrade<C> for SimpleProtocol<F>
where
C: AsyncRead + AsyncWrite,
F: Fn(C) -> O,
O: IntoFuture<Error = IoError>,
O::Future: Send + 'static,
{
type Output = O::Item;
type Error = IoError;
type Future = Box<Future<Item = O::Item, Error = Self::Error> + Send>;
#[inline]
fn upgrade_outbound(self, socket: C, _: Self::UpgradeId) -> Self::Future {
let upgrade = &self.upgrade; let upgrade = &self.upgrade;
let fut = upgrade(socket).into_future().from_err(); let fut = upgrade(socket).into_future().from_err();
Box::new(fut) as Box<_> Box::new(fut) as Box<_>

View File

@ -16,3 +16,4 @@ rand = "0.4"
tokio-codec = "0.1" tokio-codec = "0.1"
tokio-io = "0.1" tokio-io = "0.1"
unsigned-varint = { version = "0.2.1", features = ["codec"] } unsigned-varint = { version = "0.2.1", features = ["codec"] }
void = "1"

View File

@ -26,8 +26,8 @@
// Based on copy.rs from tokio-io in https://github.com/tokio-rs/tokio // Based on copy.rs from tokio-io in https://github.com/tokio-rs/tokio
use futures::{Future, Poll, try_ready};
use std::io; use std::io;
use futures::{Future, Poll};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
/// A future which will copy all data from a reader into a writer. /// A future which will copy all data from a reader into a writer.

View File

@ -0,0 +1,76 @@
// Copyright 2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use libp2p_core::{PeerId, upgrade::UpgradeError};
use std::{fmt, io};
#[derive(Debug)]
pub enum RelayError<E> {
Io(io::Error),
Upgrade(UpgradeError<E>),
NoRelayFor(PeerId),
Message(&'static str),
#[doc(hidden)]
__Nonexhaustive
}
impl<E> fmt::Display for RelayError<E>
where
E: fmt::Display
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
RelayError::Io(e) => write!(f, "i/o error: {}", e),
RelayError::Upgrade(e) => write!(f, "upgrade error: {}", e),
RelayError::NoRelayFor(p) => write!(f, "no relay for peer: {:?}", p),
RelayError::Message(m) => write!(f, "{}", m),
RelayError::__Nonexhaustive => f.write_str("__Nonexhaustive")
}
}
}
impl<E> std::error::Error for RelayError<E>
where
E: std::error::Error
{
fn cause(&self) -> Option<&dyn std::error::Error> {
match self {
RelayError::Io(e) => Some(e),
RelayError::Upgrade(e) => Some(e),
RelayError::NoRelayFor(_) => None,
RelayError::Message(_) => None,
RelayError::__Nonexhaustive => None
}
}
}
impl<E> From<io::Error> for RelayError<E> {
fn from(e: io::Error) -> Self {
RelayError::Io(e)
}
}
impl<E> From<UpgradeError<E>> for RelayError<E> {
fn from(e: UpgradeError<E>) -> Self {
RelayError::Upgrade(e)
}
}

View File

@ -19,11 +19,9 @@
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
extern crate bytes; extern crate bytes;
#[macro_use]
extern crate futures; extern crate futures;
extern crate libp2p_peerstore as peerstore; extern crate libp2p_peerstore as peerstore;
extern crate libp2p_core as core; extern crate libp2p_core;
#[macro_use]
extern crate log; extern crate log;
extern crate multiaddr; extern crate multiaddr;
extern crate protobuf; extern crate protobuf;
@ -31,8 +29,10 @@ extern crate rand;
extern crate tokio_codec; extern crate tokio_codec;
extern crate tokio_io; extern crate tokio_io;
extern crate unsigned_varint; extern crate unsigned_varint;
extern crate void;
mod copy; mod copy;
mod error;
mod message; mod message;
mod protocol; mod protocol;
mod transport; mod transport;

View File

@ -19,19 +19,27 @@
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use bytes::Bytes; use bytes::Bytes;
use copy; use crate::{
use core::{ConnectionUpgrade, Endpoint, Transport}; copy,
error::RelayError,
message::{CircuitRelay, CircuitRelay_Peer, CircuitRelay_Status, CircuitRelay_Type},
utility::{io_err, is_success, status, Io, Peer}
};
use futures::{stream, future::{self, Either::{A, B}, FutureResult}, prelude::*}; use futures::{stream, future::{self, Either::{A, B}, FutureResult}, prelude::*};
use message::{CircuitRelay, CircuitRelay_Peer, CircuitRelay_Status, CircuitRelay_Type}; use libp2p_core::{
transport::Transport,
upgrade::{apply_outbound, InboundUpgrade, OutboundUpgrade, UpgradeInfo}
};
use log::debug;
use peerstore::{PeerAccess, PeerId, Peerstore}; use peerstore::{PeerAccess, PeerId, Peerstore};
use std::{io, iter, ops::Deref}; use std::{io, iter, ops::Deref};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use utility::{io_err, is_success, status, Io, Peer}; use void::Void;
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub struct RelayConfig<T, P> { pub struct RelayConfig<T, P> {
my_id: PeerId, my_id: PeerId,
transport: T, dialer: T,
peers: P, peers: P,
// If `allow_relays` is false this node can only be used as a // If `allow_relays` is false this node can only be used as a
// destination but will not allow relaying streams to other // destination but will not allow relaying streams to other
@ -49,7 +57,16 @@ pub enum Output<C> {
Sealed(Box<Future<Item=(), Error=io::Error> + Send>) Sealed(Box<Future<Item=(), Error=io::Error> + Send>)
} }
impl<C, T, P, S> ConnectionUpgrade<C> for RelayConfig<T, P> impl<T, P> UpgradeInfo for RelayConfig<T, P> {
type UpgradeId = ();
type NamesIter = iter::Once<(Bytes, Self::UpgradeId)>;
fn protocol_names(&self) -> Self::NamesIter {
iter::once((Bytes::from("/libp2p/relay/circuit/0.1.0"), ()))
}
}
impl<C, T, P, S> InboundUpgrade<C> for RelayConfig<T, P>
where where
C: AsyncRead + AsyncWrite + Send + 'static, C: AsyncRead + AsyncWrite + Send + 'static,
T: Transport + Clone + Send + 'static, T: Transport + Clone + Send + 'static,
@ -61,34 +78,28 @@ where
S: 'static, S: 'static,
for<'a> &'a S: Peerstore for<'a> &'a S: Peerstore
{ {
type NamesIter = iter::Once<(Bytes, Self::UpgradeIdentifier)>;
type UpgradeIdentifier = ();
fn protocol_names(&self) -> Self::NamesIter {
iter::once((Bytes::from("/libp2p/relay/circuit/0.1.0"), ()))
}
type Output = Output<C>; type Output = Output<C>;
type Future = Box<Future<Item=Self::Output, Error=io::Error> + Send>; type Error = RelayError<Void>;
type Future = Box<Future<Item=Self::Output, Error=Self::Error> + Send>;
fn upgrade(self, conn: C, _: (), _: Endpoint) -> Self::Future { fn upgrade_inbound(self, conn: C, _: ()) -> Self::Future {
let future = Io::new(conn).recv().and_then(move |(message, io)| { let future = Io::new(conn).recv().from_err().and_then(move |(message, io)| {
let msg = if let Some(m) = message { let msg = if let Some(m) = message {
m m
} else { } else {
return A(A(future::err(io_err("no message received")))) return A(A(future::err(RelayError::Message("no message received"))))
}; };
match msg.get_field_type() { match msg.get_field_type() {
CircuitRelay_Type::HOP if self.allow_relays => { // act as relay CircuitRelay_Type::HOP if self.allow_relays => { // act as relay
B(A(self.on_hop(msg, io).map(|fut| Output::Sealed(Box::new(fut))))) B(A(self.on_hop(msg, io).map(|fut| Output::Sealed(Box::new(fut)))))
} }
CircuitRelay_Type::STOP => { // act as destination CircuitRelay_Type::STOP => { // act as destination
B(B(self.on_stop(msg, io).map(Output::Stream))) B(B(self.on_stop(msg, io).from_err().map(Output::Stream)))
} }
other => { other => {
debug!("invalid message type: {:?}", other); debug!("invalid message type: {:?}", other);
let resp = status(CircuitRelay_Status::MALFORMED_MESSAGE); let resp = status(CircuitRelay_Status::MALFORMED_MESSAGE);
A(B(io.send(resp).and_then(|_| Err(io_err("invalid message type"))))) A(B(io.send(resp).from_err().and_then(|_| Err(RelayError::Message("invalid message type")))))
} }
} }
}); });
@ -106,8 +117,8 @@ where
P: Deref<Target = S> + Clone + 'static, P: Deref<Target = S> + Clone + 'static,
for<'a> &'a S: Peerstore, for<'a> &'a S: Peerstore,
{ {
pub fn new(my_id: PeerId, transport: T, peers: P) -> RelayConfig<T, P> { pub fn new(my_id: PeerId, dialer: T, peers: P) -> RelayConfig<T, P> {
RelayConfig { my_id, transport, peers, allow_relays: true } RelayConfig { my_id, dialer, peers, allow_relays: true }
} }
pub fn allow_relays(&mut self, val: bool) { pub fn allow_relays(&mut self, val: bool) {
@ -115,7 +126,7 @@ where
} }
// HOP message handling (relay mode). // HOP message handling (relay mode).
fn on_hop<C>(self, mut msg: CircuitRelay, io: Io<C>) -> impl Future<Item=impl Future<Item=(), Error=io::Error>, Error=io::Error> fn on_hop<C>(self, mut msg: CircuitRelay, io: Io<C>) -> impl Future<Item=impl Future<Item=(), Error=io::Error>, Error=RelayError<Void>>
where where
C: AsyncRead + AsyncWrite + 'static, C: AsyncRead + AsyncWrite + 'static,
{ {
@ -123,14 +134,14 @@ where
peer peer
} else { } else {
let msg = status(CircuitRelay_Status::HOP_SRC_MULTIADDR_INVALID); let msg = status(CircuitRelay_Status::HOP_SRC_MULTIADDR_INVALID);
return A(io.send(msg).and_then(|_| Err(io_err("invalid src address")))) return A(io.send(msg).from_err().and_then(|_| Err(RelayError::Message("invalid src address"))))
}; };
let mut dest = if let Some(peer) = Peer::from_message(msg.take_dstPeer()) { let mut dest = if let Some(peer) = Peer::from_message(msg.take_dstPeer()) {
peer peer
} else { } else {
let msg = status(CircuitRelay_Status::HOP_DST_MULTIADDR_INVALID); let msg = status(CircuitRelay_Status::HOP_DST_MULTIADDR_INVALID);
return B(A(io.send(msg).and_then(|_| Err(io_err("invalid dest address"))))) return B(A(io.send(msg).from_err().and_then(|_| Err(RelayError::Message("invalid dest address")))))
}; };
if dest.addrs.is_empty() { if dest.addrs.is_empty() {
@ -142,13 +153,12 @@ where
let stop = stop_message(&from, &dest); let stop = stop_message(&from, &dest);
let transport = self.transport.with_upgrade(TrivialUpgrade); let dialer = self.dialer;
let dest_id = dest.id;
let future = stream::iter_ok(dest.addrs.into_iter()) let future = stream::iter_ok(dest.addrs.into_iter())
.and_then(move |dest_addr| { .and_then(move |dest_addr| {
transport.clone().dial(dest_addr).map_err(|_| io_err("failed to dial")) dialer.clone().dial(dest_addr).map_err(|_| RelayError::Message("could no dial addr"))
}) })
.and_then(|dial| dial) .and_then(|outbound| outbound.from_err().and_then(|c| apply_outbound(c, TrivialUpgrade).from_err()))
.then(|result| Ok(result.ok())) .then(|result| Ok(result.ok()))
.filter_map(|result| result) .filter_map(|result| result)
.into_future() .into_future()
@ -156,22 +166,24 @@ where
.and_then(move |(ok, _stream)| { .and_then(move |(ok, _stream)| {
if let Some(c) = ok { if let Some(c) = ok {
// send STOP message to destination and expect back a SUCCESS message // send STOP message to destination and expect back a SUCCESS message
let future = Io::new(c).send(stop) let future = Io::new(c)
.send(stop)
.and_then(Io::recv) .and_then(Io::recv)
.from_err()
.and_then(|(response, io)| { .and_then(|(response, io)| {
let rsp = match response { let rsp = match response {
Some(m) => m, Some(m) => m,
None => return Err(io_err("no message from destination")) None => return Err(RelayError::Message("no message from destination"))
}; };
if is_success(&rsp) { if is_success(&rsp) {
Ok(io.into()) Ok(io.into())
} else { } else {
Err(io_err("no success response from relay")) Err(RelayError::Message("no success response from relay"))
} }
}); });
A(future) A(future)
} else { } else {
B(future::err(io_err(format!("could not dial to {:?}", dest_id)))) B(future::err(RelayError::Message("could not dial peer")))
} }
}) })
// signal success or failure to source // signal success or failure to source
@ -179,11 +191,11 @@ where
match result { match result {
Ok(c) => { Ok(c) => {
let msg = status(CircuitRelay_Status::SUCCESS); let msg = status(CircuitRelay_Status::SUCCESS);
A(io.send(msg).map(|io| (io.into(), c))) A(io.send(msg).map(|io| (io.into(), c)).from_err())
} }
Err(e) => { Err(e) => {
let msg = status(CircuitRelay_Status::HOP_CANT_DIAL_DST); let msg = status(CircuitRelay_Status::HOP_CANT_DIAL_DST);
B(io.send(msg).and_then(|_| Err(e))) B(io.send(msg).from_err().and_then(|_| Err(e)))
} }
} }
}) })
@ -247,21 +259,24 @@ fn stop_message(from: &Peer, dest: &Peer) -> CircuitRelay {
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
struct TrivialUpgrade; struct TrivialUpgrade;
impl<C> ConnectionUpgrade<C> for TrivialUpgrade impl UpgradeInfo for TrivialUpgrade {
where type UpgradeId = ();
C: AsyncRead + AsyncWrite + 'static type NamesIter = iter::Once<(Bytes, Self::UpgradeId)>;
{
type NamesIter = iter::Once<(Bytes, Self::UpgradeIdentifier)>;
type UpgradeIdentifier = ();
fn protocol_names(&self) -> Self::NamesIter { fn protocol_names(&self) -> Self::NamesIter {
iter::once((Bytes::from("/libp2p/relay/circuit/0.1.0"), ())) iter::once((Bytes::from("/libp2p/relay/circuit/0.1.0"), ()))
} }
}
impl<C> OutboundUpgrade<C> for TrivialUpgrade
where
C: AsyncRead + AsyncWrite + 'static
{
type Output = C; type Output = C;
type Future = FutureResult<Self::Output, io::Error>; type Error = Void;
type Future = FutureResult<Self::Output, Self::Error>;
fn upgrade(self, conn: C, _: (), _: Endpoint) -> Self::Future { fn upgrade_outbound(self, conn: C, _: ()) -> Self::Future {
future::ok(conn) future::ok(conn)
} }
} }
@ -269,21 +284,24 @@ where
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub(crate) struct Source(pub(crate) CircuitRelay); pub(crate) struct Source(pub(crate) CircuitRelay);
impl<C> ConnectionUpgrade<C> for Source impl UpgradeInfo for Source {
where type UpgradeId = ();
C: AsyncRead + AsyncWrite + Send + 'static, type NamesIter = iter::Once<(Bytes, Self::UpgradeId)>;
{
type NamesIter = iter::Once<(Bytes, Self::UpgradeIdentifier)>;
type UpgradeIdentifier = ();
fn protocol_names(&self) -> Self::NamesIter { fn protocol_names(&self) -> Self::NamesIter {
iter::once((Bytes::from("/libp2p/relay/circuit/0.1.0"), ())) iter::once((Bytes::from("/libp2p/relay/circuit/0.1.0"), ()))
} }
}
impl<C> OutboundUpgrade<C> for Source
where
C: AsyncRead + AsyncWrite + Send + 'static,
{
type Output = C; type Output = C;
type Future = Box<Future<Item=Self::Output, Error=io::Error> + Send>; type Error = io::Error;
type Future = Box<Future<Item=Self::Output, Error=Self::Error> + Send>;
fn upgrade(self, conn: C, _: (), _: Endpoint) -> Self::Future { fn upgrade_outbound(self, conn: C, _: ()) -> Self::Future {
let future = Io::new(conn) let future = Io::new(conn)
.send(self.0) .send(self.0)
.and_then(Io::recv) .and_then(Io::recv)

View File

@ -18,16 +18,20 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use core::Transport; use crate::{
error::RelayError,
message::{CircuitRelay, CircuitRelay_Peer, CircuitRelay_Type},
protocol,
utility::{Peer, RelayAddr}
};
use futures::{stream, prelude::*}; use futures::{stream, prelude::*};
use message::{CircuitRelay, CircuitRelay_Peer, CircuitRelay_Type}; use libp2p_core::{transport::Transport, upgrade::apply_outbound};
use log::{debug, info, trace};
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use peerstore::{PeerAccess, PeerId, Peerstore}; use peerstore::{PeerAccess, PeerId, Peerstore};
use protocol;
use rand::{self, Rng}; use rand::{self, Rng};
use std::{io, iter::FromIterator, ops::Deref, sync::Arc}; use std::{io, iter::FromIterator, ops::Deref, sync::Arc};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use utility::{io_err, Peer, RelayAddr};
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub struct RelayTransport<T, P> { pub struct RelayTransport<T, P> {
@ -70,10 +74,10 @@ where
RelayAddr::Address { relay, dest } => { RelayAddr::Address { relay, dest } => {
if let Some(ref r) = relay { if let Some(ref r) = relay {
let f = self.relay_via(r, &dest).map_err(|this| (this, addr))?; let f = self.relay_via(r, &dest).map_err(|this| (this, addr))?;
Ok(Box::new(f)) Ok(Box::new(f.map_err(|e| io::Error::new(io::ErrorKind::Other, e))))
} else { } else {
let f = self.relay_to(&dest).map_err(|this| (this, addr))?; let f = self.relay_to(&dest).map_err(|this| (this, addr))?;
Ok(Box::new(f)) Ok(Box::new(f.map_err(|e| io::Error::new(io::ErrorKind::Other, e))))
} }
} }
} }
@ -111,7 +115,7 @@ where
} }
// Relay to destination over any available relay node. // Relay to destination over any available relay node.
fn relay_to(self, destination: &Peer) -> Result<impl Future<Item=T::Output, Error=io::Error>, Self> { fn relay_to(self, destination: &Peer) -> Result<impl Future<Item=T::Output, Error=RelayError<io::Error>>, Self> {
trace!("relay_to {:?}", destination.id); trace!("relay_to {:?}", destination.id);
let mut dials = Vec::new(); let mut dials = Vec::new();
for relay in &*self.relays { for relay in &*self.relays {
@ -142,14 +146,14 @@ where
if let Some(out) = ok { if let Some(out) = ok {
Ok(out) Ok(out)
} else { } else {
Err(io_err(format!("no relay for {:?}", dest_peer))) Err(RelayError::NoRelayFor(dest_peer))
} }
}); });
Ok(future) Ok(future)
} }
// Relay to destination via the given peer. // Relay to destination via the given peer.
fn relay_via(self, relay: &Peer, destination: &Peer) -> Result<impl Future<Item=T::Output, Error=io::Error>, Self> { fn relay_via(self, relay: &Peer, destination: &Peer) -> Result<impl Future<Item=T::Output, Error=RelayError<io::Error>>, Self> {
trace!("relay_via {:?} to {:?}", relay.id, destination.id); trace!("relay_via {:?} to {:?}", relay.id, destination.id);
let mut addresses = Vec::new(); let mut addresses = Vec::new();
@ -171,10 +175,15 @@ where
let relay = relay.clone(); let relay = relay.clone();
let message = self.hop_message(destination); let message = self.hop_message(destination);
let transport = self.transport.with_upgrade(protocol::Source(message)); let upgrade = protocol::Source(message);
let dialer = self.transport;
let future = stream::iter_ok(addresses.into_iter()) let future = stream::iter_ok(addresses.into_iter())
.filter_map(move |addr| transport.clone().dial(addr).ok()) .filter_map(move |addr| dialer.clone().dial(addr).ok())
.and_then(|dial| dial) .and_then(move |dial| {
let upgrade = upgrade.clone();
dial.map_err(|_| RelayError::Message("could not dial"))
.and_then(move |c| apply_outbound(c, upgrade).from_err())
})
.then(|result| Ok(result.ok())) .then(|result| Ok(result.ok()))
.filter_map(|result| result) .filter_map(|result| result)
.into_future() .into_future()
@ -186,7 +195,7 @@ where
} }
None => { None => {
info!("failed to dial to {:?}", relay.id); info!("failed to dial to {:?}", relay.id);
Err(io_err(format!("failed to dial to relay {:?}", relay.id))) Err(RelayError::Message("failed to dial to relay"))
} }
}); });
Ok(future) Ok(future)

View File

@ -18,8 +18,9 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::message::{CircuitRelay, CircuitRelay_Peer, CircuitRelay_Status, CircuitRelay_Type};
use futures::{future::{self, Either}, prelude::*}; use futures::{future::{self, Either}, prelude::*};
use message::{CircuitRelay, CircuitRelay_Peer, CircuitRelay_Status, CircuitRelay_Type}; use log::trace;
use multiaddr::{Protocol, Multiaddr}; use multiaddr::{Protocol, Multiaddr};
use peerstore::PeerId; use peerstore::PeerId;
use protobuf::{self, Message}; use protobuf::{self, Message};