*: Dial with handler and return handler on error and closed (#2191)

Require `NetworkBehaviourAction::{DialPeer,DialAddress}` to contain a
`ProtocolsHandler`. This allows a behaviour to attach custom state to its
handler. The behaviour would no longer need to track this state separately
during connection establishment, thus reducing state required in a behaviour.
E.g. in the case of `libp2p-kad` the behaviour can include a `GetRecord` request
in its handler, or e.g. in the case of `libp2p-request-response` the behaviour
can include the first request in the handler.

Return `ProtocolsHandler` on connection error and close. This allows a behaviour
to extract its custom state previously included in the handler on connection
failure and connection closing. E.g. in the case of `libp2p-kad` the behaviour
could extract the attached `GetRecord` from the handler of the failed connection
and then start another connection attempt with a new handler with the same
`GetRecord` or bubble up an error to the user.

Co-authored-by: Thomas Eizinger <thomas@eizinger.io>
This commit is contained in:
Max Inden
2021-08-31 17:00:51 +02:00
committed by GitHub
parent b55ee69645
commit c161acfb50
38 changed files with 1111 additions and 477 deletions

View File

@ -229,10 +229,10 @@ where
self.handler.inject_event(event);
}
/// Begins an orderly shutdown of the connection, returning a
/// `Future` that resolves when connection shutdown is complete.
pub fn close(self) -> Close<TMuxer> {
self.muxing.close().0
/// Begins an orderly shutdown of the connection, returning the connection
/// handler and a `Future` that resolves when connection shutdown is complete.
pub fn close(self) -> (THandler, Close<TMuxer>) {
(self.handler, self.muxing.close().0)
}
/// Polls the connection for events produced by the associated handler

View File

@ -29,6 +29,10 @@ pub enum ConnectionError<THandlerErr> {
// TODO: Eventually this should also be a custom error?
IO(io::Error),
/// The connection was dropped because the connection limit
/// for a peer has been reached.
ConnectionLimit(ConnectionLimit),
/// The connection handler produced an error.
Handler(THandlerErr),
}
@ -41,6 +45,9 @@ where
match self {
ConnectionError::IO(err) => write!(f, "Connection error: I/O error: {}", err),
ConnectionError::Handler(err) => write!(f, "Connection error: Handler error: {}", err),
ConnectionError::ConnectionLimit(l) => {
write!(f, "Connection error: Connection limit: {}.", l)
}
}
}
}
@ -53,6 +60,7 @@ where
match self {
ConnectionError::IO(err) => Some(err),
ConnectionError::Handler(err) => Some(err),
ConnectionError::ConnectionLimit(..) => None,
}
}
}
@ -63,14 +71,13 @@ pub enum PendingConnectionError<TTransErr> {
/// An error occurred while negotiating the transport protocol(s).
Transport(TransportError<TTransErr>),
/// Pending connection attempt has been aborted.
Aborted,
/// The peer identity obtained on the connection did not
/// match the one that was expected or is otherwise invalid.
InvalidPeerId,
/// The connection was dropped because the connection limit
/// for a peer has been reached.
ConnectionLimit(ConnectionLimit),
/// An I/O error occurred on the connection.
// TODO: Eventually this should also be a custom error?
IO(io::Error),
@ -83,15 +90,13 @@ where
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self {
PendingConnectionError::IO(err) => write!(f, "Pending connection: I/O error: {}", err),
PendingConnectionError::Aborted => write!(f, "Pending connection: Aborted."),
PendingConnectionError::Transport(err) => {
write!(f, "Pending connection: Transport error: {}", err)
}
PendingConnectionError::InvalidPeerId => {
write!(f, "Pending connection: Invalid peer ID.")
}
PendingConnectionError::ConnectionLimit(l) => {
write!(f, "Connection error: Connection limit: {}.", l)
}
}
}
}
@ -105,7 +110,7 @@ where
PendingConnectionError::IO(err) => Some(err),
PendingConnectionError::Transport(err) => Some(err),
PendingConnectionError::InvalidPeerId => None,
PendingConnectionError::ConnectionLimit(..) => None,
PendingConnectionError::Aborted => None,
}
}
}

View File

