Split ConnectionUpgrade. (#642)

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

View File

@ -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()
}
}

View File

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

View File

@ -1,4 +1,4 @@
// Copyright 2017 Parity Technologies (UK) Ltd.
// Copyright 2017-2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// 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
View File

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

View File

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

View File

@ -1,4 +1,4 @@
// Copyright 2017 Parity Technologies (UK) Ltd.
// Copyright 2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// 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))
}
}
}
}

View File

@ -1,4 +1,4 @@
// Copyright 2017 Parity Technologies (UK) Ltd.
// Copyright 2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// 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
View File

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

View File

@ -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"), ()))
}
}

View File

@ -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
{}

View File

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