mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-18 12:31:22 +00:00
Split ConnectionUpgrade
. (#642)
Introduce `InboundUpgrade` and `OutboundUpgrade`.
This commit is contained in:
committed by
Pierre Krieger
parent
466385a58a
commit
2e549884ef
21
Cargo.toml
21
Cargo.toml
@ -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"
|
||||||
]
|
]
|
||||||
|
@ -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`.
|
||||||
|
@ -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
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@ -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,
|
||||||
|
@ -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);
|
||||||
|
@ -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),
|
||||||
|
@ -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,
|
||||||
|
@ -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,
|
||||||
{
|
{
|
||||||
|
@ -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
|
||||||
|
@ -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)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -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)
|
|
||||||
}
|
|
||||||
}
|
|
@ -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
92
core/src/upgrade/error.rs
Normal 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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -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<_>
|
|
||||||
}
|
|
||||||
}
|
|
@ -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))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@ -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
108
core/src/upgrade/or.rs
Normal 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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -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
|
||||||
|
{}
|
||||||
|
|
||||||
|
@ -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;
|
|
||||||
}
|
|
@ -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});
|
||||||
|
@ -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>>,
|
||||||
|
@ -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))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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() {
|
||||||
|
@ -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() }))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
|
@ -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;
|
||||||
|
@ -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,
|
||||||
>,
|
>,
|
||||||
|
@ -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,
|
||||||
>,
|
>,
|
||||||
|
@ -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
|
||||||
|
@ -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();
|
||||||
|
@ -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;
|
||||||
|
@ -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)
|
|
||||||
})
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
|
@ -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
|
||||||
|
@ -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 || {
|
||||||
|
@ -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`
|
||||||
|
@ -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;
|
||||||
|
@ -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
|
||||||
|
@ -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();
|
||||||
|
12
protocols/plaintext/Cargo.toml
Normal file
12
protocols/plaintext/Cargo.toml
Normal 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"
|
||||||
|
|
@ -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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -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))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
|
@ -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<_>
|
||||||
|
@ -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"
|
||||||
|
@ -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.
|
||||||
|
76
transports/relay/src/error.rs
Normal file
76
transports/relay/src/error.rs
Normal 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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -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;
|
||||||
|
@ -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)
|
||||||
|
@ -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)
|
||||||
|
@ -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};
|
||||||
|
Reference in New Issue
Block a user