@ -20,8 +20,8 @@
use super::{
handler::{THandlerError, THandlerInEvent, THandlerOutEvent},
Connected, ConnectedPoint, ConnectionError, ConnectionHandler, IntoConnectionHandler,
PendingConnectionError, Substream,
Connected, ConnectedPoint, ConnectionError, ConnectionHandler, ConnectionLimit,
IntoConnectionHandler, PendingConnectionError, Substream,
};
use crate::{muxing::StreamMuxer, Executor};
use fnv::FnvHashMap;
@ -192,6 +192,7 @@ pub enum Event<'a, H: IntoConnectionHandler, TE> {
/// The error that occurred, if any. If `None`, the connection
/// has been actively closed.
error: Option<ConnectionError<THandlerError<H>>>,
handler: H::Handler,
},
/// A connection has been established.
@ -350,7 +351,7 @@ impl<H: IntoConnectionHandler, TE> Manager<H, TE> {
new_endpoint: new,
}
}
task::Event::Closed { id, error } => {
task::Event::Closed { id, error, handler } => {
let id = ConnectionId(id);
let task = task.remove();
match task.state {
@ -358,6 +359,7 @@ impl<H: IntoConnectionHandler, TE> Manager<H, TE> {
id,
connected,
error,
handler,
},
TaskState::Pending => unreachable!(
"`Event::Closed` implies (2) occurred on that task and thus (3)."
@ -437,7 +439,7 @@ impl<'a, I> EstablishedEntry<'a, I> {
///
/// When the connection is ultimately closed, [`Event::ConnectionClosed`]
/// is emitted by [`Manager::poll`].
pub fn start_close(mut self) {
pub fn start_close(mut self, error: Option<ConnectionLimit>) {
// Clone the sender so that we are guaranteed to have
// capacity for the close command (every sender gets a slot).
match self
@ -445,7 +447,7 @@ impl<'a, I> EstablishedEntry<'a, I> {
.get_mut()
.sender
.clone()
.try_send(task::Command::Close)
.try_send(task::Command::Close(error))
{
Ok(()) => {}
Err(e) => assert!(e.is_disconnected(), "No capacity for close command."),
@ -460,17 +462,6 @@ impl<'a, I> EstablishedEntry<'a, I> {
}
}
/// Instantly removes the entry from the manager, dropping
/// the command channel to the background task of the connection,
/// which will thus drop the connection asap without an orderly
/// close or emitting another event.
pub fn remove(self) -> Connected {
match self.task.remove().state {
TaskState::Established(c) => c,
TaskState::Pending => unreachable!("By Entry::new()"),
}
}
/// Returns the connection ID.
pub fn id(&self) -> ConnectionId {
ConnectionId(*self.task.key())

View File

@ -23,8 +23,8 @@ use crate::{
connection::{
self,
handler::{THandlerError, THandlerInEvent, THandlerOutEvent},
Close, Connected, Connection, ConnectionError, ConnectionHandler, IntoConnectionHandler,
PendingConnectionError, Substream,
Close, Connected, Connection, ConnectionError, ConnectionHandler, ConnectionLimit,
IntoConnectionHandler, PendingConnectionError, Substream,
},
muxing::StreamMuxer,
Multiaddr,
@ -43,7 +43,7 @@ pub enum Command<T> {
NotifyHandler(T),
/// Gracefully close the connection (active close) before
/// terminating the task.
Close,
Close(Option<ConnectionLimit>),
}
/// Events that a task can emit to its manager.
@ -71,6 +71,7 @@ pub enum Event<H: IntoConnectionHandler, TE> {
Closed {
id: TaskId,
error: Option<ConnectionError<THandlerError<H>>>,
handler: H::Handler,
},
}
@ -159,7 +160,11 @@ where
},
/// The connection is closing (active close).
Closing(Close<M>),
Closing {
closing_muxer: Close<M>,
handler: H::Handler,
error: Option<ConnectionLimit>,
},
/// The task is terminating with a final event for the `Manager`.
Terminating(Event<H, E>),
@ -204,7 +209,16 @@ where
Poll::Pending => {}
Poll::Ready(None) => {
// The manager has dropped the task; abort.
return Poll::Ready(());
// Don't accept any further commands and terminate the
// task with a final event.
this.commands.get_mut().close();
let event = Event::Failed {
id,
handler,
error: PendingConnectionError::Aborted,
};
this.state = State::Terminating(event);
continue 'poll;
}
Poll::Ready(Some(_)) => {
panic!("Task received command while the connection is pending.")
@ -243,15 +257,20 @@ where
Poll::Ready(Some(Command::NotifyHandler(event))) => {
connection.inject_event(event)
}
Poll::Ready(Some(Command::Close)) => {
Poll::Ready(Some(Command::Close(error))) => {
// Don't accept any further commands.
this.commands.get_mut().close();
// Discard the event, if any, and start a graceful close.
this.state = State::Closing(connection.close());
let (handler, closing_muxer) = connection.close();
this.state = State::Closing {
handler,
closing_muxer,
error,
};
continue 'poll;
}
Poll::Ready(None) => {
// The manager has dropped the task or disappeared; abort.
// The manager has disappeared; abort.
return Poll::Ready(());
}
}
@ -306,10 +325,12 @@ where
Poll::Ready(Err(error)) => {
// Don't accept any further commands.
this.commands.get_mut().close();
let (handler, _closing_muxer) = connection.close();
// Terminate the task with the error, dropping the connection.
let event = Event::Closed {
id,
error: Some(error),
handler,
};
this.state = State::Terminating(event);
}
@ -317,13 +338,18 @@ where
}
}
State::Closing(mut closing) => {
State::Closing {
handler,
error,
mut closing_muxer,
} => {
// Try to gracefully close the connection.
match closing.poll_unpin(cx) {
match closing_muxer.poll_unpin(cx) {
Poll::Ready(Ok(())) => {
let event = Event::Closed {
id: this.id,
error: None,
error: error.map(|limit| ConnectionError::ConnectionLimit(limit)),
handler,
};
this.state = State::Terminating(event);
}
@ -331,11 +357,16 @@ where
let event = Event::Closed {
id: this.id,
error: Some(ConnectionError::IO(e)),
handler,
};
this.state = State::Terminating(event);
}
Poll::Pending => {
this.state = State::Closing(closing);
this.state = State::Closing {
handler,
error,
closing_muxer,
};
return Poll::Pending;
}
}

View File

@ -26,6 +26,7 @@ use crate::{
IntoConnectionHandler, OutgoingInfo, PendingConnectionError, Substream,
},
muxing::StreamMuxer,
network::DialError,
ConnectedPoint, PeerId,
};
use either::Either;
@ -53,12 +54,6 @@ pub struct Pool<THandler: IntoConnectionHandler, TTransErr> {
/// The pending connections that are currently being negotiated.
pending: FnvHashMap<ConnectionId, (ConnectedPoint, Option<PeerId>)>,
/// Established connections that have been closed in the context of
/// a [`Pool::disconnect`] in order to emit a `ConnectionClosed`
/// event for each. Every `ConnectionEstablished` event must be
/// paired with (eventually) a `ConnectionClosed`.
disconnected: Vec<Disconnected>,
}
impl<THandler: IntoConnectionHandler, TTransErr> fmt::Debug for Pool<THandler, TTransErr> {
@ -101,6 +96,7 @@ pub enum PoolEvent<'a, THandler: IntoConnectionHandler, TTransErr> {
pool: &'a mut Pool<THandler, TTransErr>,
/// The remaining number of established connections to the same peer.
num_established: u32,
handler: THandler::Handler,
},
/// A connection attempt failed.
@ -113,7 +109,7 @@ pub enum PoolEvent<'a, THandler: IntoConnectionHandler, TTransErr> {
error: PendingConnectionError<TTransErr>,
/// The handler that was supposed to handle the connection,
/// if the connection failed before the handler was consumed.
handler: Option<THandler>,
handler: THandler,
/// The (expected) peer of the failed connection.
peer: Option<PeerId>,
/// A reference to the pool that managed the connection.
@ -199,7 +195,6 @@ impl<THandler: IntoConnectionHandler, TTransErr> Pool<THandler, TTransErr> {
manager: Manager::new(manager_config),
established: Default::default(),
pending: Default::default(),
disconnected: Vec::new(),
}
}
@ -245,7 +240,7 @@ impl<THandler: IntoConnectionHandler, TTransErr> Pool<THandler, TTransErr> {
future: TFut,
handler: THandler,
info: OutgoingInfo<'_>,
) -> Result<ConnectionId, ConnectionLimit>
) -> Result<ConnectionId, DialError<THandler>>
where
TFut: Future<Output = Result<(PeerId, TMuxer), PendingConnectionError<TTransErr>>>
+ Send
@ -257,7 +252,9 @@ impl<THandler: IntoConnectionHandler, TTransErr> Pool<THandler, TTransErr> {
TMuxer: StreamMuxer + Send + Sync + 'static,
TMuxer::OutboundSubstream: Send + 'static,
{
self.counters.check_max_pending_outgoing()?;
if let Err(limit) = self.counters.check_max_pending_outgoing() {
return Err(DialError::ConnectionLimit { limit, handler });
};
let endpoint = info.to_connected_point();
Ok(self.add_pending(future, handler, endpoint, info.peer_id.cloned()))
}
@ -379,45 +376,24 @@ impl<THandler: IntoConnectionHandler, TTransErr> Pool<THandler, TTransErr> {
/// (Forcefully) close all connections to the given peer.
///
/// All connections to the peer, whether pending or established are
/// dropped asap and no more events from these connections are emitted
/// closed asap and no more events from these connections are emitted
/// by the pool effective immediately.
///
/// > **Note**: Established connections are dropped without performing
/// > an orderly close. See [`EstablishedConnection::start_close`] for
/// > performing such an orderly close.
pub fn disconnect(&mut self, peer: &PeerId) {
if let Some(conns) = self.established.get(peer) {
// Count upwards because we push to / pop from the end. See also `Pool::poll`.
let mut num_established = 0;
for (&id, endpoint) in conns.iter() {
for (&id, _endpoint) in conns.iter() {
if let Some(manager::Entry::Established(e)) = self.manager.entry(id) {
let connected = e.remove();
self.disconnected.push(Disconnected {
id,
connected,
num_established,
});
num_established += 1;
e.start_close(None);
}
self.counters.dec_established(endpoint);
}
}
self.established.remove(peer);
let mut aborted = Vec::new();
for (&id, (_endpoint, peer2)) in &self.pending {
if Some(peer) == peer2.as_ref() {
if let Some(manager::Entry::Pending(e)) = self.manager.entry(id) {
e.abort();
aborted.push(id);
}
}
}
for id in aborted {
if let Some((endpoint, _)) = self.pending.remove(&id) {
self.counters.dec_pending(&endpoint);
}
}
}
/// Counts the number of established connections to the given peer.
@ -503,28 +479,6 @@ impl<THandler: IntoConnectionHandler, TTransErr> Pool<THandler, TTransErr> {
&'a mut self,
cx: &mut Context<'_>,
) -> Poll<PoolEvent<'a, THandler, TTransErr>> {
// Drain events resulting from forced disconnections.
//
// Note: The `Disconnected` entries in `self.disconnected`
// are inserted in ascending order of the remaining `num_established`
// connections. Thus we `pop()` them off from the end to emit the
// events in an order that properly counts down `num_established`.
// See also `Pool::disconnect`.
if let Some(Disconnected {
id,
connected,
num_established,
}) = self.disconnected.pop()
{
return Poll::Ready(PoolEvent::ConnectionClosed {
id,
connected,
num_established,
error: None,
pool: self,
});
}
// Poll the connection `Manager`.
loop {
let item = match self.manager.poll(cx) {
@ -540,7 +494,7 @@ impl<THandler: IntoConnectionHandler, TTransErr> Pool<THandler, TTransErr> {
id,
endpoint,
error,
handler: Some(handler),
handler,
peer,
pool: self,
});
@ -550,6 +504,7 @@ impl<THandler: IntoConnectionHandler, TTransErr> Pool<THandler, TTransErr> {
id,
connected,
error,
handler,
} => {
let num_established =
if let Some(conns) = self.established.get_mut(&connected.peer_id) {
@ -569,6 +524,7 @@ impl<THandler: IntoConnectionHandler, TTransErr> Pool<THandler, TTransErr> {
error,
num_established,
pool: self,
handler,
});
}
manager::Event::ConnectionEstablished { entry } => {
@ -578,30 +534,16 @@ impl<THandler: IntoConnectionHandler, TTransErr> Pool<THandler, TTransErr> {
// Check general established connection limit.
if let Err(e) = self.counters.check_max_established(&endpoint) {
let connected = entry.remove();
return Poll::Ready(PoolEvent::PendingConnectionError {
id,
endpoint: connected.endpoint,
error: PendingConnectionError::ConnectionLimit(e),
handler: None,
peer,
pool: self,
});
entry.start_close(Some(e));
continue;
}
// Check per-peer established connection limit.
let current =
num_peer_established(&self.established, &entry.connected().peer_id);
if let Err(e) = self.counters.check_max_established_per_peer(current) {
let connected = entry.remove();
return Poll::Ready(PoolEvent::PendingConnectionError {
id,
endpoint: connected.endpoint,
error: PendingConnectionError::ConnectionLimit(e),
handler: None,
peer,
pool: self,
});
entry.start_close(Some(e));
continue;
}
// Peer ID checks must already have happened. See `add_pending`.
@ -790,7 +732,7 @@ impl<TInEvent> EstablishedConnection<'_, TInEvent> {
///
/// Has no effect if the connection is already closing.
pub fn start_close(self) {
self.entry.start_close()
self.entry.start_close(None)
}
}
@ -1064,15 +1006,3 @@ impl ConnectionLimits {
self
}
}
/// Information about a former established connection to a peer
/// that was dropped via [`Pool::disconnect`].
struct Disconnected {
/// The unique identifier of the dropped connection.
id: ConnectionId,
/// Information about the dropped connection.
connected: Connected,
/// The remaining number of established connections
/// to the same peer.
num_established: u32,
}

View File

@ -22,7 +22,7 @@ mod event;
pub mod peer;
pub use crate::connection::{ConnectionCounters, ConnectionLimits};
pub use event::{IncomingConnection, NetworkEvent};
pub use event::{DialAttemptsRemaining, IncomingConnection, NetworkEvent};
pub use peer::Peer;
use crate::{
@ -45,7 +45,7 @@ use std::{
collections::hash_map,
convert::TryFrom as _,
error, fmt,
num::NonZeroUsize,
num::{NonZeroU32, NonZeroUsize},
pin::Pin,
task::{Context, Poll},
};
@ -202,7 +202,7 @@ where
&mut self,
address: &Multiaddr,
handler: THandler,
) -> Result<ConnectionId, DialError>
) -> Result<ConnectionId, DialError<THandler>>
where
TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::Error: Send + 'static,
@ -235,15 +235,11 @@ where
Ok(f) => {
let f =
f.map_err(|err| PendingConnectionError::Transport(TransportError::Other(err)));
self.pool
.add_outgoing(f, handler, info)
.map_err(DialError::ConnectionLimit)
self.pool.add_outgoing(f, handler, info)
}
Err(err) => {
let f = future::err(PendingConnectionError::Transport(err));
self.pool
.add_outgoing(f, handler, info)
.map_err(DialError::ConnectionLimit)
self.pool.add_outgoing(f, handler, info)
}
}
}
@ -445,12 +441,14 @@ where
connected,
error,
num_established,
handler,
..
}) => NetworkEvent::ConnectionClosed {
id,
connected,
num_established,
error,
handler,
},
Poll::Ready(PoolEvent::ConnectionEvent { connection, event }) => {
NetworkEvent::ConnectionEvent { connection, event }
@ -470,7 +468,10 @@ where
}
/// Initiates a connection attempt to a known peer.
fn dial_peer(&mut self, opts: DialingOpts<PeerId, THandler>) -> Result<ConnectionId, DialError>
fn dial_peer(
&mut self,
opts: DialingOpts<PeerId, THandler>,
) -> Result<ConnectionId, DialError<THandler>>
where
TTrans: Transport<Output = (PeerId, TMuxer)>,
TTrans::Dial: Send + 'static,
@ -502,7 +503,7 @@ fn dial_peer_impl<TMuxer, THandler, TTrans>(
pool: &mut Pool<THandler, TTrans::Error>,
dialing: &mut FnvHashMap<PeerId, SmallVec<[peer::DialingState; 10]>>,
opts: DialingOpts<PeerId, THandler>,
) -> Result<ConnectionId, DialError>
) -> Result<ConnectionId, DialError<THandler>>
where
THandler: IntoConnectionHandler + Send + 'static,
<THandler::Handler as ConnectionHandler>::Error: error::Error + Send + 'static,
@ -517,7 +518,15 @@ where
// Ensure the address to dial encapsulates the `p2p` protocol for the
// targeted peer, so that the transport has a "fully qualified" address
// to work with.
let addr = p2p_addr(opts.peer, opts.address).map_err(DialError::InvalidAddress)?;
let addr = match p2p_addr(opts.peer, opts.address) {
Ok(address) => address,
Err(address) => {
return Err(DialError::InvalidAddress {
address,
handler: opts.handler,
})
}
};
let result = match transport.dial(addr.clone()) {
Ok(fut) => {
@ -527,7 +536,6 @@ where
peer_id: Some(&opts.peer),
};
pool.add_outgoing(fut, opts.handler, info)
.map_err(DialError::ConnectionLimit)
}
Err(err) => {
let fut = future::err(PendingConnectionError::Transport(err));
@ -536,7 +544,6 @@ where
peer_id: Some(&opts.peer),
};
pool.add_outgoing(fut, opts.handler, info)
.map_err(DialError::ConnectionLimit)
}
};
@ -563,7 +570,7 @@ fn on_connection_failed<'a, TTrans, THandler>(
id: ConnectionId,
endpoint: ConnectedPoint,
error: PendingConnectionError<TTrans::Error>,
handler: Option<THandler>,
handler: THandler,
) -> (
Option<DialingOpts<PeerId, THandler>>,
NetworkEvent<'a, TTrans, THandlerInEvent<THandler>, THandlerOutEvent<THandler>, THandler>,
@ -591,24 +598,17 @@ where
let num_remain = u32::try_from(attempt.remaining.len()).unwrap();
let failed_addr = attempt.current.1.clone();
let (opts, attempts_remaining) = if num_remain > 0 {
if let Some(handler) = handler {
let next_attempt = attempt.remaining.remove(0);
let opts = DialingOpts {
peer: peer_id,
handler,
address: next_attempt,
remaining: attempt.remaining,
};
(Some(opts), num_remain)
} else {
// The error is "fatal" for the dialing attempt, since
// the handler was already consumed. All potential
// remaining connection attempts are thus void.
(None, 0)
}
let (opts, attempts_remaining) = if let Some(num_remain) = NonZeroU32::new(num_remain) {
let next_attempt = attempt.remaining.remove(0);
let opts = DialingOpts {
peer: peer_id,
handler,
address: next_attempt,
remaining: attempt.remaining,
};
(Some(opts), DialAttemptsRemaining::Some(num_remain))
} else {
(None, 0)
(None, DialAttemptsRemaining::None(handler))
};
(
@ -628,6 +628,7 @@ where
NetworkEvent::UnknownPeerDialError {
multiaddr: address,
error,
handler,
},
),
ConnectedPoint::Listener {
@ -639,6 +640,7 @@ where
local_addr,
send_back_addr,
error,
handler,
},
),
}
@ -751,13 +753,42 @@ fn p2p_addr(peer: PeerId, addr: Multiaddr) -> Result<Multiaddr, Multiaddr> {
}
/// Possible (synchronous) errors when dialing a peer.
#[derive(Clone, Debug)]
pub enum DialError {
#[derive(Clone)]
pub enum DialError<THandler> {
/// The dialing attempt is rejected because of a connection limit.
ConnectionLimit(ConnectionLimit),
ConnectionLimit {
limit: ConnectionLimit,
handler: THandler,
},
/// The address being dialed is invalid, e.g. if it refers to a different
/// remote peer than the one being dialed.
InvalidAddress(Multiaddr),
InvalidAddress {
address: Multiaddr,
handler: THandler,
},
/// The dialing attempt is rejected because the peer being dialed is the local peer.
LocalPeerId { handler: THandler },
}
impl<THandler> fmt::Debug for DialError<THandler> {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> {
match self {
DialError::ConnectionLimit { limit, handler: _ } => f
.debug_struct("DialError::ConnectionLimit")
.field("limit", limit)
.finish(),
DialError::InvalidAddress {
address,
handler: _,
} => f
.debug_struct("DialError::InvalidAddress")
.field("address", address)
.finish(),
DialError::LocalPeerId { handler: _ } => {
f.debug_struct("DialError::LocalPeerId").finish()
}
}
}
}
#[cfg(test)]

View File

@ -92,6 +92,7 @@ where
send_back_addr: Multiaddr,
/// The error that happened.
error: PendingConnectionError<TTrans::Error>,
handler: THandler,
},
/// A new connection to a peer has been established.
@ -124,12 +125,13 @@ where
error: Option<ConnectionError<<THandler::Handler as ConnectionHandler>::Error>>,
/// The remaining number of established connections to the same peer.
num_established: u32,
handler: THandler::Handler,
},
/// A dialing attempt to an address of a peer failed.
DialError {
/// The number of remaining dialing attempts.
attempts_remaining: u32,
attempts_remaining: DialAttemptsRemaining<THandler>,
/// Id of the peer we were trying to dial.
peer_id: PeerId,
@ -148,6 +150,8 @@ where
/// The error that happened.
error: PendingConnectionError<TTrans::Error>,
handler: THandler,
},
/// An established connection produced an event.
@ -169,6 +173,20 @@ where
},
}
pub enum DialAttemptsRemaining<THandler> {
Some(NonZeroU32),
None(THandler),
}
impl<THandler> DialAttemptsRemaining<THandler> {
pub fn get_attempts(&self) -> u32 {
match self {
DialAttemptsRemaining::Some(attempts) => (*attempts).into(),
DialAttemptsRemaining::None(_) => 0,
}
}
}
impl<TTrans, TInEvent, TOutEvent, THandler> fmt::Debug
for NetworkEvent<'_, TTrans, TInEvent, TOutEvent, THandler>
where
@ -221,6 +239,7 @@ where
local_addr,
send_back_addr,
error,
handler: _,
} => f
.debug_struct("IncomingConnectionError")
.field("local_addr", local_addr)
@ -249,7 +268,7 @@ where
error,
} => f
.debug_struct("DialError")
.field("attempts_remaining", attempts_remaining)
.field("attempts_remaining", &attempts_remaining.get_attempts())
.field("peer_id", peer_id)
.field("multiaddr", multiaddr)
.field("error", error)

