mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-27 08:41:36 +00:00
Split ConnectionUpgrade
. (#642)
Introduce `InboundUpgrade` and `OutboundUpgrade`.
This commit is contained in:
committed by
Pierre Krieger
parent
466385a58a
commit
2e549884ef
@ -18,11 +18,42 @@
|
||||
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
|
||||
// DEALINGS IN THE SOFTWARE.
|
||||
|
||||
use crate::{muxing::{Shutdown, StreamMuxer}, Multiaddr};
|
||||
use futures::prelude::*;
|
||||
use muxing::{Shutdown, StreamMuxer};
|
||||
use std::io::{Error as IoError, Read, Write};
|
||||
use std::{fmt, io::{Error as IoError, Read, Write}};
|
||||
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
|
||||
/// either `First` or `Second`.
|
||||
|
@ -89,7 +89,7 @@
|
||||
//!
|
||||
//! # fn main() {
|
||||
//! 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
|
||||
//! # }
|
||||
@ -132,31 +132,28 @@
|
||||
//! extern crate tokio;
|
||||
//!
|
||||
//! use futures::{Future, Stream};
|
||||
//! use libp2p_ping::protocol::{Ping, PingOutput};
|
||||
//! use libp2p_core::Transport;
|
||||
//! use libp2p_ping::protocol::Ping;
|
||||
//! use libp2p_core::{Transport, upgrade::apply_outbound};
|
||||
//! use tokio::runtime::current_thread::Runtime;
|
||||
//!
|
||||
//! # fn main() {
|
||||
//! let ping_finished_future = libp2p_tcp_transport::TcpConfig::new()
|
||||
//! // We have a `TcpConfig` struct that implements `Transport`, and apply a `Ping` upgrade on it.
|
||||
//! .with_upgrade(Ping::default())
|
||||
//! let ping_dialer = libp2p_tcp_transport::TcpConfig::new()
|
||||
//! // We have a `TcpConfig` struct that implements `Dialer`, and apply a `Ping` upgrade on it.
|
||||
//! .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
|
||||
//! // something that is more simple to use
|
||||
//! .dial("127.0.0.1:12345".parse::<libp2p_core::Multiaddr>().unwrap()).unwrap_or_else(|_| panic!())
|
||||
//! .and_then(|out| {
|
||||
//! match out {
|
||||
//! PingOutput::Ponger(processing) => Box::new(processing) as Box<Future<Item = _, Error = _>>,
|
||||
//! PingOutput::Pinger(mut pinger) => {
|
||||
//! pinger.ping(());
|
||||
//! let f = pinger.into_future().map(|_| ()).map_err(|(err, _)| err);
|
||||
//! Box::new(f) as Box<Future<Item = _, Error = _>>
|
||||
//! },
|
||||
//! }
|
||||
//! .dial("/ip4/127.0.0.1/tcp/12345".parse::<libp2p_core::Multiaddr>().unwrap()).unwrap_or_else(|_| panic!())
|
||||
//! .and_then(|mut pinger| {
|
||||
//! pinger.ping(());
|
||||
//! let f = pinger.into_future().map(|_| ()).map_err(|(e, _)| e);
|
||||
//! Box::new(f) as Box<Future<Item = _, Error = _>>
|
||||
//! });
|
||||
//!
|
||||
//! // Runs until the ping arrives.
|
||||
//! 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::public_key::PublicKey;
|
||||
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
|
||||
// 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 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 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)
|
||||
-> ReachAttemptId
|
||||
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,
|
||||
TInEvent: Send + 'static,
|
||||
TOutEvent: Send + 'static,
|
||||
|
@ -18,18 +18,25 @@
|
||||
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
|
||||
// DEALINGS IN THE SOFTWARE.
|
||||
|
||||
use crate::{
|
||||
PeerId,
|
||||
muxing::StreamMuxer,
|
||||
nodes::{
|
||||
handled_node::{HandledNode, NodeHandler},
|
||||
node::Substream
|
||||
}
|
||||
};
|
||||
use fnv::FnvHashMap;
|
||||
use futures::{prelude::*, stream, sync::mpsc};
|
||||
use muxing::StreamMuxer;
|
||||
use nodes::node::Substream;
|
||||
use nodes::handled_node::{HandledNode, NodeHandler};
|
||||
use smallvec::SmallVec;
|
||||
use std::collections::hash_map::{Entry, OccupiedEntry};
|
||||
use std::io::Error as IoError;
|
||||
use std::{fmt, mem};
|
||||
use std::{
|
||||
collections::hash_map::{Entry, OccupiedEntry},
|
||||
fmt,
|
||||
io::{self, Error as IoError},
|
||||
mem
|
||||
};
|
||||
use tokio_executor;
|
||||
use void::Void;
|
||||
use 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
|
||||
/// events.
|
||||
pub fn add_reach_attempt<TFut, TMuxer>(&mut self, future: TFut, handler: THandler)
|
||||
-> TaskId
|
||||
pub fn add_reach_attempt<TFut, TMuxer>(&mut self, future: TFut, handler: THandler) -> TaskId
|
||||
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,
|
||||
TInEvent: Send + 'static,
|
||||
TOutEvent: Send + 'static,
|
||||
@ -349,7 +356,8 @@ impl<TFut, TMuxer, THandler, TInEvent, TOutEvent> Future for
|
||||
NodeTask<TFut, TMuxer, THandler, TInEvent, TOutEvent>
|
||||
where
|
||||
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>,
|
||||
{
|
||||
type Item = ();
|
||||
@ -388,7 +396,8 @@ where
|
||||
},
|
||||
Err(err) => {
|
||||
// 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));
|
||||
return Ok(Async::Ready(()));
|
||||
}
|
||||
@ -464,6 +473,7 @@ mod tests {
|
||||
use tests::dummy_muxer::{DummyMuxer, DummyConnectionState};
|
||||
use tokio::runtime::Builder;
|
||||
use tokio::runtime::current_thread::Runtime;
|
||||
use void::Void;
|
||||
use {PeerId, PublicKey};
|
||||
|
||||
type TestNodeTask = NodeTask<
|
||||
@ -570,7 +580,7 @@ mod tests {
|
||||
let peer_id = PublicKey::Rsa((0 .. 2048).map(|_| -> u8 { random() }).collect()).into_peer_id();
|
||||
let mut task_ids = Vec::new();
|
||||
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(
|
||||
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.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.to_spawn.len(), 1);
|
||||
|
@ -18,15 +18,26 @@
|
||||
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
|
||||
// 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 nodes::handled_node::{NodeHandler, NodeHandlerEndpoint, NodeHandlerEvent};
|
||||
use std::{io, marker::PhantomData, time::Duration};
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use tokio_timer::Timeout;
|
||||
use upgrade::{self, apply::UpgradeApplyFuture, DeniedConnectionUpgrade};
|
||||
use void::Void;
|
||||
use {ConnectionUpgrade, Endpoint};
|
||||
|
||||
/// 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.
|
||||
type Substream: AsyncRead + AsyncWrite;
|
||||
/// 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`,
|
||||
/// and will be passed back in `inject_substream` or `inject_outbound_closed`.
|
||||
type OutboundOpenInfo;
|
||||
@ -92,15 +105,22 @@ pub trait ProtocolsHandler {
|
||||
/// > 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
|
||||
/// > 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.
|
||||
///
|
||||
/// This method is called when a substream has been successfully opened and negotiated.
|
||||
fn inject_fully_negotiated(
|
||||
fn inject_fully_negotiated_inbound(
|
||||
&mut self,
|
||||
protocol: <Self::Protocol as ConnectionUpgrade<Self::Substream>>::Output,
|
||||
endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>,
|
||||
protocol: <Self::InboundProtocol as InboundUpgrade<Self::Substream>>::Output
|
||||
);
|
||||
|
||||
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.
|
||||
@ -126,12 +146,7 @@ pub trait ProtocolsHandler {
|
||||
/// > **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
|
||||
/// > `shutdown()` and will eventually be closed and destroyed.
|
||||
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>;
|
||||
|
||||
/// Adds a closure that turns the input event into something else.
|
||||
#[inline]
|
||||
@ -296,19 +311,32 @@ where
|
||||
type InEvent = Void;
|
||||
type OutEvent = Void;
|
||||
type Substream = TSubstream;
|
||||
type Protocol = DeniedConnectionUpgrade;
|
||||
type InboundProtocol = DeniedUpgrade;
|
||||
type OutboundProtocol = DeniedUpgrade;
|
||||
type OutboundOpenInfo = Void;
|
||||
|
||||
#[inline]
|
||||
fn listen_protocol(&self) -> Self::Protocol {
|
||||
DeniedConnectionUpgrade
|
||||
fn listen_protocol(&self) -> Self::InboundProtocol {
|
||||
DeniedUpgrade
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn inject_fully_negotiated(
|
||||
fn dialer_protocol(&self) -> Self::OutboundProtocol {
|
||||
DeniedUpgrade
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn inject_fully_negotiated_inbound(
|
||||
&mut self,
|
||||
_: <Self::Protocol as ConnectionUpgrade<TSubstream>>::Output,
|
||||
_: NodeHandlerEndpoint<Self::OutboundOpenInfo>,
|
||||
_: <Self::InboundProtocol as InboundUpgrade<TSubstream>>::Output
|
||||
) {
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn inject_fully_negotiated_outbound(
|
||||
&mut self,
|
||||
_: <Self::OutboundProtocol as OutboundUpgrade<TSubstream>>::Output,
|
||||
_: Self::OutboundOpenInfo
|
||||
) {
|
||||
}
|
||||
|
||||
@ -330,7 +358,7 @@ where
|
||||
fn poll(
|
||||
&mut self,
|
||||
) -> Poll<
|
||||
Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>,
|
||||
Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>,
|
||||
io::Error,
|
||||
> {
|
||||
if self.shutting_down {
|
||||
@ -356,21 +384,35 @@ where
|
||||
type InEvent = TNewIn;
|
||||
type OutEvent = TProtoHandler::OutEvent;
|
||||
type Substream = TProtoHandler::Substream;
|
||||
type Protocol = TProtoHandler::Protocol;
|
||||
type InboundProtocol = TProtoHandler::InboundProtocol;
|
||||
type OutboundProtocol = TProtoHandler::OutboundProtocol;
|
||||
type OutboundOpenInfo = TProtoHandler::OutboundOpenInfo;
|
||||
|
||||
#[inline]
|
||||
fn listen_protocol(&self) -> Self::Protocol {
|
||||
fn listen_protocol(&self) -> Self::InboundProtocol {
|
||||
self.inner.listen_protocol()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn inject_fully_negotiated(
|
||||
fn dialer_protocol(&self) -> Self::OutboundProtocol {
|
||||
self.inner.dialer_protocol()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn inject_fully_negotiated_inbound(
|
||||
&mut self,
|
||||
protocol: <Self::Protocol as ConnectionUpgrade<Self::Substream>>::Output,
|
||||
endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>,
|
||||
protocol: <Self::InboundProtocol as InboundUpgrade<Self::Substream>>::Output
|
||||
) {
|
||||
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]
|
||||
@ -399,7 +441,7 @@ where
|
||||
fn poll(
|
||||
&mut self,
|
||||
) -> Poll<
|
||||
Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>,
|
||||
Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>,
|
||||
io::Error,
|
||||
> {
|
||||
self.inner.poll()
|
||||
@ -420,21 +462,35 @@ where
|
||||
type InEvent = TProtoHandler::InEvent;
|
||||
type OutEvent = TNewOut;
|
||||
type Substream = TProtoHandler::Substream;
|
||||
type Protocol = TProtoHandler::Protocol;
|
||||
type InboundProtocol = TProtoHandler::InboundProtocol;
|
||||
type OutboundProtocol = TProtoHandler::OutboundProtocol;
|
||||
type OutboundOpenInfo = TProtoHandler::OutboundOpenInfo;
|
||||
|
||||
#[inline]
|
||||
fn listen_protocol(&self) -> Self::Protocol {
|
||||
fn listen_protocol(&self) -> Self::InboundProtocol {
|
||||
self.inner.listen_protocol()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn inject_fully_negotiated(
|
||||
fn dialer_protocol(&self) -> Self::OutboundProtocol {
|
||||
self.inner.dialer_protocol()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn inject_fully_negotiated_inbound(
|
||||
&mut self,
|
||||
protocol: <Self::Protocol as ConnectionUpgrade<Self::Substream>>::Output,
|
||||
endpoint: NodeHandlerEndpoint<Self::OutboundOpenInfo>,
|
||||
protocol: <Self::InboundProtocol as InboundUpgrade<Self::Substream>>::Output
|
||||
) {
|
||||
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]
|
||||
@ -461,7 +517,7 @@ where
|
||||
fn poll(
|
||||
&mut self,
|
||||
) -> Poll<
|
||||
Option<ProtocolsHandlerEvent<Self::Protocol, Self::OutboundOpenInfo, Self::OutEvent>>,
|
||||
Option<ProtocolsHandlerEvent<Self::OutboundProtocol, Self::OutboundOpenInfo, Self::OutEvent>>,
|
||||
io::Error,
|
||||
> {
|
||||
Ok(self.inner.poll()?.map(|ev| {
|
||||
@ -531,12 +587,12 @@ where
|
||||
handler: TProtoHandler,
|
||||
/// Futures that upgrade incoming substreams.
|
||||
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
|
||||
/// to pass back once successfully opened.
|
||||
negotiating_out: Vec<(
|
||||
TProtoHandler::OutboundOpenInfo,
|
||||
Timeout<UpgradeApplyFuture<TProtoHandler::Substream, TProtoHandler::Protocol>>,
|
||||
Timeout<OutboundUpgradeApply<TProtoHandler::Substream, TProtoHandler::OutboundProtocol>>,
|
||||
)>,
|
||||
/// Timeout for incoming substreams negotiation.
|
||||
in_timeout: Duration,
|
||||
@ -544,7 +600,7 @@ where
|
||||
out_timeout: Duration,
|
||||
/// For each outbound substream request, how to upgrade it. The first element of the tuple
|
||||
/// 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_dial_upgrade_id: u64,
|
||||
}
|
||||
@ -552,7 +608,8 @@ where
|
||||
impl<TProtoHandler> NodeHandler for NodeHandlerWrapper<TProtoHandler>
|
||||
where
|
||||
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 OutEvent = TProtoHandler::OutEvent;
|
||||
@ -569,7 +626,7 @@ where
|
||||
match endpoint {
|
||||
NodeHandlerEndpoint::Listener => {
|
||||
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);
|
||||
self.negotiating_in.push(with_timeout);
|
||||
}
|
||||
@ -587,7 +644,7 @@ where
|
||||
};
|
||||
|
||||
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);
|
||||
self.negotiating_out.push((user_data, with_timeout));
|
||||
}
|
||||
@ -630,21 +687,15 @@ where
|
||||
self.handler.shutdown();
|
||||
}
|
||||
|
||||
fn poll(
|
||||
&mut self,
|
||||
) -> Poll<Option<NodeHandlerEvent<Self::OutboundOpenInfo, Self::OutEvent>>, io::Error> {
|
||||
fn poll(&mut self) -> Poll<Option<NodeHandlerEvent<Self::OutboundOpenInfo, Self::OutEvent>>, io::Error> {
|
||||
// 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.
|
||||
for n in (0..self.negotiating_in.len()).rev() {
|
||||
let mut in_progress = self.negotiating_in.swap_remove(n);
|
||||
match in_progress.poll() {
|
||||
Ok(Async::Ready(upgrade)) => {
|
||||
self.handler
|
||||
.inject_fully_negotiated(upgrade, NodeHandlerEndpoint::Listener);
|
||||
}
|
||||
Ok(Async::NotReady) => {
|
||||
self.negotiating_in.push(in_progress);
|
||||
}
|
||||
Ok(Async::Ready(upgrade)) =>
|
||||
self.handler.inject_fully_negotiated_inbound(upgrade),
|
||||
Ok(Async::NotReady) => self.negotiating_in.push(in_progress),
|
||||
// TODO: return a diagnostic event?
|
||||
Err(_err) => {}
|
||||
}
|
||||
@ -656,8 +707,7 @@ where
|
||||
let (upgr_info, mut in_progress) = self.negotiating_out.swap_remove(n);
|
||||
match in_progress.poll() {
|
||||
Ok(Async::Ready(upgrade)) => {
|
||||
let endpoint = NodeHandlerEndpoint::Dialer(upgr_info);
|
||||
self.handler.inject_fully_negotiated(upgrade, endpoint);
|
||||
self.handler.inject_fully_negotiated_outbound(upgrade, upgr_info);
|
||||
}
|
||||
Ok(Async::NotReady) => {
|
||||
self.negotiating_out.push((upgr_info, in_progress));
|
||||
@ -704,45 +754,138 @@ pub struct ProtocolsHandlerSelect<TProto1, TProto2> {
|
||||
|
||||
impl<TSubstream, TProto1, TProto2, TProto1Out, TProto2Out>
|
||||
ProtocolsHandler for ProtocolsHandlerSelect<TProto1, TProto2>
|
||||
where TProto1: ProtocolsHandler<Substream = TSubstream>,
|
||||
TProto2: ProtocolsHandler<Substream = TSubstream>,
|
||||
TSubstream: AsyncRead + AsyncWrite,
|
||||
TProto1::Protocol: ConnectionUpgrade<TSubstream, Output = TProto1Out>,
|
||||
TProto2::Protocol: ConnectionUpgrade<TSubstream, Output = TProto2Out>,
|
||||
where
|
||||
TProto1: ProtocolsHandler<Substream = TSubstream>,
|
||||
TProto2: ProtocolsHandler<Substream = TSubstream>,
|
||||
TSubstream: AsyncRead + AsyncWrite,
|
||||
TProto1::InboundProtocol: InboundUpgrade<TSubstream, Output = TProto1Out>,
|
||||
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 OutEvent = EitherOutput<TProto1::OutEvent, TProto2::OutEvent>;
|
||||
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>;
|
||||
|
||||
#[inline]
|
||||
fn listen_protocol(&self) -> Self::Protocol {
|
||||
let proto1 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(self.proto1.listen_protocol(), EitherOutput::First));
|
||||
let proto2 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(self.proto2.listen_protocol(), EitherOutput::Second));
|
||||
upgrade::or(proto1, proto2)
|
||||
fn listen_protocol(&self) -> Self::InboundProtocol {
|
||||
let proto1 = self.proto1.listen_protocol()
|
||||
.map_inbound(EitherOutput::First as fn(TProto1Out) -> EitherOutput<TProto1Out, TProto2Out>)
|
||||
.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) {
|
||||
(EitherOutput::First(protocol), NodeHandlerEndpoint::Dialer(EitherOutput::First(info))) => {
|
||||
self.proto1.inject_fully_negotiated(protocol, NodeHandlerEndpoint::Dialer(info));
|
||||
},
|
||||
(EitherOutput::Second(protocol), NodeHandlerEndpoint::Dialer(EitherOutput::Second(info))) => {
|
||||
self.proto2.inject_fully_negotiated(protocol, NodeHandlerEndpoint::Dialer(info));
|
||||
},
|
||||
(EitherOutput::First(_), NodeHandlerEndpoint::Dialer(EitherOutput::Second(_))) => {
|
||||
(EitherOutput::First(protocol), EitherOutput::First(info)) =>
|
||||
self.proto1.inject_fully_negotiated_outbound(protocol, info),
|
||||
(EitherOutput::Second(protocol), EitherOutput::Second(info)) =>
|
||||
self.proto2.inject_fully_negotiated_outbound(protocol, info),
|
||||
(EitherOutput::First(_), EitherOutput::Second(_)) =>
|
||||
panic!("wrong API usage: the protocol doesn't match the upgrade info"),
|
||||
(EitherOutput::Second(_), EitherOutput::First(_)) =>
|
||||
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")
|
||||
},
|
||||
(EitherOutput::First(protocol), NodeHandlerEndpoint::Listener) => {
|
||||
self.proto1.inject_fully_negotiated(protocol, NodeHandlerEndpoint::Listener);
|
||||
},
|
||||
(EitherOutput::Second(protocol), NodeHandlerEndpoint::Listener) => {
|
||||
self.proto2.inject_fully_negotiated(protocol, NodeHandlerEndpoint::Listener);
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
fn inject_fully_negotiated_inbound(&mut self, protocol: <Self::InboundProtocol as InboundUpgrade<TSubstream>>::Output) {
|
||||
match protocol {
|
||||
EitherOutput::First(protocol) =>
|
||||
self.proto1.inject_fully_negotiated_inbound(protocol),
|
||||
EitherOutput::Second(protocol) =>
|
||||
self.proto2.inject_fully_negotiated_inbound(protocol)
|
||||
}
|
||||
}
|
||||
|
||||
@ -774,19 +917,33 @@ where TProto1: ProtocolsHandler<Substream = TSubstream>,
|
||||
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()? {
|
||||
Async::Ready(Some(ProtocolsHandlerEvent::Custom(event))) => {
|
||||
return Ok(Async::Ready(Some(ProtocolsHandlerEvent::Custom(EitherOutput::First(event)))));
|
||||
},
|
||||
Async::Ready(Some(ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info})) => {
|
||||
let upgrade = {
|
||||
let proto1 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(upgrade, EitherOutput::First));
|
||||
let mut proto2 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(self.proto2.listen_protocol(), EitherOutput::Second));
|
||||
proto2.disable();
|
||||
upgrade::or(proto1, proto2)
|
||||
};
|
||||
let proto1 = upgrade
|
||||
.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
|
||||
>);
|
||||
|
||||
let proto1 = upgrade::toggleable(proto1);
|
||||
let mut proto2 = upgrade::toggleable(proto2);
|
||||
proto2.disable();
|
||||
proto1.or_outbound(proto2)
|
||||
};
|
||||
return Ok(Async::Ready(Some(ProtocolsHandlerEvent::OutboundSubstreamRequest {
|
||||
upgrade,
|
||||
info: EitherOutput::First(info),
|
||||
@ -802,12 +959,26 @@ where TProto1: ProtocolsHandler<Substream = TSubstream>,
|
||||
},
|
||||
Async::Ready(Some(ProtocolsHandlerEvent::OutboundSubstreamRequest { upgrade, info })) => {
|
||||
let upgrade = {
|
||||
let mut proto1 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(self.proto1.listen_protocol(), EitherOutput::First));
|
||||
proto1.disable();
|
||||
let proto2 = upgrade::toggleable(upgrade::map::<_, fn(_) -> _>(upgrade, EitherOutput::Second));
|
||||
upgrade::or(proto1, proto2)
|
||||
};
|
||||
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 = 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 {
|
||||
upgrade,
|
||||
info: EitherOutput::Second(info),
|
||||
|
@ -18,18 +18,30 @@
|
||||
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
|
||||
// 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 futures::{prelude::*, future};
|
||||
use muxing::StreamMuxer;
|
||||
use nodes::collection::{
|
||||
CollectionEvent, CollectionNodeAccept, CollectionReachEvent, CollectionStream, PeerMut as CollecPeerMut, ReachAttemptId,
|
||||
use std::{
|
||||
collections::hash_map::{Entry, OccupiedEntry},
|
||||
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.
|
||||
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>
|
||||
where
|
||||
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
|
||||
TTrans::Dial: Send + 'static,
|
||||
TTrans: Transport<Output = (PeerId, TMuxer)>,
|
||||
TTrans::ListenerUpgrade: Send + 'static,
|
||||
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static,
|
||||
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
|
||||
@ -318,7 +329,7 @@ impl ConnectedPoint {
|
||||
impl<TTrans, TInEvent, TOutEvent, TMuxer, THandler>
|
||||
RawSwarm<TTrans, TInEvent, TOutEvent, THandler>
|
||||
where
|
||||
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
|
||||
TTrans: Transport + Clone,
|
||||
TMuxer: StreamMuxer,
|
||||
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent> + Send + 'static,
|
||||
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.
|
||||
pub fn dial(&mut self, addr: Multiaddr, handler: THandler) -> Result<(), Multiaddr>
|
||||
where
|
||||
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
|
||||
TTrans: Transport<Output = (PeerId, TMuxer)>,
|
||||
TTrans::Dial: Send + 'static,
|
||||
TMuxer: StreamMuxer + Send + Sync + 'static,
|
||||
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
|
||||
/// given peer.
|
||||
fn start_dial_out(&mut self, peer_id: PeerId, handler: THandler, first: Multiaddr,
|
||||
rest: Vec<Multiaddr>)
|
||||
fn start_dial_out(&mut self, peer_id: PeerId, handler: THandler, first: Multiaddr, rest: Vec<Multiaddr>)
|
||||
where
|
||||
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
|
||||
TTrans: Transport<Output = (PeerId, TMuxer)>,
|
||||
TTrans::Dial: Send + 'static,
|
||||
TMuxer: StreamMuxer + Send + Sync + 'static,
|
||||
TMuxer::OutboundSubstream: Send,
|
||||
@ -510,7 +520,7 @@ where
|
||||
/// Provides an API similar to `Stream`, except that it cannot error.
|
||||
pub fn poll(&mut self) -> Async<RawSwarmEvent<TTrans, TInEvent, TOutEvent, THandler>>
|
||||
where
|
||||
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
|
||||
TTrans: Transport<Output = (PeerId, TMuxer)>,
|
||||
TTrans::Dial: Send + 'static,
|
||||
TTrans::ListenerUpgrade: Send + 'static,
|
||||
TMuxer: StreamMuxer + Send + Sync + 'static,
|
||||
@ -524,11 +534,7 @@ where
|
||||
// Start by polling the listeners for events.
|
||||
match self.listeners.poll() {
|
||||
Async::NotReady => (),
|
||||
Async::Ready(ListenersEvent::Incoming {
|
||||
upgrade,
|
||||
listen_addr,
|
||||
send_back_addr,
|
||||
}) => {
|
||||
Async::Ready(ListenersEvent::Incoming { upgrade, listen_addr, send_back_addr }) => {
|
||||
let event = IncomingConnectionEvent {
|
||||
upgrade,
|
||||
listen_addr,
|
||||
@ -536,14 +542,9 @@ where
|
||||
active_nodes: &mut self.active_nodes,
|
||||
other_reach_attempts: &mut self.reach_attempts.other_reach_attempts,
|
||||
};
|
||||
|
||||
return Async::Ready(RawSwarmEvent::IncomingConnection(event));
|
||||
}
|
||||
Async::Ready(ListenersEvent::Closed {
|
||||
listen_addr,
|
||||
listener,
|
||||
result,
|
||||
}) => {
|
||||
Async::Ready(ListenersEvent::Closed { listen_addr, listener, result }) => {
|
||||
return Async::Ready(RawSwarmEvent::ListenerClosed {
|
||||
listen_addr,
|
||||
listener,
|
||||
@ -654,7 +655,6 @@ fn handle_node_reached<'a, TTrans, TMuxer, TInEvent, TOutEvent, THandler>(
|
||||
) -> (ActionItem<THandler>, RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler>)
|
||||
where
|
||||
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
|
||||
TTrans::Dial: Send + 'static,
|
||||
TMuxer: StreamMuxer + Send + Sync + 'static,
|
||||
TMuxer::OutboundSubstream: Send,
|
||||
TMuxer::Substream: Send,
|
||||
@ -848,10 +848,15 @@ where
|
||||
impl<'a, TTrans, TMuxer, TInEvent, TOutEvent, THandler>
|
||||
Peer<'a, TTrans, TInEvent, TOutEvent, THandler>
|
||||
where
|
||||
TTrans: Transport<Output = (PeerId, TMuxer)>,
|
||||
TMuxer: StreamMuxer,
|
||||
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,
|
||||
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`.
|
||||
#[inline]
|
||||
@ -889,19 +894,8 @@ where
|
||||
/// > that we are expecting, in which case the handler will be used for this "wrong"
|
||||
/// > node.
|
||||
#[inline]
|
||||
pub fn or_connect(
|
||||
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,
|
||||
pub fn or_connect(self, addr: Multiaddr, handler: THandler)
|
||||
-> Result<PeerPotentialConnect<'a, TInEvent, TOutEvent, THandler>, Self>
|
||||
{
|
||||
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"
|
||||
/// > node.
|
||||
#[inline]
|
||||
pub fn or_connect_with<TFn>(
|
||||
self,
|
||||
addr: TFn,
|
||||
handler: THandler,
|
||||
) -> Result<PeerPotentialConnect<'a, TInEvent, TOutEvent, THandler>, Self>
|
||||
pub fn or_connect_with<TFn>(self, addr: TFn, handler: THandler)
|
||||
-> Result<PeerPotentialConnect<'a, TInEvent, TOutEvent, THandler>, Self>
|
||||
where
|
||||
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 {
|
||||
Peer::Connected(peer) => Ok(PeerPotentialConnect::Connected(peer)),
|
||||
@ -1013,7 +997,7 @@ impl<'a, TInEvent> PeerConnected<'a, TInEvent> {
|
||||
.expect("We insert into connected_points whenever a connection is opened and remove \
|
||||
only when a connection is closed; the underlying API is guaranteed to always \
|
||||
deliver a connection closed message after it has been opened, and no two \
|
||||
closed messages; qed")
|
||||
closed messages; qed")
|
||||
}
|
||||
|
||||
/// Sends an event to the node.
|
||||
@ -1082,26 +1066,22 @@ where
|
||||
impl<'a, TTrans, TInEvent, TOutEvent, TMuxer, THandler>
|
||||
PeerNotConnected<'a, TTrans, TInEvent, TOutEvent, THandler>
|
||||
where
|
||||
TTrans: Transport<Output = (PeerId, TMuxer)>,
|
||||
TMuxer: StreamMuxer,
|
||||
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
|
||||
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::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.
|
||||
///
|
||||
/// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then
|
||||
/// the whole connection is immediately closed.
|
||||
#[inline]
|
||||
pub fn connect(self, addr: Multiaddr, 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,
|
||||
{
|
||||
pub fn connect(self, addr: Multiaddr, handler: THandler) -> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self> {
|
||||
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
|
||||
/// the whole connection is immediately closed.
|
||||
#[inline]
|
||||
pub fn connect_iter<TIter>(
|
||||
self,
|
||||
addrs: TIter,
|
||||
handler: THandler,
|
||||
) -> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self>
|
||||
pub fn connect_iter<TIter>(self, addrs: TIter, handler: THandler)
|
||||
-> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self>
|
||||
where
|
||||
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 first = match addrs.next() {
|
||||
@ -1139,23 +1109,10 @@ where
|
||||
}
|
||||
|
||||
/// Inner implementation of `connect`.
|
||||
fn connect_inner(
|
||||
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,
|
||||
fn connect_inner(self, handler: THandler, first: Multiaddr, rest: Vec<Multiaddr>)
|
||||
-> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler>, Self>
|
||||
{
|
||||
self.nodes.start_dial_out(self.peer_id.clone(), handler, first, rest);
|
||||
|
||||
Ok(PeerPendingConnect {
|
||||
attempt: match self.nodes.reach_attempts.out_reach_attempts.entry(self.peer_id) {
|
||||
Entry::Occupied(e) => e,
|
||||
|
@ -18,15 +18,19 @@
|
||||
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
|
||||
// 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 muxing::StreamMuxer;
|
||||
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};
|
||||
use std::{fmt, io, ops::{Deref, DerefMut}};
|
||||
|
||||
/// Contains the state of the network, plus the way it should behave.
|
||||
pub struct Swarm<TTransport, TBehaviour, TTopology>
|
||||
@ -77,17 +81,23 @@ where TBehaviour: NetworkBehavior,
|
||||
<TMuxer as StreamMuxer>::OutboundSubstream: Send + 'static,
|
||||
<TMuxer as StreamMuxer>::Substream: Send + 'static,
|
||||
TTransport: Transport<Output = (PeerId, TMuxer)> + Clone,
|
||||
TTransport::Dial: Send + 'static,
|
||||
TTransport::Listener: Send + 'static,
|
||||
TTransport::ListenerUpgrade: Send + 'static,
|
||||
TTransport::Dial: Send + 'static,
|
||||
TBehaviour::ProtocolsHandler: ProtocolsHandler<Substream = Substream<TMuxer>> + Send + 'static,
|
||||
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InEvent: 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>::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
|
||||
TTopology: Topology,
|
||||
{
|
||||
@ -164,17 +174,23 @@ where TBehaviour: NetworkBehavior,
|
||||
<TMuxer as StreamMuxer>::OutboundSubstream: Send + 'static,
|
||||
<TMuxer as StreamMuxer>::Substream: Send + 'static,
|
||||
TTransport: Transport<Output = (PeerId, TMuxer)> + Clone,
|
||||
TTransport::Dial: Send + 'static,
|
||||
TTransport::Listener: Send + 'static,
|
||||
TTransport::ListenerUpgrade: Send + 'static,
|
||||
TTransport::Dial: Send + 'static,
|
||||
TBehaviour::ProtocolsHandler: ProtocolsHandler<Substream = Substream<TMuxer>> + Send + 'static,
|
||||
<TBehaviour::ProtocolsHandler as ProtocolsHandler>::InEvent: 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>::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
|
||||
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
|
||||
// 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
|
||||
//! together in a complex chain of protocols negotiation.
|
||||
|
||||
use crate::{InboundUpgrade, OutboundUpgrade, Endpoint};
|
||||
use futures::prelude::*;
|
||||
use multiaddr::Multiaddr;
|
||||
use nodes::raw_swarm::ConnectedPoint;
|
||||
use std::io::Error as IoError;
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use upgrade::{ConnectionUpgrade, Endpoint};
|
||||
|
||||
pub mod and_then;
|
||||
pub mod boxed;
|
||||
@ -50,7 +50,7 @@ pub mod upgrade;
|
||||
pub use self::choice::OrTransport;
|
||||
pub use self::denied::DeniedTransport;
|
||||
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
|
||||
/// peer.
|
||||
@ -178,13 +178,14 @@ pub trait Transport {
|
||||
/// > **Note**: The concept of an *upgrade* for example includes middlewares such *secio*
|
||||
/// > (communication encryption), *multiplex*, but also a protocol handler.
|
||||
#[inline]
|
||||
fn with_upgrade<U>(self, upgrade: U) -> UpgradedNode<Self, U>
|
||||
fn with_upgrade<U, O, E>(self, upgrade: U) -> Upgrade<Self, U>
|
||||
where
|
||||
Self: Sized,
|
||||
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
|
||||
|
@ -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
|
||||
// copy of this software and associated documentation files (the "Software"),
|
||||
@ -20,173 +20,75 @@
|
||||
|
||||
use futures::prelude::*;
|
||||
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 transport::Transport;
|
||||
use upgrade::{apply, ConnectionUpgrade, Endpoint};
|
||||
|
||||
/// Implements the `Transport` trait. Dials or listens, then upgrades any dialed or received
|
||||
/// connection.
|
||||
///
|
||||
/// See the `Transport::with_upgrade` method.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct UpgradedNode<T, C> {
|
||||
transports: T,
|
||||
upgrade: C,
|
||||
#[derive(Debug, Copy, Clone)]
|
||||
pub struct Upgrade<T, U> { inner: T, upgrade: U }
|
||||
|
||||
impl<T, U> Upgrade<T, U> {
|
||||
pub fn new(inner: T, upgrade: U) -> Self {
|
||||
Upgrade { inner, upgrade }
|
||||
}
|
||||
}
|
||||
|
||||
impl<T, C> UpgradedNode<T, C> {
|
||||
pub fn new(transports: T, upgrade: C) -> UpgradedNode<T, C> {
|
||||
UpgradedNode {
|
||||
transports,
|
||||
upgrade,
|
||||
impl<D, U, O, E> Transport for Upgrade<D, U>
|
||||
where
|
||||
D: Transport,
|
||||
D::Dial: Send + 'static,
|
||||
D::Listener: Send + 'static,
|
||||
D::ListenerUpgrade: Send + 'static,
|
||||
D::Output: AsyncRead + AsyncWrite + Send + 'static,
|
||||
U: InboundUpgrade<D::Output, Output = O, Error = E>,
|
||||
U: OutboundUpgrade<D::Output, Output = O, Error = E> + Send + Clone + 'static,
|
||||
<U as UpgradeInfo>::NamesIter: Clone + 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
|
||||
{
|
||||
type Output = O;
|
||||
type Listener = Box<Stream<Item = (Self::ListenerUpgrade, Multiaddr), Error = std::io::Error> + Send>;
|
||||
type ListenerUpgrade = Box<Future<Item = Self::Output, Error = std::io::Error> + Send>;
|
||||
type Dial = Box<Future<Item = Self::Output, Error = std::io::Error> + Send>;
|
||||
|
||||
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> {
|
||||
let upgrade = self.upgrade;
|
||||
match self.inner.dial(addr.clone()) {
|
||||
Ok(outbound) => {
|
||||
let future = outbound
|
||||
.and_then(move |x| apply_outbound(x, upgrade).map_err(|e| {
|
||||
std::io::Error::new(std::io::ErrorKind::Other, e)
|
||||
}));
|
||||
Ok(Box::new(future))
|
||||
}
|
||||
Err((dialer, addr)) => Err((Upgrade::new(dialer, upgrade), addr))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, T, C> UpgradedNode<T, C>
|
||||
where
|
||||
T: Transport + 'a,
|
||||
T::Dial: Send,
|
||||
T::Listener: Send,
|
||||
T::ListenerUpgrade: Send,
|
||||
T::Output: Send + AsyncRead + AsyncWrite,
|
||||
C: ConnectionUpgrade<T::Output> + Send + 'a,
|
||||
C::NamesIter: Send,
|
||||
C::Future: Send,
|
||||
C::UpgradeIdentifier: Send,
|
||||
{
|
||||
/// Returns a reference to the inner `Transport`.
|
||||
#[inline]
|
||||
pub fn transport(&self) -> &T {
|
||||
&self.transports
|
||||
}
|
||||
|
||||
/// Tries to dial on the `Multiaddr` using the transport that was passed to `new`, then upgrade
|
||||
/// 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 dialed_fut = match self.transports.dial(addr.clone()) {
|
||||
Ok(f) => f,
|
||||
Err((trans, addr)) => {
|
||||
let builder = UpgradedNode {
|
||||
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))
|
||||
}
|
||||
|
||||
/// 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)> {
|
||||
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> {
|
||||
self.transports.nat_traversal(server, observed)
|
||||
self.inner.nat_traversal(server, observed)
|
||||
}
|
||||
}
|
||||
|
@ -19,49 +19,53 @@
|
||||
// DEALINGS IN THE SOFTWARE.
|
||||
|
||||
use bytes::Bytes;
|
||||
use futures::{prelude::*, future::Either};
|
||||
use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeError};
|
||||
use futures::prelude::*;
|
||||
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 upgrade::{ConnectionUpgrade, Endpoint};
|
||||
|
||||
/// Applies a connection upgrade on a socket.
|
||||
///
|
||||
/// 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>
|
||||
pub fn apply_inbound<C, U>(conn: C, up: U) -> InboundUpgradeApply<C, U>
|
||||
where
|
||||
U: ConnectionUpgrade<C>,
|
||||
U::NamesIter: Clone, // TODO: not elegant
|
||||
C: AsyncRead + AsyncWrite,
|
||||
U: InboundUpgrade<C>,
|
||||
U::NamesIter: Clone
|
||||
{
|
||||
UpgradeApplyFuture {
|
||||
inner: UpgradeApplyState::Init {
|
||||
future: negotiate(conn, &upgrade, e),
|
||||
upgrade,
|
||||
endpoint: e,
|
||||
}
|
||||
let iter = ProtocolNames(up.protocol_names());
|
||||
let future = multistream_select::listener_select_proto(conn, iter);
|
||||
InboundUpgradeApply {
|
||||
inner: InboundUpgradeApplyState::Init { future, upgrade: up }
|
||||
}
|
||||
}
|
||||
|
||||
/// Future, returned from `apply` which performs a connection upgrade.
|
||||
pub struct UpgradeApplyFuture<C, U>
|
||||
pub fn apply_outbound<C, U>(conn: C, up: U) -> OutboundUpgradeApply<C, U>
|
||||
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
|
||||
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 {
|
||||
future: NegotiationFuture<C, ProtocolNames<U::NamesIter>, U::UpgradeIdentifier>,
|
||||
upgrade: U,
|
||||
endpoint: Endpoint
|
||||
future: ListenerSelectFuture<C, ProtocolNames<U::NamesIter>, U::UpgradeId>,
|
||||
upgrade: U
|
||||
},
|
||||
Upgrade {
|
||||
future: U::Future
|
||||
@ -69,34 +73,34 @@ where
|
||||
Undefined
|
||||
}
|
||||
|
||||
impl<C, U> Future for UpgradeApplyFuture<C, U>
|
||||
impl<C, U> Future for InboundUpgradeApply<C, U>
|
||||
where
|
||||
U: ConnectionUpgrade<C>,
|
||||
U::NamesIter: Clone,
|
||||
C: AsyncRead + AsyncWrite
|
||||
C: AsyncRead + AsyncWrite,
|
||||
U: InboundUpgrade<C>,
|
||||
U::NamesIter: Clone
|
||||
{
|
||||
type Item = U::Output;
|
||||
type Error = IoError;
|
||||
type Error = UpgradeError<U::Error>;
|
||||
|
||||
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
|
||||
loop {
|
||||
match mem::replace(&mut self.inner, UpgradeApplyState::Undefined) {
|
||||
UpgradeApplyState::Init { mut future, upgrade, endpoint } => {
|
||||
match mem::replace(&mut self.inner, InboundUpgradeApplyState::Undefined) {
|
||||
InboundUpgradeApplyState::Init { mut future, upgrade } => {
|
||||
let (upgrade_id, connection) = match future.poll()? {
|
||||
Async::Ready(x) => x,
|
||||
Async::NotReady => {
|
||||
self.inner = UpgradeApplyState::Init { future, upgrade, endpoint };
|
||||
self.inner = InboundUpgradeApplyState::Init { future, upgrade };
|
||||
return Ok(Async::NotReady)
|
||||
}
|
||||
};
|
||||
self.inner = UpgradeApplyState::Upgrade {
|
||||
future: upgrade.upgrade(connection, upgrade_id, endpoint)
|
||||
self.inner = InboundUpgradeApplyState::Upgrade {
|
||||
future: upgrade.upgrade_inbound(connection, upgrade_id)
|
||||
};
|
||||
}
|
||||
UpgradeApplyState::Upgrade { mut future } => {
|
||||
InboundUpgradeApplyState::Upgrade { mut future } => {
|
||||
match future.poll() {
|
||||
Ok(Async::NotReady) => {
|
||||
self.inner = UpgradeApplyState::Upgrade { future };
|
||||
self.inner = InboundUpgradeApplyState::Upgrade { future };
|
||||
return Ok(Async::NotReady)
|
||||
}
|
||||
Ok(Async::Ready(x)) => {
|
||||
@ -104,68 +108,82 @@ where
|
||||
return Ok(Async::Ready(x))
|
||||
}
|
||||
Err(e) => {
|
||||
debug!("Failed to apply negotiated protocol: {:?}", e);
|
||||
return Err(e)
|
||||
debug!("Failed to apply negotiated protocol");
|
||||
return Err(UpgradeError::Apply(e))
|
||||
}
|
||||
}
|
||||
}
|
||||
UpgradeApplyState::Undefined =>
|
||||
panic!("UpgradeApplyState::poll called after completion")
|
||||
InboundUpgradeApplyState::Undefined =>
|
||||
panic!("InboundUpgradeApplyState::poll called after completion")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/// 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>
|
||||
pub struct OutboundUpgradeApply<C, U>
|
||||
where
|
||||
U: ConnectionUpgrade<I>,
|
||||
U::NamesIter: Clone, // TODO: not elegant
|
||||
C: AsyncRead + AsyncWrite,
|
||||
U: OutboundUpgrade<C>
|
||||
{
|
||||
debug!("Starting protocol negotiation");
|
||||
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)),
|
||||
}
|
||||
}
|
||||
inner: OutboundUpgradeApplyState<C, U>
|
||||
}
|
||||
|
||||
/// Future, returned by `negotiate`, which negotiates a protocol and stream.
|
||||
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>
|
||||
enum OutboundUpgradeApplyState<C, U>
|
||||
where
|
||||
R: AsyncRead + AsyncWrite,
|
||||
I: Iterator<Item=(Bytes, M, P)> + Clone,
|
||||
M: FnMut(&Bytes, &Bytes) -> bool,
|
||||
C: AsyncRead + AsyncWrite,
|
||||
U: OutboundUpgrade<C>
|
||||
{
|
||||
type Item = (P, R);
|
||||
type Error = IoError;
|
||||
Init {
|
||||
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> {
|
||||
match self.inner.poll() {
|
||||
Ok(Async::NotReady) => Ok(Async::NotReady),
|
||||
Ok(Async::Ready(x)) => {
|
||||
debug!("Successfully negotiated protocol upgrade");
|
||||
Ok(Async::Ready(x))
|
||||
}
|
||||
Err(e) => {
|
||||
let err = IoError::new(IoErrorKind::Other, e);
|
||||
debug!("Error while negotiated protocol upgrade: {:?}", err);
|
||||
Err(err)
|
||||
loop {
|
||||
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)) => {
|
||||
debug!("Successfully applied negotiated protocol");
|
||||
return Ok(Async::Ready(x))
|
||||
}
|
||||
Err(e) => {
|
||||
debug!("Failed to apply negotiated protocol");
|
||||
return Err(UpgradeError::Apply(e))
|
||||
}
|
||||
}
|
||||
}
|
||||
OutboundUpgradeApplyState::Undefined =>
|
||||
panic!("OutboundUpgradeApplyState::poll called after completion")
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -191,5 +209,3 @@ where
|
||||
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
|
||||
// copy of this software and associated documentation files (the "Software"),
|
||||
@ -19,31 +19,40 @@
|
||||
// DEALINGS IN THE SOFTWARE.
|
||||
|
||||
use bytes::Bytes;
|
||||
use futures::prelude::*;
|
||||
use std::{io, iter};
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use upgrade::{ConnectionUpgrade, Endpoint};
|
||||
use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
|
||||
use futures::future::FutureResult;
|
||||
use std::iter;
|
||||
use void::{unreachable, Void};
|
||||
|
||||
/// Implementation of `ConnectionUpgrade` that always fails to negotiate.
|
||||
#[derive(Debug, Copy, Clone)]
|
||||
pub struct DeniedConnectionUpgrade;
|
||||
pub struct DeniedUpgrade;
|
||||
|
||||
impl<C> ConnectionUpgrade<C> for DeniedConnectionUpgrade
|
||||
where
|
||||
C: AsyncRead + AsyncWrite,
|
||||
{
|
||||
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 `!`
|
||||
impl UpgradeInfo for DeniedUpgrade {
|
||||
type UpgradeId = Void;
|
||||
type NamesIter = iter::Empty<(Bytes, Self::UpgradeId)>;
|
||||
|
||||
#[inline]
|
||||
fn protocol_names(&self) -> Self::NamesIter {
|
||||
iter::empty()
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn upgrade(self, _: C, _: Self::UpgradeIdentifier, _: Endpoint) -> Self::Future {
|
||||
unreachable!("the denied connection upgrade always fails to negotiate")
|
||||
impl<C> InboundUpgrade<C> for DeniedUpgrade {
|
||||
type Output = Void;
|
||||
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
|
||||
// copy of this software and associated documentation files (the "Software"),
|
||||
@ -19,47 +19,114 @@
|
||||
// DEALINGS IN THE SOFTWARE.
|
||||
|
||||
use futures::prelude::*;
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use upgrade::{ConnectionUpgrade, Endpoint};
|
||||
use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
|
||||
|
||||
/// Applies a closure on the output of a connection upgrade.
|
||||
#[inline]
|
||||
pub fn map<U, F>(upgrade: U, map: F) -> Map<U, F> {
|
||||
Map { upgrade, map }
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct MapUpgrade<U, F> { upgrade: U, fun: F }
|
||||
|
||||
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.
|
||||
#[derive(Debug, Copy, Clone)]
|
||||
pub struct Map<U, F> {
|
||||
upgrade: U,
|
||||
map: F,
|
||||
}
|
||||
|
||||
impl<C, U, F, O> ConnectionUpgrade<C> for Map<U, F>
|
||||
impl<U, F> UpgradeInfo for MapUpgrade<U, F>
|
||||
where
|
||||
U: ConnectionUpgrade<C>,
|
||||
C: AsyncRead + AsyncWrite,
|
||||
F: FnOnce(U::Output) -> O,
|
||||
U: UpgradeInfo
|
||||
{
|
||||
type UpgradeId = U::UpgradeId;
|
||||
type NamesIter = U::NamesIter;
|
||||
type UpgradeIdentifier = U::UpgradeIdentifier;
|
||||
|
||||
fn protocol_names(&self) -> Self::NamesIter {
|
||||
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>;
|
||||
|
||||
fn upgrade(
|
||||
self,
|
||||
socket: C,
|
||||
id: Self::UpgradeIdentifier,
|
||||
ty: Endpoint,
|
||||
) -> Self::Future {
|
||||
fn upgrade_inbound(self, sock: C, id: Self::UpgradeId) -> Self::Future {
|
||||
MapFuture {
|
||||
inner: self.upgrade.upgrade(socket, id, ty),
|
||||
map: Some(self.map),
|
||||
inner: self.upgrade.upgrade_inbound(sock, id),
|
||||
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,8 +137,9 @@ pub struct MapFuture<TInnerFut, TMap> {
|
||||
}
|
||||
|
||||
impl<TInnerFut, TIn, TMap, TOut> Future for MapFuture<TInnerFut, TMap>
|
||||
where TInnerFut: Future<Item = TIn>,
|
||||
TMap: FnOnce(TIn) -> TOut,
|
||||
where
|
||||
TInnerFut: Future<Item = TIn>,
|
||||
TMap: FnOnce(TIn) -> TOut,
|
||||
{
|
||||
type Item = TOut;
|
||||
type Error = TInnerFut::Error;
|
||||
@ -82,3 +150,29 @@ where TInnerFut: Future<Item = TIn>,
|
||||
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
|
||||
// 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
|
||||
// DEALINGS IN THE SOFTWARE.
|
||||
|
||||
pub mod apply;
|
||||
pub mod choice;
|
||||
pub mod denied;
|
||||
pub mod loop_upg;
|
||||
pub mod map;
|
||||
pub mod plaintext;
|
||||
pub mod toggleable;
|
||||
pub mod traits;
|
||||
mod apply;
|
||||
mod denied;
|
||||
mod error;
|
||||
mod map;
|
||||
mod or;
|
||||
mod toggleable;
|
||||
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
@ -1,53 +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::{self, FutureResult};
|
||||
use std::{iter, io::Error as IoError};
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use upgrade::{ConnectionUpgrade, Endpoint};
|
||||
|
||||
/// 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)]
|
||||
pub struct PlainTextConfig;
|
||||
|
||||
impl<C> ConnectionUpgrade<C> for PlainTextConfig
|
||||
where
|
||||
C: AsyncRead + AsyncWrite,
|
||||
{
|
||||
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 {
|
||||
iter::once((Bytes::from("/plaintext/1.0.0"), ()))
|
||||
}
|
||||
}
|
@ -18,12 +18,11 @@
|
||||
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
|
||||
// DEALINGS IN THE SOFTWARE.
|
||||
|
||||
use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
|
||||
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]
|
||||
pub fn toggleable<U>(upgrade: U) -> Toggleable<U> {
|
||||
Toggleable {
|
||||
@ -32,7 +31,7 @@ pub fn toggleable<U>(upgrade: U) -> Toggleable<U> {
|
||||
}
|
||||
}
|
||||
|
||||
/// See `upgrade::toggleable`.
|
||||
/// See `toggleable`.
|
||||
#[derive(Debug, Copy, Clone)]
|
||||
pub struct Toggleable<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
|
||||
C: AsyncRead + AsyncWrite,
|
||||
U: ConnectionUpgrade<C>,
|
||||
U: UpgradeInfo
|
||||
{
|
||||
type UpgradeId = U::UpgradeId;
|
||||
type NamesIter = ToggleableIter<U::NamesIter>;
|
||||
type UpgradeIdentifier = U::UpgradeIdentifier;
|
||||
|
||||
#[inline]
|
||||
fn protocol_names(&self) -> Self::NamesIter {
|
||||
@ -80,19 +78,38 @@ where
|
||||
enabled: self.enabled,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<C, U> InboundUpgrade<C> for Toggleable<U>
|
||||
where
|
||||
U: InboundUpgrade<C>
|
||||
{
|
||||
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]
|
||||
fn upgrade(
|
||||
self,
|
||||
socket: C,
|
||||
id: Self::UpgradeIdentifier,
|
||||
ty: Endpoint,
|
||||
) -> Self::Future {
|
||||
fn upgrade_inbound(self, socket: C, id: Self::UpgradeId) -> Self::Future {
|
||||
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 {
|
||||
future::Either::A(future::empty())
|
||||
}
|
||||
@ -130,4 +147,7 @@ where I: Iterator
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
Reference in New Issue
Block a user