Add an Error associated type to transports (#835)

* Add an Error associated type to transports

* Improve raw swarm a bit

* Rename map_other to map

* Use source() instead of cause()

* RawSwarmIncErr -> IncomingError
This commit is contained in:
Pierre Krieger
2019-01-10 11:27:06 +01:00
committed by GitHub
parent f55a8bc2f3
commit dbff125df2
30 changed files with 798 additions and 682 deletions

View File

@ -275,19 +275,21 @@ pub enum EitherListenStream<A, B> {
impl<AStream, BStream, AInner, BInner> Stream for EitherListenStream<AStream, BStream> impl<AStream, BStream, AInner, BInner> Stream for EitherListenStream<AStream, BStream>
where where
AStream: Stream<Item = (AInner, Multiaddr), Error = IoError>, AStream: Stream<Item = (AInner, Multiaddr)>,
BStream: Stream<Item = (BInner, Multiaddr), Error = IoError>, BStream: Stream<Item = (BInner, Multiaddr)>,
{ {
type Item = (EitherFuture<AInner, BInner>, Multiaddr); type Item = (EitherFuture<AInner, BInner>, Multiaddr);
type Error = IoError; type Error = EitherError<AStream::Error, BStream::Error>;
#[inline] #[inline]
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> { fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
match self { match self {
EitherListenStream::First(a) => a.poll() EitherListenStream::First(a) => a.poll()
.map(|i| (i.map(|v| (v.map(|(o, addr)| (EitherFuture::First(o), addr)))))), .map(|i| (i.map(|v| (v.map(|(o, addr)| (EitherFuture::First(o), addr))))))
.map_err(EitherError::A),
EitherListenStream::Second(a) => a.poll() EitherListenStream::Second(a) => a.poll()
.map(|i| (i.map(|v| (v.map(|(o, addr)| (EitherFuture::Second(o), addr)))))), .map(|i| (i.map(|v| (v.map(|(o, addr)| (EitherFuture::Second(o), addr))))))
.map_err(EitherError::B),
} }
} }
} }
@ -302,17 +304,17 @@ pub enum EitherFuture<A, B> {
impl<AFuture, BFuture, AInner, BInner> Future for EitherFuture<AFuture, BFuture> impl<AFuture, BFuture, AInner, BInner> Future for EitherFuture<AFuture, BFuture>
where where
AFuture: Future<Item = AInner, Error = IoError>, AFuture: Future<Item = AInner>,
BFuture: Future<Item = BInner, Error = IoError>, BFuture: Future<Item = BInner>,
{ {
type Item = EitherOutput<AInner, BInner>; type Item = EitherOutput<AInner, BInner>;
type Error = IoError; type Error = EitherError<AFuture::Error, BFuture::Error>;
#[inline] #[inline]
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
match self { match self {
EitherFuture::First(a) => a.poll().map(|v| v.map(EitherOutput::First)), EitherFuture::First(a) => a.poll().map(|v| v.map(EitherOutput::First)).map_err(EitherError::A),
EitherFuture::Second(a) => a.poll().map(|v| v.map(EitherOutput::Second)), EitherFuture::Second(a) => a.poll().map(|v| v.map(EitherOutput::Second)).map_err(EitherError::B),
} }
} }
} }

View File