View File

@ -22,8 +22,8 @@ use super::{DialError, DialingOpts, Network};
use crate::{
connection::{
handler::THandlerInEvent, pool::Pool, ConnectedPoint, ConnectionHandler, ConnectionId,
ConnectionLimit, EstablishedConnection, EstablishedConnectionIter, IntoConnectionHandler,
PendingConnection, Substream,
EstablishedConnection, EstablishedConnectionIter, IntoConnectionHandler, PendingConnection,
Substream,
},
Multiaddr, PeerId, StreamMuxer, Transport,
};
@ -163,7 +163,7 @@ where
address: Multiaddr,
remaining: I,
handler: THandler,
) -> Result<(ConnectionId, DialingPeer<'a, TTrans, THandler>), DialError>
) -> Result<(ConnectionId, DialingPeer<'a, TTrans, THandler>), DialError<THandler>>
where
I: IntoIterator<Item = Multiaddr>,
{
@ -171,12 +171,7 @@ where
Peer::Connected(p) => (p.peer_id, p.network),
Peer::Dialing(p) => (p.peer_id, p.network),
Peer::Disconnected(p) => (p.peer_id, p.network),
Peer::Local => {
return Err(DialError::ConnectionLimit(ConnectionLimit {
current: 0,
limit: 0,
}))
}
Peer::Local => return Err(DialError::LocalPeerId { handler }),
};
let id = network.dial_peer(DialingOpts {