Switch to stable futures (#1196)

* Switch to stable futures

* Remove from_fn

* Fix secio

* Fix core --lib tests
This commit is contained in:
Pierre Krieger
2019-09-16 11:08:44 +02:00
committed by GitHub
parent 8c119269d6
commit 170d2d268f
105 changed files with 3193 additions and 5594 deletions

View File

@ -21,34 +21,33 @@
use crate::ConnectedPoint;
use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeError};
use crate::upgrade::{ProtocolName, NegotiatedComplete};
use futures::{future::Either, prelude::*};
use futures::{future::Either, prelude::*, compat::Compat, compat::Compat01As03, compat::Future01CompatExt};
use log::debug;
use multistream_select::{self, DialerSelectFuture, ListenerSelectFuture};
use std::{iter, mem};
use tokio_io::{AsyncRead, AsyncWrite};
use std::{iter, mem, pin::Pin, task::Context, task::Poll};
/// Applies an upgrade to the inbound and outbound direction of a connection or substream.
pub fn apply<C, U>(conn: C, up: U, cp: ConnectedPoint)
-> Either<InboundUpgradeApply<C, U>, OutboundUpgradeApply<C, U>>
where
C: AsyncRead + AsyncWrite,
C: AsyncRead + AsyncWrite + Unpin,
U: InboundUpgrade<C> + OutboundUpgrade<C>,
{
if cp.is_listener() {
Either::A(apply_inbound(conn, up))
Either::Left(apply_inbound(conn, up))
} else {
Either::B(apply_outbound(conn, up))
Either::Right(apply_outbound(conn, up))
}
}
/// Tries to perform an upgrade on an inbound connection or substream.
pub fn apply_inbound<C, U>(conn: C, up: U) -> InboundUpgradeApply<C, U>
where
C: AsyncRead + AsyncWrite,
C: AsyncRead + AsyncWrite + Unpin,
U: InboundUpgrade<C>,
{
let iter = up.protocol_info().into_iter().map(NameWrap as fn(_) -> NameWrap<_>);
let future = multistream_select::listener_select_proto(conn, iter);
let future = multistream_select::listener_select_proto(Compat::new(conn), iter).compat();
InboundUpgradeApply {
inner: InboundUpgradeApplyState::Init { future, upgrade: up }
}
@ -57,11 +56,11 @@ where
/// Tries to perform an upgrade on an outbound connection or substream.
pub fn apply_outbound<C, U>(conn: C, up: U) -> OutboundUpgradeApply<C, U>
where
C: AsyncRead + AsyncWrite,
C: AsyncRead + AsyncWrite + Unpin,
U: OutboundUpgrade<C>
{
let iter = up.protocol_info().into_iter().map(NameWrap as fn(_) -> NameWrap<_>);
let future = multistream_select::dialer_select_proto(conn, iter);
let future = multistream_select::dialer_select_proto(Compat::new(conn), iter).compat();
OutboundUpgradeApply {
inner: OutboundUpgradeApplyState::Init { future, upgrade: up }
}
@ -70,7 +69,7 @@ where
/// Future returned by `apply_inbound`. Drives the upgrade process.
pub struct InboundUpgradeApply<C, U>
where
C: AsyncRead + AsyncWrite,
C: AsyncRead + AsyncWrite + Unpin,
U: InboundUpgrade<C>
{
inner: InboundUpgradeApplyState<C, U>
@ -78,11 +77,11 @@ where
enum InboundUpgradeApplyState<C, U>
where
C: AsyncRead + AsyncWrite,
C: AsyncRead + AsyncWrite + Unpin,
U: InboundUpgrade<C>,
{
Init {
future: ListenerSelectFuture<C, NameWrap<U::Info>>,
future: Compat01As03<ListenerSelectFuture<Compat<C>, NameWrap<U::Info>>>,
upgrade: U,
},
Upgrade {
@ -91,42 +90,49 @@ where
Undefined
}
impl<C, U> Future for InboundUpgradeApply<C, U>
impl<C, U> Unpin for InboundUpgradeApply<C, U>
where
C: AsyncRead + AsyncWrite,
C: AsyncRead + AsyncWrite + Unpin,
U: InboundUpgrade<C>,
{
type Item = U::Output;
type Error = UpgradeError<U::Error>;
}
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
impl<C, U> Future for InboundUpgradeApply<C, U>
where
C: AsyncRead + AsyncWrite + Unpin,
U: InboundUpgrade<C>,
U::Future: Unpin,
{
type Output = Result<U::Output, UpgradeError<U::Error>>;
fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Self::Output> {
loop {
match mem::replace(&mut self.inner, InboundUpgradeApplyState::Undefined) {
InboundUpgradeApplyState::Init { mut future, upgrade } => {
let (info, io) = match future.poll()? {
Async::Ready(x) => x,
Async::NotReady => {
let (info, io) = match Future::poll(Pin::new(&mut future), cx)? {
Poll::Ready(x) => x,
Poll::Pending => {
self.inner = InboundUpgradeApplyState::Init { future, upgrade };
return Ok(Async::NotReady)
return Poll::Pending
}
};
self.inner = InboundUpgradeApplyState::Upgrade {
future: upgrade.upgrade_inbound(io, info.0)
future: upgrade.upgrade_inbound(Compat01As03::new(io), info.0)
};
}
InboundUpgradeApplyState::Upgrade { mut future } => {
match future.poll() {
Ok(Async::NotReady) => {
match Future::poll(Pin::new(&mut future), cx) {
Poll::Pending => {
self.inner = InboundUpgradeApplyState::Upgrade { future };
return Ok(Async::NotReady)
return Poll::Pending
}
Ok(Async::Ready(x)) => {
Poll::Ready(Ok(x)) => {
debug!("Successfully applied negotiated protocol");
return Ok(Async::Ready(x))
return Poll::Ready(Ok(x))
}
Err(e) => {
Poll::Ready(Err(e)) => {
debug!("Failed to apply negotiated protocol");
return Err(UpgradeError::Apply(e))
return Poll::Ready(Err(UpgradeError::Apply(e)))
}
}
}
@ -140,7 +146,7 @@ where
/// Future returned by `apply_outbound`. Drives the upgrade process.
pub struct OutboundUpgradeApply<C, U>
where
C: AsyncRead + AsyncWrite,
C: AsyncRead + AsyncWrite + Unpin,
U: OutboundUpgrade<C>
{
inner: OutboundUpgradeApplyState<C, U>
@ -148,15 +154,15 @@ where
enum OutboundUpgradeApplyState<C, U>
where
C: AsyncRead + AsyncWrite,
C: AsyncRead + AsyncWrite + Unpin,
U: OutboundUpgrade<C>
{
Init {
future: DialerSelectFuture<C, NameWrapIter<<U::InfoIter as IntoIterator>::IntoIter>>,
future: Compat01As03<DialerSelectFuture<Compat<C>, NameWrapIter<<U::InfoIter as IntoIterator>::IntoIter>>>,
upgrade: U
},
AwaitNegotiated {
io: NegotiatedComplete<C>,
io: Compat01As03<NegotiatedComplete<Compat<C>>>,
upgrade: U,
protocol: U::Info
},
@ -166,58 +172,65 @@ where
Undefined
}
impl<C, U> Unpin for OutboundUpgradeApply<C, U>
where
C: AsyncRead + AsyncWrite + Unpin,
U: OutboundUpgrade<C>,
{
}
impl<C, U> Future for OutboundUpgradeApply<C, U>
where
C: AsyncRead + AsyncWrite,
U: OutboundUpgrade<C>
C: AsyncRead + AsyncWrite + Unpin,
U: OutboundUpgrade<C>,
U::Future: Unpin,
{
type Item = U::Output;
type Error = UpgradeError<U::Error>;
type Output = Result<U::Output, UpgradeError<U::Error>>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Self::Output> {
loop {
match mem::replace(&mut self.inner, OutboundUpgradeApplyState::Undefined) {
OutboundUpgradeApplyState::Init { mut future, upgrade } => {
let (info, connection) = match future.poll()? {
Async::Ready(x) => x,
Async::NotReady => {
let (info, connection) = match Future::poll(Pin::new(&mut future), cx)? {
Poll::Ready(x) => x,
Poll::Pending => {
self.inner = OutboundUpgradeApplyState::Init { future, upgrade };
return Ok(Async::NotReady)
return Poll::Pending
}
};
self.inner = OutboundUpgradeApplyState::AwaitNegotiated {
io: connection.complete(),
io: Compat01As03::new(connection.complete()),
protocol: info.0,
upgrade
};
}
OutboundUpgradeApplyState::AwaitNegotiated { mut io, protocol, upgrade } => {
let io = match io.poll()? {
Async::NotReady => {
let io = match Future::poll(Pin::new(&mut io), cx)? {
Poll::Pending => {
self.inner = OutboundUpgradeApplyState::AwaitNegotiated {
io, protocol, upgrade
};
return Ok(Async::NotReady)
return Poll::Pending
}
Async::Ready(io) => io
Poll::Ready(io) => io
};
self.inner = OutboundUpgradeApplyState::Upgrade {
future: upgrade.upgrade_outbound(io, protocol)
future: upgrade.upgrade_outbound(Compat01As03::new(io), protocol)
};
}
OutboundUpgradeApplyState::Upgrade { mut future } => {
match future.poll() {
Ok(Async::NotReady) => {
match Future::poll(Pin::new(&mut future), cx) {
Poll::Pending => {
self.inner = OutboundUpgradeApplyState::Upgrade { future };
return Ok(Async::NotReady)
return Poll::Pending
}
Ok(Async::Ready(x)) => {
Poll::Ready(Ok(x)) => {
debug!("Successfully applied negotiated protocol");
return Ok(Async::Ready(x))
return Poll::Ready(Ok(x))
}
Err(e) => {
Poll::Ready(Err(e)) => {
debug!("Failed to apply negotiated protocol");
return Err(UpgradeError::Apply(e))
return Poll::Ready(Err(UpgradeError::Apply(e)));
}
}
}

View File

@ -18,9 +18,9 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use crate::Negotiated;
use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
use futures::future;
use multistream_select::Negotiated;
use std::iter;
use void::Void;
@ -41,20 +41,19 @@ impl UpgradeInfo for DeniedUpgrade {
impl<C> InboundUpgrade<C> for DeniedUpgrade {
type Output = Void;
type Error = Void;
type Future = future::Empty<Self::Output, Self::Error>;
type Future = future::Pending<Result<Self::Output, Self::Error>>;
fn upgrade_inbound(self, _: Negotiated<C>, _: Self::Info) -> Self::Future {
future::empty()
future::pending()
}
}
impl<C> OutboundUpgrade<C> for DeniedUpgrade {
type Output = Void;
type Error = Void;
type Future = future::Empty<Self::Output, Self::Error>;
type Future = future::Pending<Result<Self::Output, Self::Error>>;
fn upgrade_outbound(self, _: Negotiated<C>, _: Self::Info) -> Self::Future {
future::empty()
future::pending()
}
}

View File

@ -19,10 +19,10 @@
// DEALINGS IN THE SOFTWARE.
use crate::{
Negotiated,
either::{EitherOutput, EitherError, EitherFuture2, EitherName},
upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}
};
use multistream_select::Negotiated;
/// A type to represent two possible upgrade types (inbound or outbound).
#[derive(Debug, Clone)]
@ -50,7 +50,9 @@ where
impl<C, A, B, TA, TB, EA, EB> InboundUpgrade<C> for EitherUpgrade<A, B>
where
A: InboundUpgrade<C, Output = TA, Error = EA>,
<A as InboundUpgrade<C>>::Future: Unpin,
B: InboundUpgrade<C, Output = TB, Error = EB>,
<B as InboundUpgrade<C>>::Future: Unpin,
{
type Output = EitherOutput<TA, TB>;
type Error = EitherError<EA, EB>;
@ -68,7 +70,9 @@ where
impl<C, A, B, TA, TB, EA, EB> OutboundUpgrade<C> for EitherUpgrade<A, B>
where
A: OutboundUpgrade<C, Output = TA, Error = EA>,
<A as OutboundUpgrade<C>>::Future: Unpin,
B: OutboundUpgrade<C, Output = TB, Error = EB>,
<B as OutboundUpgrade<C>>::Future: Unpin,
{
type Output = EitherOutput<TA, TB>;
type Error = EitherError<EA, EB>;

View File

@ -18,9 +18,10 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use crate::Negotiated;
use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
use futures::{prelude::*, try_ready};
use multistream_select::Negotiated;
use futures::prelude::*;
use std::{pin::Pin, task::Context, task::Poll};
/// Wraps around an upgrade and applies a closure to the output.
#[derive(Debug, Clone)]
@ -47,6 +48,7 @@ where
impl<C, U, F, T> InboundUpgrade<C> for MapInboundUpgrade<U, F>
where
U: InboundUpgrade<C>,
U::Future: Unpin,
F: FnOnce(U::Output) -> T
{
type Output = T;
@ -63,7 +65,8 @@ where
impl<C, U, F> OutboundUpgrade<C> for MapInboundUpgrade<U, F>
where
U: OutboundUpgrade<C>
U: OutboundUpgrade<C>,
U::Future: Unpin,
{
type Output = U::Output;
type Error = U::Error;
@ -98,7 +101,8 @@ where
impl<C, U, F> InboundUpgrade<C> for MapOutboundUpgrade<U, F>
where
U: InboundUpgrade<C>
U: InboundUpgrade<C>,
U::Future: Unpin,
{
type Output = U::Output;
type Error = U::Error;
@ -112,6 +116,7 @@ where
impl<C, U, F, T> OutboundUpgrade<C> for MapOutboundUpgrade<U, F>
where
U: OutboundUpgrade<C>,
U::Future: Unpin,
F: FnOnce(U::Output) -> T
{
type Output = T;
@ -151,6 +156,7 @@ where
impl<C, U, F, T> InboundUpgrade<C> for MapInboundUpgradeErr<U, F>
where
U: InboundUpgrade<C>,
U::Future: Unpin,
F: FnOnce(U::Error) -> T
{
type Output = U::Output;
@ -167,7 +173,8 @@ where
impl<C, U, F> OutboundUpgrade<C> for MapInboundUpgradeErr<U, F>
where
U: OutboundUpgrade<C>
U: OutboundUpgrade<C>,
U::Future: Unpin,
{
type Output = U::Output;
type Error = U::Error;
@ -203,6 +210,7 @@ where
impl<C, U, F, T> OutboundUpgrade<C> for MapOutboundUpgradeErr<U, F>
where
U: OutboundUpgrade<C>,
U::Future: Unpin,
F: FnOnce(U::Error) -> T
{
type Output = U::Output;
@ -235,18 +243,25 @@ pub struct MapFuture<TInnerFut, TMap> {
map: Option<TMap>,
}
impl<TInnerFut, TMap> Unpin for MapFuture<TInnerFut, TMap> {
}
impl<TInnerFut, TIn, TMap, TOut> Future for MapFuture<TInnerFut, TMap>
where
TInnerFut: Future<Item = TIn>,
TInnerFut: TryFuture<Ok = TIn> + Unpin,
TMap: FnOnce(TIn) -> TOut,
{
type Item = TOut;
type Error = TInnerFut::Error;
type Output = Result<TOut, TInnerFut::Error>;
fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Self::Output> {
let item = match TryFuture::try_poll(Pin::new(&mut self.inner), cx) {
Poll::Ready(Ok(v)) => v,
Poll::Ready(Err(err)) => return Poll::Ready(Err(err)),
Poll::Pending => return Poll::Pending,
};
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
let item = try_ready!(self.inner.poll());
let map = self.map.take().expect("Future has already finished");
Ok(Async::Ready(map(item)))
Poll::Ready(Ok(map(item)))
}
}
@ -255,21 +270,23 @@ pub struct MapErrFuture<T, F> {
fun: Option<F>,
}
impl<T, F> Unpin for MapErrFuture<T, F> {
}
impl<T, E, F, A> Future for MapErrFuture<T, F>
where
T: Future<Error = E>,
T: TryFuture<Error = E> + Unpin,
F: FnOnce(E) -> A,
{
type Item = T::Item;
type Error = A;
type Output = Result<T::Ok, 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) => {
fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Self::Output> {
match TryFuture::try_poll(Pin::new(&mut self.fut), cx) {
Poll::Pending => Poll::Pending,
Poll::Ready(Ok(x)) => Poll::Ready(Ok(x)),
Poll::Ready(Err(e)) => {
let f = self.fun.take().expect("Future has not resolved yet");
Err(f(e))
Poll::Ready(Err(f(e)))
}
}
}

View File

@ -68,7 +68,8 @@ mod transfer;
use futures::future::Future;
pub use multistream_select::{Negotiated, NegotiatedComplete, NegotiationError, ProtocolError};
pub use crate::Negotiated;
pub use multistream_select::{NegotiatedComplete, NegotiationError, ProtocolError};
pub use self::{
apply::{apply, apply_inbound, apply_outbound, InboundUpgradeApply, OutboundUpgradeApply},
denied::DeniedUpgrade,
@ -77,7 +78,7 @@ pub use self::{
map::{MapInboundUpgrade, MapOutboundUpgrade, MapInboundUpgradeErr, MapOutboundUpgradeErr},
optional::OptionalUpgrade,
select::SelectUpgrade,
transfer::{write_one, WriteOne, read_one, ReadOne, read_one_then, ReadOneThen, ReadOneError, request_response, RequestResponse, read_respond, ReadRespond},
transfer::{write_one, write_with_len_prefix, write_varint, read_one, ReadOneError, read_varint},
};
/// Types serving as protocol names.
@ -143,7 +144,8 @@ pub trait InboundUpgrade<C>: UpgradeInfo {
/// Possible error during the handshake.
type Error;
/// Future that performs the handshake with the remote.
type Future: Future<Item = Self::Output, Error = Self::Error>;
// TODO: remove Unpin
type Future: Future<Output = Result<Self::Output, Self::Error>> + Unpin;
/// After we have determined that the remote supports one of the protocols we support, this
/// method is called to start the handshake.
@ -183,7 +185,8 @@ pub trait OutboundUpgrade<C>: UpgradeInfo {
/// Possible error during the handshake.
type Error;
/// Future that performs the handshake with the remote.
type Future: Future<Item = Self::Output, Error = Self::Error>;
// TODO: remove Unpin
type Future: Future<Output = Result<Self::Output, Self::Error>> + Unpin;
/// After we have determined that the remote supports one of the protocols we support, this
/// method is called to start the handshake.

View File

@ -19,7 +19,7 @@
// DEALINGS IN THE SOFTWARE.
use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
use multistream_select::Negotiated;
use crate::Negotiated;
/// Upgrade that can be disabled at runtime.
///

View File

@ -19,10 +19,10 @@
// DEALINGS IN THE SOFTWARE.
use crate::{
Negotiated,
either::{EitherOutput, EitherError, EitherFuture2, EitherName},
upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}
};
use multistream_select::Negotiated;
/// Upgrade that combines two upgrades into one. Supports all the protocols supported by either
/// sub-upgrade.
@ -59,7 +59,9 @@ where
impl<C, A, B, TA, TB, EA, EB> InboundUpgrade<C> for SelectUpgrade<A, B>
where
A: InboundUpgrade<C, Output = TA, Error = EA>,
<A as InboundUpgrade<C>>::Future: Unpin,
B: InboundUpgrade<C, Output = TB, Error = EB>,
<B as InboundUpgrade<C>>::Future: Unpin,
{
type Output = EitherOutput<TA, TB>;
type Error = EitherError<EA, EB>;
@ -76,7 +78,9 @@ where
impl<C, A, B, TA, TB, EA, EB> OutboundUpgrade<C> for SelectUpgrade<A, B>
where
A: OutboundUpgrade<C, Output = TA, Error = EA>,
<A as OutboundUpgrade<C>>::Future: Unpin,
B: OutboundUpgrade<C, Output = TB, Error = EB>,
<B as OutboundUpgrade<C>>::Future: Unpin,
{
type Output = EitherOutput<TA, TB>;
type Error = EitherError<EA, EB>;

View File

@ -20,104 +20,93 @@
//! Contains some helper futures for creating upgrades.
use futures::{prelude::*, try_ready};
use std::{cmp, error, fmt, io::Cursor, mem};
use tokio_io::{io, AsyncRead, AsyncWrite};
use futures::prelude::*;
use std::{error, fmt, io};
// TODO: these methods could be on an Ext trait to AsyncWrite
/// Send a message to the given socket, then shuts down the writing side.
///
/// > **Note**: Prepends a variable-length prefix indicate the length of the message. This is
/// > compatible with what `read_one` expects.
pub fn write_one<TSocket, TData>(socket: TSocket, data: TData) -> WriteOne<TSocket, TData>
where
TSocket: AsyncWrite,
TData: AsRef<[u8]>,
pub async fn write_one(socket: &mut (impl AsyncWrite + Unpin), data: impl AsRef<[u8]>)
-> Result<(), io::Error>
{
let len_data = build_int_buffer(data.as_ref().len());
WriteOne {
inner: WriteOneInner::WriteLen(io::write_all(socket, len_data), data),
}
write_varint(socket, data.as_ref().len()).await?;
socket.write_all(data.as_ref()).await?;
socket.close().await?;
Ok(())
}
/// Builds a buffer that contains the given integer encoded as variable-length.
fn build_int_buffer(num: usize) -> io::Window<[u8; 10]> {
let mut len_data = unsigned_varint::encode::u64_buffer();
let encoded_len = unsigned_varint::encode::u64(num as u64, &mut len_data).len();
let mut len_data = io::Window::new(len_data);
len_data.set_end(encoded_len);
len_data
}
/// Future that makes `write_one` work.
#[derive(Debug)]
pub struct WriteOne<TSocket, TData = Vec<u8>> {
inner: WriteOneInner<TSocket, TData>,
}
#[derive(Debug)]
enum WriteOneInner<TSocket, TData> {
/// We need to write the data length to the socket.
WriteLen(io::WriteAll<TSocket, io::Window<[u8; 10]>>, TData),
/// We need to write the actual data to the socket.
Write(io::WriteAll<TSocket, TData>),
/// We need to shut down the socket.
Shutdown(io::Shutdown<TSocket>),
/// A problem happened during the processing.
Poisoned,
}
impl<TSocket, TData> Future for WriteOne<TSocket, TData>
where
TSocket: AsyncWrite,
TData: AsRef<[u8]>,
/// Send a message to the given socket with a length prefix appended to it. Also flushes the socket.
///
/// > **Note**: Prepends a variable-length prefix indicate the length of the message. This is
/// > compatible with what `read_one` expects.
pub async fn write_with_len_prefix(socket: &mut (impl AsyncWrite + Unpin), data: impl AsRef<[u8]>)
-> Result<(), io::Error>
{
type Item = ();
type Error = std::io::Error;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
Ok(self.inner.poll()?.map(|_socket| ()))
}
write_varint(socket, data.as_ref().len()).await?;
socket.write_all(data.as_ref()).await?;
socket.flush().await?;
Ok(())
}
impl<TSocket, TData> Future for WriteOneInner<TSocket, TData>
where
TSocket: AsyncWrite,
TData: AsRef<[u8]>,
/// Writes a variable-length integer to the `socket`.
///
/// > **Note**: Does **NOT** flush the socket.
pub async fn write_varint(socket: &mut (impl AsyncWrite + Unpin), len: usize)
-> Result<(), io::Error>
{
type Item = TSocket;
type Error = std::io::Error;
let mut len_data = unsigned_varint::encode::usize_buffer();
let encoded_len = unsigned_varint::encode::usize(len, &mut len_data).len();
socket.write_all(&len_data[..encoded_len]).await?;
Ok(())
}
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
loop {
match mem::replace(self, WriteOneInner::Poisoned) {
WriteOneInner::WriteLen(mut inner, data) => match inner.poll()? {
Async::Ready((socket, _)) => {
*self = WriteOneInner::Write(io::write_all(socket, data));
}
Async::NotReady => {
*self = WriteOneInner::WriteLen(inner, data);
}
},
WriteOneInner::Write(mut inner) => match inner.poll()? {
Async::Ready((socket, _)) => {
*self = WriteOneInner::Shutdown(tokio_io::io::shutdown(socket));
}
Async::NotReady => {
*self = WriteOneInner::Write(inner);
}
},
WriteOneInner::Shutdown(ref mut inner) => {
let socket = try_ready!(inner.poll());
return Ok(Async::Ready(socket));
/// Reads a variable-length integer from the `socket`.
///
/// As a special exception, if the `socket` is empty and EOFs right at the beginning, then we
/// return `Ok(0)`.
///
/// > **Note**: This function reads bytes one by one from the `socket`. It is therefore encouraged
/// > to use some sort of buffering mechanism.
pub async fn read_varint(socket: &mut (impl AsyncRead + Unpin)) -> Result<usize, io::Error> {
let mut buffer = unsigned_varint::encode::usize_buffer();
let mut buffer_len = 0;
loop {
match socket.read(&mut buffer[buffer_len..buffer_len+1]).await? {
0 => {
// Reaching EOF before finishing to read the length is an error, unless the EOF is
// at the very beginning of the substream, in which case we assume that the data is
// empty.
if buffer_len == 0 {
return Ok(0);
} else {
return Err(io::ErrorKind::UnexpectedEof.into());
}
WriteOneInner::Poisoned => panic!(),
}
n => debug_assert_eq!(n, 1),
}
buffer_len += 1;
match unsigned_varint::decode::usize(&buffer[..buffer_len]) {
Ok((len, _)) => return Ok(len),
Err(unsigned_varint::decode::Error::Overflow) => {
return Err(io::Error::new(
io::ErrorKind::InvalidData,
"overflow in variable-length integer"
));
}
// TODO: why do we have a `__Nonexhaustive` variant in the error? I don't know how to process it
// Err(unsigned_varint::decode::Error::Insufficient) => {}
Err(_) => {}
}
}
}
/// Reads a message from the given socket. Only one message is processed and the socket is dropped,
/// because we assume that the socket will not send anything more.
/// Reads a length-prefixed message from the given socket.
///
/// The `max_size` parameter is the maximum size in bytes of the message that we accept. This is
/// necessary in order to avoid DoS attacks where the remote sends us a message of several
@ -125,137 +114,20 @@ where
///
/// > **Note**: Assumes that a variable-length prefix indicates the length of the message. This is
/// > compatible with what `write_one` does.
pub fn read_one<TSocket>(
socket: TSocket,
max_size: usize,
) -> ReadOne<TSocket>
pub async fn read_one(socket: &mut (impl AsyncRead + Unpin), max_size: usize)
-> Result<Vec<u8>, ReadOneError>
{
ReadOne {
inner: ReadOneInner::ReadLen {
socket,
len_buf: Cursor::new([0; 10]),
max_size,
},
let len = read_varint(socket).await?;
if len > max_size {
return Err(ReadOneError::TooLarge {
requested: len,
max: max_size,
});
}
}
/// Future that makes `read_one` work.
#[derive(Debug)]
pub struct ReadOne<TSocket> {
inner: ReadOneInner<TSocket>,
}
#[derive(Debug)]
enum ReadOneInner<TSocket> {
// We need to read the data length from the socket.
ReadLen {
socket: TSocket,
/// A small buffer where we will right the variable-length integer representing the
/// length of the actual packet.
len_buf: Cursor<[u8; 10]>,
max_size: usize,
},
// We need to read the actual data from the socket.
ReadRest(io::ReadExact<TSocket, io::Window<Vec<u8>>>),
/// A problem happened during the processing.
Poisoned,
}
impl<TSocket> Future for ReadOne<TSocket>
where
TSocket: AsyncRead,
{
type Item = Vec<u8>;
type Error = ReadOneError;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
Ok(self.inner.poll()?.map(|(_, out)| out))
}
}
impl<TSocket> Future for ReadOneInner<TSocket>
where
TSocket: AsyncRead,
{
type Item = (TSocket, Vec<u8>);
type Error = ReadOneError;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
loop {
match mem::replace(self, ReadOneInner::Poisoned) {
ReadOneInner::ReadLen {
mut socket,
mut len_buf,
max_size,
} => {
match socket.read_buf(&mut len_buf)? {
Async::Ready(num_read) => {
// Reaching EOF before finishing to read the length is an error, unless
// the EOF is at the very beginning of the substream, in which case we
// assume that the data is empty.
if num_read == 0 {
if len_buf.position() == 0 {
return Ok(Async::Ready((socket, Vec::new())));
} else {
return Err(ReadOneError::Io(
std::io::ErrorKind::UnexpectedEof.into(),
));
}
}
let len_buf_with_data =
&len_buf.get_ref()[..len_buf.position() as usize];
if let Ok((len, data_start)) =
unsigned_varint::decode::usize(len_buf_with_data)
{
if len >= max_size {
return Err(ReadOneError::TooLarge {
requested: len,
max: max_size,
});
}
// Create `data_buf` containing the start of the data that was
// already in `len_buf`.
let n = cmp::min(data_start.len(), len);
let mut data_buf = vec![0; len];
data_buf[.. n].copy_from_slice(&data_start[.. n]);
let mut data_buf = io::Window::new(data_buf);
data_buf.set_start(data_start.len());
*self = ReadOneInner::ReadRest(io::read_exact(socket, data_buf));
} else {
*self = ReadOneInner::ReadLen {
socket,
len_buf,
max_size,
};
}
}
Async::NotReady => {
*self = ReadOneInner::ReadLen {
socket,
len_buf,
max_size,
};
return Ok(Async::NotReady);
}
}
}
ReadOneInner::ReadRest(mut inner) => {
match inner.poll()? {
Async::Ready((socket, data)) => {
return Ok(Async::Ready((socket, data.into_inner())));
}
Async::NotReady => {
*self = ReadOneInner::ReadRest(inner);
return Ok(Async::NotReady);
}
}
}
ReadOneInner::Poisoned => panic!(),
}
}
}
let mut buf = vec![0; len];
socket.read_exact(&mut buf).await?;
Ok(buf)
}
/// Error while reading one message.
@ -296,194 +168,10 @@ impl error::Error for ReadOneError {
}
}
/// Similar to `read_one`, but applies a transformation on the output buffer.
///
/// > **Note**: The `param` parameter is an arbitrary value that will be passed back to `then`.
/// > This parameter is normally not necessary, as we could just pass a closure that has
/// > ownership of any data we want. In practice, though, this would make the
/// > `ReadRespond` type impossible to express as a concrete type. Once the `impl Trait`
/// > syntax is allowed within traits, we can remove this parameter.
pub fn read_one_then<TSocket, TParam, TThen, TOut, TErr>(
socket: TSocket,
max_size: usize,
param: TParam,
then: TThen,
) -> ReadOneThen<TSocket, TParam, TThen>
where
TSocket: AsyncRead,
TThen: FnOnce(Vec<u8>, TParam) -> Result<TOut, TErr>,
TErr: From<ReadOneError>,
{
ReadOneThen {
inner: read_one(socket, max_size),
then: Some((param, then)),
}
}
/// Future that makes `read_one_then` work.
#[derive(Debug)]
pub struct ReadOneThen<TSocket, TParam, TThen> {
inner: ReadOne<TSocket>,
then: Option<(TParam, TThen)>,
}
impl<TSocket, TParam, TThen, TOut, TErr> Future for ReadOneThen<TSocket, TParam, TThen>
where
TSocket: AsyncRead,
TThen: FnOnce(Vec<u8>, TParam) -> Result<TOut, TErr>,
TErr: From<ReadOneError>,
{
type Item = TOut;
type Error = TErr;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
match self.inner.poll()? {
Async::Ready(buffer) => {
let (param, then) = self.then.take()
.expect("Future was polled after it was finished");
Ok(Async::Ready(then(buffer, param)?))
},
Async::NotReady => Ok(Async::NotReady),
}
}
}
/// Similar to `read_one`, but applies a transformation on the output buffer.
///
/// > **Note**: The `param` parameter is an arbitrary value that will be passed back to `then`.
/// > This parameter is normally not necessary, as we could just pass a closure that has
/// > ownership of any data we want. In practice, though, this would make the
/// > `ReadRespond` type impossible to express as a concrete type. Once the `impl Trait`
/// > syntax is allowed within traits, we can remove this parameter.
pub fn read_respond<TSocket, TThen, TParam, TOut, TErr>(
socket: TSocket,
max_size: usize,
param: TParam,
then: TThen,
) -> ReadRespond<TSocket, TParam, TThen>
where
TSocket: AsyncRead,
TThen: FnOnce(TSocket, Vec<u8>, TParam) -> Result<TOut, TErr>,
TErr: From<ReadOneError>,
{
ReadRespond {
inner: read_one(socket, max_size).inner,
then: Some((then, param)),
}
}
/// Future that makes `read_respond` work.
#[derive(Debug)]
pub struct ReadRespond<TSocket, TParam, TThen> {
inner: ReadOneInner<TSocket>,
then: Option<(TThen, TParam)>,
}
impl<TSocket, TThen, TParam, TOut, TErr> Future for ReadRespond<TSocket, TParam, TThen>
where
TSocket: AsyncRead,
TThen: FnOnce(TSocket, Vec<u8>, TParam) -> Result<TOut, TErr>,
TErr: From<ReadOneError>,
{
type Item = TOut;
type Error = TErr;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
match self.inner.poll()? {
Async::Ready((socket, buffer)) => {
let (then, param) = self.then.take().expect("Future was polled after it was finished");
Ok(Async::Ready(then(socket, buffer, param)?))
},
Async::NotReady => Ok(Async::NotReady),
}
}
}
/// Send a message to the given socket, then shuts down the writing side, then reads an answer.
///
/// This combines `write_one` followed with `read_one_then`.
///
/// > **Note**: The `param` parameter is an arbitrary value that will be passed back to `then`.
/// > This parameter is normally not necessary, as we could just pass a closure that has
/// > ownership of any data we want. In practice, though, this would make the
/// > `ReadRespond` type impossible to express as a concrete type. Once the `impl Trait`
/// > syntax is allowed within traits, we can remove this parameter.
pub fn request_response<TSocket, TData, TParam, TThen, TOut, TErr>(
socket: TSocket,
data: TData,
max_size: usize,
param: TParam,
then: TThen,
) -> RequestResponse<TSocket, TParam, TThen, TData>
where
TSocket: AsyncRead + AsyncWrite,
TData: AsRef<[u8]>,
TThen: FnOnce(Vec<u8>, TParam) -> Result<TOut, TErr>,
{
RequestResponse {
inner: RequestResponseInner::Write(write_one(socket, data).inner, max_size, param, then),
}
}
/// Future that makes `request_response` work.
#[derive(Debug)]
pub struct RequestResponse<TSocket, TParam, TThen, TData = Vec<u8>> {
inner: RequestResponseInner<TSocket, TData, TParam, TThen>,
}
#[derive(Debug)]
enum RequestResponseInner<TSocket, TData, TParam, TThen> {
// We need to write data to the socket.
Write(WriteOneInner<TSocket, TData>, usize, TParam, TThen),
// We need to read the message.
Read(ReadOneThen<TSocket, TParam, TThen>),
// An error happened during the processing.
Poisoned,
}
impl<TSocket, TData, TParam, TThen, TOut, TErr> Future for RequestResponse<TSocket, TParam, TThen, TData>
where
TSocket: AsyncRead + AsyncWrite,
TData: AsRef<[u8]>,
TThen: FnOnce(Vec<u8>, TParam) -> Result<TOut, TErr>,
TErr: From<ReadOneError>,
{
type Item = TOut;
type Error = TErr;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
loop {
match mem::replace(&mut self.inner, RequestResponseInner::Poisoned) {
RequestResponseInner::Write(mut inner, max_size, param, then) => {
match inner.poll().map_err(ReadOneError::Io)? {
Async::Ready(socket) => {
self.inner =
RequestResponseInner::Read(read_one_then(socket, max_size, param, then));
}
Async::NotReady => {
self.inner = RequestResponseInner::Write(inner, max_size, param, then);
return Ok(Async::NotReady);
}
}
}
RequestResponseInner::Read(mut inner) => match inner.poll()? {
Async::Ready(packet) => return Ok(Async::Ready(packet)),
Async::NotReady => {
self.inner = RequestResponseInner::Read(inner);
return Ok(Async::NotReady);
}
},
RequestResponseInner::Poisoned => panic!(),
};
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use std::io::{self, Cursor};
use tokio::runtime::current_thread::Runtime;
#[test]
fn write_one_works() {
@ -492,14 +180,17 @@ mod tests {
.collect::<Vec<_>>();
let mut out = vec![0; 10_000];
let future = write_one(Cursor::new(&mut out[..]), data.clone());
Runtime::new().unwrap().block_on(future).unwrap();
futures::executor::block_on(
write_one(&mut Cursor::new(&mut out[..]), data.clone())
).unwrap();
let (out_len, out_data) = unsigned_varint::decode::usize(&out).unwrap();
assert_eq!(out_len, data.len());
assert_eq!(&out_data[..out_len], &data[..]);
}
// TODO: rewrite these tests
/*
#[test]
fn read_one_works() {
let original_data = (0..rand::random::<usize>() % 10_000)
@ -517,7 +208,7 @@ mod tests {
Ok(())
});
Runtime::new().unwrap().block_on(future).unwrap();
futures::executor::block_on(future).unwrap();
}
#[test]
@ -527,7 +218,7 @@ mod tests {
Ok(())
});
Runtime::new().unwrap().block_on(future).unwrap();
futures::executor::block_on(future).unwrap();
}
#[test]
@ -542,7 +233,7 @@ mod tests {
Ok(())
});
match Runtime::new().unwrap().block_on(future) {
match futures::executor::block_on(future) {
Err(ReadOneError::TooLarge { .. }) => (),
_ => panic!(),
}
@ -555,7 +246,7 @@ mod tests {
Ok(())
});
Runtime::new().unwrap().block_on(future).unwrap();
futures::executor::block_on(future).unwrap();
}
#[test]
@ -564,9 +255,9 @@ mod tests {
unreachable!()
});
match Runtime::new().unwrap().block_on(future) {
match futures::executor::block_on(future) {
Err(ReadOneError::Io(ref err)) if err.kind() == io::ErrorKind::UnexpectedEof => (),
_ => panic!()
}
}
}*/
}