@ -20,11 +20,10 @@
//! Manage listening on multiple multiaddresses at once. //! Manage listening on multiple multiaddresses at once.
use crate::{Multiaddr, Transport, transport::TransportError};
use futures::prelude::*; use futures::prelude::*;
use std::fmt; use std::{collections::VecDeque, fmt};
use void::Void; use void::Void;
use crate::{Multiaddr, Transport};
use std::collections::VecDeque;
/// Implementation of `futures::Stream` that allows listening on multiaddresses. /// Implementation of `futures::Stream` that allows listening on multiaddresses.
/// ///
@ -151,15 +150,14 @@ where
/// Start listening on a multiaddress. /// Start listening on a multiaddress.
/// ///
/// Returns an error if the transport doesn't support the given multiaddress. /// Returns an error if the transport doesn't support the given multiaddress.
pub fn listen_on(&mut self, addr: Multiaddr) -> Result<Multiaddr, Multiaddr> pub fn listen_on(&mut self, addr: Multiaddr) -> Result<Multiaddr, TransportError<TTrans::Error>>
where where
TTrans: Clone, TTrans: Clone,
{ {
let (listener, new_addr) = self let (listener, new_addr) = self
.transport .transport
.clone() .clone()
.listen_on(addr) .listen_on(addr)?;
.map_err(|(_, addr)| addr)?;
self.listeners.push_back(Listener { self.listeners.push_back(Listener {
listener, listener,
@ -324,7 +322,7 @@ mod tests {
let mut listeners = ListenersStream::new(rx); let mut listeners = ListenersStream::new(rx);
listeners.listen_on("/memory".parse().unwrap()).unwrap(); listeners.listen_on("/memory".parse().unwrap()).unwrap();
let dial = tx.dial("/memory".parse().unwrap()).unwrap_or_else(|_| panic!()); let dial = tx.dial("/memory".parse().unwrap()).unwrap();
let future = listeners let future = listeners
.into_future() .into_future()

View File

@ -37,7 +37,8 @@ use crate::{
node::Substream node::Substream
}, },
nodes::listeners::{ListenersEvent, ListenersStream}, nodes::listeners::{ListenersEvent, ListenersStream},
transport::Transport transport::Transport,
transport::TransportError,
}; };
use fnv::FnvHashMap; use fnv::FnvHashMap;
use futures::{prelude::*, future}; use futures::{prelude::*, future};
@ -45,7 +46,6 @@ use std::{
collections::hash_map::{Entry, OccupiedEntry}, collections::hash_map::{Entry, OccupiedEntry},
error, error,
fmt, fmt,
io::{Error as IoError, ErrorKind as IoErrorKind}
}; };
mod tests; mod tests;
@ -60,7 +60,7 @@ where
listeners: ListenersStream<TTrans>, listeners: ListenersStream<TTrans>,
/// The nodes currently active. /// The nodes currently active.
active_nodes: CollectionStream<TInEvent, TOutEvent, THandler, RawSwarmReachError, THandlerErr>, active_nodes: CollectionStream<TInEvent, TOutEvent, THandler, RawSwarmReachError<TTrans::Error>, THandlerErr>,
/// The reach attempts of the swarm. /// The reach attempts of the swarm.
/// This needs to be a separate struct in order to handle multiple mutable borrows issues. /// This needs to be a separate struct in order to handle multiple mutable borrows issues.
@ -122,7 +122,7 @@ where
/// Address used to send back data to the remote. /// Address used to send back data to the remote.
send_back_addr: Multiaddr, send_back_addr: Multiaddr,
/// The error that happened. /// The error that happened.
error: IoError, error: IncomingError<TTrans::Error>,
}, },
/// A new connection to a peer has been opened. /// A new connection to a peer has been opened.
@ -180,7 +180,7 @@ where
multiaddr: Multiaddr, multiaddr: Multiaddr,
/// The error that happened. /// The error that happened.
error: RawSwarmReachError, error: RawSwarmReachError<TTrans::Error>,
}, },
/// Failed to reach a peer that we were trying to dial. /// Failed to reach a peer that we were trying to dial.
@ -189,7 +189,7 @@ where
multiaddr: Multiaddr, multiaddr: Multiaddr,
/// The error that happened. /// The error that happened.
error: IoError, error: TransportError<TTrans::Error>,
/// The handler that was passed to `dial()`. /// The handler that was passed to `dial()`.
handler: THandler, handler: THandler,
@ -208,6 +208,7 @@ impl<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr> fmt::Debug for RawS
where where
TOutEvent: fmt::Debug, TOutEvent: fmt::Debug,
TTrans: Transport, TTrans: Transport,
TTrans::Error: fmt::Debug,
THandlerErr: fmt::Debug, THandlerErr: fmt::Debug,
{ {
fn fmt(&self, f: &mut fmt::Formatter) -> Result<(), fmt::Error> { fn fmt(&self, f: &mut fmt::Formatter) -> Result<(), fmt::Error> {
@ -283,10 +284,10 @@ where
/// Error that can happen when trying to reach a node. /// Error that can happen when trying to reach a node.
#[derive(Debug)] #[derive(Debug)]
pub enum RawSwarmReachError { pub enum RawSwarmReachError<TTransErr> {
/// Error in the transport layer. /// Error in the transport layer.
// TODO: better error type // TODO: is a TransportError correct here?
Transport(IoError), Transport(TransportError<TTransErr>),
/// We successfully reached the peer, but there was a mismatch between the expected id and the /// We successfully reached the peer, but there was a mismatch between the expected id and the
/// actual id of the peer. /// actual id of the peer.
PeerIdMismatch { PeerIdMismatch {
@ -295,7 +296,9 @@ pub enum RawSwarmReachError {
} }
} }
impl fmt::Display for RawSwarmReachError { impl<TTransErr> fmt::Display for RawSwarmReachError<TTransErr>
where TTransErr: fmt::Display
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self { match self {
RawSwarmReachError::Transport(err) => write!(f, "{}", err), RawSwarmReachError::Transport(err) => write!(f, "{}", err),
@ -306,7 +309,9 @@ impl fmt::Display for RawSwarmReachError {
} }
} }
impl error::Error for RawSwarmReachError { impl<TTransErr> error::Error for RawSwarmReachError<TTransErr>
where TTransErr: error::Error + 'static
{
fn source(&self) -> Option<&(dyn error::Error + 'static)> { fn source(&self) -> Option<&(dyn error::Error + 'static)> {
match self { match self {
RawSwarmReachError::Transport(err) => Some(err), RawSwarmReachError::Transport(err) => Some(err),
@ -315,6 +320,41 @@ impl error::Error for RawSwarmReachError {
} }
} }
/// Error that can happen on an incoming connection.
#[derive(Debug)]
pub enum IncomingError<TTransErr> {
/// Error in the transport layer.
// TODO: just TTransError should be enough?
Transport(TransportError<TTransErr>),
/// Denied the incoming connection because we're already connected to this peer as a dialer
/// and we have a higher priority than the remote.
DeniedLowerPriority,
}
impl<TTransErr> fmt::Display for IncomingError<TTransErr>
where TTransErr: fmt::Display
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
IncomingError::Transport(err) => write!(f, "{}", err),
IncomingError::DeniedLowerPriority => {
write!(f, "Denied because of lower priority")
},
}
}
}
impl<TTransErr> error::Error for IncomingError<TTransErr>
where TTransErr: error::Error + 'static
{
fn source(&self) -> Option<&(dyn error::Error + 'static)> {
match self {
IncomingError::Transport(err) => Some(err),
IncomingError::DeniedLowerPriority => None,
}
}
}
/// A new connection arrived on a listener. /// A new connection arrived on a listener.
pub struct IncomingConnectionEvent<'a, TTrans: 'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a, THandlerErr: 'a> pub struct IncomingConnectionEvent<'a, TTrans: 'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a, THandlerErr: 'a>
where TTrans: Transport where TTrans: Transport
@ -326,7 +366,7 @@ where TTrans: Transport
/// Address used to send back data to the remote. /// Address used to send back data to the remote.
send_back_addr: Multiaddr, send_back_addr: Multiaddr,
/// Reference to the `active_nodes` field of the swarm. /// Reference to the `active_nodes` field of the swarm.
active_nodes: &'a mut CollectionStream<TInEvent, TOutEvent, THandler, RawSwarmReachError, THandlerErr>, active_nodes: &'a mut CollectionStream<TInEvent, TOutEvent, THandler, RawSwarmReachError<TTrans::Error>, THandlerErr>,
/// Reference to the `other_reach_attempts` field of the swarm. /// Reference to the `other_reach_attempts` field of the swarm.
other_reach_attempts: &'a mut Vec<(ReachAttemptId, ConnectedPoint)>, other_reach_attempts: &'a mut Vec<(ReachAttemptId, ConnectedPoint)>,
} }
@ -335,6 +375,7 @@ impl<'a, TTrans, TInEvent, TOutEvent, TMuxer, THandler, THandlerErr>
IncomingConnectionEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr> IncomingConnectionEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>
where where
TTrans: Transport<Output = (PeerId, TMuxer)>, TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::Error: Send + 'static,
TTrans::ListenerUpgrade: Send + 'static, TTrans::ListenerUpgrade: Send + 'static,
THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static, THandler: NodeHandler<Substream = Substream<TMuxer>, InEvent = TInEvent, OutEvent = TOutEvent, Error = THandlerErr> + Send + 'static,
THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary THandler::OutboundOpenInfo: Send + 'static, // TODO: shouldn't be necessary
@ -357,7 +398,7 @@ where
{ {
let connected_point = self.to_connected_point(); let connected_point = self.to_connected_point();
let handler = builder(self.info()); let handler = builder(self.info());
let id = self.active_nodes.add_reach_attempt(self.upgrade.map_err(RawSwarmReachError::Transport), handler); let id = self.active_nodes.add_reach_attempt(self.upgrade.map_err(|err| RawSwarmReachError::Transport(TransportError::Other(err))), handler);
self.other_reach_attempts.push(( self.other_reach_attempts.push((
id, id,
connected_point, connected_point,
@ -510,7 +551,7 @@ where
/// Start listening on the given multiaddress. /// Start listening on the given multiaddress.
#[inline] #[inline]
pub fn listen_on(&mut self, addr: Multiaddr) -> Result<Multiaddr, Multiaddr> { pub fn listen_on(&mut self, addr: Multiaddr) -> Result<Multiaddr, TransportError<TTrans::Error>> {
self.listeners.listen_on(addr) self.listeners.listen_on(addr)
} }
@ -550,9 +591,10 @@ where
/// Dials a multiaddress without knowing the peer ID we're going to obtain. /// Dials a multiaddress without knowing the peer ID we're going to obtain.
/// ///
/// The second parameter is the handler to use if we manage to reach a node. /// The second parameter is the handler to use if we manage to reach a node.
pub fn dial(&mut self, addr: Multiaddr, handler: THandler) -> Result<(), Multiaddr> pub fn dial(&mut self, addr: Multiaddr, handler: THandler) -> Result<(), TransportError<TTrans::Error>>
where where
TTrans: Transport<Output = (PeerId, TMuxer)>, TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::Error: Send + 'static,
TTrans::Dial: Send + 'static, TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send, TMuxer::OutboundSubstream: Send,
@ -560,13 +602,10 @@ where
TInEvent: Send + 'static, TInEvent: Send + 'static,
TOutEvent: Send + 'static, TOutEvent: Send + 'static,
{ {
let future = match self.transport().clone().dial(addr.clone()) { let future = self.transport().clone().dial(addr.clone())?;
Ok(fut) => fut,
Err((_, addr)) => return Err(addr),
};
let connected_point = ConnectedPoint::Dialer { address: addr }; let connected_point = ConnectedPoint::Dialer { address: addr };
let reach_id = self.active_nodes.add_reach_attempt(future.map_err(RawSwarmReachError::Transport), handler); let reach_id = self.active_nodes.add_reach_attempt(future.map_err(|err| RawSwarmReachError::Transport(TransportError::Other(err))), handler);
self.reach_attempts.other_reach_attempts.push((reach_id, connected_point)); self.reach_attempts.other_reach_attempts.push((reach_id, connected_point));
Ok(()) Ok(())
} }
@ -658,6 +697,7 @@ where
where where
TTrans: Transport<Output = (PeerId, TMuxer)>, TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::Dial: Send + 'static, TTrans::Dial: Send + 'static,
TTrans::Error: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send, TMuxer::OutboundSubstream: Send,
TMuxer::Substream: Send, TMuxer::Substream: Send,
@ -668,7 +708,7 @@ where
Ok(fut) => { Ok(fut) => {
let expected_peer_id = peer_id.clone(); let expected_peer_id = peer_id.clone();
let fut = fut let fut = fut
.map_err(RawSwarmReachError::Transport) .map_err(|err| RawSwarmReachError::Transport(TransportError::Other(err)))
.and_then(move |(actual_peer_id, muxer)| { .and_then(move |(actual_peer_id, muxer)| {
if actual_peer_id == expected_peer_id { if actual_peer_id == expected_peer_id {
Ok((actual_peer_id, muxer)) Ok((actual_peer_id, muxer))
@ -678,10 +718,8 @@ where
}); });
self.active_nodes.add_reach_attempt(fut, handler) self.active_nodes.add_reach_attempt(fut, handler)
}, },
Err((_, addr)) => { Err(err) => {
// TODO: better error reporting let fut = future::err(RawSwarmReachError::Transport(err));
let msg = format!("unsupported multiaddr {}", addr);
let fut = future::err(RawSwarmReachError::Transport(IoError::new(IoErrorKind::Other, msg)));
self.active_nodes.add_reach_attempt(fut, handler) self.active_nodes.add_reach_attempt(fut, handler)
}, },
}; };
@ -702,6 +740,7 @@ where
pub fn poll(&mut self) -> Async<RawSwarmEvent<TTrans, TInEvent, TOutEvent, THandler, THandlerErr>> pub fn poll(&mut self) -> Async<RawSwarmEvent<TTrans, TInEvent, TOutEvent, THandler, THandlerErr>>
where where
TTrans: Transport<Output = (PeerId, TMuxer)>, TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::Error: Send + 'static,
TTrans::Dial: Send + 'static, TTrans::Dial: Send + 'static,
TTrans::ListenerUpgrade: Send + 'static, TTrans::ListenerUpgrade: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer: StreamMuxer + Send + Sync + 'static,
@ -829,7 +868,7 @@ impl<THandler> Default for ActionItem<THandler> {
/// > panics will likely happen. /// > panics will likely happen.
fn handle_node_reached<'a, TTrans, TMuxer, TInEvent, TOutEvent, THandler, THandlerErr>( fn handle_node_reached<'a, TTrans, TMuxer, TInEvent, TOutEvent, THandler, THandlerErr>(
reach_attempts: &mut ReachAttempts, reach_attempts: &mut ReachAttempts,
event: CollectionReachEvent<TInEvent, TOutEvent, THandler, RawSwarmReachError, THandlerErr> event: CollectionReachEvent<TInEvent, TOutEvent, THandler, RawSwarmReachError<TTrans::Error>, THandlerErr>
) -> (ActionItem<THandler>, RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>) ) -> (ActionItem<THandler>, RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>)
where where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone, TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
@ -856,8 +895,7 @@ where
return (Default::default(), RawSwarmEvent::IncomingConnectionError { return (Default::default(), RawSwarmEvent::IncomingConnectionError {
listen_addr, listen_addr,
send_back_addr, send_back_addr,
error: IoError::new(IoErrorKind::PermissionDenied, error: IncomingError::DeniedLowerPriority,
"refused incoming connection".to_string()),
}); });
} }
} }
@ -959,7 +997,7 @@ fn has_dial_prio(local: &PeerId, other: &PeerId) -> bool {
fn handle_reach_error<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>( fn handle_reach_error<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>(
reach_attempts: &mut ReachAttempts, reach_attempts: &mut ReachAttempts,
reach_id: ReachAttemptId, reach_id: ReachAttemptId,
error: RawSwarmReachError, error: RawSwarmReachError<TTrans::Error>,
handler: THandler, handler: THandler,
) -> (ActionItem<THandler>, RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>) ) -> (ActionItem<THandler>, RawSwarmEvent<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>)
where TTrans: Transport where TTrans: Transport
@ -1017,6 +1055,7 @@ where TTrans: Transport
}); });
} }
ConnectedPoint::Listener { listen_addr, send_back_addr } => { ConnectedPoint::Listener { listen_addr, send_back_addr } => {
let error = IncomingError::Transport(error);
return (Default::default(), RawSwarmEvent::IncomingConnectionError { listen_addr, send_back_addr, error }); return (Default::default(), RawSwarmEvent::IncomingConnectionError { listen_addr, send_back_addr, error });
} }
} }
@ -1041,7 +1080,7 @@ where
Connected(PeerConnected<'a, TInEvent>), Connected(PeerConnected<'a, TInEvent>),
/// We are currently attempting to connect to this peer. /// We are currently attempting to connect to this peer.
PendingConnect(PeerPendingConnect<'a, TInEvent, TOutEvent, THandler, THandlerErr>), PendingConnect(PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>),
/// We are not connected to this peer at all. /// We are not connected to this peer at all.
/// ///
@ -1081,6 +1120,7 @@ impl<'a, TTrans, TMuxer, TInEvent, TOutEvent, THandler, THandlerErr>
Peer<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr> Peer<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>
where where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone, TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Error: Send + 'static,
TTrans::Dial: Send + 'static, TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send, TMuxer::OutboundSubstream: Send,
@ -1102,7 +1142,7 @@ where
/// If a connection is pending, returns the `PeerPendingConnect`. /// If a connection is pending, returns the `PeerPendingConnect`.
#[inline] #[inline]
pub fn as_pending_connect(self) -> Option<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler, THandlerErr>> { pub fn as_pending_connect(self) -> Option<PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>> {
match self { match self {
Peer::PendingConnect(peer) => Some(peer), Peer::PendingConnect(peer) => Some(peer),
_ => None, _ => None,
@ -1122,13 +1162,9 @@ where
/// ///
/// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then /// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then
/// the whole connection is immediately closed. /// the whole connection is immediately closed.
///
/// > **Note**: It is possible that the attempt reaches a node that doesn't have the peer id
/// > that we are expecting, in which case the handler will be used for this "wrong"
/// > node.
#[inline] #[inline]
pub fn or_connect(self, addr: Multiaddr, handler: THandler) pub fn or_connect(self, addr: Multiaddr, handler: THandler)
-> Result<PeerPotentialConnect<'a, TInEvent, TOutEvent, THandler, THandlerErr>, Self> -> PeerPotentialConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>
{ {
self.or_connect_with(move |_| addr, handler) self.or_connect_with(move |_| addr, handler)
} }
@ -1138,40 +1174,40 @@ where
/// ///
/// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then /// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then
/// the whole connection is immediately closed. /// the whole connection is immediately closed.
///
/// > **Note**: It is possible that the attempt reaches a node that doesn't have the peer id
/// > that we are expecting, in which case the handler will be used for this "wrong"
/// > node.
#[inline] #[inline]
pub fn or_connect_with<TFn>(self, addr: TFn, handler: THandler) pub fn or_connect_with<TFn>(self, addr: TFn, handler: THandler)
-> Result<PeerPotentialConnect<'a, TInEvent, TOutEvent, THandler, THandlerErr>, Self> -> PeerPotentialConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>
where where
TFn: FnOnce(&PeerId) -> Multiaddr, TFn: FnOnce(&PeerId) -> Multiaddr,
{ {
match self { match self {
Peer::Connected(peer) => Ok(PeerPotentialConnect::Connected(peer)), Peer::Connected(peer) => PeerPotentialConnect::Connected(peer),
Peer::PendingConnect(peer) => Ok(PeerPotentialConnect::PendingConnect(peer)), Peer::PendingConnect(peer) => PeerPotentialConnect::PendingConnect(peer),
Peer::NotConnected(peer) => { Peer::NotConnected(peer) => {
let addr = addr(&peer.peer_id); let addr = addr(&peer.peer_id);
match peer.connect(addr, handler) { PeerPotentialConnect::PendingConnect(peer.connect(addr, handler))
Ok(peer) => Ok(PeerPotentialConnect::PendingConnect(peer)),
Err(peer) => Err(Peer::NotConnected(peer)),
}
} }
} }
} }
} }
/// Peer we are potentially going to connect to. /// Peer we are potentially going to connect to.
pub enum PeerPotentialConnect<'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a, THandlerErr: 'a> { pub enum PeerPotentialConnect<'a, TTrans: 'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a, THandlerErr: 'a>
where
TTrans: Transport
{
/// We are connected to this peer. /// We are connected to this peer.
Connected(PeerConnected<'a, TInEvent>), Connected(PeerConnected<'a, TInEvent>),
/// We are currently attempting to connect to this peer. /// We are currently attempting to connect to this peer.
PendingConnect(PeerPendingConnect<'a, TInEvent, TOutEvent, THandler, THandlerErr>), PendingConnect(PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>),
} }
impl<'a, TInEvent, TOutEvent, THandler, THandlerErr> PeerPotentialConnect<'a, TInEvent, TOutEvent, THandler, THandlerErr> { impl<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>
PeerPotentialConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>
where
TTrans: Transport
{
/// Closes the connection or the connection attempt. /// Closes the connection or the connection attempt.
/// ///
/// If the connection was active, returns the list of outbound substream openings that were /// If the connection was active, returns the list of outbound substream openings that were
@ -1196,7 +1232,7 @@ impl<'a, TInEvent, TOutEvent, THandler, THandlerErr> PeerPotentialConnect<'a, TI
/// If a connection is pending, returns the `PeerPendingConnect`. /// If a connection is pending, returns the `PeerPendingConnect`.
#[inline] #[inline]
pub fn as_pending_connect(self) -> Option<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler, THandlerErr>> { pub fn as_pending_connect(self) -> Option<PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>> {
match self { match self {
PeerPotentialConnect::PendingConnect(peer) => Some(peer), PeerPotentialConnect::PendingConnect(peer) => Some(peer),
_ => None, _ => None,
@ -1242,12 +1278,18 @@ impl<'a, TInEvent> PeerConnected<'a, TInEvent> {
/// Access to a peer we are attempting to connect to. /// Access to a peer we are attempting to connect to.
#[derive(Debug)] #[derive(Debug)]
pub struct PeerPendingConnect<'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a, THandlerErr: 'a> { pub struct PeerPendingConnect<'a, TTrans: 'a, TInEvent: 'a, TOutEvent: 'a, THandler: 'a, THandlerErr: 'a>
where
TTrans: Transport
{
attempt: OccupiedEntry<'a, PeerId, OutReachAttempt>, attempt: OccupiedEntry<'a, PeerId, OutReachAttempt>,
active_nodes: &'a mut CollectionStream<TInEvent, TOutEvent, THandler, RawSwarmReachError, THandlerErr>, active_nodes: &'a mut CollectionStream<TInEvent, TOutEvent, THandler, RawSwarmReachError<TTrans::Error>, THandlerErr>,
} }
impl<'a, TInEvent, TOutEvent, THandler, THandlerErr> PeerPendingConnect<'a, TInEvent, TOutEvent, THandler, THandlerErr> { impl<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr> PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>
where
TTrans: Transport
{
/// Interrupt this connection attempt. /// Interrupt this connection attempt.
// TODO: consider returning a PeerNotConnected; however that is really pain in terms of // TODO: consider returning a PeerNotConnected; however that is really pain in terms of
// borrows // borrows
@ -1302,6 +1344,7 @@ impl<'a, TTrans, TInEvent, TOutEvent, TMuxer, THandler, THandlerErr>
PeerNotConnected<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr> PeerNotConnected<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>
where where
TTrans: Transport<Output = (PeerId, TMuxer)> + Clone, TTrans: Transport<Output = (PeerId, TMuxer)> + Clone,
TTrans::Error: Send + 'static,
TTrans::Dial: Send + 'static, TTrans::Dial: Send + 'static,
TMuxer: StreamMuxer + Send + Sync + 'static, TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send, TMuxer::OutboundSubstream: Send,
@ -1317,7 +1360,9 @@ where
/// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then /// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then
/// the whole connection is immediately closed. /// the whole connection is immediately closed.
#[inline] #[inline]
pub fn connect(self, addr: Multiaddr, handler: THandler) -> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler, THandlerErr>, Self> { pub fn connect(self, addr: Multiaddr, handler: THandler)
-> PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>
{
self.connect_inner(handler, addr, Vec::new()) self.connect_inner(handler, addr, Vec::new())
} }
@ -1325,13 +1370,13 @@ where
/// ///
/// The multiaddresses passed as parameter will be tried one by one. /// The multiaddresses passed as parameter will be tried one by one.
/// ///
/// If the iterator is empty, TODO: what to do? at the moment we unwrap /// Returns an error if the iterator is empty.
/// ///
/// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then /// If we reach a peer but the `PeerId` doesn't correspond to the one we're expecting, then
/// the whole connection is immediately closed. /// the whole connection is immediately closed.
#[inline] #[inline]
pub fn connect_iter<TIter>(self, addrs: TIter, handler: THandler) pub fn connect_iter<TIter>(self, addrs: TIter, handler: THandler)
-> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler, THandlerErr>, Self> -> Result<PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>, Self>
where where
TIter: IntoIterator<Item = Multiaddr>, TIter: IntoIterator<Item = Multiaddr>,
{ {
@ -1341,15 +1386,15 @@ where
None => return Err(self) None => return Err(self)
}; };
let rest = addrs.collect(); let rest = addrs.collect();
self.connect_inner(handler, first, rest) Ok(self.connect_inner(handler, first, rest))
} }
/// Inner implementation of `connect`. /// Inner implementation of `connect`.
fn connect_inner(self, handler: THandler, first: Multiaddr, rest: Vec<Multiaddr>) fn connect_inner(self, handler: THandler, first: Multiaddr, rest: Vec<Multiaddr>)
-> Result<PeerPendingConnect<'a, TInEvent, TOutEvent, THandler, THandlerErr>, Self> -> PeerPendingConnect<'a, TTrans, TInEvent, TOutEvent, THandler, THandlerErr>
{ {
self.nodes.start_dial_out(self.peer_id.clone(), handler, first, rest); self.nodes.start_dial_out(self.peer_id.clone(), handler, first, rest);
Ok(PeerPendingConnect { PeerPendingConnect {
attempt: match self.nodes.reach_attempts.out_reach_attempts.entry(self.peer_id) { attempt: match self.nodes.reach_attempts.out_reach_attempts.entry(self.peer_id) {
Entry::Occupied(e) => e, Entry::Occupied(e) => e,
Entry::Vacant(_) => { Entry::Vacant(_) => {
@ -1357,6 +1402,6 @@ where
}, },
}, },
active_nodes: &mut self.nodes.active_nodes, active_nodes: &mut self.nodes.active_nodes,
}) }
} }
} }

View File

@ -198,8 +198,7 @@ fn querying_for_pending_peer() {
assert_matches!(peer, Peer::NotConnected(PeerNotConnected{ .. })); assert_matches!(peer, Peer::NotConnected(PeerNotConnected{ .. }));
let addr = "/memory".parse().expect("bad multiaddr"); let addr = "/memory".parse().expect("bad multiaddr");
let pending_peer = peer.as_not_connected().unwrap().connect(addr, Handler::default()); let pending_peer = peer.as_not_connected().unwrap().connect(addr, Handler::default());
assert!(pending_peer.is_ok()); assert_matches!(pending_peer, PeerPendingConnect { .. });
assert_matches!(pending_peer, Ok(PeerPendingConnect { .. } ));
} }
#[test] #[test]
@ -307,8 +306,7 @@ fn known_peer_that_is_unreachable_yields_dial_error() {
assert_matches!(peer, Peer::NotConnected(PeerNotConnected{ .. })); assert_matches!(peer, Peer::NotConnected(PeerNotConnected{ .. }));
let addr = "/memory".parse::<Multiaddr>().expect("bad multiaddr"); let addr = "/memory".parse::<Multiaddr>().expect("bad multiaddr");
let pending_peer = peer.as_not_connected().unwrap().connect(addr, Handler::default()); let pending_peer = peer.as_not_connected().unwrap().connect(addr, Handler::default());
assert!(pending_peer.is_ok()); assert_matches!(pending_peer, PeerPendingConnect { .. });
assert_matches!(pending_peer, Ok(PeerPendingConnect { .. } ));
} }
let mut rt = Runtime::new().unwrap(); let mut rt = Runtime::new().unwrap();
// Drive it forward until we hear back from the node. // Drive it forward until we hear back from the node.
@ -349,7 +347,7 @@ fn yields_node_error_when_there_is_an_error_after_successful_connect() {
let mut handler = Handler::default(); let mut handler = Handler::default();
// Force an error // Force an error
handler.next_states = vec![ HandlerState::Err ]; handler.next_states = vec![ HandlerState::Err ];
peer.as_not_connected().unwrap().connect(addr, handler).expect("can connect unconnected peer"); peer.as_not_connected().unwrap().connect(addr, handler);
} }
// Ensure we run on a single thread // Ensure we run on a single thread
@ -403,7 +401,7 @@ fn yields_node_closed_when_the_node_closes_after_successful_connect() {
let mut handler = Handler::default(); let mut handler = Handler::default();
// Force handler to close // Force handler to close
handler.next_states = vec![ HandlerState::Ready(None) ]; handler.next_states = vec![ HandlerState::Ready(None) ];
peer.as_not_connected().unwrap().connect(addr, handler).expect("can connect unconnected peer"); peer.as_not_connected().unwrap().connect(addr, handler);
} }
// Ensure we run on a single thread // Ensure we run on a single thread

View File

@ -51,6 +51,7 @@ use crate::{
}, },
protocols_handler::{NodeHandlerWrapper, ProtocolsHandler}, protocols_handler::{NodeHandlerWrapper, ProtocolsHandler},
topology::Topology, topology::Topology,
transport::TransportError,
topology::DisconnectReason, topology::DisconnectReason,
}; };
use futures::prelude::*; use futures::prelude::*;
@ -115,6 +116,7 @@ where TBehaviour: NetworkBehaviour<TTopology>,
<TMuxer as StreamMuxer>::OutboundSubstream: Send + 'static, <TMuxer as StreamMuxer>::OutboundSubstream: Send + 'static,
<TMuxer as StreamMuxer>::Substream: Send + 'static, <TMuxer as StreamMuxer>::Substream: Send + 'static,
TTransport: Transport<Output = (PeerId, TMuxer)> + Clone, TTransport: Transport<Output = (PeerId, TMuxer)> + Clone,
TTransport::Error: Send + 'static,
TTransport::Listener: Send + 'static, TTransport::Listener: Send + 'static,
TTransport::ListenerUpgrade: Send + 'static, TTransport::ListenerUpgrade: Send + 'static,
TTransport::Dial: Send + 'static, TTransport::Dial: Send + 'static,
@ -171,7 +173,7 @@ where TBehaviour: NetworkBehaviour<TTopology>,
/// Returns an error if the address is not supported. /// Returns an error if the address is not supported.
/// On success, returns an alternative version of the address. /// On success, returns an alternative version of the address.
#[inline] #[inline]
pub fn listen_on(me: &mut Self, addr: Multiaddr) -> Result<Multiaddr, Multiaddr> { pub fn listen_on(me: &mut Self, addr: Multiaddr) -> Result<Multiaddr, TransportError<TTransport::Error>> {
let result = me.raw_swarm.listen_on(addr); let result = me.raw_swarm.listen_on(addr);
if let Ok(ref addr) = result { if let Ok(ref addr) = result {
me.listened_addrs.push(addr.clone()); me.listened_addrs.push(addr.clone());
@ -183,7 +185,7 @@ where TBehaviour: NetworkBehaviour<TTopology>,
/// ///
/// Returns an error if the address is not supported. /// Returns an error if the address is not supported.
#[inline] #[inline]
pub fn dial_addr(me: &mut Self, addr: Multiaddr) -> Result<(), Multiaddr> { pub fn dial_addr(me: &mut Self, addr: Multiaddr) -> Result<(), TransportError<TTransport::Error>> {
let handler = me.behaviour.new_handler(); let handler = me.behaviour.new_handler();
me.raw_swarm.dial(addr, handler.into_node_handler()) me.raw_swarm.dial(addr, handler.into_node_handler())
} }
@ -232,6 +234,7 @@ where TBehaviour: NetworkBehaviour<TTopology>,
<TMuxer as StreamMuxer>::OutboundSubstream: Send + 'static, <TMuxer as StreamMuxer>::OutboundSubstream: Send + 'static,
<TMuxer as StreamMuxer>::Substream: Send + 'static, <TMuxer as StreamMuxer>::Substream: Send + 'static,
TTransport: Transport<Output = (PeerId, TMuxer)> + Clone, TTransport: Transport<Output = (PeerId, TMuxer)> + Clone,
TTransport::Error: Send + 'static,
TTransport::Listener: Send + 'static, TTransport::Listener: Send + 'static,
TTransport::ListenerUpgrade: Send + 'static, TTransport::ListenerUpgrade: Send + 'static,
TTransport::Dial: Send + 'static, TTransport::Dial: Send + 'static,

View File

@ -28,7 +28,7 @@ use futures::{
stream, stream,
}; };
use std::io; use std::io;
use crate::{Multiaddr, PeerId, Transport}; use crate::{Multiaddr, PeerId, Transport, transport::TransportError};
use crate::tests::dummy_muxer::DummyMuxer; use crate::tests::dummy_muxer::DummyMuxer;
#[derive(Debug, PartialEq, Clone)] #[derive(Debug, PartialEq, Clone)]
@ -68,11 +68,12 @@ impl DummyTransport {
} }
impl Transport for DummyTransport { impl Transport for DummyTransport {
type Output = (PeerId, DummyMuxer); type Output = (PeerId, DummyMuxer);
type Error = io::Error;
type Listener = Box<Stream<Item=(Self::ListenerUpgrade, Multiaddr), Error=io::Error> + Send>; type Listener = Box<Stream<Item=(Self::ListenerUpgrade, Multiaddr), Error=io::Error> + Send>;
type ListenerUpgrade = FutureResult<Self::Output, io::Error>; type ListenerUpgrade = FutureResult<Self::Output, io::Error>;
type Dial = Box<Future<Item = Self::Output, Error = io::Error> + Send>; type Dial = Box<Future<Item = Self::Output, Error = io::Error> + Send>;
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>>
where where
Self: Sized, Self: Sized,
{ {
@ -95,11 +96,11 @@ impl Transport for DummyTransport {
} }
}) })
} }
ListenerState::Error => Err((self, addr2)), ListenerState::Error => Err(TransportError::MultiaddrNotSupported(addr)),
} }
} }
fn dial(self, _addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> fn dial(self, _addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>>
where where
Self: Sized, Self: Sized,
{ {

View File

@ -18,10 +18,11 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::{nodes::raw_swarm::ConnectedPoint, transport::Transport}; use crate::either::EitherError;
use crate::{nodes::raw_swarm::ConnectedPoint, transport::Transport, transport::TransportError};
use futures::{future::Either, prelude::*, try_ready}; use futures::{future::Either, prelude::*, try_ready};
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use std::io; use std::error;
/// See the `Transport::and_then` method. /// See the `Transport::and_then` method.
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
@ -38,22 +39,19 @@ impl<T, C, F, O> Transport for AndThen<T, C>
where where
T: Transport, T: Transport,
C: FnOnce(T::Output, ConnectedPoint) -> F + Clone, C: FnOnce(T::Output, ConnectedPoint) -> F + Clone,
F: IntoFuture<Item = O, Error = io::Error> F: IntoFuture<Item = O>,
F::Error: error::Error,
{ {
type Output = O; type Output = O;
type Error = EitherError<T::Error, F::Error>;
type Listener = AndThenStream<T::Listener, C>; type Listener = AndThenStream<T::Listener, C>;
type ListenerUpgrade = AndThenFuture<T::ListenerUpgrade, C, F::Future>; type ListenerUpgrade = AndThenFuture<T::ListenerUpgrade, C, F::Future>;
type Dial = AndThenFuture<T::Dial, C, F::Future>; type Dial = AndThenFuture<T::Dial, C, F::Future>;
#[inline] #[inline]
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
let (listening_stream, new_addr) = match self.transport.listen_on(addr) { let (listening_stream, new_addr) = self.transport.listen_on(addr)
Ok((l, new_addr)) => (l, new_addr), .map_err(|err| err.map(EitherError::A))?;
Err((transport, addr)) => {
let builder = AndThen { transport, fun: self.fun };
return Err((builder, addr));
}
};
// Try to negotiate the protocol. // Try to negotiate the protocol.
// Note that failing to negotiate a protocol will never produce a future with an error. // Note that failing to negotiate a protocol will never produce a future with an error.
@ -69,14 +67,9 @@ where
} }
#[inline] #[inline]
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
let dialed_fut = match self.transport.dial(addr.clone()) { let dialed_fut = self.transport.dial(addr.clone())
Ok(f) => f, .map_err(|err| err.map(EitherError::A))?;
Err((transport, addr)) => {
let builder = AndThen { transport, fun: self.fun };
return Err((builder, addr));
}
};
let connected_point = ConnectedPoint::Dialer { address: addr }; let connected_point = ConnectedPoint::Dialer { address: addr };
@ -98,20 +91,20 @@ where
/// ///
/// Applies a function to every stream item. /// Applies a function to every stream item.
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub struct AndThenStream<T, F> { stream: T, listen_addr: Multiaddr, fun: F } pub struct AndThenStream<TListener, TMap> { stream: TListener, listen_addr: Multiaddr, fun: TMap }
impl<T, F, A, B, X> Stream for AndThenStream<T, F> impl<TListener, TMap, TTransOut, TMapOut, TListUpgr, TTransErr> Stream for AndThenStream<TListener, TMap>
where where
T: Stream<Item = (X, Multiaddr)>, TListener: Stream<Item = (TListUpgr, Multiaddr), Error = TTransErr>,
X: Future<Item = A>, TListUpgr: Future<Item = TTransOut, Error = TTransErr>,
F: FnOnce(A, ConnectedPoint) -> B + Clone, TMap: FnOnce(TTransOut, ConnectedPoint) -> TMapOut + Clone,
B: IntoFuture<Error = X::Error> TMapOut: IntoFuture
{ {
type Item = (AndThenFuture<X, F, B::Future>, Multiaddr); type Item = (AndThenFuture<TListUpgr, TMap, TMapOut::Future>, Multiaddr);
type Error = T::Error; type Error = EitherError<TTransErr, TMapOut::Error>;
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> { fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
match self.stream.poll()? { match self.stream.poll().map_err(EitherError::A)? {
Async::Ready(Some((future, addr))) => { Async::Ready(Some((future, addr))) => {
let f = self.fun.clone(); let f = self.fun.clone();
let p = ConnectedPoint::Listener { let p = ConnectedPoint::Listener {
@ -134,29 +127,29 @@ where
/// ///
/// Applies a function to the result of the inner future. /// Applies a function to the result of the inner future.
#[derive(Debug)] #[derive(Debug)]
pub struct AndThenFuture<T, F, U> { pub struct AndThenFuture<TFut, TMap, TMapOut> {
inner: Either<T, U>, inner: Either<TFut, TMapOut>,
args: Option<(F, ConnectedPoint)> args: Option<(TMap, ConnectedPoint)>
} }
impl<T, A, F, B> Future for AndThenFuture<T, F, B::Future> impl<TFut, TMap, TMapOut> Future for AndThenFuture<TFut, TMap, TMapOut::Future>
where where
T: Future<Item = A>, TFut: Future,
F: FnOnce(A, ConnectedPoint) -> B, TMap: FnOnce(TFut::Item, ConnectedPoint) -> TMapOut,
B: IntoFuture<Error = T::Error> TMapOut: IntoFuture
{ {
type Item = <B::Future as Future>::Item; type Item = <TMapOut::Future as Future>::Item;
type Error = T::Error; type Error = EitherError<TFut::Error, TMapOut::Error>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
loop { loop {
let future = match self.inner { let future = match self.inner {
Either::A(ref mut future) => { Either::A(ref mut future) => {
let item = try_ready!(future.poll()); let item = try_ready!(future.poll().map_err(EitherError::A));
let (f, a) = self.args.take().expect("AndThenFuture has already finished."); let (f, a) = self.args.take().expect("AndThenFuture has already finished.");
f(item, a).into_future() f(item, a).into_future()
} }
Either::B(ref mut future) => return future.poll() Either::B(ref mut future) => return future.poll().map_err(EitherError::B)
}; };
self.inner = Either::B(future); self.inner = Either::B(future);

View File

@ -18,16 +18,14 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::transport::{Transport, TransportError};
use futures::prelude::*; use futures::prelude::*;
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use std::fmt; use std::{error, fmt, sync::Arc};
use std::io::Error as IoError;
use std::sync::Arc;
use crate::transport::Transport;
/// See the `Transport::boxed` method. /// See the `Transport::boxed` method.
#[inline] #[inline]
pub fn boxed<T>(transport: T) -> Boxed<T::Output> pub fn boxed<T>(transport: T) -> Boxed<T::Output, T::Error>
where where
T: Transport + Clone + Send + Sync + 'static, T: Transport + Clone + Send + Sync + 'static,
T::Dial: Send + 'static, T::Dial: Send + 'static,
@ -39,37 +37,34 @@ where
} }
} }
pub type Dial<O> = Box<Future<Item = O, Error = IoError> + Send>; pub type Dial<O, E> = Box<Future<Item = O, Error = E> + Send>;
pub type Listener<O> = Box<Stream<Item = (ListenerUpgrade<O>, Multiaddr), Error = IoError> + Send>; pub type Listener<O, E> = Box<Stream<Item = (ListenerUpgrade<O, E>, Multiaddr), Error = E> + Send>;
pub type ListenerUpgrade<O> = Box<Future<Item = O, Error = IoError> + Send>; pub type ListenerUpgrade<O, E> = Box<Future<Item = O, Error = E> + Send>;
pub type Incoming<O> = Box<Future<Item = (IncomingUpgrade<O>, Multiaddr), Error = IoError> + Send>;
pub type IncomingUpgrade<O> = Box<Future<Item = O, Error = IoError> + Send>;
trait Abstract<O> { trait Abstract<O, E> {
fn listen_on(&self, addr: Multiaddr) -> Result<(Listener<O>, Multiaddr), Multiaddr>; fn listen_on(&self, addr: Multiaddr) -> Result<(Listener<O, E>, Multiaddr), TransportError<E>>;
fn dial(&self, addr: Multiaddr) -> Result<Dial<O>, Multiaddr>; fn dial(&self, addr: Multiaddr) -> Result<Dial<O, E>, TransportError<E>>;
fn nat_traversal(&self, server: &Multiaddr, observed: &Multiaddr) -> Option<Multiaddr>; fn nat_traversal(&self, server: &Multiaddr, observed: &Multiaddr) -> Option<Multiaddr>;
} }
impl<T, O> Abstract<O> for T impl<T, O, E> Abstract<O, E> for T
where where
T: Transport<Output = O> + Clone + 'static, T: Transport<Output = O, Error = E> + Clone + 'static,
E: error::Error,
T::Dial: Send + 'static, T::Dial: Send + 'static,
T::Listener: Send + 'static, T::Listener: Send + 'static,
T::ListenerUpgrade: Send + 'static, T::ListenerUpgrade: Send + 'static,
{ {
fn listen_on(&self, addr: Multiaddr) -> Result<(Listener<O>, Multiaddr), Multiaddr> { fn listen_on(&self, addr: Multiaddr) -> Result<(Listener<O, E>, Multiaddr), TransportError<E>> {
let (listener, new_addr) = let (listener, new_addr) = Transport::listen_on(self.clone(), addr)?;
Transport::listen_on(self.clone(), addr).map_err(|(_, addr)| addr)?;
let fut = listener.map(|(upgrade, addr)| { let fut = listener.map(|(upgrade, addr)| {
(Box::new(upgrade) as ListenerUpgrade<O>, addr) (Box::new(upgrade) as ListenerUpgrade<O, E>, addr)
}); });
Ok((Box::new(fut) as Box<_>, new_addr)) Ok((Box::new(fut) as Box<_>, new_addr))
} }
fn dial(&self, addr: Multiaddr) -> Result<Dial<O>, Multiaddr> { fn dial(&self, addr: Multiaddr) -> Result<Dial<O, E>, TransportError<E>> {
let fut = Transport::dial(self.clone(), addr) let fut = Transport::dial(self.clone(), addr)?;
.map_err(|(_, addr)| addr)?;
Ok(Box::new(fut) as Box<_>) Ok(Box::new(fut) as Box<_>)
} }
@ -80,17 +75,17 @@ where
} }
/// See the `Transport::boxed` method. /// See the `Transport::boxed` method.
pub struct Boxed<O> { pub struct Boxed<O, E> {
inner: Arc<Abstract<O> + Send + Sync>, inner: Arc<Abstract<O, E> + Send + Sync>,
} }
impl<O> fmt::Debug for Boxed<O> { impl<O, E> fmt::Debug for Boxed<O, E> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "BoxedTransport") write!(f, "BoxedTransport")
} }
} }
impl<O> Clone for Boxed<O> { impl<O, E> Clone for Boxed<O, E> {
#[inline] #[inline]
fn clone(&self) -> Self { fn clone(&self) -> Self {
Boxed { Boxed {
@ -99,26 +94,23 @@ impl<O> Clone for Boxed<O> {
} }
} }
impl<O> Transport for Boxed<O> { impl<O, E> Transport for Boxed<O, E>
where E: error::Error,
{
type Output = O; type Output = O;
type Listener = Listener<O>; type Error = E;
type ListenerUpgrade = ListenerUpgrade<O>; type Listener = Listener<O, E>;
type Dial = Dial<O>; type ListenerUpgrade = ListenerUpgrade<O, E>;
type Dial = Dial<O, E>;
#[inline] #[inline]
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
match self.inner.listen_on(addr) { self.inner.listen_on(addr)
Ok(listen) => Ok(listen),
Err(addr) => Err((self, addr)),
}
} }
#[inline] #[inline]
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
match self.inner.dial(addr) { self.inner.dial(addr)
Ok(dial) => Ok(dial),
Err(addr) => Err((self, addr)),
}
} }
#[inline] #[inline]

View File

@ -18,9 +18,9 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::either::{EitherListenStream, EitherOutput, EitherFuture}; use crate::either::{EitherListenStream, EitherOutput, EitherError, EitherFuture};
use crate::transport::{Transport, TransportError};
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use crate::transport::Transport;
/// Struct returned by `or_transport()`. /// Struct returned by `or_transport()`.
#[derive(Debug, Copy, Clone)] #[derive(Debug, Copy, Clone)]
@ -34,36 +34,45 @@ impl<A, B> OrTransport<A, B> {
impl<A, B> Transport for OrTransport<A, B> impl<A, B> Transport for OrTransport<A, B>
where where
A: Transport,
B: Transport, B: Transport,
A: Transport,
{ {
type Output = EitherOutput<A::Output, B::Output>; type Output = EitherOutput<A::Output, B::Output>;
type Error = EitherError<A::Error, B::Error>;
type Listener = EitherListenStream<A::Listener, B::Listener>; type Listener = EitherListenStream<A::Listener, B::Listener>;
type ListenerUpgrade = EitherFuture<A::ListenerUpgrade, B::ListenerUpgrade>; type ListenerUpgrade = EitherFuture<A::ListenerUpgrade, B::ListenerUpgrade>;
type Dial = EitherFuture<A::Dial, B::Dial>; type Dial = EitherFuture<A::Dial, B::Dial>;
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
let (first, addr) = match self.0.listen_on(addr) { let addr = match self.0.listen_on(addr) {
Ok((connec, addr)) => return Ok((EitherListenStream::First(connec), addr)), Ok((connec, addr)) => return Ok((EitherListenStream::First(connec), addr)),
Err(err) => err, Err(TransportError::MultiaddrNotSupported(addr)) => addr,
Err(TransportError::Other(err)) => return Err(TransportError::Other(EitherError::A(err))),
}; };
match self.1.listen_on(addr) { let addr = match self.1.listen_on(addr) {
Ok((connec, addr)) => Ok((EitherListenStream::Second(connec), addr)), Ok((connec, addr)) => return Ok((EitherListenStream::Second(connec), addr)),
Err((second, addr)) => Err((OrTransport(first, second), addr)), Err(TransportError::MultiaddrNotSupported(addr)) => addr,
} Err(TransportError::Other(err)) => return Err(TransportError::Other(EitherError::B(err))),
};
Err(TransportError::MultiaddrNotSupported(addr))
} }
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
let (first, addr) = match self.0.dial(addr) { let addr = match self.0.dial(addr) {
Ok(connec) => return Ok(EitherFuture::First(connec)), Ok(connec) => return Ok(EitherFuture::First(connec)),
Err(err) => err, Err(TransportError::MultiaddrNotSupported(addr)) => addr,
Err(TransportError::Other(err)) => return Err(TransportError::Other(EitherError::A(err))),
}; };
match self.1.dial(addr) { let addr = match self.1.dial(addr) {
Ok(connec) => Ok(EitherFuture::Second(connec)), Ok(connec) => return Ok(EitherFuture::Second(connec)),
Err((second, addr)) => Err((OrTransport(first, second), addr)), Err(TransportError::MultiaddrNotSupported(addr)) => addr,
} Err(TransportError::Other(err)) => return Err(TransportError::Other(EitherError::B(err))),
};
Err(TransportError::MultiaddrNotSupported(addr))
} }
#[inline] #[inline]

View File

@ -18,7 +18,7 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::{nodes::raw_swarm::ConnectedPoint, transport::Transport}; use crate::{nodes::raw_swarm::ConnectedPoint, transport::Transport, transport::TransportError};
use futures::{prelude::*, try_ready}; use futures::{prelude::*, try_ready};
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
@ -39,35 +39,28 @@ where
F: FnOnce(T::Output, ConnectedPoint) -> D + Clone F: FnOnce(T::Output, ConnectedPoint) -> D + Clone
{ {
type Output = D; type Output = D;
type Error = T::Error;
type Listener = MapStream<T::Listener, F>; type Listener = MapStream<T::Listener, F>;
type ListenerUpgrade = MapFuture<T::ListenerUpgrade, F>; type ListenerUpgrade = MapFuture<T::ListenerUpgrade, F>;
type Dial = MapFuture<T::Dial, F>; type Dial = MapFuture<T::Dial, F>;
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
match self.transport.listen_on(addr) { let (stream, listen_addr) = self.transport.listen_on(addr)?;
Ok((stream, listen_addr)) => { let stream = MapStream {
let stream = MapStream { stream,
stream, listen_addr: listen_addr.clone(),
listen_addr: listen_addr.clone(), fun: self.fun
fun: self.fun };
}; Ok((stream, listen_addr))
Ok((stream, listen_addr))
}
Err((transport, addr)) => Err((Map { transport, fun: self.fun }, addr)),
}
} }
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
match self.transport.dial(addr.clone()) { let future = self.transport.dial(addr.clone())?;
Ok(future) => { let p = ConnectedPoint::Dialer { address: addr };
let p = ConnectedPoint::Dialer { address: addr }; Ok(MapFuture {
Ok(MapFuture { inner: future,
inner: future, args: Some((self.fun, p))
args: Some((self.fun, p)) })
})
}
Err((transport, addr)) => Err((Map { transport, fun: self.fun }, addr)),
}
} }
#[inline] #[inline]

View File

@ -18,10 +18,10 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::transport::Transport; use crate::transport::{Transport, TransportError};
use futures::{prelude::*, try_ready}; use futures::prelude::*;
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use std::io::Error as IoError; use std::error;
/// See `Transport::map_err`. /// See `Transport::map_err`.
#[derive(Debug, Copy, Clone)] #[derive(Debug, Copy, Clone)]
@ -38,17 +38,19 @@ impl<T, F> MapErr<T, F> {
} }
} }
impl<T, F> Transport for MapErr<T, F> impl<T, F, TErr> Transport for MapErr<T, F>
where where
T: Transport, T: Transport,
F: FnOnce(IoError) -> IoError + Clone, F: FnOnce(T::Error) -> TErr + Clone,
TErr: error::Error,
{ {
type Output = T::Output; type Output = T::Output;
type Error = TErr;
type Listener = MapErrListener<T, F>; type Listener = MapErrListener<T, F>;
type ListenerUpgrade = MapErrListenerUpgrade<T, F>; type ListenerUpgrade = MapErrListenerUpgrade<T, F>;
type Dial = MapErrDial<T, F>; type Dial = MapErrDial<T, F>;
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
let map = self.map; let map = self.map;
match self.transport.listen_on(addr) { match self.transport.listen_on(addr) {
@ -56,16 +58,16 @@ where
let stream = MapErrListener { inner: stream, map }; let stream = MapErrListener { inner: stream, map };
Ok((stream, listen_addr)) Ok((stream, listen_addr))
} }
Err((transport, addr)) => Err((MapErr { transport, map }, addr)), Err(err) => Err(err.map(move |err| map(err))),
} }
} }
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
let map = self.map; let map = self.map;
match self.transport.dial(addr) { match self.transport.dial(addr) {
Ok(future) => Ok(MapErrDial { inner: future, map: Some(map) }), Ok(future) => Ok(MapErrDial { inner: future, map: Some(map) }),
Err((transport, addr)) => Err((MapErr { transport, map }, addr)), Err(err) => Err(err.map(move |err| map(err))),
} }
} }
@ -82,19 +84,22 @@ where T: Transport {
map: F, map: F,
} }
impl<T, F> Stream for MapErrListener<T, F> impl<T, F, TErr> Stream for MapErrListener<T, F>
where T: Transport, where T: Transport,
F: FnOnce(IoError) -> IoError + Clone, F: FnOnce(T::Error) -> TErr + Clone,
TErr: error::Error,
{ {
type Item = (MapErrListenerUpgrade<T, F>, Multiaddr); type Item = (MapErrListenerUpgrade<T, F>, Multiaddr);
type Error = IoError; type Error = TErr;
#[inline] #[inline]
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> { fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
match try_ready!(self.inner.poll()) { match self.inner.poll() {
Some((value, addr)) => Ok(Async::Ready( Ok(Async::Ready(Some((value, addr)))) => Ok(Async::Ready(
Some((MapErrListenerUpgrade { inner: value, map: Some(self.map.clone()) }, addr)))), Some((MapErrListenerUpgrade { inner: value, map: Some(self.map.clone()) }, addr)))),
None => Ok(Async::Ready(None)) Ok(Async::Ready(None)) => Ok(Async::Ready(None)),
Ok(Async::NotReady) => Ok(Async::NotReady),
Err(err) => Err((self.map.clone())(err)),
} }
} }
} }
@ -106,12 +111,12 @@ where T: Transport {
map: Option<F>, map: Option<F>,
} }
impl<T, F> Future for MapErrListenerUpgrade<T, F> impl<T, F, TErr> Future for MapErrListenerUpgrade<T, F>
where T: Transport, where T: Transport,
F: FnOnce(IoError) -> IoError, F: FnOnce(T::Error) -> TErr,
{ {
type Item = T::Output; type Item = T::Output;
type Error = IoError; type Error = TErr;
#[inline] #[inline]
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
@ -130,19 +135,18 @@ where T: Transport,
/// Dialing future for `MapErr`. /// Dialing future for `MapErr`.
pub struct MapErrDial<T, F> pub struct MapErrDial<T, F>
where T: Transport, where T: Transport
F: FnOnce(IoError) -> IoError,
{ {
inner: T::Dial, inner: T::Dial,
map: Option<F>, map: Option<F>,
} }
impl<T, F> Future for MapErrDial<T, F> impl<T, F, TErr> Future for MapErrDial<T, F>
where T: Transport, where T: Transport,
F: FnOnce(IoError) -> IoError, F: FnOnce(T::Error) -> TErr,
{ {
type Item = T::Output; type Item = T::Output;
type Error = IoError; type Error = TErr;
#[inline] #[inline]
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {

View File

@ -1,98 +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 crate::transport::Transport;
use futures::prelude::*;
use multiaddr::Multiaddr;
use std::io;
/// See `Transport::map_err_dial`.
#[derive(Debug, Copy, Clone)]
pub struct MapErrDial<T, F> { transport: T, fun: F }
impl<T, F> MapErrDial<T, F> {
/// Internal function that builds a `MapErrDial`.
#[inline]
pub(crate) fn new(transport: T, fun: F) -> MapErrDial<T, F> {
MapErrDial { transport, fun }
}
}
impl<T, F> Transport for MapErrDial<T, F>
where
T: Transport,
F: FnOnce(io::Error, Multiaddr) -> io::Error
{
type Output = T::Output;
type Listener = T::Listener;
type ListenerUpgrade = T::ListenerUpgrade;
type Dial = MapErrFuture<T::Dial, F>;
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> {
let fun = self.fun;
self.transport.listen_on(addr)
.map_err(move |(transport, addr)| (MapErrDial { transport, fun }, addr))
}
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> {
match self.transport.dial(addr.clone()) {
Ok(future) => Ok(MapErrFuture {
inner: future,
args: Some((self.fun, addr))
}),
Err((transport, addr)) => Err((MapErrDial { transport, fun: self.fun }, addr))
}
}
#[inline]
fn nat_traversal(&self, server: &Multiaddr, observed: &Multiaddr) -> Option<Multiaddr> {
self.transport.nat_traversal(server, observed)
}
}
/// Custom `Future` to avoid boxing.
///
/// Applies a function to the inner future's error type.
#[derive(Debug, Clone)]
pub struct MapErrFuture<T, F> {
inner: T,
args: Option<(F, Multiaddr)>,
}
impl<T, E, F, A> Future for MapErrFuture<T, F>
where
T: Future<Error = E>,
F: FnOnce(E, Multiaddr) -> A
{
type Item = T::Item;
type Error = A;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
match self.inner.poll() {
Ok(Async::NotReady) => Ok(Async::NotReady),
Ok(Async::Ready(x)) => Ok(Async::Ready(x)),
Err(e) => {
let (f, a) = self.args.take().expect("MapErrFuture has already finished.");
Err(f(e, a))
}
}
}
}

View File

@ -18,13 +18,13 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use crate::{Transport, transport::TransportError};
use bytes::{Bytes, IntoBuf}; use bytes::{Bytes, IntoBuf};
use futures::{future::{self, FutureResult}, prelude::*, stream, sync::mpsc}; use futures::{future::{self, FutureResult}, prelude::*, stream, sync::mpsc};
use multiaddr::{Protocol, Multiaddr}; use multiaddr::{Protocol, Multiaddr};
use parking_lot::Mutex; use parking_lot::Mutex;
use rw_stream_sink::RwStreamSink; use rw_stream_sink::RwStreamSink;
use std::{io, sync::Arc}; use std::{error, fmt, sync::Arc};
use crate::Transport;
/// Builds a new pair of `Transport`s. The dialer can reach the listener by dialing `/memory`. /// Builds a new pair of `Transport`s. The dialer can reach the listener by dialing `/memory`.
#[inline] #[inline]
@ -52,17 +52,18 @@ impl<T> Clone for Dialer<T> {
impl<T: IntoBuf + Send + 'static> Transport for Dialer<T> { impl<T: IntoBuf + Send + 'static> Transport for Dialer<T> {
type Output = Channel<T>; type Output = Channel<T>;
type Listener = Box<Stream<Item=(Self::ListenerUpgrade, Multiaddr), Error=io::Error> + Send>; type Error = MemoryTransportError;
type ListenerUpgrade = FutureResult<Self::Output, io::Error>; type Listener = Box<Stream<Item=(Self::ListenerUpgrade, Multiaddr), Error=MemoryTransportError> + Send>;
type Dial = Box<Future<Item=Self::Output, Error=io::Error> + Send>; type ListenerUpgrade = FutureResult<Self::Output, MemoryTransportError>;
type Dial = Box<Future<Item=Self::Output, Error=MemoryTransportError> + Send>;
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
Err((self, addr)) Err(TransportError::MultiaddrNotSupported(addr))
} }
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
if !is_memory_addr(&addr) { if !is_memory_addr(&addr) {
return Err((self, addr)) return Err(TransportError::MultiaddrNotSupported(addr))
} }
let (a_tx, a_rx) = mpsc::unbounded(); let (a_tx, a_rx) = mpsc::unbounded();
let (b_tx, b_rx) = mpsc::unbounded(); let (b_tx, b_rx) = mpsc::unbounded();
@ -70,7 +71,7 @@ impl<T: IntoBuf + Send + 'static> Transport for Dialer<T> {
let b = Chan { incoming: b_rx, outgoing: a_tx }; let b = Chan { incoming: b_rx, outgoing: a_tx };
let future = self.0.send(b) let future = self.0.send(b)
.map(move |_| a.into()) .map(move |_| a.into())
.map_err(|_| io::ErrorKind::ConnectionRefused.into()); .map_err(|_| MemoryTransportError::RemoteClosed);
Ok(Box::new(future)) Ok(Box::new(future))
} }
@ -83,6 +84,24 @@ impl<T: IntoBuf + Send + 'static> Transport for Dialer<T> {
} }
} }
/// Error that can be produced from the `MemoryTransport`.
#[derive(Debug, Copy, Clone)]
pub enum MemoryTransportError {
/// The other side of the transport has been closed earlier.
RemoteClosed,
}
impl fmt::Display for MemoryTransportError {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match *self {
MemoryTransportError::RemoteClosed =>
write!(f, "The other side of the memory transport has been closed."),
}
}
}
impl error::Error for MemoryTransportError {}
/// Receiving end of the memory transport. /// Receiving end of the memory transport.
pub struct Listener<T = Bytes>(Arc<Mutex<mpsc::UnboundedReceiver<Chan<T>>>>); pub struct Listener<T = Bytes>(Arc<Mutex<mpsc::UnboundedReceiver<Chan<T>>>>);
@ -94,13 +113,14 @@ impl<T> Clone for Listener<T> {
impl<T: IntoBuf + Send + 'static> Transport for Listener<T> { impl<T: IntoBuf + Send + 'static> Transport for Listener<T> {
type Output = Channel<T>; type Output = Channel<T>;
type Listener = Box<Stream<Item=(Self::ListenerUpgrade, Multiaddr), Error=io::Error> + Send>; type Error = MemoryTransportError;
type ListenerUpgrade = FutureResult<Self::Output, io::Error>; type Listener = Box<Stream<Item=(Self::ListenerUpgrade, Multiaddr), Error=MemoryTransportError> + Send>;
type Dial = Box<Future<Item=Self::Output, Error=io::Error> + Send>; type ListenerUpgrade = FutureResult<Self::Output, MemoryTransportError>;
type Dial = Box<Future<Item=Self::Output, Error=MemoryTransportError> + Send>;
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
if !is_memory_addr(&addr) { if !is_memory_addr(&addr) {
return Err((self, addr)) return Err(TransportError::MultiaddrNotSupported(addr));
} }
let addr2 = addr.clone(); let addr2 = addr.clone();
let receiver = self.0.clone(); let receiver = self.0.clone();
@ -113,8 +133,8 @@ impl<T: IntoBuf + Send + 'static> Transport for Listener<T> {
} }
#[inline] #[inline]
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
Err((self, addr)) Err(TransportError::MultiaddrNotSupported(addr))
} }
#[inline] #[inline]
@ -154,31 +174,31 @@ pub struct Chan<T = Bytes> {
impl<T> Stream for Chan<T> { impl<T> Stream for Chan<T> {
type Item = T; type Item = T;
type Error = io::Error; type Error = MemoryTransportError;
#[inline] #[inline]
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> { fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
self.incoming.poll().map_err(|()| io::ErrorKind::ConnectionReset.into()) self.incoming.poll().map_err(|()| MemoryTransportError::RemoteClosed)
} }
} }
impl<T> Sink for Chan<T> { impl<T> Sink for Chan<T> {
type SinkItem = T; type SinkItem = T;
type SinkError = io::Error; type SinkError = MemoryTransportError;
#[inline] #[inline]
fn start_send(&mut self, item: Self::SinkItem) -> StartSend<Self::SinkItem, Self::SinkError> { fn start_send(&mut self, item: Self::SinkItem) -> StartSend<Self::SinkItem, Self::SinkError> {
self.outgoing.start_send(item).map_err(|_| io::ErrorKind::ConnectionReset.into()) self.outgoing.start_send(item).map_err(|_| MemoryTransportError::RemoteClosed)
} }
#[inline] #[inline]
fn poll_complete(&mut self) -> Poll<(), Self::SinkError> { fn poll_complete(&mut self) -> Poll<(), Self::SinkError> {
self.outgoing.poll_complete().map_err(|_| io::ErrorKind::ConnectionReset.into()) self.outgoing.poll_complete().map_err(|_| MemoryTransportError::RemoteClosed)
} }
#[inline] #[inline]
fn close(&mut self) -> Poll<(), Self::SinkError> { fn close(&mut self) -> Poll<(), Self::SinkError> {
self.outgoing.close().map_err(|_| io::ErrorKind::ConnectionReset.into()) self.outgoing.close().map_err(|_| MemoryTransportError::RemoteClosed)
} }
} }

