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:
Pierre Krieger
2018-08-31 10:31:34 +02:00
committed by Benjamin Kampmann
parent c02dea8128
commit 0e1483f02e
8 changed files with 573 additions and 301 deletions

View File

@ -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.

View File

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

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