mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-12 09:31:20 +00:00
Rework the StreamMuxer trait (#437)
* Stronger typing for the swarm handler future * The swarm future is now a swarm stream of events * Rewrite StreamMuxer in core * Update libp2p_mplex for the new stream muxer * Update yamux for new stream muxer
This commit is contained in:
committed by
Benjamin Kampmann
parent
c02dea8128
commit
0e1483f02e
@ -45,7 +45,7 @@ use futures::{Async, Future, Poll, Stream};
|
||||
use futures::stream::FuturesUnordered;
|
||||
use futures::sync::mpsc;
|
||||
use multiaddr::Multiaddr;
|
||||
use muxing::StreamMuxer;
|
||||
use muxing::{self, StreamMuxer};
|
||||
use parking_lot::Mutex;
|
||||
use std::io::{self, Error as IoError};
|
||||
use std::sync::Arc;
|
||||
@ -58,7 +58,6 @@ use upgrade::ConnectionUpgrade;
|
||||
/// Can be created from an `UpgradedNode` through the `From` trait.
|
||||
///
|
||||
/// Implements the `Transport` trait.
|
||||
#[derive(Clone)]
|
||||
pub struct ConnectionReuse<T, C>
|
||||
where
|
||||
T: Transport,
|
||||
@ -73,23 +72,36 @@ where
|
||||
shared: Arc<Mutex<Shared<C::Output>>>,
|
||||
}
|
||||
|
||||
struct Shared<M>
|
||||
impl<T, C> Clone for ConnectionReuse<T, C>
|
||||
where
|
||||
M: StreamMuxer,
|
||||
T: Transport + Clone,
|
||||
T::Output: AsyncRead + AsyncWrite,
|
||||
C: ConnectionUpgrade<T::Output, T::MultiaddrFuture> + Clone,
|
||||
C::Output: StreamMuxer
|
||||
{
|
||||
#[inline]
|
||||
fn clone(&self) -> Self {
|
||||
ConnectionReuse {
|
||||
inner: self.inner.clone(),
|
||||
shared: self.shared.clone(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
struct Shared<M> {
|
||||
// List of active muxers.
|
||||
active_connections: FnvHashMap<Multiaddr, M>,
|
||||
active_connections: FnvHashMap<Multiaddr, Arc<M>>,
|
||||
|
||||
// List of pending inbound substreams from dialed nodes.
|
||||
// Only add to this list elements received through `add_to_next_rx`.
|
||||
next_incoming: Vec<(M, M::InboundSubstream, Multiaddr)>,
|
||||
next_incoming: Vec<(Arc<M>, Multiaddr)>,
|
||||
|
||||
// New elements are not directly added to `next_incoming`. Instead they are sent to this
|
||||
// channel. This is done so that we can wake up tasks whenever a new element is added.
|
||||
add_to_next_rx: mpsc::UnboundedReceiver<(M, M::InboundSubstream, Multiaddr)>,
|
||||
add_to_next_rx: mpsc::UnboundedReceiver<(Arc<M>, Multiaddr)>,
|
||||
|
||||
// Other side of `add_to_next_rx`.
|
||||
add_to_next_tx: mpsc::UnboundedSender<(M, M::InboundSubstream, Multiaddr)>,
|
||||
add_to_next_tx: mpsc::UnboundedSender<(Arc<M>, Multiaddr)>,
|
||||
}
|
||||
|
||||
impl<T, C> From<UpgradedNode<T, C>> for ConnectionReuse<T, C>
|
||||
@ -120,11 +132,11 @@ where
|
||||
T: Transport + 'static, // TODO: 'static :(
|
||||
T::Output: AsyncRead + AsyncWrite,
|
||||
C: ConnectionUpgrade<T::Output, T::MultiaddrFuture> + Clone + 'static, // TODO: 'static :(
|
||||
C::Output: StreamMuxer + Clone,
|
||||
C::Output: StreamMuxer,
|
||||
C::MultiaddrFuture: Future<Item = Multiaddr, Error = IoError>,
|
||||
C::NamesIter: Clone, // TODO: not elegant
|
||||
{
|
||||
type Output = <C::Output as StreamMuxer>::Substream;
|
||||
type Output = muxing::SubstreamRef<Arc<C::Output>>;
|
||||
type MultiaddrFuture = future::FutureResult<Multiaddr, IoError>;
|
||||
type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = IoError>>;
|
||||
type ListenerUpgrade = FutureResult<(Self::Output, Self::MultiaddrFuture), IoError>;
|
||||
@ -171,7 +183,7 @@ where
|
||||
.map(|muxer| muxer.clone())
|
||||
{
|
||||
let a = addr.clone();
|
||||
Either::A(muxer.outbound().map(move |s| s.map(move |s| (s, future::ok(a)))))
|
||||
Either::A(muxing::outbound_from_ref_and_wrap(muxer).map(|o| o.map(move |s| (s, future::ok(a)))))
|
||||
} else {
|
||||
Either::B(future::ok(None))
|
||||
};
|
||||
@ -190,10 +202,10 @@ where
|
||||
let future = dial
|
||||
.and_then(move |(muxer, addr_fut)| {
|
||||
trace!("Waiting for remote's address");
|
||||
addr_fut.map(move |addr| (muxer, addr))
|
||||
addr_fut.map(move |addr| (Arc::new(muxer), addr))
|
||||
})
|
||||
.and_then(move |(muxer, addr)| {
|
||||
muxer.clone().outbound().and_then(move |substream| {
|
||||
muxing::outbound_from_ref(muxer.clone()).and_then(move |substream| {
|
||||
if let Some(s) = substream {
|
||||
// Replace the active connection because we are the most recent.
|
||||
let mut lock = shared.lock();
|
||||
@ -201,9 +213,9 @@ where
|
||||
// TODO: doesn't need locking ; the sender could be extracted
|
||||
let _ = lock.add_to_next_tx.unbounded_send((
|
||||
muxer.clone(),
|
||||
muxer.inbound(),
|
||||
addr.clone(),
|
||||
));
|
||||
let s = muxing::substream_from_ref(muxer, s);
|
||||
Ok((s, future::ok(addr)))
|
||||
} else {
|
||||
error!("failed to dial to {}", addr);
|
||||
@ -235,13 +247,13 @@ where
|
||||
T: Transport + 'static, // TODO: 'static :(
|
||||
T::Output: AsyncRead + AsyncWrite,
|
||||
C: ConnectionUpgrade<T::Output, T::MultiaddrFuture> + Clone + 'static, // TODO: 'static :(
|
||||
C::Output: StreamMuxer + Clone,
|
||||
C::Output: StreamMuxer,
|
||||
C::MultiaddrFuture: Future<Item = Multiaddr, Error = IoError>,
|
||||
C::NamesIter: Clone, // TODO: not elegant
|
||||
{
|
||||
type Incoming = ConnectionReuseIncoming<C::Output>;
|
||||
type IncomingUpgrade =
|
||||
future::FutureResult<(<C::Output as StreamMuxer>::Substream, Self::MultiaddrFuture), IoError>;
|
||||
future::FutureResult<(muxing::SubstreamRef<Arc<C::Output>>, Self::MultiaddrFuture), IoError>;
|
||||
|
||||
#[inline]
|
||||
fn next_incoming(self) -> Self::Incoming {
|
||||
@ -259,7 +271,7 @@ where
|
||||
// The main listener. `S` is from the underlying transport.
|
||||
listener: S,
|
||||
current_upgrades: FuturesUnordered<F>,
|
||||
connections: Vec<(M, <M as StreamMuxer>::InboundSubstream, Multiaddr)>,
|
||||
connections: Vec<(Arc<M>, Multiaddr)>,
|
||||
|
||||
// Shared between the whole connection reuse mechanism.
|
||||
shared: Arc<Mutex<Shared<M>>>,
|
||||
@ -269,9 +281,9 @@ impl<S, F, M> Stream for ConnectionReuseListener<S, F, M>
|
||||
where
|
||||
S: Stream<Item = F, Error = IoError>,
|
||||
F: Future<Item = (M, Multiaddr), Error = IoError>,
|
||||
M: StreamMuxer + Clone + 'static, // TODO: 'static :(
|
||||
M: StreamMuxer + 'static, // TODO: 'static :(
|
||||
{
|
||||
type Item = FutureResult<(M::Substream, FutureResult<Multiaddr, IoError>), IoError>;
|
||||
type Item = FutureResult<(muxing::SubstreamRef<Arc<M>>, FutureResult<Multiaddr, IoError>), IoError>;
|
||||
type Error = IoError;
|
||||
|
||||
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
|
||||
@ -304,9 +316,7 @@ where
|
||||
loop {
|
||||
match self.current_upgrades.poll() {
|
||||
Ok(Async::Ready(Some((muxer, client_addr)))) => {
|
||||
let next_incoming = muxer.clone().inbound();
|
||||
self.connections
|
||||
.push((muxer.clone(), next_incoming, client_addr.clone()));
|
||||
self.connections.push((Arc::new(muxer), client_addr.clone()));
|
||||
}
|
||||
Err(err) => {
|
||||
debug!("error while upgrading listener connection: {:?}", err);
|
||||
@ -318,8 +328,8 @@ where
|
||||
|
||||
// Check whether any incoming substream is ready.
|
||||
for n in (0..self.connections.len()).rev() {
|
||||
let (muxer, mut next_incoming, client_addr) = self.connections.swap_remove(n);
|
||||
match next_incoming.poll() {
|
||||
let (muxer, client_addr) = self.connections.swap_remove(n);
|
||||
match muxer.poll_inbound() {
|
||||
Ok(Async::Ready(None)) => {
|
||||
// stream muxer gave us a `None` => connection should be considered closed
|
||||
debug!("no more inbound substreams on {}", client_addr);
|
||||
@ -333,15 +343,15 @@ where
|
||||
.active_connections
|
||||
.insert(client_addr.clone(), muxer.clone());
|
||||
// A new substream is ready.
|
||||
let mut new_next = muxer.clone().inbound();
|
||||
self.connections
|
||||
.push((muxer, new_next, client_addr.clone()));
|
||||
.push((muxer.clone(), client_addr.clone()));
|
||||
let incoming = muxing::substream_from_ref(muxer, incoming);
|
||||
return Ok(Async::Ready(Some(
|
||||
future::ok((incoming, future::ok(client_addr))),
|
||||
)));
|
||||
}
|
||||
Ok(Async::NotReady) => {
|
||||
self.connections.push((muxer, next_incoming, client_addr));
|
||||
self.connections.push((muxer, client_addr));
|
||||
}
|
||||
Err(err) => {
|
||||
debug!("error while upgrading the multiplexed incoming connection: {:?}", err);
|
||||
@ -367,9 +377,9 @@ where
|
||||
|
||||
impl<M> Future for ConnectionReuseIncoming<M>
|
||||
where
|
||||
M: Clone + StreamMuxer,
|
||||
M: StreamMuxer,
|
||||
{
|
||||
type Item = future::FutureResult<(M::Substream, future::FutureResult<Multiaddr, IoError>), IoError>;
|
||||
type Item = future::FutureResult<(muxing::SubstreamRef<Arc<M>>, future::FutureResult<Multiaddr, IoError>), IoError>;
|
||||
type Error = IoError;
|
||||
|
||||
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
|
||||
@ -393,8 +403,8 @@ where
|
||||
|
||||
// Check whether any incoming substream is ready.
|
||||
for n in (0..lock.next_incoming.len()).rev() {
|
||||
let (muxer, mut future, addr) = lock.next_incoming.swap_remove(n);
|
||||
match future.poll() {
|
||||
let (muxer, addr) = lock.next_incoming.swap_remove(n);
|
||||
match muxer.poll_inbound() {
|
||||
Ok(Async::Ready(None)) => {
|
||||
debug!("no inbound substream for {}", addr);
|
||||
lock.active_connections.remove(&addr);
|
||||
@ -403,12 +413,12 @@ where
|
||||
// A substream is ready ; push back the muxer for the next time this function
|
||||
// is called, then return.
|
||||
debug!("New incoming substream");
|
||||
let next = muxer.clone().inbound();
|
||||
lock.next_incoming.push((muxer, next, addr.clone()));
|
||||
return Ok(Async::Ready(future::ok((value, future::ok(addr)))));
|
||||
lock.next_incoming.push((muxer.clone(), addr.clone()));
|
||||
let substream = muxing::substream_from_ref(muxer, value);
|
||||
return Ok(Async::Ready(future::ok((substream, future::ok(addr)))));
|
||||
}
|
||||
Ok(Async::NotReady) => {
|
||||
lock.next_incoming.push((muxer, future, addr));
|
||||
lock.next_incoming.push((muxer, addr));
|
||||
}
|
||||
Err(err) => {
|
||||
// In case of error, we just not push back the element, which drops it.
|
||||
|
@ -101,51 +101,113 @@ where
|
||||
B: StreamMuxer,
|
||||
{
|
||||
type Substream = EitherOutput<A::Substream, B::Substream>;
|
||||
type InboundSubstream = EitherInbound<A, B>;
|
||||
type OutboundSubstream = EitherOutbound<A, B>;
|
||||
|
||||
#[inline]
|
||||
fn inbound(self) -> Self::InboundSubstream {
|
||||
match self {
|
||||
EitherOutput::First(a) => EitherInbound::A(a.inbound()),
|
||||
EitherOutput::Second(b) => EitherInbound::B(b.inbound()),
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn outbound(self) -> Self::OutboundSubstream {
|
||||
match self {
|
||||
EitherOutput::First(a) => EitherOutbound::A(a.outbound()),
|
||||
EitherOutput::Second(b) => EitherOutbound::B(b.outbound()),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Copy, Clone)]
|
||||
pub enum EitherInbound<A: StreamMuxer, B: StreamMuxer> {
|
||||
A(A::InboundSubstream),
|
||||
B(B::InboundSubstream),
|
||||
}
|
||||
|
||||
impl<A, B> Future for EitherInbound<A, B>
|
||||
where
|
||||
A: StreamMuxer,
|
||||
B: StreamMuxer,
|
||||
{
|
||||
type Item = Option<EitherOutput<A::Substream, B::Substream>>;
|
||||
type Error = IoError;
|
||||
|
||||
#[inline]
|
||||
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
|
||||
fn poll_inbound(&self) -> Poll<Option<Self::Substream>, IoError> {
|
||||
match *self {
|
||||
EitherInbound::A(ref mut a) => {
|
||||
let item = try_ready!(a.poll());
|
||||
Ok(Async::Ready(item.map(EitherOutput::First)))
|
||||
}
|
||||
EitherInbound::B(ref mut b) => {
|
||||
let item = try_ready!(b.poll());
|
||||
Ok(Async::Ready(item.map(EitherOutput::Second)))
|
||||
}
|
||||
EitherOutput::First(ref inner) => inner.poll_inbound().map(|p| p.map(|o| o.map(EitherOutput::First))),
|
||||
EitherOutput::Second(ref inner) => inner.poll_inbound().map(|p| p.map(|o| o.map(EitherOutput::Second))),
|
||||
}
|
||||
}
|
||||
|
||||
fn open_outbound(&self) -> Self::OutboundSubstream {
|
||||
match *self {
|
||||
EitherOutput::First(ref inner) => EitherOutbound::A(inner.open_outbound()),
|
||||
EitherOutput::Second(ref inner) => EitherOutbound::B(inner.open_outbound()),
|
||||
}
|
||||
}
|
||||
|
||||
fn poll_outbound(&self, substream: &mut Self::OutboundSubstream) -> Poll<Option<Self::Substream>, IoError> {
|
||||
match (self, substream) {
|
||||
(EitherOutput::First(ref inner), EitherOutbound::A(ref mut substream)) => {
|
||||
inner.poll_outbound(substream).map(|p| p.map(|o| o.map(EitherOutput::First)))
|
||||
},
|
||||
(EitherOutput::Second(ref inner), EitherOutbound::B(ref mut substream)) => {
|
||||
inner.poll_outbound(substream).map(|p| p.map(|o| o.map(EitherOutput::Second)))
|
||||
},
|
||||
_ => panic!("Wrong API usage")
|
||||
}
|
||||
}
|
||||
|
||||
fn destroy_outbound(&self, substream: Self::OutboundSubstream) {
|
||||
match *self {
|
||||
EitherOutput::First(ref inner) => {
|
||||
match substream {
|
||||
EitherOutbound::A(substream) => inner.destroy_outbound(substream),
|
||||
_ => panic!("Wrong API usage")
|
||||
}
|
||||
},
|
||||
EitherOutput::Second(ref inner) => {
|
||||
match substream {
|
||||
EitherOutbound::B(substream) => inner.destroy_outbound(substream),
|
||||
_ => panic!("Wrong API usage")
|
||||
}
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
fn read_substream(&self, substream: &mut Self::Substream, buf: &mut [u8]) -> Result<usize, IoError> {
|
||||
match (self, substream) {
|
||||
(EitherOutput::First(ref inner), EitherOutput::First(ref mut substream)) => {
|
||||
inner.read_substream(substream, buf)
|
||||
},
|
||||
(EitherOutput::Second(ref inner), EitherOutput::Second(ref mut substream)) => {
|
||||
inner.read_substream(substream, buf)
|
||||
},
|
||||
_ => panic!("Wrong API usage")
|
||||
}
|
||||
}
|
||||
|
||||
fn write_substream(&self, substream: &mut Self::Substream, buf: &[u8]) -> Result<usize, IoError> {
|
||||
match (self, substream) {
|
||||
(EitherOutput::First(ref inner), EitherOutput::First(ref mut substream)) => {
|
||||
inner.write_substream(substream, buf)
|
||||
},
|
||||
(EitherOutput::Second(ref inner), EitherOutput::Second(ref mut substream)) => {
|
||||
inner.write_substream(substream, buf)
|
||||
},
|
||||
_ => panic!("Wrong API usage")
|
||||
}
|
||||
}
|
||||
|
||||
fn flush_substream(&self, substream: &mut Self::Substream) -> Result<(), IoError> {
|
||||
match (self, substream) {
|
||||
(EitherOutput::First(ref inner), EitherOutput::First(ref mut substream)) => {
|
||||
inner.flush_substream(substream)
|
||||
},
|
||||
(EitherOutput::Second(ref inner), EitherOutput::Second(ref mut substream)) => {
|
||||
inner.flush_substream(substream)
|
||||
},
|
||||
_ => panic!("Wrong API usage")
|
||||
}
|
||||
}
|
||||
|
||||
fn shutdown_substream(&self, substream: &mut Self::Substream) -> Poll<(), IoError> {
|
||||
match (self, substream) {
|
||||
(EitherOutput::First(ref inner), EitherOutput::First(ref mut substream)) => {
|
||||
inner.shutdown_substream(substream)
|
||||
},
|
||||
(EitherOutput::Second(ref inner), EitherOutput::Second(ref mut substream)) => {
|
||||
inner.shutdown_substream(substream)
|
||||
},
|
||||
_ => panic!("Wrong API usage")
|
||||
}
|
||||
}
|
||||
|
||||
fn destroy_substream(&self, substream: Self::Substream) {
|
||||
match *self {
|
||||
EitherOutput::First(ref inner) => {
|
||||
match substream {
|
||||
EitherOutput::First(substream) => inner.destroy_substream(substream),
|
||||
_ => panic!("Wrong API usage")
|
||||
}
|
||||
},
|
||||
EitherOutput::Second(ref inner) => {
|
||||
match substream {
|
||||
EitherOutput::Second(substream) => inner.destroy_substream(substream),
|
||||
_ => panic!("Wrong API usage")
|
||||
}
|
||||
},
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -156,29 +218,6 @@ pub enum EitherOutbound<A: StreamMuxer, B: StreamMuxer> {
|
||||
B(B::OutboundSubstream),
|
||||
}
|
||||
|
||||
impl<A, B> Future for EitherOutbound<A, B>
|
||||
where
|
||||
A: StreamMuxer,
|
||||
B: StreamMuxer,
|
||||
{
|
||||
type Item = Option<EitherOutput<A::Substream, B::Substream>>;
|
||||
type Error = IoError;
|
||||
|
||||
#[inline]
|
||||
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
|
||||
match *self {
|
||||
EitherOutbound::A(ref mut a) => {
|
||||
let item = try_ready!(a.poll());
|
||||
Ok(Async::Ready(item.map(EitherOutput::First)))
|
||||
}
|
||||
EitherOutbound::B(ref mut b) => {
|
||||
let item = try_ready!(b.poll());
|
||||
Ok(Async::Ready(item.map(EitherOutput::Second)))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Implements `Stream` and dispatches all method calls to either `First` or `Second`.
|
||||
#[derive(Debug, Copy, Clone)]
|
||||
pub enum EitherListenStream<A, B> {
|
||||
|
@ -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,34 +18,257 @@
|
||||
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
|
||||
// DEALINGS IN THE SOFTWARE.
|
||||
|
||||
use futures::future::Future;
|
||||
use std::io::Error as IoError;
|
||||
use futures::{future, prelude::*};
|
||||
use std::io::{Error as IoError, Read, Write};
|
||||
use std::ops::Deref;
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
|
||||
/// Implemented on objects that can be turned into a substream.
|
||||
///
|
||||
/// > **Note**: The methods of this trait consume the object, but if the object implements `Clone`
|
||||
/// > then you can clone it and keep the original in order to open additional substreams.
|
||||
/// Implemented on objects that can open and manage substreams.
|
||||
pub trait StreamMuxer {
|
||||
/// Type of the object that represents the raw substream where data can be read and written.
|
||||
type Substream: AsyncRead + AsyncWrite;
|
||||
|
||||
/// Future that will be resolved when a new incoming substream is open.
|
||||
///
|
||||
/// A `None` item signals that the underlying resource has been exhausted and
|
||||
/// no more substreams can be created.
|
||||
type InboundSubstream: Future<Item = Option<Self::Substream>, Error = IoError>;
|
||||
type Substream;
|
||||
|
||||
/// Future that will be resolved when the outgoing substream is open.
|
||||
///
|
||||
/// A `None` item signals that the underlying resource has been exhausted and
|
||||
/// no more substreams can be created.
|
||||
type OutboundSubstream: Future<Item = Option<Self::Substream>, Error = IoError>;
|
||||
type OutboundSubstream;
|
||||
|
||||
/// Produces a future that will be resolved when a new incoming substream arrives.
|
||||
fn inbound(self) -> Self::InboundSubstream;
|
||||
/// Polls for an inbound substream.
|
||||
///
|
||||
/// This function behaves the same as a `Stream`.
|
||||
fn poll_inbound(&self) -> Poll<Option<Self::Substream>, IoError>;
|
||||
|
||||
/// Opens a new outgoing substream, and produces a future that will be resolved when it becomes
|
||||
/// available.
|
||||
fn outbound(self) -> Self::OutboundSubstream;
|
||||
fn open_outbound(&self) -> Self::OutboundSubstream;
|
||||
|
||||
/// Polls the outbound substream.
|
||||
///
|
||||
/// May panic or produce an undefined result if an earlier polling returned `Ready` or `Err`.
|
||||
fn poll_outbound(
|
||||
&self,
|
||||
substream: &mut Self::OutboundSubstream,
|
||||
) -> Poll<Option<Self::Substream>, IoError>;
|
||||
|
||||
/// Destroys an outbound substream. Use this after the outbound substream has finished, or if
|
||||
/// you want to interrupt it.
|
||||
fn destroy_outbound(&self, substream: Self::OutboundSubstream);
|
||||
|
||||
/// Reads data from a substream. The behaviour is the same as `std::io::Read::read`.
|
||||
fn read_substream(
|
||||
&self,
|
||||
substream: &mut Self::Substream,
|
||||
buf: &mut [u8],
|
||||
) -> Result<usize, IoError>;
|
||||
|
||||
/// Write data to a substream. The behaviour is the same as `std::io::Write::write`.
|
||||
fn write_substream(
|
||||
&self,
|
||||
substream: &mut Self::Substream,
|
||||
buf: &[u8],
|
||||
) -> Result<usize, IoError>;
|
||||
|
||||
/// Flushes a substream. The behaviour is the same as `std::io::Write::flush`.
|
||||
fn flush_substream(&self, substream: &mut Self::Substream) -> Result<(), IoError>;
|
||||
|
||||
/// Attempts to shut down a substream. The behaviour is the same as
|
||||
/// `tokio_io::AsyncWrite::shutdown`.
|
||||
fn shutdown_substream(&self, substream: &mut Self::Substream) -> Poll<(), IoError>;
|
||||
|
||||
/// Destroys a substream.
|
||||
fn destroy_substream(&self, substream: Self::Substream);
|
||||
}
|
||||
|
||||
/// Polls for an inbound from the muxer but wraps the output in an object that
|
||||
/// implements `Read`/`Write`/`AsyncRead`/`AsyncWrite`.
|
||||
#[inline]
|
||||
pub fn inbound_from_ref_and_wrap<P>(
|
||||
muxer: P,
|
||||
) -> impl Future<Item = Option<SubstreamRef<P>>, Error = IoError>
|
||||
where
|
||||
P: Deref + Clone,
|
||||
P::Target: StreamMuxer,
|
||||
{
|
||||
let muxer2 = muxer.clone();
|
||||
future::poll_fn(move || muxer.poll_inbound())
|
||||
.map(|substream| substream.map(move |s| substream_from_ref(muxer2, s)))
|
||||
}
|
||||
|
||||
/// Same as `outbound_from_ref`, but wraps the output in an object that
|
||||
/// implements `Read`/`Write`/`AsyncRead`/`AsyncWrite`.
|
||||
#[inline]
|
||||
pub fn outbound_from_ref_and_wrap<P>(muxer: P) -> OutboundSubstreamRefWrapFuture<P>
|
||||
where
|
||||
P: Deref + Clone,
|
||||
P::Target: StreamMuxer,
|
||||
{
|
||||
let inner = outbound_from_ref(muxer);
|
||||
OutboundSubstreamRefWrapFuture { inner }
|
||||
}
|
||||
|
||||
/// Future returned by `outbound_from_ref_and_wrap`.
|
||||
pub struct OutboundSubstreamRefWrapFuture<P>
|
||||
where
|
||||
P: Deref + Clone,
|
||||
P::Target: StreamMuxer,
|
||||
{
|
||||
inner: OutboundSubstreamRefFuture<P>,
|
||||
}
|
||||
|
||||
impl<P> Future for OutboundSubstreamRefWrapFuture<P>
|
||||
where
|
||||
P: Deref + Clone,
|
||||
P::Target: StreamMuxer,
|
||||
{
|
||||
type Item = Option<SubstreamRef<P>>;
|
||||
type Error = IoError;
|
||||
|
||||
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
|
||||
match self.inner.poll() {
|
||||
Ok(Async::Ready(Some(substream))) => {
|
||||
let out = substream_from_ref(self.inner.muxer.clone(), substream);
|
||||
Ok(Async::Ready(Some(out)))
|
||||
}
|
||||
Ok(Async::Ready(None)) => Ok(Async::Ready(None)),
|
||||
Ok(Async::NotReady) => Ok(Async::NotReady),
|
||||
Err(err) => Err(err),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Builds a new future for an outbound substream, where the muxer is a reference.
|
||||
#[inline]
|
||||
pub fn outbound_from_ref<P>(muxer: P) -> OutboundSubstreamRefFuture<P>
|
||||
where
|
||||
P: Deref,
|
||||
P::Target: StreamMuxer,
|
||||
{
|
||||
let outbound = muxer.open_outbound();
|
||||
OutboundSubstreamRefFuture {
|
||||
muxer,
|
||||
outbound: Some(outbound),
|
||||
}
|
||||
}
|
||||
|
||||
/// Future returned by `outbound_from_ref`.
|
||||
pub struct OutboundSubstreamRefFuture<P>
|
||||
where
|
||||
P: Deref,
|
||||
P::Target: StreamMuxer,
|
||||
{
|
||||
muxer: P,
|
||||
outbound: Option<<P::Target as StreamMuxer>::OutboundSubstream>,
|
||||
}
|
||||
|
||||
impl<P> Future for OutboundSubstreamRefFuture<P>
|
||||
where
|
||||
P: Deref,
|
||||
P::Target: StreamMuxer,
|
||||
{
|
||||
type Item = Option<<P::Target as StreamMuxer>::Substream>;
|
||||
type Error = IoError;
|
||||
|
||||
#[inline]
|
||||
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
|
||||
self.muxer
|
||||
.poll_outbound(self.outbound.as_mut().expect("outbound was empty"))
|
||||
}
|
||||
}
|
||||
|
||||
impl<P> Drop for OutboundSubstreamRefFuture<P>
|
||||
where
|
||||
P: Deref,
|
||||
P::Target: StreamMuxer,
|
||||
{
|
||||
#[inline]
|
||||
fn drop(&mut self) {
|
||||
self.muxer
|
||||
.destroy_outbound(self.outbound.take().expect("outbound was empty"))
|
||||
}
|
||||
}
|
||||
|
||||
/// Builds an implementation of `Read`/`Write`/`AsyncRead`/`AsyncWrite` from an `Arc` to the
|
||||
/// muxer and a substream.
|
||||
#[inline]
|
||||
pub fn substream_from_ref<P>(
|
||||
muxer: P,
|
||||
substream: <P::Target as StreamMuxer>::Substream,
|
||||
) -> SubstreamRef<P>
|
||||
where
|
||||
P: Deref,
|
||||
P::Target: StreamMuxer,
|
||||
{
|
||||
SubstreamRef {
|
||||
muxer,
|
||||
substream: Some(substream),
|
||||
}
|
||||
}
|
||||
|
||||
/// Stream returned by `substream_from_ref`.
|
||||
pub struct SubstreamRef<P>
|
||||
where
|
||||
P: Deref,
|
||||
P::Target: StreamMuxer,
|
||||
{
|
||||
muxer: P,
|
||||
substream: Option<<P::Target as StreamMuxer>::Substream>,
|
||||
}
|
||||
|
||||
impl<P> Read for SubstreamRef<P>
|
||||
where
|
||||
P: Deref,
|
||||
P::Target: StreamMuxer,
|
||||
{
|
||||
#[inline]
|
||||
fn read(&mut self, buf: &mut [u8]) -> Result<usize, IoError> {
|
||||
self.muxer
|
||||
.read_substream(self.substream.as_mut().expect("substream was empty"), buf)
|
||||
}
|
||||
}
|
||||
|
||||
impl<P> AsyncRead for SubstreamRef<P>
|
||||
where
|
||||
P: Deref,
|
||||
P::Target: StreamMuxer,
|
||||
{
|
||||
}
|
||||
|
||||
impl<P> Write for SubstreamRef<P>
|
||||
where
|
||||
P: Deref,
|
||||
P::Target: StreamMuxer,
|
||||
{
|
||||
#[inline]
|
||||
fn write(&mut self, buf: &[u8]) -> Result<usize, IoError> {
|
||||
self.muxer
|
||||
.write_substream(self.substream.as_mut().expect("substream was empty"), buf)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn flush(&mut self) -> Result<(), IoError> {
|
||||
self.muxer
|
||||
.flush_substream(self.substream.as_mut().expect("substream was empty"))
|
||||
}
|
||||
}
|
||||
|
||||
impl<P> AsyncWrite for SubstreamRef<P>
|
||||
where
|
||||
P: Deref,
|
||||
P::Target: StreamMuxer,
|
||||
{
|
||||
#[inline]
|
||||
fn shutdown(&mut self) -> Poll<(), IoError> {
|
||||
self.muxer
|
||||
.shutdown_substream(self.substream.as_mut().expect("substream was empty"))
|
||||
}
|
||||
}
|
||||
|
||||
impl<P> Drop for SubstreamRef<P>
|
||||
where
|
||||
P: Deref,
|
||||
P::Target: StreamMuxer,
|
||||
{
|
||||
#[inline]
|
||||
fn drop(&mut self) {
|
||||
self.muxer
|
||||
.destroy_substream(self.substream.take().expect("substream was empty"))
|
||||
}
|
||||
}
|
||||
|
Reference in New Issue
Block a user