View File

@ -29,7 +29,7 @@
use crate::{InboundUpgrade, OutboundUpgrade, nodes::raw_swarm::ConnectedPoint}; use crate::{InboundUpgrade, OutboundUpgrade, nodes::raw_swarm::ConnectedPoint};
use futures::prelude::*; use futures::prelude::*;
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use std::io::Error as IoError; use std::{error, fmt};
use std::time::Duration; use std::time::Duration;
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
@ -38,7 +38,6 @@ pub mod boxed;
pub mod choice; pub mod choice;
pub mod map; pub mod map;
pub mod map_err; pub mod map_err;
pub mod map_err_dial;
pub mod memory; pub mod memory;
pub mod timeout; pub mod timeout;
pub mod upgrade; pub mod upgrade;
@ -60,39 +59,43 @@ pub use self::upgrade::Upgrade;
pub trait Transport { pub trait Transport {
/// The raw connection to a peer. /// The raw connection to a peer.
type Output; type Output;
/// Error that can happen when dialing or listening.
type Error: error::Error;
/// The listener produces incoming connections. /// The listener produces incoming connections.
/// ///
/// An item should be produced whenever a connection is received at the lowest level of the /// An item should be produced whenever a connection is received at the lowest level of the
/// transport stack. The item is a `Future` that is signalled once some pre-processing has /// transport stack. The item is a `Future` that is signalled once some pre-processing has
/// taken place, and that connection has been upgraded to the wanted protocols. /// taken place, and that connection has been upgraded to the wanted protocols.
type Listener: Stream<Item = (Self::ListenerUpgrade, Multiaddr), Error = IoError>; type Listener: Stream<Item = (Self::ListenerUpgrade, Multiaddr), Error = Self::Error>;
/// After a connection has been received, we may need to do some asynchronous pre-processing /// After a connection has been received, we may need to do some asynchronous pre-processing
/// on it (e.g. an intermediary protocol negotiation). While this pre-processing takes place, we /// on it (e.g. an intermediary protocol negotiation). While this pre-processing takes place,
/// want to be able to continue polling on the listener. /// we want to be able to continue polling on the listener.
type ListenerUpgrade: Future<Item = Self::Output, Error = IoError>; type ListenerUpgrade: Future<Item = Self::Output, Error = Self::Error>;
/// A future which indicates that we are currently dialing to a peer. /// A future which indicates that we are currently dialing to a peer.
type Dial: Future<Item = Self::Output, Error = IoError>; type Dial: Future<Item = Self::Output, Error = Self::Error>;
/// Listen on the given multiaddr. Returns a stream of incoming connections, plus a modified /// Listen on the given multiaddr. Returns a stream of incoming connections, plus a modified
/// version of the `Multiaddr`. This new `Multiaddr` is the one that that should be advertised /// version of the `Multiaddr`. This new `Multiaddr` is the one that that should be advertised
/// to other nodes, instead of the one passed as parameter. /// to other nodes, instead of the one passed as parameter.
/// ///
/// Returns the address back if it isn't supported.
///
/// > **Note**: The reason why we need to change the `Multiaddr` on success is to handle /// > **Note**: The reason why we need to change the `Multiaddr` on success is to handle
/// > situations such as turning `/ip4/127.0.0.1/tcp/0` into /// > situations such as turning `/ip4/127.0.0.1/tcp/0` into
/// > `/ip4/127.0.0.1/tcp/<actual port>`. /// > `/ip4/127.0.0.1/tcp/<actual port>`.
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>>
where where
Self: Sized; Self: Sized;
/// Dial the given multi-addr. /// Dial the given multi-addr.
/// ///
/// Returns either a future which may resolve to a connection, or gives back the multiaddress. /// Returns either a future which may resolve to a connection.
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> ///
/// If `MultiaddrNotSupported` is returned, then caller can try another implementation of
/// `Transport` if there is any. If instead an error is returned, then we assume that there is
/// no point in trying another `Transport`.
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>>
where where
Self: Sized; Self: Sized;
@ -114,7 +117,7 @@ pub trait Transport {
/// Turns this `Transport` into an abstract boxed transport. /// Turns this `Transport` into an abstract boxed transport.
#[inline] #[inline]
fn boxed(self) -> boxed::Boxed<Self::Output> fn boxed(self) -> boxed::Boxed<Self::Output, Self::Error>
where Self: Sized + Clone + Send + Sync + 'static, where Self: Sized + Clone + Send + Sync + 'static,
Self::Dial: Send + 'static, Self::Dial: Send + 'static,
Self::Listener: Send + 'static, Self::Listener: Send + 'static,
@ -135,26 +138,14 @@ pub trait Transport {
/// Applies a function on the errors generated by the futures of the `Transport`. /// Applies a function on the errors generated by the futures of the `Transport`.
#[inline] #[inline]
fn map_err<F>(self, map_err: F) -> map_err::MapErr<Self, F> fn map_err<F, TNewErr>(self, map_err: F) -> map_err::MapErr<Self, F>
where where
Self: Sized, Self: Sized,
F: FnOnce(IoError) -> IoError + Clone F: FnOnce(Self::Error) -> TNewErr + Clone
{ {
map_err::MapErr::new(self, map_err) map_err::MapErr::new(self, map_err)
} }
/// Applies a function on the errors generated by the futures of the `Transport` when dialing.
///
/// Contrary to `map_err`, this gives access to the `Multiaddr` that we tried to dial.
#[inline]
fn map_err_dial<F>(self, map_err: F) -> map_err_dial::MapErrDial<Self, F>
where
Self: Sized,
F: FnOnce(IoError, Multiaddr) -> IoError
{
map_err_dial::MapErrDial::new(self, map_err)
}
/// Builds a new struct that implements `Transport` that contains both `self` and `other`. /// Builds a new struct that implements `Transport` that contains both `self` and `other`.
/// ///
/// The returned object will redirect its calls to `self`, except that if `listen_on` or `dial` /// The returned object will redirect its calls to `self`, except that if `listen_on` or `dial`
@ -193,7 +184,7 @@ pub trait Transport {
where where
Self: Sized, Self: Sized,
C: FnOnce(Self::Output, ConnectedPoint) -> F + Clone, C: FnOnce(Self::Output, ConnectedPoint) -> F + Clone,
F: IntoFuture<Item = O, Error = IoError> F: IntoFuture<Item = O>
{ {
and_then::AndThen::new(self, upgrade) and_then::AndThen::new(self, upgrade)
} }
@ -228,3 +219,48 @@ pub trait Transport {
timeout::TransportTimeout::with_ingoing_timeout(self, timeout) timeout::TransportTimeout::with_ingoing_timeout(self, timeout)
} }
} }
/// Error that can happen when dialing or listening.
#[derive(Debug, Clone)]
pub enum TransportError<TErr> {
/// The `Multiaddr` passed as parameter is not supported.
///
/// Contains back the same address.
MultiaddrNotSupported(Multiaddr),
/// Any other error that the `Transport` may produce.
Other(TErr),
}
impl<TErr> TransportError<TErr> {
/// Applies a map to the `Other` variant.
#[inline]
pub fn map<TNewErr>(self, map: impl FnOnce(TErr) -> TNewErr) -> TransportError<TNewErr> {
match self {
TransportError::MultiaddrNotSupported(addr) => TransportError::MultiaddrNotSupported(addr),
TransportError::Other(err) => TransportError::Other(map(err)),
}
}
}
impl<TErr> fmt::Display for TransportError<TErr>
where TErr: fmt::Display,
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
TransportError::MultiaddrNotSupported(addr) => write!(f, "Multiaddr is not supported: {}", addr),
TransportError::Other(err) => write!(f, "{}", err),
}
}
}
impl<TErr> error::Error for TransportError<TErr>
where TErr: error::Error + 'static,
{
fn source(&self) -> Option<&(dyn error::Error + 'static)> {
match self {
TransportError::MultiaddrNotSupported(_) => None,
TransportError::Other(err) => Some(err),
}
}
}

View File

@ -23,11 +23,10 @@
//! The timeout includes the upgrading process. //! The timeout includes the upgrading process.
// TODO: add example // TODO: add example
use crate::{Multiaddr, Transport}; use crate::{Multiaddr, Transport, transport::TransportError};
use futures::{try_ready, Async, Future, Poll, Stream}; use futures::{try_ready, Async, Future, Poll, Stream};
use log::debug; use log::debug;
use std::io::{Error as IoError, ErrorKind as IoErrorKind}; use std::{error, fmt, time::Duration};
use std::time::Duration;
use tokio_timer::Timeout; use tokio_timer::Timeout;
use tokio_timer::timeout::Error as TimeoutError; use tokio_timer::timeout::Error as TimeoutError;
@ -77,49 +76,32 @@ impl<InnerTrans> TransportTimeout<InnerTrans> {
impl<InnerTrans> Transport for TransportTimeout<InnerTrans> impl<InnerTrans> Transport for TransportTimeout<InnerTrans>
where where
InnerTrans: Transport, InnerTrans: Transport,
InnerTrans::Error: 'static,
{ {
type Output = InnerTrans::Output; type Output = InnerTrans::Output;
type Error = TransportTimeoutError<InnerTrans::Error>;
type Listener = TimeoutListener<InnerTrans::Listener>; type Listener = TimeoutListener<InnerTrans::Listener>;
type ListenerUpgrade = TokioTimerMapErr<Timeout<InnerTrans::ListenerUpgrade>>; type ListenerUpgrade = TokioTimerMapErr<Timeout<InnerTrans::ListenerUpgrade>>;
type Dial = TokioTimerMapErr<Timeout<InnerTrans::Dial>>; type Dial = TokioTimerMapErr<Timeout<InnerTrans::Dial>>;
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
match self.inner.listen_on(addr) { let (listener, addr) = self.inner.listen_on(addr)
Ok((listener, addr)) => { .map_err(|err| err.map(TransportTimeoutError::Other))?;
let listener = TimeoutListener {
inner: listener,
timeout: self.incoming_timeout,
};
Ok((listener, addr)) let listener = TimeoutListener {
} inner: listener,
Err((inner, addr)) => { timeout: self.incoming_timeout,
let transport = TransportTimeout { };
inner,
outgoing_timeout: self.outgoing_timeout,
incoming_timeout: self.incoming_timeout,
};
Err((transport, addr)) Ok((listener, addr))
}
}
} }
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
match self.inner.dial(addr) { let dial = self.inner.dial(addr)
Ok(dial) => Ok(TokioTimerMapErr { .map_err(|err| err.map(TransportTimeoutError::Other))?;
inner: Timeout::new(dial, self.outgoing_timeout), Ok(TokioTimerMapErr {
}), inner: Timeout::new(dial, self.outgoing_timeout),
Err((inner, addr)) => { })
let transport = TransportTimeout {
inner,
outgoing_timeout: self.outgoing_timeout,
incoming_timeout: self.incoming_timeout,
};
Err((transport, addr))
}
}
} }
#[inline] #[inline]
@ -140,10 +122,10 @@ where
InnerStream: Stream<Item = (O, Multiaddr)>, InnerStream: Stream<Item = (O, Multiaddr)>,
{ {
type Item = (TokioTimerMapErr<Timeout<O>>, Multiaddr); type Item = (TokioTimerMapErr<Timeout<O>>, Multiaddr);
type Error = InnerStream::Error; type Error = TransportTimeoutError<InnerStream::Error>;
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> { fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
let poll_out = try_ready!(self.inner.poll()); let poll_out = try_ready!(self.inner.poll().map_err(TransportTimeoutError::Other));
if let Some((inner_fut, addr)) = poll_out { if let Some((inner_fut, addr)) = poll_out {
let fut = TokioTimerMapErr { let fut = TokioTimerMapErr {
inner: Timeout::new(inner_fut, self.timeout), inner: Timeout::new(inner_fut, self.timeout),
@ -155,7 +137,8 @@ where
} }
} }
/// Wraps around a `Future`. Turns the error type from `TimeoutError<IoError>` to `IoError`. /// Wraps around a `Future`. Turns the error type from `TimeoutError<Err>` to
/// `TransportTimeoutError<Err>`.
// TODO: can be replaced with `impl Future` once `impl Trait` are fully stable in Rust // TODO: can be replaced with `impl Future` once `impl Trait` are fully stable in Rust
// (https://github.com/rust-lang/rust/issues/34511) // (https://github.com/rust-lang/rust/issues/34511)
#[must_use = "futures do nothing unless polled"] #[must_use = "futures do nothing unless polled"]
@ -163,26 +146,60 @@ pub struct TokioTimerMapErr<InnerFut> {
inner: InnerFut, inner: InnerFut,
} }
impl<InnerFut> Future for TokioTimerMapErr<InnerFut> impl<InnerFut, TErr> Future for TokioTimerMapErr<InnerFut>
where where
InnerFut: Future<Error = TimeoutError<IoError>>, InnerFut: Future<Error = TimeoutError<TErr>>,
{ {
type Item = InnerFut::Item; type Item = InnerFut::Item;
type Error = IoError; type Error = TransportTimeoutError<TErr>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
self.inner.poll().map_err(|err: TimeoutError<IoError>| { self.inner.poll().map_err(|err: TimeoutError<TErr>| {
if err.is_inner() { if err.is_inner() {
err.into_inner().expect("ensured by is_inner()") TransportTimeoutError::Other(err.into_inner().expect("ensured by is_inner()"))
} else if err.is_elapsed() { } else if err.is_elapsed() {
debug!("timeout elapsed for connection"); debug!("timeout elapsed for connection");
IoErrorKind::TimedOut.into() TransportTimeoutError::Timeout
} else { } else {
assert!(err.is_timer()); assert!(err.is_timer());
debug!("tokio timer error in timeout wrapper"); debug!("tokio timer error in timeout wrapper");
let err = err.into_timer().expect("ensure by is_timer()"); TransportTimeoutError::TimerError
IoError::new(IoErrorKind::Other, err)
} }
}) })
} }
} }
/// Error that can be produced by the `TransportTimeout` layer.
#[derive(Debug, Copy, Clone)]
pub enum TransportTimeoutError<TErr> {
/// The transport timed out.
Timeout,
/// An error happened in the timer.
TimerError,
/// Other kind of error.
Other(TErr),
}
impl<TErr> fmt::Display for TransportTimeoutError<TErr>
where TErr: fmt::Display,
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
TransportTimeoutError::Timeout => write!(f, "Timeout has been reached"),
TransportTimeoutError::TimerError => write!(f, "Error in the timer"),
TransportTimeoutError::Other(err) => write!(f, "{}", err),
}
}
}
impl<TErr> error::Error for TransportTimeoutError<TErr>
where TErr: error::Error + 'static,
{
fn source(&self) -> Option<&(dyn error::Error + 'static)> {
match self {
TransportTimeoutError::Timeout => None,
TransportTimeoutError::TimerError => None,
TransportTimeoutError::Other(err) => Some(err),
}
}
}

View File

@ -20,6 +20,7 @@
use crate::{ use crate::{
transport::Transport, transport::Transport,
transport::TransportError,
upgrade::{ upgrade::{
OutboundUpgrade, OutboundUpgrade,
InboundUpgrade, InboundUpgrade,
@ -32,6 +33,7 @@ use crate::{
}; };
use futures::{future::Either, prelude::*, try_ready}; use futures::{future::Either, prelude::*, try_ready};
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use std::{error, fmt};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
#[derive(Debug, Copy, Clone)] #[derive(Debug, Copy, Clone)]
@ -43,36 +45,34 @@ impl<T, U> Upgrade<T, U> {
} }
} }
impl<D, U, O, E> Transport for Upgrade<D, U> impl<D, U, O, TUpgrErr> Transport for Upgrade<D, U>
where where
D: Transport, D: Transport,
D::Output: AsyncRead + AsyncWrite, D::Output: AsyncRead + AsyncWrite,
U: InboundUpgrade<D::Output, Output = O, Error = E>, D::Error: 'static,
U: OutboundUpgrade<D::Output, Output = O, Error = E> + Clone, U: InboundUpgrade<D::Output, Output = O, Error = TUpgrErr>,
E: std::error::Error + Send + Sync + 'static U: OutboundUpgrade<D::Output, Output = O, Error = TUpgrErr> + Clone,
TUpgrErr: std::error::Error + Send + Sync + 'static // TODO: remove bounds
{ {
type Output = O; type Output = O;
type Error = TransportUpgradeError<D::Error, TUpgrErr>;
type Listener = ListenerStream<D::Listener, U>; type Listener = ListenerStream<D::Listener, U>;
type ListenerUpgrade = ListenerUpgradeFuture<D::ListenerUpgrade, U>; type ListenerUpgrade = ListenerUpgradeFuture<D::ListenerUpgrade, U>;
type Dial = DialUpgradeFuture<D::Dial, U>; type Dial = DialUpgradeFuture<D::Dial, U>;
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
match self.inner.dial(addr.clone()) { let outbound = self.inner.dial(addr.clone())
Ok(outbound) => Ok(DialUpgradeFuture { .map_err(|err| err.map(TransportUpgradeError::Transport))?;
future: outbound, Ok(DialUpgradeFuture {
upgrade: Either::A(Some(self.upgrade)) future: outbound,
}), upgrade: Either::A(Some(self.upgrade))
Err((dialer, addr)) => Err((Upgrade::new(dialer, self.upgrade), addr)) })
}
} }
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
match self.inner.listen_on(addr) { let (inbound, addr) = self.inner.listen_on(addr)
Ok((inbound, addr)) => .map_err(|err| err.map(TransportUpgradeError::Transport))?;
Ok((ListenerStream { stream: inbound, upgrade: self.upgrade }, addr)), Ok((ListenerStream { stream: inbound, upgrade: self.upgrade }, addr))
Err((listener, addr)) =>
Err((Upgrade::new(listener, self.upgrade), addr))
}
} }
fn nat_traversal(&self, server: &Multiaddr, observed: &Multiaddr) -> Option<Multiaddr> { fn nat_traversal(&self, server: &Multiaddr, observed: &Multiaddr) -> Option<Multiaddr> {
@ -80,6 +80,41 @@ where
} }
} }
/// Error produced by a transport upgrade.
#[derive(Debug)]
pub enum TransportUpgradeError<TTransErr, TUpgrErr> {
/// Error in the transport.
Transport(TTransErr),
/// Error while upgrading to a protocol.
Upgrade(UpgradeError<TUpgrErr>),
}
impl<TTransErr, TUpgrErr> fmt::Display for TransportUpgradeError<TTransErr, TUpgrErr>
where
TTransErr: fmt::Display,
TUpgrErr: fmt::Display,
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
TransportUpgradeError::Transport(e) => write!(f, "Transport error: {}", e),
TransportUpgradeError::Upgrade(e) => write!(f, "Upgrade error: {}", e),
}
}
}
impl<TTransErr, TUpgrErr> error::Error for TransportUpgradeError<TTransErr, TUpgrErr>
where
TTransErr: error::Error + 'static,
TUpgrErr: error::Error + 'static,
{
fn source(&self) -> Option<&(dyn error::Error + 'static)> {
match self {
TransportUpgradeError::Transport(e) => Some(e),
TransportUpgradeError::Upgrade(e) => Some(e),
}
}
}
pub struct DialUpgradeFuture<T, U> pub struct DialUpgradeFuture<T, U>
where where
T: Future, T: Future,
@ -92,23 +127,23 @@ where
impl<T, U> Future for DialUpgradeFuture<T, U> impl<T, U> Future for DialUpgradeFuture<T, U>
where where
T: Future<Error = std::io::Error>, T: Future,
T::Item: AsyncRead + AsyncWrite, T::Item: AsyncRead + AsyncWrite,
U: OutboundUpgrade<T::Item>, U: OutboundUpgrade<T::Item>,
U::Error: std::error::Error + Send + Sync + 'static U::Error: std::error::Error + Send + Sync + 'static
{ {
type Item = U::Output; type Item = U::Output;
type Error = std::io::Error; type Error = TransportUpgradeError<T::Error, U::Error>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
loop { loop {
let next = match self.upgrade { let next = match self.upgrade {
Either::A(ref mut up) => { Either::A(ref mut up) => {
let x = try_ready!(self.future.poll()); let x = try_ready!(self.future.poll().map_err(TransportUpgradeError::Transport));
let u = up.take().expect("DialUpgradeFuture is constructed with Either::A(Some)."); let u = up.take().expect("DialUpgradeFuture is constructed with Either::A(Some).");
Either::B(apply_outbound(x, u)) Either::B(apply_outbound(x, u))
} }
Either::B(ref mut up) => return up.poll().map_err(UpgradeError::into_io_error) Either::B(ref mut up) => return up.poll().map_err(TransportUpgradeError::Upgrade)
}; };
self.upgrade = next self.upgrade = next
} }
@ -128,10 +163,10 @@ where
U: InboundUpgrade<F::Item> + Clone U: InboundUpgrade<F::Item> + Clone
{ {
type Item = (ListenerUpgradeFuture<F, U>, Multiaddr); type Item = (ListenerUpgradeFuture<F, U>, Multiaddr);
type Error = T::Error; type Error = TransportUpgradeError<T::Error, U::Error>;
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> { fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
match try_ready!(self.stream.poll()) { match try_ready!(self.stream.poll().map_err(TransportUpgradeError::Transport)) {
Some((x, a)) => { Some((x, a)) => {
let f = ListenerUpgradeFuture { let f = ListenerUpgradeFuture {
future: x, future: x,
@ -156,23 +191,23 @@ where
impl<T, U> Future for ListenerUpgradeFuture<T, U> impl<T, U> Future for ListenerUpgradeFuture<T, U>
where where
T: Future<Error = std::io::Error>, T: Future,
T::Item: AsyncRead + AsyncWrite, T::Item: AsyncRead + AsyncWrite,
U: InboundUpgrade<T::Item>, U: InboundUpgrade<T::Item>,
U::Error: std::error::Error + Send + Sync + 'static U::Error: std::error::Error + Send + Sync + 'static
{ {
type Item = U::Output; type Item = U::Output;
type Error = std::io::Error; type Error = TransportUpgradeError<T::Error, U::Error>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
loop { loop {
let next = match self.upgrade { let next = match self.upgrade {
Either::A(ref mut up) => { Either::A(ref mut up) => {
let x = try_ready!(self.future.poll()); let x = try_ready!(self.future.poll().map_err(TransportUpgradeError::Transport));
let u = up.take().expect("ListenerUpgradeFuture is constructed with Either::A(Some)."); let u = up.take().expect("ListenerUpgradeFuture is constructed with Either::A(Some).");
Either::B(apply_inbound(x, u)) Either::B(apply_inbound(x, u))
} }
Either::B(ref mut up) => return up.poll().map_err(UpgradeError::into_io_error) Either::B(ref mut up) => return up.poll().map_err(TransportUpgradeError::Upgrade)
}; };
self.upgrade = next self.upgrade = next
} }

View File

@ -30,15 +30,6 @@ pub enum UpgradeError<E> {
Apply(E), Apply(E),
} }
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> { impl<E> UpgradeError<E> {
pub fn map_err<F, T>(self, f: F) -> UpgradeError<T> pub fn map_err<F, T>(self, f: F) -> UpgradeError<T>
where where
@ -72,9 +63,9 @@ where
impl<E> std::error::Error for UpgradeError<E> impl<E> std::error::Error for UpgradeError<E>
where where
E: std::error::Error E: std::error::Error + 'static
{ {
fn cause(&self) -> Option<&dyn std::error::Error> { fn source(&self) -> Option<&(dyn std::error::Error + 'static)> {
match self { match self {
UpgradeError::Select(e) => Some(e), UpgradeError::Select(e) => Some(e),
UpgradeError::Apply(e) => Some(e), UpgradeError::Apply(e) => Some(e),

View File

@ -45,8 +45,9 @@ fn client_to_server_outbound() {
let future = listener let future = listener
.into_future() .into_future()
.map_err(|(err, _)| err) .map_err(|(err, _)| panic!("{:?}", err))
.and_then(|(client, _)| client.unwrap().0) .and_then(|(client, _)| client.unwrap().0)
.map_err(|err| panic!("{:?}", err))
.and_then(|client| muxing::outbound_from_ref_and_wrap(Arc::new(client))) .and_then(|client| muxing::outbound_from_ref_and_wrap(Arc::new(client)))
.map(|client| Builder::new().new_read(client.unwrap())) .map(|client| Builder::new().new_read(client.unwrap()))
.and_then(|client| { .and_then(|client| {
@ -70,6 +71,7 @@ fn client_to_server_outbound() {
let future = transport let future = transport
.dial(rx.recv().unwrap()) .dial(rx.recv().unwrap())
.unwrap() .unwrap()
.map_err(|err| panic!("{:?}", err))
.and_then(|client| muxing::inbound_from_ref_and_wrap(Arc::new(client))) .and_then(|client| muxing::inbound_from_ref_and_wrap(Arc::new(client)))
.map(|server| Builder::new().new_write(server.unwrap())) .map(|server| Builder::new().new_write(server.unwrap()))
.and_then(|server| server.send("hello world".into())) .and_then(|server| server.send("hello world".into()))
@ -97,8 +99,9 @@ fn client_to_server_inbound() {
let future = listener let future = listener
.into_future() .into_future()
.map_err(|(err, _)| err) .map_err(|(err, _)| panic!("{:?}", err))
.and_then(|(client, _)| client.unwrap().0) .and_then(|(client, _)| client.unwrap().0)
.map_err(|err| panic!("{:?}", err))
.and_then(|client| muxing::inbound_from_ref_and_wrap(Arc::new(client))) .and_then(|client| muxing::inbound_from_ref_and_wrap(Arc::new(client)))
.map(|client| Builder::new().new_read(client.unwrap())) .map(|client| Builder::new().new_read(client.unwrap()))
.and_then(|client| { .and_then(|client| {
@ -122,6 +125,7 @@ fn client_to_server_inbound() {
let future = transport let future = transport
.dial(rx.recv().unwrap()) .dial(rx.recv().unwrap())
.unwrap() .unwrap()
.map_err(|err| panic!("{:?}", err))
.and_then(|client| muxing::outbound_from_ref_and_wrap(Arc::new(client))) .and_then(|client| muxing::outbound_from_ref_and_wrap(Arc::new(client)))
.map(|server| Builder::new().new_write(server.unwrap())) .map(|server| Builder::new().new_write(server.unwrap()))
.and_then(|server| server.send("hello world".into())) .and_then(|server| server.send("hello world".into()))

View File

@ -23,6 +23,7 @@
use futures::{future, prelude::*, stream, AndThen, MapErr}; use futures::{future, prelude::*, stream, AndThen, MapErr};
use libp2p_core::{ use libp2p_core::{
Multiaddr, PeerId, PublicKey, muxing, Transport, Multiaddr, PeerId, PublicKey, muxing, Transport,
transport::{TransportError, upgrade::TransportUpgradeError},
upgrade::{self, OutboundUpgradeApply, UpgradeError} upgrade::{self, OutboundUpgradeApply, UpgradeError}
}; };
use protocol::{RemoteInfo, IdentifyProtocolConfig}; use protocol::{RemoteInfo, IdentifyProtocolConfig};
@ -59,38 +60,32 @@ impl<TTrans> IdentifyTransport<TTrans> {
impl<TTrans, TMuxer> Transport for IdentifyTransport<TTrans> impl<TTrans, TMuxer> Transport for IdentifyTransport<TTrans>
where where
TTrans: Transport<Output = TMuxer>, TTrans: Transport<Output = TMuxer>,
TTrans::Error: 'static,
TMuxer: muxing::StreamMuxer + Send + Sync + 'static, // TODO: remove unnecessary bounds TMuxer: muxing::StreamMuxer + Send + Sync + 'static, // TODO: remove unnecessary bounds
TMuxer::Substream: Send + Sync + 'static, // TODO: remove unnecessary bounds TMuxer::Substream: Send + Sync + 'static, // TODO: remove unnecessary bounds
{ {
type Output = (PeerId, TMuxer); type Output = (PeerId, TMuxer);
type Listener = stream::Empty<(Self::ListenerUpgrade, Multiaddr), IoError>; type Error = TransportUpgradeError<TTrans::Error, IoError>; // TODO: better than IoError
type ListenerUpgrade = future::Empty<Self::Output, IoError>; type Listener = stream::Empty<(Self::ListenerUpgrade, Multiaddr), Self::Error>;
type ListenerUpgrade = future::Empty<Self::Output, Self::Error>;
type Dial = AndThen< type Dial = AndThen<
TTrans::Dial, MapErr<TTrans::Dial, fn(TTrans::Error) -> Self::Error>,
MapErr<IdRetriever<TMuxer>, fn(UpgradeError<IoError>) -> IoError>, MapErr<IdRetriever<TMuxer>, fn(UpgradeError<IoError>) -> Self::Error>,
fn(TMuxer) -> MapErr<IdRetriever<TMuxer>, fn(UpgradeError<IoError>) -> IoError> fn(TMuxer) -> MapErr<IdRetriever<TMuxer>, fn(UpgradeError<IoError>) -> Self::Error>
>; >;
#[inline] #[inline]
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
Err((self, addr)) Err(TransportError::MultiaddrNotSupported(addr))
} }
#[inline] #[inline]
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
// We dial a first time the node. // We dial a first time the node.
let dial = match self.transport.dial(addr.clone()) { let dial = self.transport.dial(addr)
Ok(d) => d, .map_err(|err| err.map(TransportUpgradeError::Transport))?;
Err((transport, addr)) => { Ok(dial.map_err::<fn(_) -> _, _>(TransportUpgradeError::Transport).and_then(|muxer| {
let id = IdentifyTransport { IdRetriever::new(muxer, IdentifyProtocolConfig).map_err(TransportUpgradeError::Upgrade)
transport,
};
return Err((id, addr));
}
};
Ok(dial.and_then(|muxer| {
IdRetriever::new(muxer, IdentifyProtocolConfig).map_err(|e| e.into_io_error())
})) }))
} }

View File

@ -260,8 +260,7 @@ mod tests {
use self::libp2p_tcp::TcpConfig; use self::libp2p_tcp::TcpConfig;
use futures::{Future, Stream}; use futures::{Future, Stream};
use libp2p_core::{PublicKey, Transport, upgrade::{apply_outbound, apply_inbound}}; use libp2p_core::{PublicKey, Transport, upgrade::{apply_outbound, apply_inbound}};
use std::sync::mpsc; use std::{io, sync::mpsc, thread};
use std::thread;
#[test] #[test]
fn correct_transfer() { fn correct_transfer() {
@ -284,7 +283,8 @@ mod tests {
.map_err(|(err, _)| err) .map_err(|(err, _)| err)
.and_then(|(client, _)| client.unwrap().0) .and_then(|(client, _)| client.unwrap().0)
.and_then(|socket| { .and_then(|socket| {
apply_inbound(socket, IdentifyProtocolConfig).map_err(|e| e.into_io_error()) apply_inbound(socket, IdentifyProtocolConfig)
.map_err(|e| io::Error::new(io::ErrorKind::Other, e))
}) })
.and_then(|sender| { .and_then(|sender| {
sender.send( sender.send(
@ -308,9 +308,10 @@ mod tests {
let transport = TcpConfig::new(); let transport = TcpConfig::new();
let future = transport.dial(rx.recv().unwrap()) let future = transport.dial(rx.recv().unwrap())
.unwrap_or_else(|_| panic!()) .unwrap()
.and_then(|socket| { .and_then(|socket| {
apply_outbound(socket, IdentifyProtocolConfig).map_err(|e| e.into_io_error()) apply_outbound(socket, IdentifyProtocolConfig)
.map_err(|e| io::Error::new(io::ErrorKind::Other, e))
}) })
.and_then(|RemoteInfo { info, observed_addr, .. }| { .and_then(|RemoteInfo { info, observed_addr, .. }| {
assert_eq!(observed_addr, "/ip4/100.101.102.103/tcp/5000".parse().unwrap()); assert_eq!(observed_addr, "/ip4/100.101.102.103/tcp/5000".parse().unwrap());

View File

@ -536,7 +536,7 @@ mod tests {
let future = transport let future = transport
.dial(rx.recv().unwrap()) .dial(rx.recv().unwrap())
.unwrap_or_else(|_| panic!()) .unwrap()
.and_then(|proto| proto.send(msg_client)) .and_then(|proto| proto.send(msg_client))
.map(|_| ()); .map(|_| ());
let mut rt = Runtime::new().unwrap(); let mut rt = Runtime::new().unwrap();

View File

@ -51,7 +51,8 @@
//! .map(|out: SecioOutput<_>, _| out.stream); //! .map(|out: SecioOutput<_>, _| out.stream);
//! //!
//! let future = dialer.dial("/ip4/127.0.0.1/tcp/12345".parse::<Multiaddr>().unwrap()) //! let future = dialer.dial("/ip4/127.0.0.1/tcp/12345".parse::<Multiaddr>().unwrap())
//! .unwrap_or_else(|_| panic!("Unable to dial node")) //! .unwrap()
//! .map_err(|e| panic!("error: {:?}", e))
//! .and_then(|connection| { //! .and_then(|connection| {
//! // Sends "hello world" on the connection, will be encrypted. //! // Sends "hello world" on the connection, will be encrypted.
//! write_all(connection, "hello world") //! write_all(connection, "hello world")

View File

@ -165,6 +165,7 @@ pub mod simple;
pub use self::core::{ pub use self::core::{
Transport, PeerId, Swarm, Transport, PeerId, Swarm,
transport::TransportError,
upgrade::{InboundUpgrade, InboundUpgradeExt, OutboundUpgrade, OutboundUpgradeExt} upgrade::{InboundUpgrade, InboundUpgradeExt, OutboundUpgrade, OutboundUpgradeExt}
}; };
pub use libp2p_core_derive::NetworkBehaviour; pub use libp2p_core_derive::NetworkBehaviour;
@ -173,7 +174,7 @@ pub use self::simple::SimpleProtocol;
pub use self::transport_ext::TransportExt; pub use self::transport_ext::TransportExt;
use futures::prelude::*; use futures::prelude::*;
use std::time::Duration; use std::{error, time::Duration};
/// Builds a `Transport` that supports the most commonly-used protocols that libp2p supports. /// Builds a `Transport` that supports the most commonly-used protocols that libp2p supports.
/// ///
@ -181,7 +182,7 @@ use std::time::Duration;
/// > reserves the right to support additional protocols or remove deprecated protocols. /// > reserves the right to support additional protocols or remove deprecated protocols.
#[inline] #[inline]
pub fn build_development_transport(local_private_key: secio::SecioKeyPair) pub fn build_development_transport(local_private_key: secio::SecioKeyPair)
-> impl Transport<Output = (PeerId, impl core::muxing::StreamMuxer<OutboundSubstream = impl Send, Substream = impl Send> + Send + Sync), Listener = impl Send, Dial = impl Send, ListenerUpgrade = impl Send> + Clone -> impl Transport<Output = (PeerId, impl core::muxing::StreamMuxer<OutboundSubstream = impl Send, Substream = impl Send> + Send + Sync), Error = impl error::Error + Send, Listener = impl Send, Dial = impl Send, ListenerUpgrade = impl Send> + Clone
{ {
build_tcp_ws_secio_mplex_yamux(local_private_key) build_tcp_ws_secio_mplex_yamux(local_private_key)
} }
@ -193,7 +194,7 @@ pub fn build_development_transport(local_private_key: secio::SecioKeyPair)
/// ///
/// > **Note**: If you ever need to express the type of this `Transport`. /// > **Note**: If you ever need to express the type of this `Transport`.
pub fn build_tcp_ws_secio_mplex_yamux(local_private_key: secio::SecioKeyPair) pub fn build_tcp_ws_secio_mplex_yamux(local_private_key: secio::SecioKeyPair)
-> impl Transport<Output = (PeerId, impl core::muxing::StreamMuxer<OutboundSubstream = impl Send, Substream = impl Send> + Send + Sync), Listener = impl Send, Dial = impl Send, ListenerUpgrade = impl Send> + Clone -> impl Transport<Output = (PeerId, impl core::muxing::StreamMuxer<OutboundSubstream = impl Send, Substream = impl Send> + Send + Sync), Error = impl error::Error + Send, Listener = impl Send, Dial = impl Send, ListenerUpgrade = impl Send> + Clone
{ {
CommonTransport::new() CommonTransport::new()
.with_upgrade(secio::SecioConfig::new(local_private_key)) .with_upgrade(secio::SecioConfig::new(local_private_key))
@ -207,7 +208,6 @@ pub fn build_tcp_ws_secio_mplex_yamux(local_private_key: secio::SecioKeyPair)
core::upgrade::apply(out.stream, upgrade, endpoint) core::upgrade::apply(out.stream, upgrade, endpoint)
.map(|(id, muxer)| (id, core::muxing::StreamMuxerBox::new(muxer))) .map(|(id, muxer)| (id, core::muxing::StreamMuxerBox::new(muxer)))
.map_err(|e| e.into_io_error())
}) })
.with_timeout(Duration::from_secs(20)) .with_timeout(Duration::from_secs(20))
} }
@ -265,30 +265,19 @@ impl CommonTransport {
impl Transport for CommonTransport { impl Transport for CommonTransport {
type Output = <InnerImplementation as Transport>::Output; type Output = <InnerImplementation as Transport>::Output;
type Error = <InnerImplementation as Transport>::Error;
type Listener = <InnerImplementation as Transport>::Listener; type Listener = <InnerImplementation as Transport>::Listener;
type ListenerUpgrade = <InnerImplementation as Transport>::ListenerUpgrade; type ListenerUpgrade = <InnerImplementation as Transport>::ListenerUpgrade;
type Dial = <InnerImplementation as Transport>::Dial; type Dial = <InnerImplementation as Transport>::Dial;
#[inline] #[inline]
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
match self.inner.inner.listen_on(addr) { self.inner.inner.listen_on(addr)
Ok(res) => Ok(res),
Err((inner, addr)) => {
let trans = CommonTransport { inner: CommonTransportInner { inner: inner } };
Err((trans, addr))
}
}
} }
#[inline] #[inline]
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
match self.inner.inner.dial(addr) { self.inner.inner.dial(addr)
Ok(res) => Ok(res),
Err((inner, addr)) => {
let trans = CommonTransport { inner: CommonTransportInner { inner: inner } };
Err((trans, addr))
}
}
} }
#[inline] #[inline]

View File

@ -41,13 +41,11 @@ extern crate multiaddr;
extern crate tokio_dns; extern crate tokio_dns;
extern crate tokio_io; extern crate tokio_io;
use futures::{future::{self, Either, FutureResult, JoinAll}, prelude::*, try_ready}; use futures::{future::{self, Either, FutureResult, JoinAll}, prelude::*, stream, try_ready};
use log::Level; use log::Level;
use multiaddr::{Protocol, Multiaddr}; use multiaddr::{Protocol, Multiaddr};
use std::fmt; use std::{error, fmt, io, marker::PhantomData, net::IpAddr};
use std::io::{Error as IoError, ErrorKind as IoErrorKind}; use swarm::{Transport, transport::TransportError};
use std::net::IpAddr;
use swarm::Transport;
use tokio_dns::{CpuPoolResolver, Resolver}; use tokio_dns::{CpuPoolResolver, Resolver};
/// Represents the configuration for a DNS transport capability of libp2p. /// Represents the configuration for a DNS transport capability of libp2p.
@ -94,33 +92,34 @@ where
impl<T> Transport for DnsConfig<T> impl<T> Transport for DnsConfig<T>
where where
T: Transport T: Transport,
T::Error: 'static,
{ {
type Output = T::Output; type Output = T::Output;
type Listener = T::Listener; type Error = DnsErr<T::Error>;
type ListenerUpgrade = T::ListenerUpgrade; type Listener = stream::MapErr<
type Dial = Either<T::Dial, stream::Map<T::Listener,
fn((T::ListenerUpgrade, Multiaddr)) -> (Self::ListenerUpgrade, Multiaddr)>,
fn(T::Error) -> Self::Error>;
type ListenerUpgrade = future::MapErr<T::ListenerUpgrade, fn(T::Error) -> Self::Error>;
type Dial = Either<future::MapErr<T::Dial, fn(T::Error) -> Self::Error>,
DialFuture<T, JoinFuture<JoinAll<std::vec::IntoIter<Either< DialFuture<T, JoinFuture<JoinAll<std::vec::IntoIter<Either<
ResolveFuture<tokio_dns::IoFuture<Vec<IpAddr>>>, ResolveFuture<tokio_dns::IoFuture<Vec<IpAddr>>, T::Error>,
FutureResult<Protocol<'static>, IoError>>>> FutureResult<Protocol<'static>, Self::Error>>>>
>> >>
>; >;
#[inline] #[inline]
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
match self.inner.listen_on(addr) { let (listener, new_addr) = self.inner.listen_on(addr)
Ok(r) => Ok(r), .map_err(|err| err.map(DnsErr::Underlying))?;
Err((inner, addr)) => Err(( let listener = listener
DnsConfig { .map::<_, fn(_) -> _>(|(upgr, multiaddr)| (upgr.map_err::<fn(_) -> _, _>(DnsErr::Underlying), multiaddr))
inner, .map_err::<_, fn(_) -> _>(DnsErr::Underlying);
resolver: self.resolver, Ok((listener, new_addr))
},
addr,
)),
}
} }
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
let contains_dns = addr.iter().any(|cmp| match cmp { let contains_dns = addr.iter().any(|cmp| match cmp {
Protocol::Dns4(_) => true, Protocol::Dns4(_) => true,
Protocol::Dns6(_) => true, Protocol::Dns6(_) => true,
@ -129,16 +128,8 @@ where
if !contains_dns { if !contains_dns {
trace!("Pass-through address without DNS: {}", addr); trace!("Pass-through address without DNS: {}", addr);
return match self.inner.dial(addr) { let inner_dial = self.inner.dial(addr).map_err(|err| err.map(DnsErr::Underlying))?;
Ok(d) => Ok(Either::A(d)), return Ok(Either::A(inner_dial.map_err(DnsErr::Underlying)));
Err((inner, addr)) => Err((
DnsConfig {
inner,
resolver: self.resolver,
},
addr,
)),
};
} }
let resolver = self.resolver; let resolver = self.resolver;
@ -154,7 +145,8 @@ where
None None
}, },
inner: resolver.resolve(name), inner: resolver.resolve(name),
ty: ResolveTy::Dns4 ty: ResolveTy::Dns4,
error_ty: PhantomData,
}), }),
Protocol::Dns6(ref name) => Protocol::Dns6(ref name) =>
Either::A(ResolveFuture { Either::A(ResolveFuture {
@ -164,7 +156,8 @@ where
None None
}, },
inner: resolver.resolve(name), inner: resolver.resolve(name),
ty: ResolveTy::Dns6 ty: ResolveTy::Dns6,
error_ty: PhantomData,
}), }),
cmp => Either::B(future::ok(cmp.acquire())) cmp => Either::B(future::ok(cmp.acquire()))
}) })
@ -183,6 +176,50 @@ where
} }
} }
/// Error that can be generated by the DNS layer.
#[derive(Debug)]
pub enum DnsErr<TErr> {
/// Error in the underlying transport layer.
Underlying(TErr),
/// Failed to find any IP address for this DNS address.
ResolveFail(String),
/// Error while resolving a DNS address.
ResolveError {
domain_name: String,
error: io::Error,
},
/// Found an IP address, but the underlying transport doesn't support the multiaddr.
MultiaddrNotSupported,
}
impl<TErr> fmt::Display for DnsErr<TErr>
where TErr: fmt::Display
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
DnsErr::Underlying(err) => write!(f, "{}", err),
DnsErr::ResolveFail(addr) => write!(f, "Failed to resolve DNS address: {:?}", addr),
DnsErr::ResolveError { domain_name, error } => {
write!(f, "Failed to resolve DNS address: {:?}; {:?}", domain_name, error)
},
DnsErr::MultiaddrNotSupported => write!(f, "Resolve multiaddr not supported"),
}
}
}
impl<TErr> error::Error for DnsErr<TErr>
where TErr: error::Error + 'static
{
fn source(&self) -> Option<&(dyn error::Error + 'static)> {
match self {
DnsErr::Underlying(err) => Some(err),
DnsErr::ResolveFail(_) => None,
DnsErr::ResolveError { error, .. } => Some(error),
DnsErr::MultiaddrNotSupported => None,
}
}
}
// How to resolve; to an IPv4 address or an IPv6 address? // How to resolve; to an IPv4 address or an IPv6 address?
#[derive(Debug, Copy, Clone, PartialEq, Eq)] #[derive(Debug, Copy, Clone, PartialEq, Eq)]
enum ResolveTy { enum ResolveTy {
@ -192,22 +229,27 @@ enum ResolveTy {
/// Future, performing DNS resolution. /// Future, performing DNS resolution.
#[derive(Debug)] #[derive(Debug)]
pub struct ResolveFuture<T> { pub struct ResolveFuture<T, E> {
name: Option<String>, name: Option<String>,
inner: T, inner: T,
ty: ResolveTy ty: ResolveTy,
error_ty: PhantomData<E>,
} }
impl<T> Future for ResolveFuture<T> impl<T, E> Future for ResolveFuture<T, E>
where where
T: Future<Item = Vec<IpAddr>, Error = IoError> T: Future<Item = Vec<IpAddr>, Error = io::Error>
{ {
type Item = Protocol<'static>; type Item = Protocol<'static>;
type Error = IoError; type Error = DnsErr<E>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
let ty = self.ty; let ty = self.ty;
let addrs = try_ready!(self.inner.poll()); let addrs = try_ready!(self.inner.poll().map_err(|error| {
let domain_name = self.name.take().unwrap_or(String::new());
DnsErr::ResolveError { domain_name, error }
}));
trace!("DNS component resolution: {:?} => {:?}", self.name, addrs); trace!("DNS component resolution: {:?} => {:?}", self.name, addrs);
let mut addrs = addrs let mut addrs = addrs
.into_iter() .into_iter()
@ -218,7 +260,7 @@ where
}); });
match addrs.next() { match addrs.next() {
Some(a) => Ok(Async::Ready(a)), Some(a) => Ok(Async::Ready(a)),
None => Err(IoError::new(IoErrorKind::Other, "couldn't find any relevant IP address")) None => Err(DnsErr::ResolveFail(self.name.take().unwrap_or(String::new())))
} }
} }
} }
@ -232,10 +274,10 @@ pub struct JoinFuture<T> {
impl<T> Future for JoinFuture<T> impl<T> Future for JoinFuture<T>
where where
T: Future<Item = Vec<Protocol<'static>>, Error = IoError> T: Future<Item = Vec<Protocol<'static>>>
{ {
type Item = Multiaddr; type Item = Multiaddr;
type Error = IoError; type Error = T::Error;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
let outcome = try_ready!(self.future.poll()); let outcome = try_ready!(self.future.poll());
@ -252,13 +294,14 @@ pub struct DialFuture<T: Transport, F> {
future: Either<F, T::Dial>, future: Either<F, T::Dial>,
} }
impl<T, F> Future for DialFuture<T, F> impl<T, F, TErr> Future for DialFuture<T, F>
where where
T: Transport, T: Transport<Error = TErr>,
F: Future<Item = Multiaddr, Error = IoError> F: Future<Item = Multiaddr, Error = DnsErr<TErr>>,
TErr: error::Error,
{ {
type Item = T::Output; type Item = T::Output;
type Error = IoError; type Error = DnsErr<TErr>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
loop { loop {
@ -267,10 +310,10 @@ where
let addr = try_ready!(f.poll()); let addr = try_ready!(f.poll());
match self.trans.take().unwrap().dial(addr) { match self.trans.take().unwrap().dial(addr) {
Ok(dial) => Either::B(dial), Ok(dial) => Either::B(dial),
Err(_) => return Err(IoError::new(IoErrorKind::Other, "multiaddr not supported")) Err(_) => return Err(DnsErr::MultiaddrNotSupported)
} }
} }
Either::B(ref mut f) => return f.poll() Either::B(ref mut f) => return f.poll().map_err(DnsErr::Underlying)
}; };
self.future = next self.future = next
} }
@ -282,9 +325,8 @@ mod tests {
extern crate libp2p_tcp; extern crate libp2p_tcp;
use self::libp2p_tcp::TcpConfig; use self::libp2p_tcp::TcpConfig;
use futures::future; use futures::future;
use swarm::Transport; use swarm::{Transport, transport::TransportError};
use multiaddr::{Protocol, Multiaddr}; use multiaddr::{Protocol, Multiaddr};
use std::io::Error as IoError;
use DnsConfig; use DnsConfig;
#[test] #[test]
@ -293,19 +335,20 @@ mod tests {
struct CustomTransport; struct CustomTransport;
impl Transport for CustomTransport { impl Transport for CustomTransport {
type Output = <TcpConfig as Transport>::Output; type Output = <TcpConfig as Transport>::Output;
type Error = <TcpConfig as Transport>::Error;
type Listener = <TcpConfig as Transport>::Listener; type Listener = <TcpConfig as Transport>::Listener;
type ListenerUpgrade = <TcpConfig as Transport>::ListenerUpgrade; type ListenerUpgrade = <TcpConfig as Transport>::ListenerUpgrade;
type Dial = future::Empty<Self::Output, IoError>; type Dial = future::Empty<Self::Output, Self::Error>;
#[inline] #[inline]
fn listen_on( fn listen_on(
self, self,
_addr: Multiaddr, _addr: Multiaddr,
) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { ) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
unreachable!() unreachable!()
} }
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
let addr = addr.iter().collect::<Vec<_>>(); let addr = addr.iter().collect::<Vec<_>>();
assert_eq!(addr.len(), 2); assert_eq!(addr.len(), 2);
match addr[1] { match addr[1] {
@ -331,9 +374,9 @@ mod tests {
let _ = transport let _ = transport
.clone() .clone()
.dial("/dns4/example.com/tcp/20000".parse().unwrap()) .dial("/dns4/example.com/tcp/20000".parse().unwrap())
.unwrap_or_else(|_| panic!()); .unwrap();
let _ = transport let _ = transport
.dial("/dns6/example.com/tcp/20000".parse().unwrap()) .dial("/dns6/example.com/tcp/20000".parse().unwrap())
.unwrap_or_else(|_| panic!()); .unwrap();
} }
} }

View File

@ -29,8 +29,8 @@ extern crate tokio_io;
use aio_limited::{Limited, Limiter}; use aio_limited::{Limited, Limiter};
use futures::prelude::*; use futures::prelude::*;
use libp2p_core::{Multiaddr, Transport}; use libp2p_core::{Multiaddr, Transport, transport::TransportError};
use std::io; use std::{error, fmt, io};
use tokio_executor::Executor; use tokio_executor::Executor;
use tokio_io::{AsyncRead, AsyncWrite, io::{ReadHalf, WriteHalf}}; use tokio_io::{AsyncRead, AsyncWrite, io::{ReadHalf, WriteHalf}};
@ -70,6 +70,37 @@ impl<T> RateLimited<T> {
} }
} }
/// Error that can be generated by the rate limiter.
#[derive(Debug)]
pub enum RateLimitedErr<TErr> {
/// Error in the underlying transport layer.
Underlying(TErr),
/// Error while creating a rate limiter.
LimiterError(io::Error),
}
impl<TErr> fmt::Display for RateLimitedErr<TErr>
where TErr: fmt::Display
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
RateLimitedErr::LimiterError(err) => write!(f, "Limiter initialization error: {}", err),
RateLimitedErr::Underlying(err) => write!(f, "{}", err),
}
}
}
impl<TErr> error::Error for RateLimitedErr<TErr>
where TErr: error::Error + 'static
{
fn source(&self) -> Option<&(dyn error::Error + 'static)> {
match self {
RateLimitedErr::LimiterError(err) => Some(err),
RateLimitedErr::Underlying(err) => Some(err),
}
}
}
/// A rate-limited connection. /// A rate-limited connection.
pub struct Connection<C: AsyncRead + AsyncWrite> { pub struct Connection<C: AsyncRead + AsyncWrite> {
reader: Limited<ReadHalf<C>>, reader: Limited<ReadHalf<C>>,
@ -120,10 +151,10 @@ pub struct Listener<T: Transport>(RateLimited<T::Listener>);
impl<T: Transport> Stream for Listener<T> { impl<T: Transport> Stream for Listener<T> {
type Item = (ListenerUpgrade<T>, Multiaddr); type Item = (ListenerUpgrade<T>, Multiaddr);
type Error = io::Error; type Error = RateLimitedErr<T::Error>;
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> { fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
match try_ready!(self.0.value.poll()) { match try_ready!(self.0.value.poll().map_err(RateLimitedErr::Underlying)) {
Some((upgrade, addr)) => { Some((upgrade, addr)) => {
let r = self.0.rlimiter.clone(); let r = self.0.rlimiter.clone();
let w = self.0.wlimiter.clone(); let w = self.0.wlimiter.clone();
@ -144,47 +175,47 @@ where
T::Output: AsyncRead + AsyncWrite T::Output: AsyncRead + AsyncWrite
{ {
type Item = Connection<T::Output>; type Item = Connection<T::Output>;
type Error = io::Error; type Error = RateLimitedErr<T::Error>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
let conn = try_ready!(self.0.value.poll()); let conn = try_ready!(self.0.value.poll().map_err(RateLimitedErr::Underlying));
let r = self.0.rlimiter.clone(); let r = self.0.rlimiter.clone();
let w = self.0.wlimiter.clone(); let w = self.0.wlimiter.clone();
Ok(Async::Ready(Connection::new(conn, r, w)?)) Ok(Async::Ready(Connection::new(conn, r, w).map_err(RateLimitedErr::LimiterError)?))
} }
} }
impl<T> Transport for RateLimited<T> impl<T> Transport for RateLimited<T>
where where
T: Transport, T: Transport,
T::Output: AsyncRead + AsyncWrite T::Output: AsyncRead + AsyncWrite,
T::Error: 'static,
{ {
type Output = Connection<T::Output>; type Output = Connection<T::Output>;
type Error = RateLimitedErr<T::Error>;
type Listener = Listener<T>; type Listener = Listener<T>;
type ListenerUpgrade = ListenerUpgrade<T>; type ListenerUpgrade = ListenerUpgrade<T>;
type Dial = DialFuture<T::Dial>; type Dial = DialFuture<T::Dial>;
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
let r = self.rlimiter; let r = self.rlimiter;
let w = self.wlimiter; let w = self.wlimiter;
self.value self.value
.listen_on(addr) .listen_on(addr)
.map_err(|err| err.map(RateLimitedErr::Underlying))
.map(|(listener, a)| { .map(|(listener, a)| {
( (
Listener(RateLimited::from_parts(listener, r.clone(), w.clone())), Listener(RateLimited::from_parts(listener, r.clone(), w.clone())),
a, a,
) )
}) })
.map_err(|(transport, a)| (RateLimited::from_parts(transport, r, w), a))
} }
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
let r = self.rlimiter; let r = self.rlimiter;
let w = self.wlimiter; let w = self.wlimiter;
match self.value.dial(addr) { let dial = self.value.dial(addr).map_err(|err| err.map(RateLimitedErr::Underlying))?;
Ok(dial) => Ok(DialFuture { r, w, f: dial }), Ok(DialFuture { r, w, f: dial })
Err((t, a)) => Err((RateLimited::from_parts(t, r, w), a))
}
} }
fn nat_traversal(&self, server: &Multiaddr, observed: &Multiaddr) -> Option<Multiaddr> { fn nat_traversal(&self, server: &Multiaddr, observed: &Multiaddr) -> Option<Multiaddr> {
@ -203,13 +234,13 @@ impl<T> Future for DialFuture<T>
where where
T: Future, T: Future,
T::Item: AsyncRead + AsyncWrite, T::Item: AsyncRead + AsyncWrite,
T::Error: From<io::Error>
{ {
type Item = Connection<T::Item>; type Item = Connection<T::Item>;
type Error = T::Error; type Error = RateLimitedErr<T::Error>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
let item = try_ready!(self.f.poll()); let item = try_ready!(self.f.poll().map_err(RateLimitedErr::Underlying));
Ok(Async::Ready(Connection::new(item, self.r.clone(), self.w.clone())?)) Ok(Async::Ready(Connection::new(item, self.r.clone(), self.w.clone())
.map_err(RateLimitedErr::LimiterError)?))
} }
} }

View File

@ -50,10 +50,10 @@ extern crate tokio_tcp;
use futures::{future, future::FutureResult, prelude::*, Async, Poll}; use futures::{future, future::FutureResult, prelude::*, Async, Poll};
use multiaddr::{Protocol, Multiaddr, ToMultiaddr}; use multiaddr::{Protocol, Multiaddr, ToMultiaddr};
use std::fmt; use std::fmt;
use std::io::{Error as IoError, Read, Write}; use std::io::{self, Read, Write};
use std::net::SocketAddr; use std::net::SocketAddr;
use std::time::Duration; use std::time::Duration;
use swarm::Transport; use swarm::{Transport, transport::TransportError};
use tk_listen::{ListenExt, SleepOnError}; use tk_listen::{ListenExt, SleepOnError};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use tokio_tcp::{ConnectFuture, Incoming, TcpListener, TcpStream}; use tokio_tcp::{ConnectFuture, Incoming, TcpListener, TcpStream};
@ -130,11 +130,12 @@ impl TcpConfig {
impl Transport for TcpConfig { impl Transport for TcpConfig {
type Output = TcpTransStream; type Output = TcpTransStream;
type Error = io::Error;
type Listener = TcpListenStream; type Listener = TcpListenStream;
type ListenerUpgrade = FutureResult<Self::Output, IoError>; type ListenerUpgrade = FutureResult<Self::Output, io::Error>;
type Dial = TcpDialFut; type Dial = TcpDialFut;
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
if let Ok(socket_addr) = multiaddr_to_socketaddr(&addr) { if let Ok(socket_addr) = multiaddr_to_socketaddr(&addr) {
let listener = TcpListener::bind(&socket_addr); let listener = TcpListener::bind(&socket_addr);
// We need to build the `Multiaddr` to return from this function. If an error happened, // We need to build the `Multiaddr` to return from this function. If an error happened,
@ -154,21 +155,23 @@ impl Transport for TcpConfig {
debug!("Now listening on {}", new_addr); debug!("Now listening on {}", new_addr);
let sleep_on_error = self.sleep_on_error; let sleep_on_error = self.sleep_on_error;
let inner = listener let inner = listener
.map_err(Some) .map_err(TransportError::Other)?
.map(move |l| l.incoming().sleep_on_error(sleep_on_error)); .incoming()
.sleep_on_error(sleep_on_error);
Ok(( Ok((
TcpListenStream { TcpListenStream {
inner, inner: Ok(inner),
config: self, config: self,
}, },
new_addr, new_addr,
)) ))
} else { } else {
Err((self, addr)) Err(TransportError::MultiaddrNotSupported(addr))
} }
} }
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
if let Ok(socket_addr) = multiaddr_to_socketaddr(&addr) { if let Ok(socket_addr) = multiaddr_to_socketaddr(&addr) {
// As an optimization, we check that the address is not of the form `0.0.0.0`. // As an optimization, we check that the address is not of the form `0.0.0.0`.
// If so, we instantly refuse dialing instead of going through the kernel. // If so, we instantly refuse dialing instead of going through the kernel.
@ -180,10 +183,10 @@ impl Transport for TcpConfig {
}) })
} else { } else {
debug!("Instantly refusing dialing {}, as it is invalid", addr); debug!("Instantly refusing dialing {}, as it is invalid", addr);
Err((self, addr)) Err(TransportError::Other(io::ErrorKind::ConnectionRefused.into()))
} }
} else { } else {
Err((self, addr)) Err(TransportError::MultiaddrNotSupported(addr))
} }
} }
@ -224,7 +227,7 @@ fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Result<SocketAddr, ()> {
} }
/// Applies the socket configuration parameters to a socket. /// Applies the socket configuration parameters to a socket.
fn apply_config(config: &TcpConfig, socket: &TcpStream) -> Result<(), IoError> { fn apply_config(config: &TcpConfig, socket: &TcpStream) -> Result<(), io::Error> {
if let Some(recv_buffer_size) = config.recv_buffer_size { if let Some(recv_buffer_size) = config.recv_buffer_size {
socket.set_recv_buffer_size(recv_buffer_size)?; socket.set_recv_buffer_size(recv_buffer_size)?;
} }
@ -259,9 +262,9 @@ pub struct TcpDialFut {
impl Future for TcpDialFut { impl Future for TcpDialFut {
type Item = TcpTransStream; type Item = TcpTransStream;
type Error = IoError; type Error = io::Error;
fn poll(&mut self) -> Poll<TcpTransStream, IoError> { fn poll(&mut self) -> Poll<TcpTransStream, io::Error> {
match self.inner.poll() { match self.inner.poll() {
Ok(Async::Ready(stream)) => { Ok(Async::Ready(stream)) => {
apply_config(&self.config, &stream)?; apply_config(&self.config, &stream)?;
@ -278,20 +281,20 @@ impl Future for TcpDialFut {
/// Stream that listens on an TCP/IP address. /// Stream that listens on an TCP/IP address.
pub struct TcpListenStream { pub struct TcpListenStream {
inner: Result<SleepOnError<Incoming>, Option<IoError>>, inner: Result<SleepOnError<Incoming>, Option<io::Error>>,
/// Original configuration. /// Original configuration.
config: TcpConfig, config: TcpConfig,
} }
impl Stream for TcpListenStream { impl Stream for TcpListenStream {
type Item = (FutureResult<TcpTransStream, IoError>, Multiaddr); type Item = (FutureResult<TcpTransStream, io::Error>, Multiaddr);
type Error = IoError; type Error = io::Error;
fn poll( fn poll(
&mut self, &mut self,
) -> Poll< ) -> Poll<
Option<(FutureResult<TcpTransStream, IoError>, Multiaddr)>, Option<(FutureResult<TcpTransStream, io::Error>, Multiaddr)>,
IoError, io::Error,
> { > {
let inner = match self.inner { let inner = match self.inner {
Ok(ref mut inc) => inc, Ok(ref mut inc) => inc,
@ -352,7 +355,7 @@ pub struct TcpTransStream {
impl Read for TcpTransStream { impl Read for TcpTransStream {
#[inline] #[inline]
fn read(&mut self, buf: &mut [u8]) -> Result<usize, IoError> { fn read(&mut self, buf: &mut [u8]) -> Result<usize, io::Error> {
self.inner.read(buf) self.inner.read(buf)
} }
} }
@ -361,19 +364,19 @@ impl AsyncRead for TcpTransStream {}
impl Write for TcpTransStream { impl Write for TcpTransStream {
#[inline] #[inline]
fn write(&mut self, buf: &[u8]) -> Result<usize, IoError> { fn write(&mut self, buf: &[u8]) -> Result<usize, io::Error> {
self.inner.write(buf) self.inner.write(buf)
} }
#[inline] #[inline]
fn flush(&mut self) -> Result<(), IoError> { fn flush(&mut self) -> Result<(), io::Error> {
self.inner.flush() self.inner.flush()
} }
} }
impl AsyncWrite for TcpTransStream { impl AsyncWrite for TcpTransStream {
#[inline] #[inline]
fn shutdown(&mut self) -> Poll<(), IoError> { fn shutdown(&mut self) -> Poll<(), io::Error> {
AsyncWrite::shutdown(&mut self.inner) AsyncWrite::shutdown(&mut self.inner)
} }
} }

View File

@ -8,14 +8,14 @@ repository = "https://github.com/libp2p/rust-libp2p"
keywords = ["peer-to-peer", "libp2p", "networking"] keywords = ["peer-to-peer", "libp2p", "networking"]
categories = ["network-programming", "asynchronous"] categories = ["network-programming", "asynchronous"]
[target.'cfg(all(unix, any(target_os = "emscripten", target_os = "unknown")))'.dependencies] [target.'cfg(all(unix, not(any(target_os = "emscripten", target_os = "unknown"))))'.dependencies]
libp2p-core = { version = "0.1.0", path = "../../core" } libp2p-core = { version = "0.1.0", path = "../../core" }
log = "0.4.1" log = "0.4.1"
futures = "0.1" futures = "0.1"
multiaddr = { package = "parity-multiaddr", version = "0.1.0", path = "../../misc/multiaddr" } multiaddr = { package = "parity-multiaddr", version = "0.1.0", path = "../../misc/multiaddr" }
tokio-uds = "0.2" tokio-uds = "0.2"
[target.'cfg(all(unix, any(target_os = "emscripten", target_os = "unknown")))'.dev-dependencies] [target.'cfg(all(unix, not(any(target_os = "emscripten", target_os = "unknown"))))'.dev-dependencies]
tempfile = "3.0" tempfile = "3.0"
tokio = "0.1" tokio = "0.1"
tokio-io = "0.1" tokio-io = "0.1"

View File

@ -44,7 +44,7 @@
//! The `UdsConfig` structs implements the `Transport` trait of the `core` library. See the //! The `UdsConfig` structs implements the `Transport` trait of the `core` library. See the
//! documentation of `core` and of libp2p in general to learn how to use the `Transport` trait. //! documentation of `core` and of libp2p in general to learn how to use the `Transport` trait.
#![cfg(all(unix, any(target_os = "emscripten", target_os = "unknown")))] #![cfg(all(unix, not(any(target_os = "emscripten", target_os = "unknown"))))]
extern crate futures; extern crate futures;
extern crate libp2p_core; extern crate libp2p_core;
@ -63,9 +63,8 @@ extern crate tokio;
use futures::{future::{self, FutureResult}, prelude::*, try_ready}; use futures::{future::{self, FutureResult}, prelude::*, try_ready};
use futures::stream::Stream; use futures::stream::Stream;
use multiaddr::{Protocol, Multiaddr}; use multiaddr::{Protocol, Multiaddr};
use std::io::Error as IoError; use std::{io, path::PathBuf};
use std::path::PathBuf; use libp2p_core::{Transport, transport::TransportError};
use libp2p_core::Transport;
use tokio_uds::{UnixListener, UnixStream}; use tokio_uds::{UnixListener, UnixStream};
/// Represents the configuration for a Unix domain sockets transport capability for libp2p. /// Represents the configuration for a Unix domain sockets transport capability for libp2p.
@ -86,11 +85,12 @@ impl UdsConfig {
impl Transport for UdsConfig { impl Transport for UdsConfig {
type Output = UnixStream; type Output = UnixStream;
type Error = io::Error;
type Listener = ListenerStream<tokio_uds::Incoming>; type Listener = ListenerStream<tokio_uds::Incoming>;
type ListenerUpgrade = FutureResult<Self::Output, IoError>; type ListenerUpgrade = FutureResult<Self::Output, io::Error>;
type Dial = tokio_uds::ConnectFuture; type Dial = tokio_uds::ConnectFuture;
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
if let Ok(path) = multiaddr_to_path(&addr) { if let Ok(path) = multiaddr_to_path(&addr) {
let listener = UnixListener::bind(&path); let listener = UnixListener::bind(&path);
// We need to build the `Multiaddr` to return from this function. If an error happened, // We need to build the `Multiaddr` to return from this function. If an error happened,
@ -104,19 +104,19 @@ impl Transport for UdsConfig {
}; };
Ok((future, addr)) Ok((future, addr))
} }
Err(_) => return Err((self, addr)), Err(_) => return Err(TransportError::MultiaddrNotSupported(addr)),
} }
} else { } else {
Err((self, addr)) Err(TransportError::MultiaddrNotSupported(addr))
} }
} }
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
if let Ok(path) = multiaddr_to_path(&addr) { if let Ok(path) = multiaddr_to_path(&addr) {
debug!("Dialing {}", addr); debug!("Dialing {}", addr);
Ok(UnixStream::connect(&path)) Ok(UnixStream::connect(&path))
} else { } else {
Err((self, addr)) Err(TransportError::MultiaddrNotSupported(addr))
} }
} }

View File

@ -29,7 +29,7 @@ use std::io::{Read, Write};
use std::sync::{Arc, Mutex}; use std::sync::{Arc, Mutex};
use stdweb::web::TypedArray; use stdweb::web::TypedArray;
use stdweb::{self, Reference}; use stdweb::{self, Reference};
use swarm::Transport; use swarm::{Transport, transport::TransportError};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
/// Represents the configuration for a websocket transport capability for libp2p. /// Represents the configuration for a websocket transport capability for libp2p.
@ -53,17 +53,18 @@ impl BrowserWsConfig {
impl Transport for BrowserWsConfig { impl Transport for BrowserWsConfig {
type Output = BrowserWsConn; type Output = BrowserWsConn;
type Error = IoError; // TODO: better error type?
type Listener = stream::Empty<(Self::ListenerUpgrade, Multiaddr), IoError>; type Listener = stream::Empty<(Self::ListenerUpgrade, Multiaddr), IoError>;
type ListenerUpgrade = future::Empty<Self::Output, IoError>; type ListenerUpgrade = future::Empty<Self::Output, IoError>;
type Dial = Box<Future<Item = Self::Output, Error = IoError> + Send>; type Dial = Box<Future<Item = Self::Output, Error = IoError> + Send>;
#[inline] #[inline]
fn listen_on(self, a: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, a: Multiaddr) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
// Listening is never supported. // Listening is never supported.
Err((self, a)) Err(TransportError::MultiaddrNotSupported(a))
} }
fn dial(self, original_addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, original_addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
// Making sure we are initialized before we dial. Initialization is protected by a simple // Making sure we are initialized before we dial. Initialization is protected by a simple
// boolean static variable, so it's not a problem to call it multiple times and the cost // boolean static variable, so it's not a problem to call it multiple times and the cost
// is negligible. // is negligible.
@ -73,7 +74,7 @@ impl Transport for BrowserWsConfig {
// a string) on success. // a string) on success.
let inner_addr = match multiaddr_to_target(&original_addr) { let inner_addr = match multiaddr_to_target(&original_addr) {
Ok(a) => a, Ok(a) => a,
Err(_) => return Err((self, original_addr)), Err(_) => return Err(TransportError::MultiaddrNotSupported(original_addr)),
}; };
debug!("Dialing {}", original_addr); debug!("Dialing {}", original_addr);
@ -89,7 +90,8 @@ impl Transport for BrowserWsConfig {
}; };
match val.into_reference() { match val.into_reference() {
Some(ws) => ws, Some(ws) => ws,
None => return Err((self, original_addr)), // `false` was returned by `js!` // TODO: more descriptive error
None => return Err(TransportError::Other(IoErrorKind::Other.into())), // `false` was returned by `js!`
} }
}; };

View File

@ -18,11 +18,12 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use futures::{stream, Future, IntoFuture, Sink, Stream}; use futures::{Future, IntoFuture, Sink, Stream};
use multiaddr::{Protocol, Multiaddr}; use multiaddr::{Protocol, Multiaddr};
use rw_stream_sink::RwStreamSink; use rw_stream_sink::RwStreamSink;
use std::{error, fmt};
use std::io::{Error as IoError, ErrorKind as IoErrorKind}; use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use swarm::Transport; use swarm::{Transport, transport::TransportError};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use websocket::client::builder::ClientBuilder; use websocket::client::builder::ClientBuilder;
use websocket::message::OwnedMessage; use websocket::message::OwnedMessage;
@ -59,6 +60,7 @@ impl<T> Transport for WsConfig<T>
where where
// TODO: this 'static is pretty arbitrary and is necessary because of the websocket library // TODO: this 'static is pretty arbitrary and is necessary because of the websocket library
T: Transport + 'static, T: Transport + 'static,
T::Error: Send,
T::Dial: Send, T::Dial: Send,
T::Listener: Send, T::Listener: Send,
T::ListenerUpgrade: Send, T::ListenerUpgrade: Send,
@ -66,55 +68,42 @@ where
T::Output: AsyncRead + AsyncWrite + Send, T::Output: AsyncRead + AsyncWrite + Send,
{ {
type Output = Box<AsyncStream + Send>; type Output = Box<AsyncStream + Send>;
type Listener = type Error = WsError<T::Error>;
stream::Map<T::Listener, fn((<T as Transport>::ListenerUpgrade, Multiaddr)) -> (Self::ListenerUpgrade, Multiaddr)>; type Listener = Box<Stream<Item = (Self::ListenerUpgrade, Multiaddr), Error = Self::Error> + Send>;
type ListenerUpgrade = Box<Future<Item = Self::Output, Error = IoError> + Send>; type ListenerUpgrade = Box<Future<Item = Self::Output, Error = Self::Error> + Send>;
type Dial = Box<Future<Item = Self::Output, Error = IoError> + Send>; type Dial = Box<Future<Item = Self::Output, Error = Self::Error> + Send>;
fn listen_on( fn listen_on(
self, self,
original_addr: Multiaddr, original_addr: Multiaddr,
) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { ) -> Result<(Self::Listener, Multiaddr), TransportError<Self::Error>> {
let mut inner_addr = original_addr.clone(); let mut inner_addr = original_addr.clone();
match inner_addr.pop() { match inner_addr.pop() {
Some(Protocol::Ws) => {} Some(Protocol::Ws) => {}
_ => return Err((self, original_addr)), _ => return Err(TransportError::MultiaddrNotSupported(original_addr)),
};
let (inner_listen, new_addr) = match self.transport.listen_on(inner_addr) {
Ok((listen, mut new_addr)) => {
// Need to suffix `/ws` to the listening address.
new_addr.append(Protocol::Ws);
(listen, new_addr)
}
Err((transport, _)) => {
return Err((
WsConfig {
transport: transport,
},
original_addr,
));
}
}; };
let (inner_listen, mut new_addr) = self.transport.listen_on(inner_addr)
.map_err(|err| err.map(WsError::Underlying))?;
new_addr.append(Protocol::Ws);
debug!("Listening on {}", new_addr); debug!("Listening on {}", new_addr);
let listen = inner_listen.map::<_, fn(_) -> _>(|(stream, mut client_addr)| { let listen = inner_listen.map_err(WsError::Underlying).map(|(stream, mut client_addr)| {
// Need to suffix `/ws` to each client address. // Need to suffix `/ws` to each client address.
client_addr.append(Protocol::Ws); client_addr.append(Protocol::Ws);
// Upgrade the listener to websockets like the websockets library requires us to do. // Upgrade the listener to websockets like the websockets library requires us to do.
let upgraded = stream.and_then(move |stream| { let upgraded = stream.map_err(WsError::Underlying).and_then(move |stream| {
debug!("Incoming connection"); debug!("Incoming connection");
stream stream
.into_ws() .into_ws()
.map_err(|e| IoError::new(IoErrorKind::Other, e.3)) .map_err(|e| WsError::WebSocket(Box::new(e.3)))
.and_then(|stream| { .and_then(|stream| {
// Accept the next incoming connection. // Accept the next incoming connection.
stream stream
.accept() .accept()
.map_err(|err| IoError::new(IoErrorKind::Other, err)) .map_err(|e| WsError::WebSocket(Box::new(e)))
.map(|(client, _http_headers)| { .map(|(client, _http_headers)| {
debug!("Upgraded incoming connection to websockets"); debug!("Upgraded incoming connection to websockets");
@ -149,10 +138,10 @@ where
(Box::new(upgraded) as Box<Future<Item = _, Error = _> + Send>, client_addr) (Box::new(upgraded) as Box<Future<Item = _, Error = _> + Send>, client_addr)
}); });
Ok((listen, new_addr)) Ok((Box::new(listen) as Box<_>, new_addr))
} }
fn dial(self, original_addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> { fn dial(self, original_addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
let mut inner_addr = original_addr.clone(); let mut inner_addr = original_addr.clone();
let is_wss = match inner_addr.pop() { let is_wss = match inner_addr.pop() {
Some(Protocol::Ws) => false, Some(Protocol::Ws) => false,
@ -162,7 +151,7 @@ where
"Ignoring dial attempt for {} because it is not a websocket multiaddr", "Ignoring dial attempt for {} because it is not a websocket multiaddr",
original_addr original_addr
); );
return Err((self, original_addr)); return Err(TransportError::MultiaddrNotSupported(original_addr));
} }
}; };
@ -170,29 +159,17 @@ where
let ws_addr = client_addr_to_ws(&inner_addr, is_wss); let ws_addr = client_addr_to_ws(&inner_addr, is_wss);
let inner_dial = match self.transport.dial(inner_addr) { let inner_dial = self.transport.dial(inner_addr)
Ok(d) => d, .map_err(|err| err.map(WsError::Underlying))?;
Err((transport, old_addr)) => {
debug!(
"Failed to dial {} because {} is not supported by the underlying transport",
original_addr, old_addr
);
return Err((
WsConfig {
transport: transport,
},
original_addr,
));
}
};
let dial = inner_dial let dial = inner_dial
.map_err(WsError::Underlying)
.into_future() .into_future()
.and_then(move |connec| { .and_then(move |connec| {
ClientBuilder::new(&ws_addr) ClientBuilder::new(&ws_addr)
.expect("generated ws address is always valid") .expect("generated ws address is always valid")
.async_connect_on(connec) .async_connect_on(connec)
.map_err(|err| IoError::new(IoErrorKind::Other, err)) .map_err(|e| WsError::WebSocket(Box::new(e)))
.map(|(client, _)| { .map(|(client, _)| {
debug!("Upgraded outgoing connection to websockets"); debug!("Upgraded outgoing connection to websockets");
@ -224,6 +201,37 @@ where
} }
} }
/// Error in WebSockets.
#[derive(Debug)]
pub enum WsError<TErr> {
/// Error in the WebSocket layer.
WebSocket(Box<dyn error::Error + Send + Sync>),
/// Error in the transport layer underneath.
Underlying(TErr),
}
impl<TErr> fmt::Display for WsError<TErr>
where TErr: fmt::Display
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
WsError::WebSocket(err) => write!(f, "{}", err),
WsError::Underlying(err) => write!(f, "{}", err),
}
}
}
impl<TErr> error::Error for WsError<TErr>
where TErr: error::Error + 'static
{
fn source(&self) -> Option<&(dyn error::Error + 'static)> {
match self {
WsError::WebSocket(err) => Some(&**err),
WsError::Underlying(err) => Some(err),
}
}
}
fn client_addr_to_ws(client_addr: &Multiaddr, is_wss: bool) -> String { fn client_addr_to_ws(client_addr: &Multiaddr, is_wss: bool) -> String {
let inner = { let inner = {
let protocols: Vec<_> = client_addr.iter().collect(); let protocols: Vec<_> = client_addr.iter().collect();